Browse Source

YARN-5959. RM changes to support change of container ExecutionType. (Arun Suresh via wangda)

(cherry picked from commit 0a55bd841ec0f2eb89a0383f4c589526e8b138d4)
Wangda Tan 8 years ago
parent
commit
650ff95e00
46 changed files with 1586 additions and 287 deletions
  1. 5 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
  2. 4 5
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
  3. 7 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerUpdateType.java
  4. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/UpdateContainerError.java
  5. 10 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/UpdateContainerRequest.java
  6. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
  7. 5 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
  8. 12 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java
  9. 5 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerContext.java
  10. 41 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/SchedulerRequestKey.java
  11. 41 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
  12. 6 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
  13. 88 54
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
  14. 4 2
      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
  15. 20 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
  16. 101 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
  17. 24 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Allocation.java
  18. 43 27
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
  19. 265 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerUpdateContext.java
  20. 68 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerUpdates.java
  21. 160 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
  22. 22 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
  23. 2 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
  24. 12 4
      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
  25. 1 1
      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
  26. 9 4
      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
  27. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
  28. 10 4
      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
  29. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java
  30. 2 3
      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
  31. 16 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalitySchedulingPlacementSet.java
  32. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/SchedulingPlacementSet.java
  33. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
  34. 7 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
  35. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
  36. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
  37. 453 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
  38. 11 8
      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
  39. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
  40. 21 16
      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
  41. 4 1
      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
  42. 6 3
      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
  43. 11 3
      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
  44. 5 4
      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
  45. 21 12
      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
  46. 31 15
      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

+ 5 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java

@@ -108,7 +108,6 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
@@ -130,6 +129,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
@@ -1705,8 +1705,7 @@ public class TestRMContainerAllocator {
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
         List<ContainerId> release, List<String> blacklistAdditions,
         List<ContainerId> release, List<String> blacklistAdditions,
         List<String> blacklistRemovals,
         List<String> blacklistRemovals,
-        List<UpdateContainerRequest> increaseRequests,
-        List<UpdateContainerRequest> decreaseRequests) {
+        ContainerUpdates updateRequests) {
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       for (ResourceRequest req : ask) {
       for (ResourceRequest req : ask) {
         ResourceRequest reqCopy = ResourceRequest.newInstance(req
         ResourceRequest reqCopy = ResourceRequest.newInstance(req
@@ -1721,7 +1720,7 @@ public class TestRMContainerAllocator {
       lastBlacklistRemovals = blacklistRemovals;
       lastBlacklistRemovals = blacklistRemovals;
       Allocation allocation = super.allocate(
       Allocation allocation = super.allocate(
           applicationAttemptId, askCopy, release, blacklistAdditions,
           applicationAttemptId, askCopy, release, blacklistAdditions,
-          blacklistRemovals, increaseRequests, decreaseRequests);
+          blacklistRemovals, updateRequests);
       if (forceResourceLimit != null) {
       if (forceResourceLimit != null) {
         // Test wants to force the non-default resource limit
         // Test wants to force the non-default resource limit
         allocation.setResourceLimit(forceResourceLimit);
         allocation.setResourceLimit(forceResourceLimit);
@@ -1752,8 +1751,7 @@ public class TestRMContainerAllocator {
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
         List<ContainerId> release, List<String> blacklistAdditions,
         List<ContainerId> release, List<String> blacklistAdditions,
         List<String> blacklistRemovals,
         List<String> blacklistRemovals,
-        List<UpdateContainerRequest> increaseRequest,
-        List<UpdateContainerRequest> decreaseRequests) {
+        ContainerUpdates updateRequests) {
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       for (ResourceRequest req : ask) {
       for (ResourceRequest req : ask) {
         ResourceRequest reqCopy = ResourceRequest.newInstance(req
         ResourceRequest reqCopy = ResourceRequest.newInstance(req
@@ -1764,7 +1762,7 @@ public class TestRMContainerAllocator {
       SecurityUtil.setTokenServiceUseIp(false);
       SecurityUtil.setTokenServiceUseIp(false);
       Allocation normalAlloc = super.allocate(
       Allocation normalAlloc = super.allocate(
           applicationAttemptId, askCopy, release,
           applicationAttemptId, askCopy, release,
-          blacklistAdditions, blacklistRemovals, null, null);
+          blacklistAdditions, blacklistRemovals, updateRequests);
       List<Container> containers = normalAlloc.getContainers();
       List<Container> containers = normalAlloc.getContainers();
       if(containers.size() > 0) {
       if(containers.size() > 0) {
         // allocate excess container
         // allocate excess container

+ 4 - 5
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java

@@ -59,7 +59,6 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
@@ -68,6 +67,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 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.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
@@ -207,14 +207,13 @@ public class ResourceSchedulerWrapper
   public Allocation allocate(ApplicationAttemptId attemptId,
   public Allocation allocate(ApplicationAttemptId attemptId,
       List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
       List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
       List<String> strings, List<String> strings2,
       List<String> strings, List<String> strings2,
-      List<UpdateContainerRequest> increaseRequests,
-      List<UpdateContainerRequest> decreaseRequests) {
+      ContainerUpdates updateRequests) {
     if (metricsON) {
     if (metricsON) {
       final Timer.Context context = schedulerAllocateTimer.time();
       final Timer.Context context = schedulerAllocateTimer.time();
       Allocation allocation = null;
       Allocation allocation = null;
       try {
       try {
         allocation = scheduler.allocate(attemptId, resourceRequests,
         allocation = scheduler.allocate(attemptId, resourceRequests,
-                containerIds, strings, strings2, null, null);
+                containerIds, strings, strings2, updateRequests);
         return allocation;
         return allocation;
       } finally {
       } finally {
         context.stop();
         context.stop();
@@ -228,7 +227,7 @@ public class ResourceSchedulerWrapper
       }
       }
     } else {
     } else {
       return scheduler.allocate(attemptId,
       return scheduler.allocate(attemptId,
-              resourceRequests, containerIds, strings, strings2, null, null);
+              resourceRequests, containerIds, strings, strings2, updateRequests);
     }
     }
   }
   }
 
 

+ 7 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerUpdateType.java

@@ -39,7 +39,12 @@ public enum ContainerUpdateType {
   DECREASE_RESOURCE,
   DECREASE_RESOURCE,
 
 
   /**
   /**
-   * Execution Type change.
+   * Execution Type promotion.
    */
    */
-  UPDATE_EXECUTION_TYPE
+  PROMOTE_EXECUTION_TYPE,
+
+  /**
+   * Execution Type demotion.
+   */
+  DEMOTE_EXECUTION_TYPE
 }
 }

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/UpdateContainerError.java

@@ -86,6 +86,12 @@ public abstract class UpdateContainerError {
     return result;
     return result;
   }
   }
 
 
+  @Override
+  public String toString() {
+    return "UpdateContainerError{reason=" + getReason() + ", "
+        + "req=" + getUpdateContainerRequest() + "}";
+  }
+
   @Override
   @Override
   public boolean equals(Object obj) {
   public boolean equals(Object obj) {
     if (this == obj) {
     if (this == obj) {

+ 10 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/UpdateContainerRequest.java

@@ -166,11 +166,13 @@ public abstract class UpdateContainerRequest {
     ContainerId cId = getContainerId();
     ContainerId cId = getContainerId();
     ExecutionType execType = getExecutionType();
     ExecutionType execType = getExecutionType();
     Resource capability = getCapability();
     Resource capability = getCapability();
+    ContainerUpdateType updateType = getContainerUpdateType();
     result =
     result =
         prime * result + ((capability == null) ? 0 : capability.hashCode());
         prime * result + ((capability == null) ? 0 : capability.hashCode());
     result = prime * result + ((cId == null) ? 0 : cId.hashCode());
     result = prime * result + ((cId == null) ? 0 : cId.hashCode());
     result = prime * result + getContainerVersion();
     result = prime * result + getContainerVersion();
     result = prime * result + ((execType == null) ? 0 : execType.hashCode());
     result = prime * result + ((execType == null) ? 0 : execType.hashCode());
+    result = prime * result + ((updateType== null) ? 0 : updateType.hashCode());
     return result;
     return result;
   }
   }
 
 
@@ -224,6 +226,14 @@ public abstract class UpdateContainerRequest {
     } else if (!execType.equals(other.getExecutionType())) {
     } else if (!execType.equals(other.getExecutionType())) {
       return false;
       return false;
     }
     }
+    ContainerUpdateType updateType = getContainerUpdateType();
+    if (updateType == null) {
+      if (other.getContainerUpdateType() != null) {
+        return false;
+      }
+    } else if (!updateType.equals(other.getContainerUpdateType())) {
+      return false;
+    }
     return true;
     return true;
   }
   }
 }
 }

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto

@@ -63,7 +63,8 @@ message FinishApplicationMasterResponseProto {
 enum ContainerUpdateTypeProto {
 enum ContainerUpdateTypeProto {
   INCREASE_RESOURCE = 0;
   INCREASE_RESOURCE = 0;
   DECREASE_RESOURCE = 1;
   DECREASE_RESOURCE = 1;
-  UPDATE_EXECUTION_TYPE = 2;
+  PROMOTE_EXECUTION_TYPE = 2;
+  DEMOTE_EXECUTION_TYPE = 3;
 }
 }
 
 
 message UpdateContainerRequestProto {
 message UpdateContainerRequestProto {

+ 5 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java

@@ -66,6 +66,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 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.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
@@ -574,8 +575,7 @@ public class TestAMRMClientOnRMRestart {
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
         List<ContainerId> release, List<String> blacklistAdditions,
         List<ContainerId> release, List<String> blacklistAdditions,
         List<String> blacklistRemovals,
         List<String> blacklistRemovals,
-        List<UpdateContainerRequest> increaseRequests,
-        List<UpdateContainerRequest> decreaseRequests) {
+        ContainerUpdates updateRequests) {
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       for (ResourceRequest req : ask) {
       for (ResourceRequest req : ask) {
         ResourceRequest reqCopy =
         ResourceRequest reqCopy =
@@ -586,13 +586,12 @@ public class TestAMRMClientOnRMRestart {
       }
       }
       lastAsk = ask;
       lastAsk = ask;
       lastRelease = release;
       lastRelease = release;
-      lastIncrease = increaseRequests;
-      lastDecrease = decreaseRequests;
+      lastIncrease = updateRequests.getIncreaseRequests();
+      lastDecrease = updateRequests.getDecreaseRequests();
       lastBlacklistAdditions = blacklistAdditions;
       lastBlacklistAdditions = blacklistAdditions;
       lastBlacklistRemovals = blacklistRemovals;
       lastBlacklistRemovals = blacklistRemovals;
       return super.allocate(applicationAttemptId, askCopy, release,
       return super.allocate(applicationAttemptId, askCopy, release,
-          blacklistAdditions, blacklistRemovals, increaseRequests,
-          decreaseRequests);
+          blacklistAdditions, blacklistRemovals, updateRequests);
     }
     }
   }
   }
 
 

+ 12 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java

@@ -321,13 +321,21 @@ public class OpportunisticContainerAllocator {
     // before accepting an ask)
     // before accepting an ask)
     Resource capability = normalizeCapability(appParams, rr);
     Resource capability = normalizeCapability(appParams, rr);
 
 
+    return createContainer(
+        rmIdentifier, appParams.getContainerTokenExpiryInterval(),
+        SchedulerRequestKey.create(rr), userName, node, cId, capability);
+  }
+
+  private Container createContainer(long rmIdentifier, long tokenExpiry,
+      SchedulerRequestKey schedulerKey, String userName, RemoteNode node,
+      ContainerId cId, Resource capability) {
     long currTime = System.currentTimeMillis();
     long currTime = System.currentTimeMillis();
     ContainerTokenIdentifier containerTokenIdentifier =
     ContainerTokenIdentifier containerTokenIdentifier =
         new ContainerTokenIdentifier(
         new ContainerTokenIdentifier(
             cId, 0, node.getNodeId().toString(), userName,
             cId, 0, node.getNodeId().toString(), userName,
-            capability, currTime + appParams.containerTokenExpiryInterval,
+            capability, currTime + tokenExpiry,
             tokenSecretManager.getCurrentKey().getKeyId(), rmIdentifier,
             tokenSecretManager.getCurrentKey().getKeyId(), rmIdentifier,
-            rr.getPriority(), currTime,
+            schedulerKey.getPriority(), currTime,
             null, CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK,
             null, CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK,
             ExecutionType.OPPORTUNISTIC);
             ExecutionType.OPPORTUNISTIC);
     byte[] pwd =
     byte[] pwd =
@@ -336,9 +344,9 @@ public class OpportunisticContainerAllocator {
         containerTokenIdentifier);
         containerTokenIdentifier);
     Container container = BuilderUtils.newContainer(
     Container container = BuilderUtils.newContainer(
         cId, node.getNodeId(), node.getHttpAddress(),
         cId, node.getNodeId(), node.getHttpAddress(),
-        capability, rr.getPriority(), containerToken,
+        capability, schedulerKey.getPriority(), containerToken,
         containerTokenIdentifier.getExecutionType(),
         containerTokenIdentifier.getExecutionType(),
-        rr.getAllocationRequestId());
+        schedulerKey.getAllocationRequestId());
     return container;
     return container;
   }
   }
 
 

+ 5 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerContext.java

@@ -150,8 +150,9 @@ public class OpportunisticContainerContext {
             resourceRequest.getNumContainers() + request.getNumContainers());
             resourceRequest.getNumContainers() + request.getNumContainers());
       }
       }
       if (ResourceRequest.isAnyLocation(request.getResourceName())) {
       if (ResourceRequest.isAnyLocation(request.getResourceName())) {
-        LOG.info("# of outstandingOpReqs in ANY (at" +
-            "priority = "+ schedulerKey.getPriority()
+        LOG.info("# of outstandingOpReqs in ANY (at "
+            + "priority = " + schedulerKey.getPriority()
+            + ", allocationReqId = " + schedulerKey.getAllocationRequestId()
             + ", with capability = " + request.getCapability() + " ) : "
             + ", with capability = " + request.getCapability() + " ) : "
             + resourceRequest.getNumContainers());
             + resourceRequest.getNumContainers());
       }
       }
@@ -167,7 +168,8 @@ public class OpportunisticContainerContext {
   public void matchAllocationToOutstandingRequest(Resource capability,
   public void matchAllocationToOutstandingRequest(Resource capability,
       List<Container> allocatedContainers) {
       List<Container> allocatedContainers) {
     for (Container c : allocatedContainers) {
     for (Container c : allocatedContainers) {
-      SchedulerRequestKey schedulerKey = SchedulerRequestKey.extractFrom(c);
+      SchedulerRequestKey schedulerKey =
+          SchedulerRequestKey.extractFrom(c);
       Map<Resource, ResourceRequest> asks =
       Map<Resource, ResourceRequest> asks =
           outstandingOpReqs.get(schedulerKey);
           outstandingOpReqs.get(schedulerKey);
 
 

+ 41 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/SchedulerRequestKey.java

@@ -19,8 +19,10 @@
 package org.apache.hadoop.yarn.server.scheduler;
 package org.apache.hadoop.yarn.server.scheduler;
 
 
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 
 
 /**
 /**
  * Composite key for outstanding scheduler requests for any schedulable entity.
  * Composite key for outstanding scheduler requests for any schedulable entity.
@@ -31,6 +33,7 @@ public final class SchedulerRequestKey implements
 
 
   private final Priority priority;
   private final Priority priority;
   private final long allocationRequestId;
   private final long allocationRequestId;
+  private final ContainerId containerToUpdate;
 
 
   /**
   /**
    * Factory method to generate a SchedulerRequestKey from a ResourceRequest.
    * Factory method to generate a SchedulerRequestKey from a ResourceRequest.
@@ -39,7 +42,13 @@ public final class SchedulerRequestKey implements
    */
    */
   public static SchedulerRequestKey create(ResourceRequest req) {
   public static SchedulerRequestKey create(ResourceRequest req) {
     return new SchedulerRequestKey(req.getPriority(),
     return new SchedulerRequestKey(req.getPriority(),
-        req.getAllocationRequestId());
+        req.getAllocationRequestId(), null);
+  }
+
+  public static SchedulerRequestKey create(UpdateContainerRequest req,
+      SchedulerRequestKey schedulerRequestKey) {
+    return new SchedulerRequestKey(schedulerRequestKey.getPriority(),
+        schedulerRequestKey.getAllocationRequestId(), req.getContainerId());
   }
   }
 
 
   /**
   /**
@@ -50,12 +59,16 @@ public final class SchedulerRequestKey implements
    */
    */
   public static SchedulerRequestKey extractFrom(Container container) {
   public static SchedulerRequestKey extractFrom(Container container) {
     return new SchedulerRequestKey(container.getPriority(),
     return new SchedulerRequestKey(container.getPriority(),
-        container.getAllocationRequestId());
+        container.getAllocationRequestId(), null);
   }
   }
 
 
-  SchedulerRequestKey(Priority priority, long allocationRequestId) {
+
+
+  public SchedulerRequestKey(Priority priority, long allocationRequestId,
+      ContainerId containerToUpdate) {
     this.priority = priority;
     this.priority = priority;
     this.allocationRequestId = allocationRequestId;
     this.allocationRequestId = allocationRequestId;
+    this.containerToUpdate = containerToUpdate;
   }
   }
 
 
   /**
   /**
@@ -76,6 +89,10 @@ public final class SchedulerRequestKey implements
     return allocationRequestId;
     return allocationRequestId;
   }
   }
 
 
+  public ContainerId getContainerToUpdate() {
+    return containerToUpdate;
+  }
+
   @Override
   @Override
   public int compareTo(SchedulerRequestKey o) {
   public int compareTo(SchedulerRequestKey o) {
     if (o == null) {
     if (o == null) {
@@ -85,6 +102,15 @@ public final class SchedulerRequestKey implements
         return 1;
         return 1;
       }
       }
     }
     }
+
+    // Ensure updates are ranked higher
+    if (this.containerToUpdate == null && o.containerToUpdate != null) {
+      return -1;
+    }
+    if (this.containerToUpdate != null && o.containerToUpdate == null) {
+      return 1;
+    }
+
     int priorityCompare = o.getPriority().compareTo(priority);
     int priorityCompare = o.getPriority().compareTo(priority);
     // we first sort by priority and then by allocationRequestId
     // we first sort by priority and then by allocationRequestId
     if (priorityCompare != 0) {
     if (priorityCompare != 0) {
@@ -107,16 +133,21 @@ public final class SchedulerRequestKey implements
     if (getAllocationRequestId() != that.getAllocationRequestId()) {
     if (getAllocationRequestId() != that.getAllocationRequestId()) {
       return false;
       return false;
     }
     }
-    return getPriority() != null ?
-        getPriority().equals(that.getPriority()) :
-        that.getPriority() == null;
+    if (!getPriority().equals(that.getPriority())) {
+      return false;
+    }
+    return containerToUpdate != null ?
+        containerToUpdate.equals(that.containerToUpdate) :
+        that.containerToUpdate == null;
   }
   }
 
 
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
-    int result = getPriority() != null ? getPriority().hashCode() : 0;
-    result = 31 * result + (int) (getAllocationRequestId() ^ (
-        getAllocationRequestId() >>> 32));
+    int result = priority != null ? priority.hashCode() : 0;
+    result = 31 * result + (int) (allocationRequestId ^ (allocationRequestId
+        >>> 32));
+    result = 31 * result + (containerToUpdate != null ? containerToUpdate
+        .hashCode() : 0);
     return result;
     return result;
   }
   }
 
 
@@ -125,6 +156,7 @@ public final class SchedulerRequestKey implements
     return "SchedulerRequestKey{" +
     return "SchedulerRequestKey{" +
         "priority=" + priority +
         "priority=" + priority +
         ", allocationRequestId=" + allocationRequestId +
         ", allocationRequestId=" + allocationRequestId +
+        ", containerToUpdate=" + containerToUpdate +
         '}';
         '}';
   }
   }
 }
 }

+ 41 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java

@@ -68,7 +68,6 @@ import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
 import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
 import org.apache.hadoop.yarn.api.records.UpdateContainerError;
 import org.apache.hadoop.yarn.api.records.UpdateContainerError;
-import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
@@ -93,7 +92,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
 
 
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security
 import org.apache.hadoop.yarn.server.resourcemanager.security
@@ -554,12 +558,10 @@ public class ApplicationMasterService extends AbstractService implements
     // Split Update Resource Requests into increase and decrease.
     // Split Update Resource Requests into increase and decrease.
     // No Exceptions are thrown here. All update errors are aggregated
     // No Exceptions are thrown here. All update errors are aggregated
     // and returned to the AM.
     // and returned to the AM.
-    List<UpdateContainerRequest> increaseResourceReqs = new ArrayList<>();
-    List<UpdateContainerRequest> decreaseResourceReqs = new ArrayList<>();
-    List<UpdateContainerError> updateContainerErrors =
+    List<UpdateContainerError> updateErrors = new ArrayList<>();
+    ContainerUpdates containerUpdateRequests =
         RMServerUtils.validateAndSplitUpdateResourceRequests(
         RMServerUtils.validateAndSplitUpdateResourceRequests(
-            rmContext, request, maximumCapacity,
-            increaseResourceReqs, decreaseResourceReqs);
+        rmContext, request, maximumCapacity, updateErrors);
 
 
     // Send new requests to appAttempt.
     // Send new requests to appAttempt.
     Allocation allocation;
     Allocation allocation;
@@ -575,7 +577,7 @@ public class ApplicationMasterService extends AbstractService implements
       allocation =
       allocation =
           this.rScheduler.allocate(appAttemptId, ask, release,
           this.rScheduler.allocate(appAttemptId, ask, release,
               blacklistAdditions, blacklistRemovals,
               blacklistAdditions, blacklistRemovals,
-              increaseResourceReqs, decreaseResourceReqs);
+              containerUpdateRequests);
     }
     }
 
 
     if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) {
     if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) {
@@ -591,7 +593,7 @@ public class ApplicationMasterService extends AbstractService implements
     }
     }
 
 
     // Notify the AM of container update errors
     // Notify the AM of container update errors
-    addToUpdateContainerErrors(allocateResponse, updateContainerErrors);
+    addToUpdateContainerErrors(allocateResponse, updateErrors);
 
 
     // update the response with the deltas of node status changes
     // update the response with the deltas of node status changes
     List<RMNode> updatedNodes = new ArrayList<RMNode>();
     List<RMNode> updatedNodes = new ArrayList<RMNode>();
@@ -625,6 +627,21 @@ public class ApplicationMasterService extends AbstractService implements
         .pullJustFinishedContainers());
         .pullJustFinishedContainers());
     allocateResponse.setAvailableResources(allocation.getResourceLimit());
     allocateResponse.setAvailableResources(allocation.getResourceLimit());
 
 
+    addToContainerUpdates(appAttemptId, allocateResponse, allocation);
+
+    allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
+
+    // add preemption to the allocateResponse message (if any)
+    allocateResponse
+        .setPreemptionMessage(generatePreemptionMessage(allocation));
+
+    // Set application priority
+    allocateResponse.setApplicationPriority(app
+        .getApplicationPriority());
+  }
+
+  private void addToContainerUpdates(ApplicationAttemptId appAttemptId,
+      AllocateResponse allocateResponse, Allocation allocation) {
     // Handling increased containers
     // Handling increased containers
     addToUpdatedContainers(
     addToUpdatedContainers(
         allocateResponse, ContainerUpdateType.INCREASE_RESOURCE,
         allocateResponse, ContainerUpdateType.INCREASE_RESOURCE,
@@ -635,15 +652,23 @@ public class ApplicationMasterService extends AbstractService implements
         allocateResponse, ContainerUpdateType.DECREASE_RESOURCE,
         allocateResponse, ContainerUpdateType.DECREASE_RESOURCE,
         allocation.getDecreasedContainers());
         allocation.getDecreasedContainers());
 
 
-    allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
-
-    // add preemption to the allocateResponse message (if any)
-    allocateResponse
-        .setPreemptionMessage(generatePreemptionMessage(allocation));
+    // Handling promoted containers
+    addToUpdatedContainers(
+        allocateResponse, ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+        allocation.getPromotedContainers());
 
 
-    // Set application priority
-    allocateResponse.setApplicationPriority(app
-        .getApplicationPriority());
+    // Handling demoted containers
+    addToUpdatedContainers(
+        allocateResponse, ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
+        allocation.getDemotedContainers());
+
+    SchedulerApplicationAttempt applicationAttempt = ((AbstractYarnScheduler)
+        rScheduler).getApplicationAttempt(appAttemptId);
+    if (applicationAttempt != null) {
+      addToUpdateContainerErrors(allocateResponse,
+          ((AbstractYarnScheduler)rScheduler)
+              .getApplicationAttempt(appAttemptId).pullUpdateContainerErrors());
+    }
   }
   }
 
 
   protected void addToUpdateContainerErrors(AllocateResponse allocateResponse,
   protected void addToUpdateContainerErrors(AllocateResponse allocateResponse,

+ 6 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java

@@ -32,7 +32,6 @@ import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
 import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
 import org.apache.hadoop.yarn.api.impl.pb.service.ApplicationMasterProtocolPBServiceImpl;
 import org.apache.hadoop.yarn.api.impl.pb.service.ApplicationMasterProtocolPBServiceImpl;
 
 
-
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
@@ -48,15 +47,14 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService;
 import org.apache.hadoop.yarn.proto.ApplicationMasterProtocol.ApplicationMasterProtocolService;
 
 
-
 import org.apache.hadoop.yarn.server.api.protocolrecords.RemoteNode;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RemoteNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 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.RMContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 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.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.NodeQueueLoadMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.NodeQueueLoadMonitor;
 
 
@@ -69,9 +67,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateS
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 
 
-
 import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator;
 import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator;
 import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
 import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
+import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
 import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
 
 
 import java.io.IOException;
 import java.io.IOException;
@@ -251,6 +249,7 @@ public class OpportunisticContainerAllocatorAMService
 
 
     // Allocate GUARANTEED containers.
     // Allocate GUARANTEED containers.
     request.setAskList(partitionedAsks.getGuaranteed());
     request.setAskList(partitionedAsks.getGuaranteed());
+
     super.allocateInternal(appAttemptId, request, allocateResponse);
     super.allocateInternal(appAttemptId, request, allocateResponse);
   }
   }
 
 
@@ -298,15 +297,9 @@ public class OpportunisticContainerAllocatorAMService
       boolean isRemotelyAllocated) {
       boolean isRemotelyAllocated) {
     for (Container container : allocContainers) {
     for (Container container : allocContainers) {
       // Create RMContainer
       // Create RMContainer
-      SchedulerApplicationAttempt appAttempt =
-          ((AbstractYarnScheduler) rmContext.getScheduler())
-              .getCurrentAttemptForContainer(container.getId());
-      RMContainer rmContainer = new RMContainerImpl(container,
-          appAttempt.getApplicationAttemptId(), container.getNodeId(),
-          appAttempt.getUser(), rmContext, isRemotelyAllocated);
-      appAttempt.addRMContainer(container.getId(), rmContainer);
-      ((AbstractYarnScheduler) rmContext.getScheduler()).getNode(
-          container.getNodeId()).allocateContainer(rmContainer);
+      RMContainer rmContainer =
+          SchedulerUtils.createOpportunisticRmContainer(
+              rmContext, container, isRemotelyAllocated);
       rmContainer.handle(
       rmContainer.handle(
           new RMContainerEvent(container.getId(),
           new RMContainerEvent(container.getId(),
               RMContainerEventType.ACQUIRED));
               RMContainerEventType.ACQUIRED));

+ 88 - 54
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java

@@ -39,6 +39,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -64,6 +66,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt
     .RMAppAttemptState;
     .RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler
     .ResourceScheduler;
     .ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler
@@ -81,7 +84,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
  */
  */
 public class RMServerUtils {
 public class RMServerUtils {
 
 
-  private static final String UPDATE_OUTSTANDING_ERROR =
+  public static final String UPDATE_OUTSTANDING_ERROR =
       "UPDATE_OUTSTANDING_ERROR";
       "UPDATE_OUTSTANDING_ERROR";
   private static final String INCORRECT_CONTAINER_VERSION_ERROR =
   private static final String INCORRECT_CONTAINER_VERSION_ERROR =
       "INCORRECT_CONTAINER_VERSION_ERROR";
       "INCORRECT_CONTAINER_VERSION_ERROR";
@@ -125,74 +128,105 @@ public class RMServerUtils {
 
 
   /**
   /**
    * Check if we have:
    * Check if we have:
-   * - Request for same containerId and different target resource
-   * - If targetResources violates maximum/minimumAllocation
-   * @param rmContext RM context
-   * @param request Allocate Request
-   * @param maximumAllocation Maximum Allocation
-   * @param increaseResourceReqs Increase Resource Request
-   * @param decreaseResourceReqs Decrease Resource Request
-   * @return List of container Errors
+   * - Request for same containerId and different target resource.
+   * - If targetResources violates maximum/minimumAllocation.
+   * @param rmContext RM context.
+   * @param request Allocate Request.
+   * @param maximumAllocation Maximum Allocation.
+   * @param updateErrors Container update errors.
+   * @return ContainerUpdateRequests.
    */
    */
-  public static List<UpdateContainerError>
+  public static ContainerUpdates
       validateAndSplitUpdateResourceRequests(RMContext rmContext,
       validateAndSplitUpdateResourceRequests(RMContext rmContext,
       AllocateRequest request, Resource maximumAllocation,
       AllocateRequest request, Resource maximumAllocation,
-      List<UpdateContainerRequest> increaseResourceReqs,
-      List<UpdateContainerRequest> decreaseResourceReqs) {
-    List<UpdateContainerError> errors = new ArrayList<>();
+      List<UpdateContainerError> updateErrors) {
+    ContainerUpdates updateRequests =
+        new ContainerUpdates();
     Set<ContainerId> outstandingUpdate = new HashSet<>();
     Set<ContainerId> outstandingUpdate = new HashSet<>();
     for (UpdateContainerRequest updateReq : request.getUpdateRequests()) {
     for (UpdateContainerRequest updateReq : request.getUpdateRequests()) {
       RMContainer rmContainer = rmContext.getScheduler().getRMContainer(
       RMContainer rmContainer = rmContext.getScheduler().getRMContainer(
           updateReq.getContainerId());
           updateReq.getContainerId());
-      String msg = null;
-      if (rmContainer == null) {
-        msg = INVALID_CONTAINER_ID;
-      }
-      // Only allow updates if the requested version matches the current
-      // version
-      if (msg == null && updateReq.getContainerVersion() !=
-          rmContainer.getContainer().getVersion()) {
-        msg = INCORRECT_CONTAINER_VERSION_ERROR + "|"
-            + updateReq.getContainerVersion() + "|"
-            + rmContainer.getContainer().getVersion();
-      }
-      // No more than 1 container update per request.
-      if (msg == null &&
-          outstandingUpdate.contains(updateReq.getContainerId())) {
-        msg = UPDATE_OUTSTANDING_ERROR;
-      }
+      String msg = validateContainerIdAndVersion(outstandingUpdate,
+          updateReq, rmContainer);
+      ContainerUpdateType updateType = updateReq.getContainerUpdateType();
       if (msg == null) {
       if (msg == null) {
-        Resource original = rmContainer.getContainer().getResource();
-        Resource target = updateReq.getCapability();
-        if (Resources.fitsIn(target, original)) {
-          // This is a decrease request
-          if (validateIncreaseDecreaseRequest(rmContext, updateReq,
-              maximumAllocation, false)) {
-            decreaseResourceReqs.add(updateReq);
-            outstandingUpdate.add(updateReq.getContainerId());
+        if ((updateType != ContainerUpdateType.PROMOTE_EXECUTION_TYPE) &&
+            (updateType !=ContainerUpdateType.DEMOTE_EXECUTION_TYPE)) {
+          Resource original = rmContainer.getContainer().getResource();
+          Resource target = updateReq.getCapability();
+          if (Resources.fitsIn(target, original)) {
+            // This is a decrease request
+            if (validateIncreaseDecreaseRequest(rmContext, updateReq,
+                maximumAllocation, false)) {
+              updateRequests.getDecreaseRequests().add(updateReq);
+              outstandingUpdate.add(updateReq.getContainerId());
+            } else {
+              msg = RESOURCE_OUTSIDE_ALLOWED_RANGE;
+            }
           } else {
           } else {
-            msg = RESOURCE_OUTSIDE_ALLOWED_RANGE;
+            // This is an increase request
+            if (validateIncreaseDecreaseRequest(rmContext, updateReq,
+                maximumAllocation, true)) {
+              updateRequests.getIncreaseRequests().add(updateReq);
+              outstandingUpdate.add(updateReq.getContainerId());
+            } else {
+              msg = RESOURCE_OUTSIDE_ALLOWED_RANGE;
+            }
           }
           }
         } else {
         } else {
-          // This is an increase request
-          if (validateIncreaseDecreaseRequest(rmContext, updateReq,
-              maximumAllocation, true)) {
-            increaseResourceReqs.add(updateReq);
-            outstandingUpdate.add(updateReq.getContainerId());
-          } else {
-            msg = RESOURCE_OUTSIDE_ALLOWED_RANGE;
+          ExecutionType original = rmContainer.getExecutionType();
+          ExecutionType target = updateReq.getExecutionType();
+          if (target != original) {
+            if (target == ExecutionType.GUARANTEED &&
+                original == ExecutionType.OPPORTUNISTIC) {
+              updateRequests.getPromotionRequests().add(updateReq);
+              outstandingUpdate.add(updateReq.getContainerId());
+            } else if (target == ExecutionType.OPPORTUNISTIC &&
+                original == ExecutionType.GUARANTEED) {
+              updateRequests.getDemotionRequests().add(updateReq);
+              outstandingUpdate.add(updateReq.getContainerId());
+            }
           }
           }
         }
         }
       }
       }
-      if (msg != null) {
-        UpdateContainerError updateError = RECORD_FACTORY
-            .newRecordInstance(UpdateContainerError.class);
-        updateError.setReason(msg);
-        updateError.setUpdateContainerRequest(updateReq);
-        errors.add(updateError);
-      }
+      checkAndcreateUpdateError(updateErrors, updateReq, msg);
+    }
+    return updateRequests;
+  }
+
+  private static void checkAndcreateUpdateError(
+      List<UpdateContainerError> errors, UpdateContainerRequest updateReq,
+      String msg) {
+    if (msg != null) {
+      UpdateContainerError updateError = RECORD_FACTORY
+          .newRecordInstance(UpdateContainerError.class);
+      updateError.setReason(msg);
+      updateError.setUpdateContainerRequest(updateReq);
+      errors.add(updateError);
+    }
+  }
+
+  private static String validateContainerIdAndVersion(
+      Set<ContainerId> outstandingUpdate, UpdateContainerRequest updateReq,
+      RMContainer rmContainer) {
+    String msg = null;
+    if (rmContainer == null) {
+      msg = INVALID_CONTAINER_ID;
+    }
+    // Only allow updates if the requested version matches the current
+    // version
+    if (msg == null && updateReq.getContainerVersion() !=
+        rmContainer.getContainer().getVersion()) {
+      msg = INCORRECT_CONTAINER_VERSION_ERROR + "|"
+          + updateReq.getContainerVersion() + "|"
+          + rmContainer.getContainer().getVersion();
+    }
+    // No more than 1 container update per request.
+    if (msg == null &&
+        outstandingUpdate.contains(updateReq.getContainerId())) {
+      msg = UPDATE_OUTSTANDING_ERROR;
     }
     }
-    return errors;
+    return msg;
   }
   }
 
 
   /**
   /**

+ 4 - 2
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

@@ -96,6 +96,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContainersPulledByAMEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContainersPulledByAMEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
@@ -1123,7 +1124,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
                 Collections.singletonList(appAttempt.amReq),
                 Collections.singletonList(appAttempt.amReq),
                 EMPTY_CONTAINER_RELEASE_LIST,
                 EMPTY_CONTAINER_RELEASE_LIST,
                 amBlacklist.getBlacklistAdditions(),
                 amBlacklist.getBlacklistAdditions(),
-                amBlacklist.getBlacklistRemovals(), null, null);
+                amBlacklist.getBlacklistRemovals(),
+                new ContainerUpdates());
         if (amContainerAllocation != null
         if (amContainerAllocation != null
             && amContainerAllocation.getContainers() != null) {
             && amContainerAllocation.getContainers() != null) {
           assert (amContainerAllocation.getContainers().size() == 0);
           assert (amContainerAllocation.getContainers().size() == 0);
@@ -1147,7 +1149,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       Allocation amContainerAllocation =
       Allocation amContainerAllocation =
           appAttempt.scheduler.allocate(appAttempt.applicationAttemptId,
           appAttempt.scheduler.allocate(appAttempt.applicationAttemptId,
             EMPTY_CONTAINER_REQUEST_LIST, EMPTY_CONTAINER_RELEASE_LIST, null,
             EMPTY_CONTAINER_REQUEST_LIST, EMPTY_CONTAINER_RELEASE_LIST, null,
-            null, null, null);
+            null, new ContainerUpdates());
       // There must be at least one container allocated, because a
       // There must be at least one container allocated, because a
       // CONTAINER_ALLOCATED is emitted after an RMContainer is constructed,
       // CONTAINER_ALLOCATED is emitted after an RMContainer is constructed,
       // and is put in SchedulerApplication#newlyAllocatedContainers.
       // and is put in SchedulerApplication#newlyAllocatedContainers.

+ 20 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java

@@ -108,6 +108,8 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
     // Transitions from ACQUIRED state
     // Transitions from ACQUIRED state
     .addTransition(RMContainerState.ACQUIRED, RMContainerState.RUNNING,
     .addTransition(RMContainerState.ACQUIRED, RMContainerState.RUNNING,
         RMContainerEventType.LAUNCHED)
         RMContainerEventType.LAUNCHED)
+    .addTransition(RMContainerState.ACQUIRED, RMContainerState.ACQUIRED,
+        RMContainerEventType.ACQUIRED)
     .addTransition(RMContainerState.ACQUIRED, RMContainerState.COMPLETED,
     .addTransition(RMContainerState.ACQUIRED, RMContainerState.COMPLETED,
         RMContainerEventType.FINISHED, new FinishedTransition())
         RMContainerEventType.FINISHED, new FinishedTransition())
     .addTransition(RMContainerState.ACQUIRED, RMContainerState.RELEASED,
     .addTransition(RMContainerState.ACQUIRED, RMContainerState.RELEASED,
@@ -124,6 +126,8 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
         RMContainerEventType.KILL, new KillTransition())
         RMContainerEventType.KILL, new KillTransition())
     .addTransition(RMContainerState.RUNNING, RMContainerState.RELEASED,
     .addTransition(RMContainerState.RUNNING, RMContainerState.RELEASED,
         RMContainerEventType.RELEASED, new KillTransition())
         RMContainerEventType.RELEASED, new KillTransition())
+    .addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
+        RMContainerEventType.ACQUIRED)
     .addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
     .addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
         RMContainerEventType.RESERVED, new ContainerReservedTransition())
         RMContainerEventType.RESERVED, new ContainerReservedTransition())
     .addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
     .addTransition(RMContainerState.RUNNING, RMContainerState.RUNNING,
@@ -163,13 +167,13 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
   private final WriteLock writeLock;
   private final WriteLock writeLock;
   private final ApplicationAttemptId appAttemptId;
   private final ApplicationAttemptId appAttemptId;
   private final NodeId nodeId;
   private final NodeId nodeId;
-  private final Container container;
   private final RMContext rmContext;
   private final RMContext rmContext;
   private final EventHandler eventHandler;
   private final EventHandler eventHandler;
   private final ContainerAllocationExpirer containerAllocationExpirer;
   private final ContainerAllocationExpirer containerAllocationExpirer;
   private final String user;
   private final String user;
   private final String nodeLabelExpression;
   private final String nodeLabelExpression;
 
 
+  private volatile Container container;
   private Resource reservedResource;
   private Resource reservedResource;
   private NodeId reservedNode;
   private NodeId reservedNode;
   private SchedulerRequestKey reservedSchedulerKey;
   private SchedulerRequestKey reservedSchedulerKey;
@@ -188,44 +192,44 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
   private boolean isExternallyAllocated;
   private boolean isExternallyAllocated;
   private SchedulerRequestKey allocatedSchedulerKey;
   private SchedulerRequestKey allocatedSchedulerKey;
 
 
-  public RMContainerImpl(Container container,
+  public RMContainerImpl(Container container, SchedulerRequestKey schedulerKey,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       RMContext rmContext) {
       RMContext rmContext) {
-    this(container, appAttemptId, nodeId, user, rmContext, System
+    this(container, schedulerKey, appAttemptId, nodeId, user, rmContext, System
         .currentTimeMillis(), "");
         .currentTimeMillis(), "");
   }
   }
 
 
-  public RMContainerImpl(Container container,
+  public RMContainerImpl(Container container, SchedulerRequestKey schedulerKey,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       RMContext rmContext, boolean isExternallyAllocated) {
       RMContext rmContext, boolean isExternallyAllocated) {
-    this(container, appAttemptId, nodeId, user, rmContext, System
+    this(container, schedulerKey, appAttemptId, nodeId, user, rmContext, System
         .currentTimeMillis(), "", isExternallyAllocated);
         .currentTimeMillis(), "", isExternallyAllocated);
   }
   }
 
 
   private boolean saveNonAMContainerMetaInfo;
   private boolean saveNonAMContainerMetaInfo;
 
 
-  public RMContainerImpl(Container container,
+  public RMContainerImpl(Container container, SchedulerRequestKey schedulerKey,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       RMContext rmContext, String nodeLabelExpression) {
       RMContext rmContext, String nodeLabelExpression) {
-    this(container, appAttemptId, nodeId, user, rmContext, System
+    this(container, schedulerKey, appAttemptId, nodeId, user, rmContext, System
       .currentTimeMillis(), nodeLabelExpression);
       .currentTimeMillis(), nodeLabelExpression);
   }
   }
 
 
-  public RMContainerImpl(Container container,
+  public RMContainerImpl(Container container, SchedulerRequestKey schedulerKey,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       RMContext rmContext, long creationTime, String nodeLabelExpression) {
       RMContext rmContext, long creationTime, String nodeLabelExpression) {
-    this(container, appAttemptId, nodeId, user, rmContext, creationTime,
-        nodeLabelExpression, false);
+    this(container, schedulerKey, appAttemptId, nodeId, user, rmContext,
+        creationTime, nodeLabelExpression, false);
   }
   }
 
 
-  public RMContainerImpl(Container container,
+  public RMContainerImpl(Container container, SchedulerRequestKey schedulerKey,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       ApplicationAttemptId appAttemptId, NodeId nodeId, String user,
       RMContext rmContext, long creationTime, String nodeLabelExpression,
       RMContext rmContext, long creationTime, String nodeLabelExpression,
       boolean isExternallyAllocated) {
       boolean isExternallyAllocated) {
     this.stateMachine = stateMachineFactory.make(this);
     this.stateMachine = stateMachineFactory.make(this);
     this.nodeId = nodeId;
     this.nodeId = nodeId;
     this.container = container;
     this.container = container;
-    this.allocatedSchedulerKey = SchedulerRequestKey.extractFrom(container);
+    this.allocatedSchedulerKey = schedulerKey;
     this.appAttemptId = appAttemptId;
     this.appAttemptId = appAttemptId;
     this.user = user;
     this.user = user;
     this.creationTime = creationTime;
     this.creationTime = creationTime;
@@ -276,6 +280,10 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
     return this.container;
     return this.container;
   }
   }
 
 
+  public void setContainer(Container container) {
+    this.container = container;
+  }
+
   @Override
   @Override
   public RMContainerState getState() {
   public RMContainerState getState() {
     this.readLock.lock();
     this.readLock.lock();

+ 101 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.UpdateContainerError;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
@@ -62,6 +63,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 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.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
@@ -81,6 +83,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdate
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
+
+
+import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
+import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.Lock;
 import org.apache.hadoop.yarn.server.utils.Lock;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.SystemClock;
@@ -509,9 +516,11 @@ public abstract class AbstractYarnScheduler
     ApplicationAttemptId attemptId =
     ApplicationAttemptId attemptId =
         container.getId().getApplicationAttemptId();
         container.getId().getApplicationAttemptId();
     RMContainer rmContainer =
     RMContainer rmContainer =
-        new RMContainerImpl(container, attemptId, node.getNodeID(),
-          applications.get(attemptId.getApplicationId()).getUser(), rmContext,
-          status.getCreationTime(), status.getNodeLabelExpression());
+        new RMContainerImpl(container,
+            SchedulerRequestKey.extractFrom(container), attemptId,
+            node.getNodeID(), applications.get(
+            attemptId.getApplicationId()).getUser(), rmContext,
+            status.getCreationTime(), status.getNodeLabelExpression());
     return rmContainer;
     return rmContainer;
   }
   }
 
 
@@ -1064,4 +1073,93 @@ public abstract class AbstractYarnScheduler
       ask.setCapability(getNormalizedResource(ask.getCapability()));
       ask.setCapability(getNormalizedResource(ask.getCapability()));
     }
     }
   }
   }
+
+  protected void handleExecutionTypeUpdates(
+      SchedulerApplicationAttempt appAttempt,
+      List<UpdateContainerRequest> promotionRequests,
+      List<UpdateContainerRequest> demotionRequests) {
+    if (promotionRequests != null && !promotionRequests.isEmpty()) {
+      LOG.info("Promotion Update requests : " + promotionRequests);
+      handlePromotionRequests(appAttempt, promotionRequests);
+    }
+    if (demotionRequests != null && !demotionRequests.isEmpty()) {
+      LOG.info("Demotion Update requests : " + demotionRequests);
+      handleDemotionRequests(appAttempt, demotionRequests);
+    }
+  }
+
+  private void handlePromotionRequests(
+      SchedulerApplicationAttempt applicationAttempt,
+      List<UpdateContainerRequest> updateContainerRequests) {
+    for (UpdateContainerRequest uReq : updateContainerRequests) {
+      RMContainer rmContainer =
+          rmContext.getScheduler().getRMContainer(uReq.getContainerId());
+      // Check if this is a container update
+      // And not in the middle of a Demotion
+      if (rmContainer != null) {
+        // Check if this is an executionType change request
+        // If so, fix the rr to make it look like a normal rr
+        // with relaxLocality=false and numContainers=1
+        SchedulerNode schedulerNode = rmContext.getScheduler()
+            .getSchedulerNode(rmContainer.getContainer().getNodeId());
+
+        // Add only if no outstanding promote requests exist.
+        if (!applicationAttempt.getUpdateContext()
+            .checkAndAddToOutstandingIncreases(
+                rmContainer, schedulerNode, uReq)) {
+          applicationAttempt.addToUpdateContainerErrors(
+              UpdateContainerError.newInstance(
+              RMServerUtils.UPDATE_OUTSTANDING_ERROR, uReq));
+        }
+      } else {
+        LOG.warn("Cannot promote non-existent (or completed) Container ["
+            + uReq.getContainerId() + "]");
+      }
+    }
+  }
+
+  private void handleDemotionRequests(SchedulerApplicationAttempt appAttempt,
+      List<UpdateContainerRequest> demotionRequests) {
+    OpportunisticContainerContext oppCntxt =
+        appAttempt.getOpportunisticContainerContext();
+    for (UpdateContainerRequest uReq : demotionRequests) {
+      RMContainer rmContainer =
+          rmContext.getScheduler().getRMContainer(uReq.getContainerId());
+      if (rmContainer != null) {
+        if (appAttempt.getUpdateContext().checkAndAddToOutstandingDecreases(
+            rmContainer.getContainer())) {
+          RMContainer demotedRMContainer =
+              createDemotedRMContainer(appAttempt, oppCntxt, rmContainer);
+          appAttempt.addToNewlyDemotedContainers(
+              uReq.getContainerId(), demotedRMContainer);
+        } else {
+          appAttempt.addToUpdateContainerErrors(
+              UpdateContainerError.newInstance(
+              RMServerUtils.UPDATE_OUTSTANDING_ERROR, uReq));
+        }
+      } else {
+        LOG.warn("Cannot demote non-existent (or completed) Container ["
+            + uReq.getContainerId() + "]");
+      }
+    }
+  }
+
+  private RMContainer createDemotedRMContainer(
+      SchedulerApplicationAttempt appAttempt,
+      OpportunisticContainerContext oppCntxt,
+      RMContainer rmContainer) {
+    SchedulerRequestKey sk =
+        SchedulerRequestKey.extractFrom(rmContainer.getContainer());
+    Container demotedContainer = BuilderUtils.newContainer(
+        ContainerId.newContainerId(appAttempt.getApplicationAttemptId(),
+            oppCntxt.getContainerIdGenerator().generateContainerId()),
+        rmContainer.getContainer().getNodeId(),
+        rmContainer.getContainer().getNodeHttpAddress(),
+        rmContainer.getContainer().getResource(),
+        sk.getPriority(), null, ExecutionType.OPPORTUNISTIC,
+        sk.getAllocationRequestId());
+    demotedContainer.setVersion(rmContainer.getContainer().getVersion());
+    return SchedulerUtils.createOpportunisticRmContainer(
+        rmContext, demotedContainer, false);
+  }
 }
 }

+ 24 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Allocation.java

@@ -37,6 +37,8 @@ public class Allocation {
   final List<NMToken> nmTokens;
   final List<NMToken> nmTokens;
   final List<Container> increasedContainers;
   final List<Container> increasedContainers;
   final List<Container> decreasedContainers;
   final List<Container> decreasedContainers;
+  final List<Container> promotedContainers;
+  final List<Container> demotedContainers;
   private Resource resourceLimit;
   private Resource resourceLimit;
 
 
 
 
@@ -51,13 +53,23 @@ public class Allocation {
       Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
       Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
       List<ResourceRequest> fungibleResources, List<NMToken> nmTokens) {
       List<ResourceRequest> fungibleResources, List<NMToken> nmTokens) {
     this(containers,  resourceLimit,strictContainers,  fungibleContainers,
     this(containers,  resourceLimit,strictContainers,  fungibleContainers,
-      fungibleResources, nmTokens, null, null);
+      fungibleResources, nmTokens, null, null, null, null);
   }
   }
-  
+
   public Allocation(List<Container> containers, Resource resourceLimit,
   public Allocation(List<Container> containers, Resource resourceLimit,
       Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
       Set<ContainerId> strictContainers, Set<ContainerId> fungibleContainers,
       List<ResourceRequest> fungibleResources, List<NMToken> nmTokens,
       List<ResourceRequest> fungibleResources, List<NMToken> nmTokens,
       List<Container> increasedContainers, List<Container> decreasedContainer) {
       List<Container> increasedContainers, List<Container> decreasedContainer) {
+    this(containers,  resourceLimit,strictContainers,  fungibleContainers,
+        fungibleResources, nmTokens, increasedContainers, decreasedContainer,
+        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,
+      List<Container> promotedContainers, List<Container> demotedContainer) {
     this.containers = containers;
     this.containers = containers;
     this.resourceLimit = resourceLimit;
     this.resourceLimit = resourceLimit;
     this.strictContainers = strictContainers;
     this.strictContainers = strictContainers;
@@ -66,6 +78,8 @@ public class Allocation {
     this.nmTokens = nmTokens;
     this.nmTokens = nmTokens;
     this.increasedContainers = increasedContainers;
     this.increasedContainers = increasedContainers;
     this.decreasedContainers = decreasedContainer;
     this.decreasedContainers = decreasedContainer;
+    this.promotedContainers = promotedContainers;
+    this.demotedContainers = demotedContainer;
   }
   }
 
 
   public List<Container> getContainers() {
   public List<Container> getContainers() {
@@ -100,6 +114,14 @@ public class Allocation {
     return decreasedContainers;
     return decreasedContainers;
   }
   }
 
 
+  public List<Container> getPromotedContainers() {
+    return promotedContainers;
+  }
+
+  public List<Container> getDemotedContainers() {
+    return demotedContainers;
+  }
+
   @VisibleForTesting
   @VisibleForTesting
   public void setResourceLimit(Resource resource) {
   public void setResourceLimit(Resource resource) {
     this.resourceLimit = resource;
     this.resourceLimit = resource;

+ 43 - 27
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java

@@ -55,7 +55,6 @@ import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 /**
 /**
  * This class keeps track of all the consumption of an application. This also
  * This class keeps track of all the consumption of an application. This also
  * keeps track of current running/completed containers for the application.
  * keeps track of current running/completed containers for the application.
@@ -92,10 +91,11 @@ public class AppSchedulingInfo {
   final Map<NodeId, Map<SchedulerRequestKey, Map<ContainerId,
   final Map<NodeId, Map<SchedulerRequestKey, Map<ContainerId,
       SchedContainerChangeRequest>>> containerIncreaseRequestMap =
       SchedContainerChangeRequest>>> containerIncreaseRequestMap =
       new ConcurrentHashMap<>();
       new ConcurrentHashMap<>();
-
   private final ReentrantReadWriteLock.ReadLock readLock;
   private final ReentrantReadWriteLock.ReadLock readLock;
   private final ReentrantReadWriteLock.WriteLock writeLock;
   private final ReentrantReadWriteLock.WriteLock writeLock;
 
 
+  public final ContainerUpdateContext updateContext;
+
   public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
   public AppSchedulingInfo(ApplicationAttemptId appAttemptId,
       String user, Queue queue, ActiveUsersManager activeUsersManager,
       String user, Queue queue, ActiveUsersManager activeUsersManager,
       long epoch, ResourceUsage appResourceUsage) {
       long epoch, ResourceUsage appResourceUsage) {
@@ -109,6 +109,7 @@ public class AppSchedulingInfo {
     this.appResourceUsage = appResourceUsage;
     this.appResourceUsage = appResourceUsage;
 
 
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    updateContext = new ContainerUpdateContext(this);
     readLock = lock.readLock();
     readLock = lock.readLock();
     writeLock = lock.writeLock();
     writeLock = lock.writeLock();
   }
   }
@@ -376,6 +377,10 @@ public class AppSchedulingInfo {
     }
     }
   }
   }
 
 
+  public ContainerUpdateContext getUpdateContext() {
+    return updateContext;
+  }
+
   /**
   /**
    * The ApplicationMaster is updating resource requirements for the
    * The ApplicationMaster is updating resource requirements for the
    * application, by asking for more resources and releasing resources acquired
    * application, by asking for more resources and releasing resources acquired
@@ -414,29 +419,9 @@ public class AppSchedulingInfo {
       }
       }
 
 
       // Update scheduling placement set
       // Update scheduling placement set
-      for (Map.Entry<SchedulerRequestKey, Map<String, ResourceRequest>> entry : dedupRequests.entrySet()) {
-        SchedulerRequestKey schedulerRequestKey = entry.getKey();
-
-        if (!schedulerKeyToPlacementSets.containsKey(schedulerRequestKey)) {
-          schedulerKeyToPlacementSets.put(schedulerRequestKey,
-              new LocalitySchedulingPlacementSet<>(this));
-        }
-
-        // Update placement set
-        ResourceRequestUpdateResult pendingAmountChanges =
-            schedulerKeyToPlacementSets.get(schedulerRequestKey)
-                .updateResourceRequests(
-                    entry.getValue().values(),
-                    recoverPreemptedRequestForAContainer);
-
-        if (null != pendingAmountChanges) {
-          updatePendingResources(
-              pendingAmountChanges.getLastAnyResourceRequest(),
-              pendingAmountChanges.getNewResourceRequest(), schedulerRequestKey,
-              queue.getMetrics());
-          offswitchResourcesUpdated = true;
-        }
-      }
+      offswitchResourcesUpdated =
+          addToPlacementSets(
+              recoverPreemptedRequestForAContainer, dedupRequests);
 
 
       return offswitchResourcesUpdated;
       return offswitchResourcesUpdated;
     } finally {
     } finally {
@@ -444,6 +429,37 @@ public class AppSchedulingInfo {
     }
     }
   }
   }
 
 
+  boolean addToPlacementSets(
+      boolean recoverPreemptedRequestForAContainer,
+      Map<SchedulerRequestKey, Map<String, ResourceRequest>> dedupRequests) {
+    boolean offswitchResourcesUpdated = false;
+    for (Map.Entry<SchedulerRequestKey, Map<String, ResourceRequest>> entry :
+        dedupRequests.entrySet()) {
+      SchedulerRequestKey schedulerRequestKey = entry.getKey();
+
+      if (!schedulerKeyToPlacementSets.containsKey(schedulerRequestKey)) {
+        schedulerKeyToPlacementSets.put(schedulerRequestKey,
+            new LocalitySchedulingPlacementSet<>(this));
+      }
+
+      // Update placement set
+      ResourceRequestUpdateResult pendingAmountChanges =
+          schedulerKeyToPlacementSets.get(schedulerRequestKey)
+              .updateResourceRequests(
+                  entry.getValue().values(),
+                  recoverPreemptedRequestForAContainer);
+
+      if (null != pendingAmountChanges) {
+        updatePendingResources(
+            pendingAmountChanges.getLastAnyResourceRequest(),
+            pendingAmountChanges.getNewResourceRequest(), schedulerRequestKey,
+            queue.getMetrics());
+        offswitchResourcesUpdated = true;
+      }
+    }
+    return offswitchResourcesUpdated;
+  }
+
   private void updatePendingResources(ResourceRequest lastRequest,
   private void updatePendingResources(ResourceRequest lastRequest,
       ResourceRequest request, SchedulerRequestKey schedulerKey,
       ResourceRequest request, SchedulerRequestKey schedulerKey,
       QueueMetrics metrics) {
       QueueMetrics metrics) {
@@ -717,8 +733,8 @@ public class AppSchedulingInfo {
         updateMetricsForAllocatedContainer(type, containerAllocated);
         updateMetricsForAllocatedContainer(type, containerAllocated);
       }
       }
 
 
-      return schedulerKeyToPlacementSets.get(schedulerKey).allocate(type, node,
-          request);
+      return schedulerKeyToPlacementSets.get(schedulerKey)
+          .allocate(schedulerKey, type, node, request);
     } finally {
     } finally {
       writeLock.unlock();
       writeLock.unlock();
     }
     }

+ 265 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerUpdateContext.java

@@ -0,0 +1,265 @@
+/**
+ * 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.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.ContainerUpdateType;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Class encapsulates all outstanding container increase and decrease
+ * requests for an application.
+ */
+public class ContainerUpdateContext {
+
+  public static final ContainerId UNDEFINED =
+      ContainerId.newContainerId(ApplicationAttemptId.newInstance(
+              ApplicationId.newInstance(-1, -1), -1), -1);
+  protected static final RecordFactory RECORD_FACTORY =
+      RecordFactoryProvider.getRecordFactory(null);
+
+  // Keep track of containers that are undergoing promotion
+  private final Map<SchedulerRequestKey, Map<Resource,
+      Map<NodeId, Set<ContainerId>>>> outstandingIncreases = new HashMap<>();
+
+  private final Set<ContainerId> outstandingDecreases = new HashSet<>();
+  private final AppSchedulingInfo appSchedulingInfo;
+
+  ContainerUpdateContext(AppSchedulingInfo appSchedulingInfo) {
+    this.appSchedulingInfo = appSchedulingInfo;
+  }
+
+  private synchronized boolean isBeingIncreased(Container container) {
+    Map<Resource, Map<NodeId, Set<ContainerId>>> resourceMap =
+        outstandingIncreases.get(
+            new SchedulerRequestKey(container.getPriority(),
+                container.getAllocationRequestId(), container.getId()));
+    if (resourceMap != null) {
+      Map<NodeId, Set<ContainerId>> locationMap =
+          resourceMap.get(container.getResource());
+      if (locationMap != null) {
+        Set<ContainerId> containerIds = locationMap.get(container.getNodeId());
+        if (containerIds != null && !containerIds.isEmpty()) {
+          return containerIds.contains(container.getId());
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Add the container to outstanding decreases.
+   * @param container Container.
+   * @return true if updated to outstanding decreases was successful.
+   */
+  public synchronized boolean checkAndAddToOutstandingDecreases(
+      Container container) {
+    if (isBeingIncreased(container)
+        || outstandingDecreases.contains(container.getId())) {
+      return false;
+    }
+    outstandingDecreases.add(container.getId());
+    return true;
+  }
+
+  /**
+   * Add the container to outstanding increases.
+   * @param rmContainer RMContainer.
+   * @param schedulerNode SchedulerNode.
+   * @param updateRequest UpdateContainerRequest.
+   * @return true if updated to outstanding increases was successful.
+   */
+  public synchronized boolean checkAndAddToOutstandingIncreases(
+      RMContainer rmContainer, SchedulerNode schedulerNode,
+      UpdateContainerRequest updateRequest) {
+    Container container = rmContainer.getContainer();
+    SchedulerRequestKey schedulerKey =
+        SchedulerRequestKey.create(updateRequest,
+            rmContainer.getAllocatedSchedulerKey());
+    Map<Resource, Map<NodeId, Set<ContainerId>>> resourceMap =
+        outstandingIncreases.get(schedulerKey);
+    if (resourceMap == null) {
+      resourceMap = new HashMap<>();
+      outstandingIncreases.put(schedulerKey, resourceMap);
+    }
+    Map<NodeId, Set<ContainerId>> locationMap =
+        resourceMap.get(container.getResource());
+    if (locationMap == null) {
+      locationMap = new HashMap<>();
+      resourceMap.put(container.getResource(), locationMap);
+    }
+    Set<ContainerId> containerIds = locationMap.get(container.getNodeId());
+    if (containerIds == null) {
+      containerIds = new HashSet<>();
+      locationMap.put(container.getNodeId(), containerIds);
+    }
+    if (containerIds.contains(container.getId())
+        || outstandingDecreases.contains(container.getId())) {
+      return false;
+    }
+    containerIds.add(container.getId());
+
+    Map<SchedulerRequestKey, Map<String, ResourceRequest>> updateResReqs =
+        new HashMap<>();
+    Resource resToIncrease = getResourceToIncrease(updateRequest, rmContainer);
+    Map<String, ResourceRequest> resMap =
+        createResourceRequests(rmContainer, schedulerNode,
+            schedulerKey, resToIncrease);
+    updateResReqs.put(schedulerKey, resMap);
+    appSchedulingInfo.addToPlacementSets(false, updateResReqs);
+    return true;
+  }
+
+  private Map<String, ResourceRequest> createResourceRequests(
+      RMContainer rmContainer, SchedulerNode schedulerNode,
+      SchedulerRequestKey schedulerKey, Resource resToIncrease) {
+    Map<String, ResourceRequest> resMap = new HashMap<>();
+    resMap.put(rmContainer.getContainer().getNodeId().getHost(),
+        createResourceReqForIncrease(schedulerKey, resToIncrease,
+            RECORD_FACTORY.newRecordInstance(ResourceRequest.class),
+            rmContainer, rmContainer.getContainer().getNodeId().getHost()));
+    resMap.put(schedulerNode.getRackName(),
+        createResourceReqForIncrease(schedulerKey, resToIncrease,
+            RECORD_FACTORY.newRecordInstance(ResourceRequest.class),
+            rmContainer, schedulerNode.getRackName()));
+    resMap.put(ResourceRequest.ANY,
+        createResourceReqForIncrease(schedulerKey, resToIncrease,
+            RECORD_FACTORY.newRecordInstance(ResourceRequest.class),
+            rmContainer, ResourceRequest.ANY));
+    return resMap;
+  }
+
+  private Resource getResourceToIncrease(UpdateContainerRequest updateReq,
+      RMContainer rmContainer) {
+    if (updateReq.getContainerUpdateType() ==
+        ContainerUpdateType.PROMOTE_EXECUTION_TYPE) {
+      return rmContainer.getContainer().getResource();
+    }
+    // TODO: Fix this for container increase..
+    //       This has to equal the Resources in excess of fitsIn()
+    //       for container increase and is equal to the container total
+    //       resource for Promotion.
+    return null;
+  }
+
+  private static ResourceRequest createResourceReqForIncrease(
+      SchedulerRequestKey schedulerRequestKey, Resource resToIncrease,
+      ResourceRequest rr, RMContainer rmContainer, String resourceName) {
+    rr.setResourceName(resourceName);
+    rr.setNumContainers(1);
+    rr.setRelaxLocality(false);
+    rr.setPriority(rmContainer.getContainer().getPriority());
+    rr.setAllocationRequestId(schedulerRequestKey.getAllocationRequestId());
+    rr.setCapability(resToIncrease);
+    rr.setNodeLabelExpression(rmContainer.getNodeLabelExpression());
+    rr.setExecutionTypeRequest(ExecutionTypeRequest.newInstance(
+        ExecutionType.GUARANTEED, true));
+    return rr;
+  }
+
+  /**
+   * Remove Container from outstanding increases / decreases. Calling this
+   * method essentially completes the update process.
+   * @param schedulerKey SchedulerRequestKey.
+   * @param container Container.
+   */
+  public synchronized void removeFromOutstandingUpdate(
+      SchedulerRequestKey schedulerKey, Container container) {
+    Map<Resource, Map<NodeId, Set<ContainerId>>> resourceMap =
+        outstandingIncreases.get(schedulerKey);
+    if (resourceMap != null) {
+      Map<NodeId, Set<ContainerId>> locationMap =
+          resourceMap.get(container.getResource());
+      if (locationMap != null) {
+        Set<ContainerId> containerIds = locationMap.get(container.getNodeId());
+        if (containerIds != null && !containerIds.isEmpty()) {
+          containerIds.remove(container.getId());
+          if (containerIds.isEmpty()) {
+            locationMap.remove(container.getNodeId());
+          }
+        }
+        if (locationMap.isEmpty()) {
+          resourceMap.remove(container.getResource());
+        }
+      }
+      if (resourceMap.isEmpty()) {
+        outstandingIncreases.remove(schedulerKey);
+      }
+    }
+    outstandingDecreases.remove(container.getId());
+  }
+
+  /**
+   * Check if a new container is to be matched up against an outstanding
+   * Container increase request.
+   * @param node SchedulerNode.
+   * @param schedulerKey SchedulerRequestKey.
+   * @param rmContainer RMContainer.
+   * @return ContainerId.
+   */
+  public ContainerId matchContainerToOutstandingIncreaseReq(
+      SchedulerNode node, SchedulerRequestKey schedulerKey,
+      RMContainer rmContainer) {
+    ContainerId retVal = null;
+    Container container = rmContainer.getContainer();
+    Map<Resource, Map<NodeId, Set<ContainerId>>> resourceMap =
+        outstandingIncreases.get(schedulerKey);
+    if (resourceMap != null) {
+      Map<NodeId, Set<ContainerId>> locationMap =
+          resourceMap.get(container.getResource());
+      if (locationMap != null) {
+        Set<ContainerId> containerIds = locationMap.get(container.getNodeId());
+        if (containerIds != null && !containerIds.isEmpty()) {
+          retVal = containerIds.iterator().next();
+        }
+      }
+    }
+    // Allocation happened on NM on the same host, but not on the NM
+    // we need.. We need to signal that this container has to be released.
+    // We also need to add these requests back.. to be reallocated.
+    if (resourceMap != null && retVal == null) {
+      Map<SchedulerRequestKey, Map<String, ResourceRequest>> reqsToUpdate =
+          new HashMap<>();
+      Map<String, ResourceRequest> resMap = createResourceRequests
+          (rmContainer, node, schedulerKey,
+          rmContainer.getContainer().getResource());
+      reqsToUpdate.put(schedulerKey, resMap);
+      appSchedulingInfo.addToPlacementSets(true, reqsToUpdate);
+      return UNDEFINED;
+    }
+    return retVal;
+  }
+}

+ 68 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ContainerUpdates.java

@@ -0,0 +1,68 @@
+/**
+ * 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.UpdateContainerRequest;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Holder class that maintains list of container update requests
+ */
+public class ContainerUpdates {
+
+  final List<UpdateContainerRequest> increaseRequests = new ArrayList<>();
+  final List<UpdateContainerRequest> decreaseRequests = new ArrayList<>();
+  final List<UpdateContainerRequest> promotionRequests = new ArrayList<>();
+  final List<UpdateContainerRequest> demotionRequests = new ArrayList<>();
+
+  /**
+   * Returns Container Increase Requests.
+   * @return Container Increase Requests.
+   */
+  public List<UpdateContainerRequest> getIncreaseRequests() {
+    return increaseRequests;
+  }
+
+  /**
+   * Returns Container Decrease Requests.
+   * @return Container Decrease Requests.
+   */
+  public List<UpdateContainerRequest> getDecreaseRequests() {
+    return decreaseRequests;
+  }
+
+  /**
+   * Returns Container Promotion Requests.
+   * @return Container Promotion Requests.
+   */
+  public List<UpdateContainerRequest> getPromotionRequests() {
+    return promotionRequests;
+  }
+
+  /**
+   * Returns Container Demotion Requests.
+   * @return Container Demotion Requests.
+   */
+  public List<UpdateContainerRequest> getDemotionRequests() {
+    return demotionRequests;
+  }
+
+}

+ 160 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashMap;
@@ -47,6 +48,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NMToken;
@@ -54,6 +56,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.UpdateContainerError;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.server.api.ContainerType;
 import org.apache.hadoop.yarn.server.api.ContainerType;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -133,10 +136,15 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   private AtomicLong firstContainerAllocatedTime = new AtomicLong(0);
   private AtomicLong firstContainerAllocatedTime = new AtomicLong(0);
 
 
   protected List<RMContainer> newlyAllocatedContainers = new ArrayList<>();
   protected List<RMContainer> newlyAllocatedContainers = new ArrayList<>();
+  protected Map<ContainerId, RMContainer> newlyPromotedContainers = new HashMap<>();
+  protected Map<ContainerId, RMContainer> newlyDemotedContainers = new HashMap<>();
+  protected List<RMContainer> tempContainerToKill = new ArrayList<>();
   protected Map<ContainerId, RMContainer> newlyDecreasedContainers = new HashMap<>();
   protected Map<ContainerId, RMContainer> newlyDecreasedContainers = new HashMap<>();
   protected Map<ContainerId, RMContainer> newlyIncreasedContainers = new HashMap<>();
   protected Map<ContainerId, RMContainer> newlyIncreasedContainers = new HashMap<>();
   protected Set<NMToken> updatedNMTokens = new HashSet<>();
   protected Set<NMToken> updatedNMTokens = new HashSet<>();
 
 
+  protected List<UpdateContainerError> updateContainerErrors = new ArrayList<>();
+
   // This pendingRelease is used in work-preserving recovery scenario to keep
   // This pendingRelease is used in work-preserving recovery scenario to keep
   // track of the AM's outstanding release requests. RM on recovery could
   // track of the AM's outstanding release requests. RM on recovery could
   // receive the release request form AM before it receives the container status
   // receive the release request form AM before it receives the container status
@@ -247,6 +255,10 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return this.appSchedulingInfo;
     return this.appSchedulingInfo;
   }
   }
 
 
+  public ContainerUpdateContext getUpdateContext() {
+    return this.appSchedulingInfo.getUpdateContext();
+  }
+
   /**
   /**
    * Is this application pending?
    * Is this application pending?
    * @return true if it is else false.
    * @return true if it is else false.
@@ -537,8 +549,9 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
       writeLock.lock();
       writeLock.lock();
       // Create RMContainer if necessary
       // Create RMContainer if necessary
       if (rmContainer == null) {
       if (rmContainer == null) {
-        rmContainer = new RMContainerImpl(container, getApplicationAttemptId(),
-            node.getNodeID(), appSchedulingInfo.getUser(), rmContext);
+        rmContainer = new RMContainerImpl(container, schedulerKey,
+            getApplicationAttemptId(), node.getNodeID(),
+            appSchedulingInfo.getUser(), rmContext);
       }
       }
       if (rmContainer.getState() == RMContainerState.NEW) {
       if (rmContainer.getState() == RMContainerState.NEW) {
         attemptResourceUsage.incReserved(node.getPartition(),
         attemptResourceUsage.incReserved(node.getPartition(),
@@ -635,10 +648,10 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   }
   }
   
   
   private Container updateContainerAndNMToken(RMContainer rmContainer,
   private Container updateContainerAndNMToken(RMContainer rmContainer,
-      boolean newContainer, boolean increasedContainer) {
+      ContainerUpdateType updateType) {
     Container container = rmContainer.getContainer();
     Container container = rmContainer.getContainer();
     ContainerType containerType = ContainerType.TASK;
     ContainerType containerType = ContainerType.TASK;
-    if (!newContainer) {
+    if (updateType != null) {
       container.setVersion(container.getVersion() + 1);
       container.setVersion(container.getVersion() + 1);
     }
     }
     // The working knowledge is that masterContainer for AM is null as it
     // The working knowledge is that masterContainer for AM is null as it
@@ -662,12 +675,15 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
       return null;
       return null;
     }
     }
     
     
-    if (newContainer) {
+    if (updateType == null ||
+        ContainerUpdateType.PROMOTE_EXECUTION_TYPE == updateType ||
+        ContainerUpdateType.DEMOTE_EXECUTION_TYPE == updateType) {
       rmContainer.handle(new RMContainerEvent(
       rmContainer.handle(new RMContainerEvent(
           rmContainer.getContainerId(), RMContainerEventType.ACQUIRED));
           rmContainer.getContainerId(), RMContainerEventType.ACQUIRED));
     } else {
     } else {
       rmContainer.handle(new RMContainerUpdatesAcquiredEvent(
       rmContainer.handle(new RMContainerUpdatesAcquiredEvent(
-          rmContainer.getContainerId(), increasedContainer));
+          rmContainer.getContainerId(),
+          ContainerUpdateType.INCREASE_RESOURCE == updateType));
     }
     }
     return container;
     return container;
   }
   }
@@ -699,8 +715,8 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
       Iterator<RMContainer> i = newlyAllocatedContainers.iterator();
       Iterator<RMContainer> i = newlyAllocatedContainers.iterator();
       while (i.hasNext()) {
       while (i.hasNext()) {
         RMContainer rmContainer = i.next();
         RMContainer rmContainer = i.next();
-        Container updatedContainer = updateContainerAndNMToken(rmContainer,
-            true, false);
+        Container updatedContainer =
+            updateContainerAndNMToken(rmContainer, null);
         // Only add container to return list when it's not null.
         // Only add container to return list when it's not null.
         // updatedContainer could be null when generate token failed, it can be
         // updatedContainer could be null when generate token failed, it can be
         // caused by DNS resolving failed.
         // caused by DNS resolving failed.
@@ -713,9 +729,142 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     } finally {
     } finally {
       writeLock.unlock();
       writeLock.unlock();
     }
     }
+  }
 
 
+  public void addToNewlyDemotedContainers(ContainerId containerId,
+      RMContainer rmContainer) {
+    newlyDemotedContainers.put(containerId, rmContainer);
   }
   }
-  
+
+  protected synchronized void addToUpdateContainerErrors(
+      UpdateContainerError error) {
+    updateContainerErrors.add(error);
+  }
+
+  protected synchronized void addToNewlyAllocatedContainers(
+      SchedulerNode node, RMContainer rmContainer) {
+    if (oppContainerContext == null) {
+      newlyAllocatedContainers.add(rmContainer);
+      return;
+    }
+    ContainerId matchedContainerId =
+        getUpdateContext().matchContainerToOutstandingIncreaseReq(
+            node, rmContainer.getAllocatedSchedulerKey(), rmContainer);
+    if (matchedContainerId != null) {
+      if (ContainerUpdateContext.UNDEFINED == matchedContainerId) {
+        // This is a spurious allocation (relaxLocality = false
+        // resulted in the Container being allocated on an NM on the same host
+        // but not on the NM running the container to be updated. Can
+        // happen if more than one NM exists on the same host.. usually
+        // occurs when using MiniYARNCluster to test).
+        tempContainerToKill.add(rmContainer);
+      } else {
+        newlyPromotedContainers.put(matchedContainerId, rmContainer);
+      }
+    } else {
+      newlyAllocatedContainers.add(rmContainer);
+    }
+  }
+
+  public List<Container> pullNewlyPromotedContainers() {
+    return pullContainersWithUpdatedExecType(newlyPromotedContainers,
+        ContainerUpdateType.PROMOTE_EXECUTION_TYPE);
+  }
+
+  public List<Container> pullNewlyDemotedContainers() {
+    return pullContainersWithUpdatedExecType(newlyDemotedContainers,
+        ContainerUpdateType.DEMOTE_EXECUTION_TYPE);
+  }
+
+  public List<UpdateContainerError> pullUpdateContainerErrors() {
+    List<UpdateContainerError> errors =
+        new ArrayList<>(updateContainerErrors);
+    updateContainerErrors.clear();
+    return errors;
+  }
+
+  /**
+   * A container is promoted if its executionType is changed from
+   * OPPORTUNISTIC to GUARANTEED. It id demoted if the change is from
+   * GUARANTEED to OPPORTUNISTIC.
+   * @return Newly Promoted and Demoted containers
+   */
+  private List<Container> pullContainersWithUpdatedExecType(
+      Map<ContainerId, RMContainer> newlyUpdatedContainers,
+      ContainerUpdateType updateTpe) {
+    List<Container> updatedContainers = new ArrayList<>();
+    if (oppContainerContext == null) {
+      return updatedContainers;
+    }
+    try {
+      writeLock.lock();
+      Iterator<Map.Entry<ContainerId, RMContainer>> i =
+          newlyUpdatedContainers.entrySet().iterator();
+      while (i.hasNext()) {
+        Map.Entry<ContainerId, RMContainer> entry = i.next();
+        ContainerId matchedContainerId = entry.getKey();
+        RMContainer rmContainer = entry.getValue();
+
+        // swap containers
+        RMContainer existingRMContainer = swapContainer(
+            rmContainer, matchedContainerId);
+        getUpdateContext().removeFromOutstandingUpdate(
+            rmContainer.getAllocatedSchedulerKey(),
+            existingRMContainer.getContainer());
+        Container updatedContainer = updateContainerAndNMToken(
+            existingRMContainer, updateTpe);
+        updatedContainers.add(updatedContainer);
+
+        tempContainerToKill.add(rmContainer);
+        i.remove();
+      }
+      // Release all temporary containers
+      Iterator<RMContainer> tempIter = tempContainerToKill.iterator();
+      while (tempIter.hasNext()) {
+        RMContainer c = tempIter.next();
+        // Mark container for release (set RRs to null, so RM does not think
+        // it is a recoverable container)
+        ((RMContainerImpl) c).setResourceRequests(null);
+        ((AbstractYarnScheduler) rmContext.getScheduler()).completedContainer(c,
+            SchedulerUtils.createAbnormalContainerStatus(c.getContainerId(),
+                SchedulerUtils.UPDATED_CONTAINER),
+            RMContainerEventType.KILL);
+        tempIter.remove();
+      }
+      return updatedContainers;
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private RMContainer swapContainer(RMContainer rmContainer, ContainerId
+      matchedContainerId) {
+    RMContainer existingRMContainer =
+        getRMContainer(matchedContainerId);
+    if (existingRMContainer != null) {
+      // Swap updated container with the existing container
+      Container updatedContainer = rmContainer.getContainer();
+
+      Container newContainer = Container.newInstance(matchedContainerId,
+          existingRMContainer.getContainer().getNodeId(),
+          existingRMContainer.getContainer().getNodeHttpAddress(),
+          updatedContainer.getResource(),
+          existingRMContainer.getContainer().getPriority(), null,
+          updatedContainer.getExecutionType());
+      newContainer.setAllocationRequestId(
+          existingRMContainer.getContainer().getAllocationRequestId());
+      newContainer.setVersion(existingRMContainer.getContainer().getVersion());
+
+      rmContainer.getContainer().setResource(
+          existingRMContainer.getContainer().getResource());
+      rmContainer.getContainer().setExecutionType(
+          existingRMContainer.getContainer().getExecutionType());
+
+      ((RMContainerImpl)existingRMContainer).setContainer(newContainer);
+    }
+    return existingRMContainer;
+  }
+
   private List<Container> pullNewlyUpdatedContainers(
   private List<Container> pullNewlyUpdatedContainers(
       Map<ContainerId, RMContainer> updatedContainerMap, boolean increase) {
       Map<ContainerId, RMContainer> updatedContainerMap, boolean increase) {
     try {
     try {
@@ -728,7 +877,8 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
       while (i.hasNext()) {
       while (i.hasNext()) {
         RMContainer rmContainer = i.next().getValue();
         RMContainer rmContainer = i.next().getValue();
         Container updatedContainer = updateContainerAndNMToken(rmContainer,
         Container updatedContainer = updateContainerAndNMToken(rmContainer,
-            false, increase);
+            increase ? ContainerUpdateType.INCREASE_RESOURCE :
+                ContainerUpdateType.DECREASE_RESOURCE);
         if (updatedContainer != null) {
         if (updatedContainer != null) {
           returnContainerList.add(updatedContainer);
           returnContainerList.add(updatedContainer);
           i.remove();
           i.remove();

+ 22 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java

@@ -25,6 +25,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -41,7 +42,10 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 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.RMContainerImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 
@@ -57,6 +61,9 @@ public class SchedulerUtils {
 
 
   public static final String RELEASED_CONTAINER = 
   public static final String RELEASED_CONTAINER = 
       "Container released by application";
       "Container released by application";
+
+  public static final String UPDATED_CONTAINER =
+      "Temporary container killed by application for ExeutionType update";
   
   
   public static final String LOST_CONTAINER = 
   public static final String LOST_CONTAINER = 
       "Container released on a *lost* node";
       "Container released on a *lost* node";
@@ -380,4 +387,19 @@ public class SchedulerUtils {
     }
     }
     return hasPendingResourceRequest(rc, usage, partitionToLookAt, cluster);
     return hasPendingResourceRequest(rc, usage, partitionToLookAt, cluster);
   }
   }
+
+  public static RMContainer createOpportunisticRmContainer(RMContext rmContext,
+      Container container, boolean isRemotelyAllocated) {
+    SchedulerApplicationAttempt appAttempt =
+        ((AbstractYarnScheduler) rmContext.getScheduler())
+            .getCurrentAttemptForContainer(container.getId());
+    RMContainer rmContainer = new RMContainerImpl(container,
+        SchedulerRequestKey.extractFrom(container),
+        appAttempt.getApplicationAttemptId(), container.getNodeId(),
+        appAttempt.getUser(), rmContext, isRemotelyAllocated);
+    appAttempt.addRMContainer(container.getId(), rmContainer);
+    ((AbstractYarnScheduler) rmContext.getScheduler()).getNode(
+        container.getNodeId()).allocateContainer(rmContainer);
+    return rmContainer;
+  }
 }
 }

+ 2 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java

@@ -136,8 +136,7 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
    * @param release
    * @param release
    * @param blacklistAdditions 
    * @param blacklistAdditions 
    * @param blacklistRemovals 
    * @param blacklistRemovals 
-   * @param increaseRequests
-   * @param decreaseRequests
+   * @param updateRequests
    * @return the {@link Allocation} for the application
    * @return the {@link Allocation} for the application
    */
    */
   @Public
   @Public
@@ -145,8 +144,7 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
   Allocation allocate(ApplicationAttemptId appAttemptId,
   Allocation allocate(ApplicationAttemptId appAttemptId,
       List<ResourceRequest> ask, List<ContainerId> release,
       List<ResourceRequest> ask, List<ContainerId> release,
       List<String> blacklistAdditions, List<String> blacklistRemovals,
       List<String> blacklistAdditions, List<String> blacklistRemovals,
-      List<UpdateContainerRequest> increaseRequests,
-      List<UpdateContainerRequest> decreaseRequests);
+      ContainerUpdates updateRequests);
 
 
   /**
   /**
    * Get node resource usage report.
    * Get node resource usage report.

+ 12 - 4
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

@@ -75,6 +75,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeDecreaseContai
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
+
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
@@ -913,22 +916,27 @@ public class CapacityScheduler extends
   public Allocation allocate(ApplicationAttemptId applicationAttemptId,
   public Allocation allocate(ApplicationAttemptId applicationAttemptId,
       List<ResourceRequest> ask, List<ContainerId> release,
       List<ResourceRequest> ask, List<ContainerId> release,
       List<String> blacklistAdditions, List<String> blacklistRemovals,
       List<String> blacklistAdditions, List<String> blacklistRemovals,
-      List<UpdateContainerRequest> increaseRequests,
-      List<UpdateContainerRequest> decreaseRequests) {
+      ContainerUpdates updateRequests) {
     FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
     FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {
     if (application == null) {
       return EMPTY_ALLOCATION;
       return EMPTY_ALLOCATION;
     }
     }
 
 
+    // Handle promotions and demotions
+    handleExecutionTypeUpdates(
+        application, updateRequests.getPromotionRequests(),
+        updateRequests.getDemotionRequests());
+
     // Release containers
     // Release containers
     releaseContainers(release, application);
     releaseContainers(release, application);
 
 
     // update increase requests
     // update increase requests
-    LeafQueue updateDemandForQueue = updateIncreaseRequests(increaseRequests,
+    LeafQueue updateDemandForQueue =
+        updateIncreaseRequests(updateRequests.getIncreaseRequests(),
         application);
         application);
 
 
     // Decrease containers
     // Decrease containers
-    decreaseContainers(decreaseRequests, application);
+    decreaseContainers(updateRequests.getDecreaseRequests(), application);
 
 
     // Sanity check for new allocation requests
     // Sanity check for new allocation requests
     normalizeRequests(ask);
     normalizeRequests(ask);

+ 1 - 1
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

@@ -746,7 +746,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
       // When reserving container
       // When reserving container
       RMContainer updatedContainer = reservedContainer;
       RMContainer updatedContainer = reservedContainer;
       if (updatedContainer == null) {
       if (updatedContainer == null) {
-        updatedContainer = new RMContainerImpl(container,
+        updatedContainer = new RMContainerImpl(container, schedulerKey,
             application.getApplicationAttemptId(), node.getNodeID(),
             application.getApplicationAttemptId(), node.getNodeID(),
             application.getAppSchedulingInfo().getUser(), rmContext);
             application.getAppSchedulingInfo().getUser(), rmContext);
       }
       }

+ 9 - 4
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

@@ -223,7 +223,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
       }
       }
 
 
       // Create RMContainer
       // Create RMContainer
-      RMContainer rmContainer = new RMContainerImpl(container,
+      RMContainer rmContainer = new RMContainerImpl(container, schedulerKey,
           this.getApplicationAttemptId(), node.getNodeID(),
           this.getApplicationAttemptId(), node.getNodeID(),
           appSchedulingInfo.getUser(), this.rmContext,
           appSchedulingInfo.getUser(), this.rmContext,
           request.getNodeLabelExpression());
           request.getNodeLabelExpression());
@@ -555,12 +555,14 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
           // Update this application for the allocated container
           // Update this application for the allocated container
           if (!allocation.isIncreasedAllocation()) {
           if (!allocation.isIncreasedAllocation()) {
             // Allocate a new container
             // Allocate a new container
-            newlyAllocatedContainers.add(rmContainer);
+            addToNewlyAllocatedContainers(
+                schedulerContainer.getSchedulerNode(), rmContainer);
             liveContainers.put(containerId, rmContainer);
             liveContainers.put(containerId, rmContainer);
 
 
             // Deduct pending resource requests
             // Deduct pending resource requests
             List<ResourceRequest> requests = appSchedulingInfo.allocate(
             List<ResourceRequest> requests = appSchedulingInfo.allocate(
-                allocation.getAllocationLocalityType(), schedulerContainer.getSchedulerNode(),
+                allocation.getAllocationLocalityType(),
+                schedulerContainer.getSchedulerNode(),
                 schedulerContainer.getSchedulerRequestKey(),
                 schedulerContainer.getSchedulerRequestKey(),
                 schedulerContainer.getRmContainer().getContainer());
                 schedulerContainer.getRmContainer().getContainer());
             ((RMContainerImpl) rmContainer).setResourceRequests(requests);
             ((RMContainerImpl) rmContainer).setResourceRequests(requests);
@@ -752,12 +754,15 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
       List<Container> newlyAllocatedContainers = pullNewlyAllocatedContainers();
       List<Container> newlyAllocatedContainers = pullNewlyAllocatedContainers();
       List<Container> newlyIncreasedContainers = pullNewlyIncreasedContainers();
       List<Container> newlyIncreasedContainers = pullNewlyIncreasedContainers();
       List<Container> newlyDecreasedContainers = pullNewlyDecreasedContainers();
       List<Container> newlyDecreasedContainers = pullNewlyDecreasedContainers();
+      List<Container> newlyPromotedContainers = pullNewlyPromotedContainers();
+      List<Container> newlyDemotedContainers = pullNewlyDemotedContainers();
       List<NMToken> updatedNMTokens = pullUpdatedNMTokens();
       List<NMToken> updatedNMTokens = pullUpdatedNMTokens();
       Resource headroom = getHeadroom();
       Resource headroom = getHeadroom();
       setApplicationHeadroomForMetrics(headroom);
       setApplicationHeadroomForMetrics(headroom);
       return new Allocation(newlyAllocatedContainers, headroom, null,
       return new Allocation(newlyAllocatedContainers, headroom, null,
           currentContPreemption, Collections.singletonList(rr), updatedNMTokens,
           currentContPreemption, Collections.singletonList(rr), updatedNMTokens,
-          newlyIncreasedContainers, newlyDecreasedContainers);
+          newlyIncreasedContainers, newlyDecreasedContainers,
+          newlyPromotedContainers, newlyDemotedContainers);
     } finally {
     } finally {
       writeLock.unlock();
       writeLock.unlock();
     }
     }

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

@@ -449,13 +449,13 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
       }
       }
 
 
       // Create RMContainer
       // Create RMContainer
-      rmContainer = new RMContainerImpl(container,
+      rmContainer = new RMContainerImpl(container, schedulerKey,
           getApplicationAttemptId(), node.getNodeID(),
           getApplicationAttemptId(), node.getNodeID(),
           appSchedulingInfo.getUser(), rmContext);
           appSchedulingInfo.getUser(), rmContext);
       ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
       ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
 
 
       // Add it to allContainers list.
       // Add it to allContainers list.
-      newlyAllocatedContainers.add(rmContainer);
+      addToNewlyAllocatedContainers(node, rmContainer);
       liveContainers.put(container.getId(), rmContainer);
       liveContainers.put(container.getId(), rmContainer);
 
 
       // Update consumption and track allocations
       // Update consumption and track allocations

+ 10 - 4
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

@@ -54,7 +54,6 @@ import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -84,6 +83,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
@@ -811,8 +811,7 @@ public class FairScheduler extends
   public Allocation allocate(ApplicationAttemptId appAttemptId,
   public Allocation allocate(ApplicationAttemptId appAttemptId,
       List<ResourceRequest> ask, List<ContainerId> release,
       List<ResourceRequest> ask, List<ContainerId> release,
       List<String> blacklistAdditions, List<String> blacklistRemovals,
       List<String> blacklistAdditions, List<String> blacklistRemovals,
-      List<UpdateContainerRequest> increaseRequests,
-      List<UpdateContainerRequest> decreaseRequests) {
+      ContainerUpdates updateRequests) {
 
 
     // Make sure this application exists
     // Make sure this application exists
     FSAppAttempt application = getSchedulerApp(appAttemptId);
     FSAppAttempt application = getSchedulerApp(appAttemptId);
@@ -822,6 +821,11 @@ public class FairScheduler extends
       return EMPTY_ALLOCATION;
       return EMPTY_ALLOCATION;
     }
     }
 
 
+    // Handle promotions and demotions
+    handleExecutionTypeUpdates(
+        application, updateRequests.getPromotionRequests(),
+        updateRequests.getDemotionRequests());
+
     // Sanity check
     // Sanity check
     normalizeRequests(ask);
     normalizeRequests(ask);
 
 
@@ -876,7 +880,9 @@ public class FairScheduler extends
     application.setApplicationHeadroomForMetrics(headroom);
     application.setApplicationHeadroomForMetrics(headroom);
     return new Allocation(newlyAllocatedContainers, headroom,
     return new Allocation(newlyAllocatedContainers, headroom,
         preemptionContainerIds, null, null,
         preemptionContainerIds, null, null,
-        application.pullUpdatedNMTokens());
+        application.pullUpdatedNMTokens(), null, null,
+        application.pullNewlyPromotedContainers(),
+        application.pullNewlyDemotedContainers());
   }
   }
 
 
   @Override
   @Override

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java

@@ -68,7 +68,7 @@ public class FifoAppAttempt extends FiCaSchedulerApp {
 
 
       // Create RMContainer
       // Create RMContainer
       RMContainer rmContainer = new RMContainerImpl(container,
       RMContainer rmContainer = new RMContainerImpl(container,
-          this.getApplicationAttemptId(), node.getNodeID(),
+          schedulerKey, this.getApplicationAttemptId(), node.getNodeID(),
           appSchedulingInfo.getUser(), this.rmContext,
           appSchedulingInfo.getUser(), this.rmContext,
           request.getNodeLabelExpression());
           request.getNodeLabelExpression());
       ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
       ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
@@ -76,7 +76,7 @@ public class FifoAppAttempt extends FiCaSchedulerApp {
       updateAMContainerDiagnostics(AMState.ASSIGNED, null);
       updateAMContainerDiagnostics(AMState.ASSIGNED, null);
 
 
       // Add it to allContainers list.
       // Add it to allContainers list.
-      newlyAllocatedContainers.add(rmContainer);
+      addToNewlyAllocatedContainers(node, rmContainer);
 
 
       ContainerId containerId = container.getId();
       ContainerId containerId = container.getId();
       liveContainers.put(containerId, rmContainer);
       liveContainers.put(containerId, rmContainer);

+ 2 - 3
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,7 +40,6 @@ import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -62,6 +61,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
@@ -325,8 +325,7 @@ public class FifoScheduler extends
   public Allocation allocate(ApplicationAttemptId applicationAttemptId,
   public Allocation allocate(ApplicationAttemptId applicationAttemptId,
       List<ResourceRequest> ask, List<ContainerId> release,
       List<ResourceRequest> ask, List<ContainerId> release,
       List<String> blacklistAdditions, List<String> blacklistRemovals,
       List<String> blacklistAdditions, List<String> blacklistRemovals,
-      List<UpdateContainerRequest> increaseRequests,
-      List<UpdateContainerRequest> decreaseRequests) {
+      ContainerUpdates updateRequests) {
     FifoAppAttempt application = getApplicationAttempt(applicationAttemptId);
     FifoAppAttempt application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {
     if (application == null) {
       LOG.error("Calling allocate on removed " +
       LOG.error("Calling allocate on removed " +

+ 16 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalitySchedulingPlacementSet.java

@@ -157,7 +157,8 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
     return resourceRequestMap.get(resourceName);
     return resourceRequestMap.get(resourceName);
   }
   }
 
 
-  private void decrementOutstanding(ResourceRequest offSwitchRequest) {
+  private void decrementOutstanding(SchedulerRequestKey schedulerRequestKey,
+      ResourceRequest offSwitchRequest) {
     int numOffSwitchContainers = offSwitchRequest.getNumContainers() - 1;
     int numOffSwitchContainers = offSwitchRequest.getNumContainers() - 1;
 
 
     // Do not remove ANY
     // Do not remove ANY
@@ -166,8 +167,6 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
     // Do we have any outstanding requests?
     // Do we have any outstanding requests?
     // If there is nothing, we need to deactivate this application
     // If there is nothing, we need to deactivate this application
     if (numOffSwitchContainers == 0) {
     if (numOffSwitchContainers == 0) {
-      SchedulerRequestKey schedulerRequestKey = SchedulerRequestKey.create(
-          offSwitchRequest);
       appSchedulingInfo.decrementSchedulerKeyReference(schedulerRequestKey);
       appSchedulingInfo.decrementSchedulerKeyReference(schedulerRequestKey);
       appSchedulingInfo.checkForDeactivation();
       appSchedulingInfo.checkForDeactivation();
     }
     }
@@ -193,15 +192,15 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
    * The {@link ResourceScheduler} is allocating data-local resources to the
    * The {@link ResourceScheduler} is allocating data-local resources to the
    * application.
    * application.
    */
    */
-  private void allocateRackLocal(SchedulerNode node,
-      ResourceRequest rackLocalRequest,
+  private void allocateRackLocal(SchedulerRequestKey schedulerKey,
+      SchedulerNode node, ResourceRequest rackLocalRequest,
       List<ResourceRequest> resourceRequests) {
       List<ResourceRequest> resourceRequests) {
     // Update future requirements
     // Update future requirements
     decResourceRequest(node.getRackName(), rackLocalRequest);
     decResourceRequest(node.getRackName(), rackLocalRequest);
 
 
     ResourceRequest offRackRequest = resourceRequestMap.get(
     ResourceRequest offRackRequest = resourceRequestMap.get(
         ResourceRequest.ANY);
         ResourceRequest.ANY);
-    decrementOutstanding(offRackRequest);
+    decrementOutstanding(schedulerKey, offRackRequest);
 
 
     // Update cloned RackLocal and OffRack requests for recovery
     // Update cloned RackLocal and OffRack requests for recovery
     resourceRequests.add(cloneResourceRequest(rackLocalRequest));
     resourceRequests.add(cloneResourceRequest(rackLocalRequest));
@@ -212,10 +211,11 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
    * The {@link ResourceScheduler} is allocating data-local resources to the
    * The {@link ResourceScheduler} is allocating data-local resources to the
    * application.
    * application.
    */
    */
-  private void allocateOffSwitch(ResourceRequest offSwitchRequest,
+  private void allocateOffSwitch(SchedulerRequestKey schedulerKey,
+      ResourceRequest offSwitchRequest,
       List<ResourceRequest> resourceRequests) {
       List<ResourceRequest> resourceRequests) {
     // Update future requirements
     // Update future requirements
-    decrementOutstanding(offSwitchRequest);
+    decrementOutstanding(schedulerKey, offSwitchRequest);
     // Update cloned OffRack requests for recovery
     // Update cloned OffRack requests for recovery
     resourceRequests.add(cloneResourceRequest(offSwitchRequest));
     resourceRequests.add(cloneResourceRequest(offSwitchRequest));
   }
   }
@@ -225,8 +225,8 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
    * The {@link ResourceScheduler} is allocating data-local resources to the
    * The {@link ResourceScheduler} is allocating data-local resources to the
    * application.
    * application.
    */
    */
-  private void allocateNodeLocal(SchedulerNode node,
-      ResourceRequest nodeLocalRequest,
+  private void allocateNodeLocal(SchedulerRequestKey schedulerKey,
+      SchedulerNode node, ResourceRequest nodeLocalRequest,
       List<ResourceRequest> resourceRequests) {
       List<ResourceRequest> resourceRequests) {
     // Update future requirements
     // Update future requirements
     decResourceRequest(node.getNodeName(), nodeLocalRequest);
     decResourceRequest(node.getNodeName(), nodeLocalRequest);
@@ -237,7 +237,7 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
 
 
     ResourceRequest offRackRequest = resourceRequestMap.get(
     ResourceRequest offRackRequest = resourceRequestMap.get(
         ResourceRequest.ANY);
         ResourceRequest.ANY);
-    decrementOutstanding(offRackRequest);
+    decrementOutstanding(schedulerKey, offRackRequest);
 
 
     // Update cloned NodeLocal, RackLocal and OffRack requests for recovery
     // Update cloned NodeLocal, RackLocal and OffRack requests for recovery
     resourceRequests.add(cloneResourceRequest(nodeLocalRequest));
     resourceRequests.add(cloneResourceRequest(nodeLocalRequest));
@@ -282,8 +282,8 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
   }
   }
 
 
   @Override
   @Override
-  public List<ResourceRequest> allocate(NodeType type, SchedulerNode node,
-      ResourceRequest request) {
+  public List<ResourceRequest> allocate(SchedulerRequestKey schedulerKey,
+      NodeType type, SchedulerNode node, ResourceRequest request) {
     try {
     try {
       writeLock.lock();
       writeLock.lock();
 
 
@@ -300,11 +300,11 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode>
       }
       }
 
 
       if (type == NodeType.NODE_LOCAL) {
       if (type == NodeType.NODE_LOCAL) {
-        allocateNodeLocal(node, request, resourceRequests);
+        allocateNodeLocal(schedulerKey, node, request, resourceRequests);
       } else if (type == NodeType.RACK_LOCAL) {
       } else if (type == NodeType.RACK_LOCAL) {
-        allocateRackLocal(node, request, resourceRequests);
+        allocateRackLocal(schedulerKey, node, request, resourceRequests);
       } else{
       } else{
-        allocateOffSwitch(request, resourceRequests);
+        allocateOffSwitch(schedulerKey, request, resourceRequests);
       }
       }
 
 
       return resourceRequests;
       return resourceRequests;

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

@@ -78,13 +78,14 @@ public interface SchedulingPlacementSet<N extends SchedulerNode> {
 
 
   /**
   /**
    * Notify container allocated.
    * Notify container allocated.
+   * @param schedulerKey SchedulerRequestKey for this ResourceRequest
    * @param type Type of the allocation
    * @param type Type of the allocation
    * @param node Which node this container allocated on
    * @param node Which node this container allocated on
    * @param request Which resource request to allocate
    * @param request Which resource request to allocate
    * @return list of ResourceRequests deducted
    * @return list of ResourceRequests deducted
    */
    */
-  List<ResourceRequest> allocate(NodeType type, SchedulerNode node,
-      ResourceRequest request);
+  List<ResourceRequest> allocate(SchedulerRequestKey schedulerKey,
+      NodeType type, SchedulerNode node, ResourceRequest request);
 
 
   /**
   /**
    * We can still have pending requirement for a given NodeType and node
    * We can still have pending requirement for a given NodeType and node

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsMana
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 
 
@@ -331,7 +332,8 @@ public class Application {
     // Get resources from the ResourceManager
     // Get resources from the ResourceManager
     Allocation allocation = resourceManager.getResourceScheduler().allocate(
     Allocation allocation = resourceManager.getResourceScheduler().allocate(
         applicationAttemptId, new ArrayList<ResourceRequest>(ask),
         applicationAttemptId, new ArrayList<ResourceRequest>(ask),
-        new ArrayList<ContainerId>(), null, null, null, null);
+        new ArrayList<ContainerId>(), null, null,
+        new ContainerUpdates());
 
 
     if (LOG.isInfoEnabled()) {
     if (LOG.isInfoEnabled()) {
       LOG.info("-=======" + applicationAttemptId + System.lineSeparator() +
       LOG.info("-=======" + applicationAttemptId + System.lineSeparator() +

+ 7 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java

@@ -251,6 +251,13 @@ public class MockAM {
     return allocate(req);
     return allocate(req);
   }
   }
 
 
+  public AllocateResponse sendContainerUpdateRequest(
+      List<UpdateContainerRequest> updateRequests) throws Exception {
+    final AllocateRequest req = AllocateRequest.newInstance(0, 0F, null, null,
+        updateRequests, null);
+    return allocate(req);
+  }
+
   public AllocateResponse allocate(AllocateRequest allocateRequest)
   public AllocateResponse allocate(AllocateRequest allocateRequest)
             throws Exception {
             throws Exception {
     UserGroupInformation ugi =
     UserGroupInformation ugi =

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java

@@ -195,6 +195,12 @@ public class MockNM {
         isHealthy, resId);
         isHealthy, resId);
   }
   }
 
 
+  public NodeHeartbeatResponse nodeHeartbeat(
+      List<ContainerStatus> updatedStats, boolean isHealthy) throws Exception {
+    return nodeHeartbeat(updatedStats, Collections.<Container>emptyList(),
+        isHealthy, ++responseId);
+  }
+
   public NodeHeartbeatResponse nodeHeartbeat(List<ContainerStatus> updatedStats,
   public NodeHeartbeatResponse nodeHeartbeat(List<ContainerStatus> updatedStats,
       List<Container> increasedConts, boolean isHealthy, int resId)
       List<Container> increasedConts, boolean isHealthy, int resId)
           throws Exception {
           throws Exception {

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java

@@ -140,6 +140,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 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.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
+
+import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Clock;
@@ -1060,7 +1062,8 @@ public class TestClientRMService {
     Container container = Container.newInstance(
     Container container = Container.newInstance(
         ContainerId.newContainerId(attemptId, 1), null, "", null, null, null);
         ContainerId.newContainerId(attemptId, 1), null, "", null, null, null);
     RMContainerImpl containerimpl = spy(new RMContainerImpl(container,
     RMContainerImpl containerimpl = spy(new RMContainerImpl(container,
-        attemptId, null, "", rmContext));
+        SchedulerRequestKey.extractFrom(container), attemptId, null, "",
+        rmContext));
     Map<ApplicationAttemptId, RMAppAttempt> attempts = 
     Map<ApplicationAttemptId, RMAppAttempt> attempts = 
       new HashMap<ApplicationAttemptId, RMAppAttempt>();
       new HashMap<ApplicationAttemptId, RMAppAttempt>();
     attempts.put(attemptId, rmAppAttemptImpl);
     attempts.put(attemptId, rmAppAttemptImpl);

+ 453 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java

@@ -34,11 +34,15 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocolPB;
 import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocolPB;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
@@ -64,8 +68,11 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterDistrib
 import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
 import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
+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.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 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.capacity.CapacitySchedulerConfiguration;
@@ -75,13 +82,17 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateS
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
 import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 
 
 /**
 /**
@@ -91,8 +102,10 @@ public class TestOpportunisticContainerAllocatorAMService {
 
 
   private static final int GB = 1024;
   private static final int GB = 1024;
 
 
-  @Test(timeout = 60000)
-  public void testNodeRemovalDuringAllocate() throws Exception {
+  private MockRM rm;
+
+  @Before
+  public void createAndStartRM() {
     CapacitySchedulerConfiguration csConf =
     CapacitySchedulerConfiguration csConf =
         new CapacitySchedulerConfiguration();
         new CapacitySchedulerConfiguration();
     YarnConfiguration conf = new YarnConfiguration(csConf);
     YarnConfiguration conf = new YarnConfiguration(csConf);
@@ -102,8 +115,445 @@ public class TestOpportunisticContainerAllocatorAMService {
         YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
         YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
     conf.setInt(
     conf.setInt(
         YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS, 100);
         YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS, 100);
-    MockRM rm = new MockRM(conf);
+    rm = new MockRM(conf);
     rm.start();
     rm.start();
+  }
+
+  @After
+  public void stopRM() {
+    if (rm != null) {
+      rm.stop();
+    }
+  }
+
+  @Test(timeout = 600000)
+  public void testContainerPromoteAndDemoteBeforeContainerStart() throws Exception {
+    HashMap<NodeId, MockNM> nodes = new HashMap<>();
+    MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
+    nodes.put(nm1.getNodeId(), nm1);
+    MockNM nm2 = new MockNM("h1:4321", 4096, rm.getResourceTrackerService());
+    nodes.put(nm2.getNodeId(), nm2);
+    MockNM nm3 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
+    nodes.put(nm3.getNodeId(), nm3);
+    MockNM nm4 = new MockNM("h2:4321", 4096, rm.getResourceTrackerService());
+    nodes.put(nm4.getNodeId(), nm4);
+    nm1.registerNode();
+    nm2.registerNode();
+    nm3.registerNode();
+    nm4.registerNode();
+
+    OpportunisticContainerAllocatorAMService amservice =
+        (OpportunisticContainerAllocatorAMService) rm
+            .getApplicationMasterService();
+    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
+    ApplicationAttemptId attemptId =
+        app1.getCurrentAppAttempt().getAppAttemptId();
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
+    ResourceScheduler scheduler = rm.getResourceScheduler();
+    RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm.getRMContext().getRMNodes().get(nm2.getNodeId());
+    RMNode rmNode3 = rm.getRMContext().getRMNodes().get(nm3.getNodeId());
+    RMNode rmNode4 = rm.getRMContext().getRMNodes().get(nm4.getNodeId());
+
+    nm1.nodeHeartbeat(true);
+    nm2.nodeHeartbeat(true);
+    nm3.nodeHeartbeat(true);
+    nm4.nodeHeartbeat(true);
+
+    ((RMNodeImpl) rmNode1)
+        .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
+    ((RMNodeImpl) rmNode2)
+        .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
+    ((RMNodeImpl) rmNode3)
+        .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
+    ((RMNodeImpl) rmNode4)
+        .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
+
+    OpportunisticContainerContext ctxt = ((CapacityScheduler) scheduler)
+        .getApplicationAttempt(attemptId).getOpportunisticContainerContext();
+    // Send add and update node events to AM Service.
+    amservice.handle(new NodeAddedSchedulerEvent(rmNode1));
+    amservice.handle(new NodeAddedSchedulerEvent(rmNode2));
+    amservice.handle(new NodeAddedSchedulerEvent(rmNode3));
+    amservice.handle(new NodeAddedSchedulerEvent(rmNode4));
+    amservice.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    amservice.handle(new NodeUpdateSchedulerEvent(rmNode2));
+    amservice.handle(new NodeUpdateSchedulerEvent(rmNode3));
+    amservice.handle(new NodeUpdateSchedulerEvent(rmNode4));
+    // All nodes 1 - 4 will be applicable for scheduling.
+    nm1.nodeHeartbeat(true);
+    nm2.nodeHeartbeat(true);
+    nm3.nodeHeartbeat(true);
+    nm4.nodeHeartbeat(true);
+
+    Thread.sleep(1000);
+
+    QueueMetrics metrics = ((CapacityScheduler) scheduler).getRootQueue()
+        .getMetrics();
+
+    // Verify Metrics
+    verifyMetrics(metrics, 15360, 15, 1024, 1, 1);
+
+    AllocateResponse allocateResponse = am1.allocate(
+        Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1),
+            "*", Resources.createResource(1 * GB), 2, true, null,
+            ExecutionTypeRequest.newInstance(
+                ExecutionType.OPPORTUNISTIC, true))),
+        null);
+    List<Container> allocatedContainers = allocateResponse
+        .getAllocatedContainers();
+    Assert.assertEquals(2, allocatedContainers.size());
+    Container container = allocatedContainers.get(0);
+    MockNM allocNode = nodes.get(container.getNodeId());
+    MockNM sameHostDiffNode = null;
+    for (NodeId n : nodes.keySet()) {
+      if (n.getHost().equals(allocNode.getNodeId().getHost()) &&
+          n.getPort() != allocNode.getNodeId().getPort()) {
+        sameHostDiffNode = nodes.get(n);
+      }
+    }
+
+    // Verify Metrics After OPP allocation (Nothing should change)
+    verifyMetrics(metrics, 15360, 15, 1024, 1, 1);
+
+    am1.sendContainerUpdateRequest(
+        Arrays.asList(UpdateContainerRequest.newInstance(0,
+            container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+            null, ExecutionType.GUARANTEED)));
+    // Node on same host should not result in allocation
+    sameHostDiffNode.nodeHeartbeat(true);
+    Thread.sleep(200);
+    allocateResponse =  am1.allocate(null, null);
+    Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
+
+    // Verify Metrics After OPP allocation (Nothing should change again)
+    verifyMetrics(metrics, 15360, 15, 1024, 1, 1);
+
+    // Send Promotion req again... this should result in update error
+    allocateResponse = am1.sendContainerUpdateRequest(
+        Arrays.asList(UpdateContainerRequest.newInstance(0,
+            container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+            null, ExecutionType.GUARANTEED)));
+    Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
+    Assert.assertEquals(1, allocateResponse.getUpdateErrors().size());
+    Assert.assertEquals("UPDATE_OUTSTANDING_ERROR",
+        allocateResponse.getUpdateErrors().get(0).getReason());
+    Assert.assertEquals(container.getId(),
+        allocateResponse.getUpdateErrors().get(0)
+            .getUpdateContainerRequest().getContainerId());
+
+    // Send Promotion req again with incorrect version...
+    // this should also result in update error
+    allocateResponse = am1.sendContainerUpdateRequest(
+        Arrays.asList(UpdateContainerRequest.newInstance(1,
+            container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+            null, ExecutionType.GUARANTEED)));
+
+    Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
+    Assert.assertEquals(1, allocateResponse.getUpdateErrors().size());
+    Assert.assertEquals("INCORRECT_CONTAINER_VERSION_ERROR|1|0",
+        allocateResponse.getUpdateErrors().get(0).getReason());
+    Assert.assertEquals(container.getId(),
+        allocateResponse.getUpdateErrors().get(0)
+            .getUpdateContainerRequest().getContainerId());
+
+    // Ensure after correct node heartbeats, we should get the allocation
+    allocNode.nodeHeartbeat(true);
+    Thread.sleep(200);
+    allocateResponse =  am1.allocate(null, null);
+    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+    Container uc =
+        allocateResponse.getUpdatedContainers().get(0).getContainer();
+    Assert.assertEquals(ExecutionType.GUARANTEED, uc.getExecutionType());
+    Assert.assertEquals(uc.getId(), container.getId());
+    Assert.assertEquals(uc.getVersion(), container.getVersion() + 1);
+
+    // Verify Metrics After OPP allocation :
+    // Allocated cores+mem should have increased, available should decrease
+    verifyMetrics(metrics, 14336, 14, 2048, 2, 2);
+
+    nm1.nodeHeartbeat(true);
+    nm2.nodeHeartbeat(true);
+    nm3.nodeHeartbeat(true);
+    nm4.nodeHeartbeat(true);
+    Thread.sleep(200);
+
+    // Verify that the container is still in ACQUIRED state wrt the RM.
+    RMContainer rmContainer = ((CapacityScheduler) scheduler)
+        .getApplicationAttempt(
+        uc.getId().getApplicationAttemptId()).getRMContainer(uc.getId());
+    Assert.assertEquals(RMContainerState.ACQUIRED, rmContainer.getState());
+
+    // Now demote the container back..
+    allocateResponse = am1.sendContainerUpdateRequest(
+        Arrays.asList(UpdateContainerRequest.newInstance(uc.getVersion(),
+            uc.getId(), ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
+            null, ExecutionType.OPPORTUNISTIC)));
+    // This should happen in the same heartbeat..
+    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+    uc = allocateResponse.getUpdatedContainers().get(0).getContainer();
+    Assert.assertEquals(ExecutionType.OPPORTUNISTIC, uc.getExecutionType());
+    Assert.assertEquals(uc.getId(), container.getId());
+    Assert.assertEquals(uc.getVersion(), container.getVersion() + 2);
+
+    // Verify Metrics After OPP allocation :
+    // Everything should have reverted to what it was
+    verifyMetrics(metrics, 15360, 15, 1024, 1, 1);
+  }
+
+  @Test(timeout = 60000)
+  public void testContainerPromoteAfterContainerStart() throws Exception {
+    HashMap<NodeId, MockNM> nodes = new HashMap<>();
+    MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
+    nodes.put(nm1.getNodeId(), nm1);
+    MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
+    nodes.put(nm2.getNodeId(), nm2);
+    nm1.registerNode();
+    nm2.registerNode();
+
+    OpportunisticContainerAllocatorAMService amservice =
+        (OpportunisticContainerAllocatorAMService) rm
+            .getApplicationMasterService();
+    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
+    ApplicationAttemptId attemptId =
+        app1.getCurrentAppAttempt().getAppAttemptId();
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
+    ResourceScheduler scheduler = rm.getResourceScheduler();
+    RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    nm1.nodeHeartbeat(true);
+    nm2.nodeHeartbeat(true);
+
+    ((RMNodeImpl) rmNode1)
+        .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
+    ((RMNodeImpl) rmNode2)
+        .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
+
+    OpportunisticContainerContext ctxt = ((CapacityScheduler) scheduler)
+        .getApplicationAttempt(attemptId).getOpportunisticContainerContext();
+    // Send add and update node events to AM Service.
+    amservice.handle(new NodeAddedSchedulerEvent(rmNode1));
+    amservice.handle(new NodeAddedSchedulerEvent(rmNode2));
+    amservice.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    amservice.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    // All nodes 1 to 2 will be applicable for scheduling.
+    nm1.nodeHeartbeat(true);
+    nm2.nodeHeartbeat(true);
+
+    Thread.sleep(1000);
+
+    QueueMetrics metrics = ((CapacityScheduler) scheduler).getRootQueue()
+        .getMetrics();
+
+    // Verify Metrics
+    verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
+
+    AllocateResponse allocateResponse = am1.allocate(
+        Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1),
+            "*", Resources.createResource(1 * GB), 2, true, null,
+            ExecutionTypeRequest.newInstance(
+                ExecutionType.OPPORTUNISTIC, true))),
+        null);
+    List<Container> allocatedContainers = allocateResponse
+        .getAllocatedContainers();
+    Assert.assertEquals(2, allocatedContainers.size());
+    Container container = allocatedContainers.get(0);
+    MockNM allocNode = nodes.get(container.getNodeId());
+
+    // Start Container in NM
+    allocNode.nodeHeartbeat(Arrays.asList(
+        ContainerStatus.newInstance(container.getId(),
+            ExecutionType.OPPORTUNISTIC, ContainerState.RUNNING, "", 0)),
+        true);
+    Thread.sleep(200);
+
+    // Verify that container is actually running wrt the RM..
+    RMContainer rmContainer = ((CapacityScheduler) scheduler)
+        .getApplicationAttempt(
+            container.getId().getApplicationAttemptId()).getRMContainer(
+            container.getId());
+    Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+
+    // Verify Metrics After OPP allocation (Nothing should change)
+    verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
+
+    am1.sendContainerUpdateRequest(
+        Arrays.asList(UpdateContainerRequest.newInstance(0,
+            container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+            null, ExecutionType.GUARANTEED)));
+
+    // Verify Metrics After OPP allocation (Nothing should change again)
+    verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
+
+    // Send Promotion req again... this should result in update error
+    allocateResponse = am1.sendContainerUpdateRequest(
+        Arrays.asList(UpdateContainerRequest.newInstance(0,
+            container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+            null, ExecutionType.GUARANTEED)));
+    Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
+    Assert.assertEquals(1, allocateResponse.getUpdateErrors().size());
+    Assert.assertEquals("UPDATE_OUTSTANDING_ERROR",
+        allocateResponse.getUpdateErrors().get(0).getReason());
+    Assert.assertEquals(container.getId(),
+        allocateResponse.getUpdateErrors().get(0)
+            .getUpdateContainerRequest().getContainerId());
+
+    // Start Container in NM
+    allocNode.nodeHeartbeat(Arrays.asList(
+        ContainerStatus.newInstance(container.getId(),
+            ExecutionType.OPPORTUNISTIC, ContainerState.RUNNING, "", 0)),
+        true);
+    Thread.sleep(200);
+
+    allocateResponse =  am1.allocate(null, null);
+    Assert.assertEquals(1, allocateResponse.getUpdatedContainers().size());
+    Container uc =
+        allocateResponse.getUpdatedContainers().get(0).getContainer();
+    Assert.assertEquals(ExecutionType.GUARANTEED, uc.getExecutionType());
+    Assert.assertEquals(uc.getId(), container.getId());
+    Assert.assertEquals(uc.getVersion(), container.getVersion() + 1);
+
+    // Verify that the Container is still in RUNNING state wrt RM..
+    rmContainer = ((CapacityScheduler) scheduler)
+        .getApplicationAttempt(
+            uc.getId().getApplicationAttemptId()).getRMContainer(uc.getId());
+    Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+
+    // Verify Metrics After OPP allocation :
+    // Allocated cores+mem should have increased, available should decrease
+    verifyMetrics(metrics, 6144, 6, 2048, 2, 2);
+  }
+
+  @Test(timeout = 600000)
+  public void testContainerPromoteAfterContainerComplete() throws Exception {
+    HashMap<NodeId, MockNM> nodes = new HashMap<>();
+    MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
+    nodes.put(nm1.getNodeId(), nm1);
+    MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
+    nodes.put(nm2.getNodeId(), nm2);
+    nm1.registerNode();
+    nm2.registerNode();
+
+    OpportunisticContainerAllocatorAMService amservice =
+        (OpportunisticContainerAllocatorAMService) rm
+            .getApplicationMasterService();
+    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "default");
+    ApplicationAttemptId attemptId =
+        app1.getCurrentAppAttempt().getAppAttemptId();
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
+    ResourceScheduler scheduler = rm.getResourceScheduler();
+    RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
+    RMNode rmNode2 = rm.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    nm1.nodeHeartbeat(true);
+    nm2.nodeHeartbeat(true);
+
+    ((RMNodeImpl) rmNode1)
+        .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
+    ((RMNodeImpl) rmNode2)
+        .setOpportunisticContainersStatus(getOppurtunisticStatus(-1, 100));
+
+    OpportunisticContainerContext ctxt = ((CapacityScheduler) scheduler)
+        .getApplicationAttempt(attemptId).getOpportunisticContainerContext();
+    // Send add and update node events to AM Service.
+    amservice.handle(new NodeAddedSchedulerEvent(rmNode1));
+    amservice.handle(new NodeAddedSchedulerEvent(rmNode2));
+    amservice.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    amservice.handle(new NodeUpdateSchedulerEvent(rmNode2));
+
+    // All nodes 1 to 2 will be applicable for scheduling.
+    nm1.nodeHeartbeat(true);
+    nm2.nodeHeartbeat(true);
+
+    Thread.sleep(1000);
+
+    QueueMetrics metrics = ((CapacityScheduler) scheduler).getRootQueue()
+        .getMetrics();
+
+    // Verify Metrics
+    verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
+
+    AllocateResponse allocateResponse = am1.allocate(
+        Arrays.asList(ResourceRequest.newInstance(Priority.newInstance(1),
+            "*", Resources.createResource(1 * GB), 2, true, null,
+            ExecutionTypeRequest.newInstance(
+                ExecutionType.OPPORTUNISTIC, true))),
+        null);
+    List<Container> allocatedContainers = allocateResponse
+        .getAllocatedContainers();
+    Assert.assertEquals(2, allocatedContainers.size());
+    Container container = allocatedContainers.get(0);
+    MockNM allocNode = nodes.get(container.getNodeId());
+
+    // Start Container in NM
+    allocNode.nodeHeartbeat(Arrays.asList(
+        ContainerStatus.newInstance(container.getId(),
+            ExecutionType.OPPORTUNISTIC, ContainerState.RUNNING, "", 0)),
+        true);
+    Thread.sleep(200);
+
+    // Verify that container is actually running wrt the RM..
+    RMContainer rmContainer = ((CapacityScheduler) scheduler)
+        .getApplicationAttempt(
+            container.getId().getApplicationAttemptId()).getRMContainer(
+            container.getId());
+    Assert.assertEquals(RMContainerState.RUNNING, rmContainer.getState());
+
+    // Container Completed in the NM
+    allocNode.nodeHeartbeat(Arrays.asList(
+        ContainerStatus.newInstance(container.getId(),
+            ExecutionType.OPPORTUNISTIC, ContainerState.COMPLETE, "", 0)),
+        true);
+    Thread.sleep(200);
+
+    // Verify that container has been removed..
+    rmContainer = ((CapacityScheduler) scheduler)
+        .getApplicationAttempt(
+            container.getId().getApplicationAttemptId()).getRMContainer(
+            container.getId());
+    Assert.assertNull(rmContainer);
+
+    // Verify Metrics After OPP allocation (Nothing should change)
+    verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
+
+    // Send Promotion req... this should result in update error
+    // Since the container doesn't exist anymore..
+    allocateResponse = am1.sendContainerUpdateRequest(
+        Arrays.asList(UpdateContainerRequest.newInstance(0,
+            container.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+            null, ExecutionType.GUARANTEED)));
+
+    Assert.assertEquals(1,
+        allocateResponse.getCompletedContainersStatuses().size());
+    Assert.assertEquals(container.getId(),
+        allocateResponse.getCompletedContainersStatuses().get(0)
+            .getContainerId());
+    Assert.assertEquals(0, allocateResponse.getUpdatedContainers().size());
+    Assert.assertEquals(1, allocateResponse.getUpdateErrors().size());
+    Assert.assertEquals("INVALID_CONTAINER_ID",
+        allocateResponse.getUpdateErrors().get(0).getReason());
+    Assert.assertEquals(container.getId(),
+        allocateResponse.getUpdateErrors().get(0)
+            .getUpdateContainerRequest().getContainerId());
+
+    // Verify Metrics After OPP allocation (Nothing should change again)
+    verifyMetrics(metrics, 7168, 7, 1024, 1, 1);
+  }
+
+  private void verifyMetrics(QueueMetrics metrics, long availableMB,
+      int availableVirtualCores, long allocatedMB,
+      int allocatedVirtualCores, int allocatedContainers) {
+    Assert.assertEquals(availableMB, metrics.getAvailableMB());
+    Assert.assertEquals(availableVirtualCores, metrics.getAvailableVirtualCores());
+    Assert.assertEquals(allocatedMB, metrics.getAllocatedMB());
+    Assert.assertEquals(allocatedVirtualCores, metrics.getAllocatedVirtualCores());
+    Assert.assertEquals(allocatedContainers, metrics.getAllocatedContainers());
+  }
+
+  @Test(timeout = 60000)
+  public void testNodeRemovalDuringAllocate() throws Exception {
     MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
     MockNM nm1 = new MockNM("h1:1234", 4096, rm.getResourceTrackerService());
     MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
     MockNM nm2 = new MockNM("h2:1234", 4096, rm.getResourceTrackerService());
     nm1.registerNode();
     nm1.registerNode();

+ 11 - 8
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

@@ -100,6 +100,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContainersPulledByAMEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContainersPulledByAMEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
@@ -478,7 +479,7 @@ public class TestRMAppAttemptTransitions {
     assertEquals(expectedState, applicationAttempt.getAppAttemptState());
     assertEquals(expectedState, applicationAttempt.getAppAttemptState());
     verify(scheduler, times(expectedAllocateCount)).allocate(
     verify(scheduler, times(expectedAllocateCount)).allocate(
         any(ApplicationAttemptId.class), any(List.class), any(List.class),
         any(ApplicationAttemptId.class), any(List.class), any(List.class),
-        any(List.class), any(List.class), any(List.class), any(List.class));
+        any(List.class), any(List.class), any(ContainerUpdates.class));
 
 
     assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
     assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
     assertNull(applicationAttempt.getMasterContainer());
     assertNull(applicationAttempt.getMasterContainer());
@@ -499,7 +500,7 @@ public class TestRMAppAttemptTransitions {
     verify(applicationMasterLauncher).handle(any(AMLauncherEvent.class));
     verify(applicationMasterLauncher).handle(any(AMLauncherEvent.class));
     verify(scheduler, times(2)).allocate(any(ApplicationAttemptId.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), any(List.class), any(List.class),
-        any(List.class), any(List.class));
+        any(ContainerUpdates.class));
     verify(nmTokenManager).clearNodeSetForAttempt(
     verify(nmTokenManager).clearNodeSetForAttempt(
       applicationAttempt.getAppAttemptId());
       applicationAttempt.getAppAttemptId());
   }
   }
@@ -526,7 +527,7 @@ public class TestRMAppAttemptTransitions {
   }
   }
 
 
   /**
   /**
-   * {@link RMAppAttemptState#LAUNCH}
+   * {@link RMAppAttemptState#LAUNCHED}
    */
    */
   private void testAppAttemptLaunchedState(Container container) {
   private void testAppAttemptLaunchedState(Container container) {
     assertEquals(RMAppAttemptState.LAUNCHED, 
     assertEquals(RMAppAttemptState.LAUNCHED, 
@@ -649,8 +650,8 @@ public class TestRMAppAttemptTransitions {
     when(allocation.getContainers()).
     when(allocation.getContainers()).
         thenReturn(Collections.singletonList(container));
         thenReturn(Collections.singletonList(container));
     when(scheduler.allocate(any(ApplicationAttemptId.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))).
+        any(List.class), any(List.class), any(List.class),
+        any(ContainerUpdates.class))).
     thenReturn(allocation);
     thenReturn(allocation);
     RMContainer rmContainer = mock(RMContainerImpl.class);
     RMContainer rmContainer = mock(RMContainerImpl.class);
     when(scheduler.getRMContainer(container.getId())).
     when(scheduler.getRMContainer(container.getId())).
@@ -1129,8 +1130,9 @@ public class TestRMAppAttemptTransitions {
     when(allocation.getContainers()).
     when(allocation.getContainers()).
         thenReturn(Collections.singletonList(amContainer));
         thenReturn(Collections.singletonList(amContainer));
     when(scheduler.allocate(any(ApplicationAttemptId.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);
+        any(List.class), any(List.class), any(List.class),
+        any(ContainerUpdates.class)))
+        .thenReturn(allocation);
     RMContainer rmContainer = mock(RMContainerImpl.class);
     RMContainer rmContainer = mock(RMContainerImpl.class);
     when(scheduler.getRMContainer(amContainer.getId())).thenReturn(rmContainer);
     when(scheduler.getRMContainer(amContainer.getId())).thenReturn(rmContainer);
 
 
@@ -1610,7 +1612,8 @@ public class TestRMAppAttemptTransitions {
     YarnScheduler mockScheduler = mock(YarnScheduler.class);
     YarnScheduler mockScheduler = mock(YarnScheduler.class);
     when(mockScheduler.allocate(any(ApplicationAttemptId.class),
     when(mockScheduler.allocate(any(ApplicationAttemptId.class),
         any(List.class), any(List.class), any(List.class), any(List.class),
         any(List.class), any(List.class), any(List.class), any(List.class),
-        any(List.class), any(List.class))).thenAnswer(new Answer<Allocation>() {
+        any(ContainerUpdates.class)))
+        .thenAnswer(new Answer<Allocation>() {
 
 
           @SuppressWarnings("rawtypes")
           @SuppressWarnings("rawtypes")
           @Override
           @Override

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java

@@ -62,6 +62,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAt
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
@@ -114,7 +115,8 @@ public class TestRMContainerImpl {
         YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
         YarnConfiguration.APPLICATION_HISTORY_SAVE_NON_AM_CONTAINER_META_INFO,
         true);
         true);
     when(rmContext.getYarnConfiguration()).thenReturn(conf);
     when(rmContext.getYarnConfiguration()).thenReturn(conf);
-    RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
+    RMContainer rmContainer = new RMContainerImpl(container,
+        SchedulerRequestKey.extractFrom(container), appAttemptId,
         nodeId, "user", rmContext);
         nodeId, "user", rmContext);
 
 
     assertEquals(RMContainerState.NEW, rmContainer.getState());
     assertEquals(RMContainerState.NEW, rmContainer.getState());
@@ -216,7 +218,8 @@ public class TestRMContainerImpl {
     when(rmContext.getYarnConfiguration()).thenReturn(conf);
     when(rmContext.getYarnConfiguration()).thenReturn(conf);
     when(rmContext.getRMApps()).thenReturn(appMap);
     when(rmContext.getRMApps()).thenReturn(appMap);
     
     
-    RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
+    RMContainer rmContainer = new RMContainerImpl(container,
+        SchedulerRequestKey.extractFrom(container), appAttemptId,
         nodeId, "user", rmContext);
         nodeId, "user", rmContext);
 
 
     assertEquals(RMContainerState.NEW, rmContainer.getState());
     assertEquals(RMContainerState.NEW, rmContainer.getState());

+ 21 - 16
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

@@ -97,6 +97,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
@@ -158,7 +159,9 @@ import static org.mockito.Mockito.when;
 public class TestCapacityScheduler {
 public class TestCapacityScheduler {
   private static final Log LOG = LogFactory.getLog(TestCapacityScheduler.class);
   private static final Log LOG = LogFactory.getLog(TestCapacityScheduler.class);
   private final int GB = 1024;
   private final int GB = 1024;
-  
+  private final static ContainerUpdates NULL_UPDATE_REQUESTS =
+      new ContainerUpdates();
+
   private static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
   private static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
   private static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
   private static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
   private static final String A1 = A + ".a1";
   private static final String A1 = A + ".a1";
@@ -807,12 +810,12 @@ public class TestCapacityScheduler {
     // Verify the blacklist can be updated independent of requesting containers
     // Verify the blacklist can be updated independent of requesting containers
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(),
         Collections.<ContainerId>emptyList(),
-        Collections.singletonList(host), null, null, null);
+        Collections.singletonList(host), null, NULL_UPDATE_REQUESTS);
     Assert.assertTrue(cs.getApplicationAttempt(appAttemptId)
     Assert.assertTrue(cs.getApplicationAttempt(appAttemptId)
         .isPlaceBlacklisted(host));
         .isPlaceBlacklisted(host));
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
     cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(), null,
         Collections.<ContainerId>emptyList(), null,
-        Collections.singletonList(host), null, null);
+        Collections.singletonList(host), NULL_UPDATE_REQUESTS);
     Assert.assertFalse(cs.getApplicationAttempt(appAttemptId)
     Assert.assertFalse(cs.getApplicationAttempt(appAttemptId)
         .isPlaceBlacklisted(host));
         .isPlaceBlacklisted(host));
     rm.stop();
     rm.stop();
@@ -908,7 +911,7 @@ public class TestCapacityScheduler {
     cs.allocate(appAttemptId1,
     cs.allocate(appAttemptId1,
         Collections.<ResourceRequest>singletonList(r1),
         Collections.<ResourceRequest>singletonList(r1),
         Collections.<ContainerId>emptyList(),
         Collections.<ContainerId>emptyList(),
-        null, null, null, null);
+        null, null, NULL_UPDATE_REQUESTS);
 
 
     //And this will result in container assignment for app1
     //And this will result in container assignment for app1
     CapacityScheduler.schedule(cs);
     CapacityScheduler.schedule(cs);
@@ -925,7 +928,7 @@ public class TestCapacityScheduler {
     cs.allocate(appAttemptId2,
     cs.allocate(appAttemptId2,
         Collections.<ResourceRequest>singletonList(r2),
         Collections.<ResourceRequest>singletonList(r2),
         Collections.<ContainerId>emptyList(),
         Collections.<ContainerId>emptyList(),
-        null, null, null, null);
+        null, null, NULL_UPDATE_REQUESTS);
 
 
     //In this case we do not perform container assignment because we want to
     //In this case we do not perform container assignment because we want to
     //verify re-ordering based on the allocation alone
     //verify re-ordering based on the allocation alone
@@ -3050,7 +3053,8 @@ public class TestCapacityScheduler {
 
 
     Allocation allocate =
     Allocation allocate =
         cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
         cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
-            Collections.<ContainerId> emptyList(), null, null, null, null);
+            Collections.<ContainerId> emptyList(), null, null,
+            NULL_UPDATE_REQUESTS);
 
 
     Assert.assertNotNull(attempt);
     Assert.assertNotNull(attempt);
 
 
@@ -3066,7 +3070,8 @@ public class TestCapacityScheduler {
 
 
     allocate =
     allocate =
         cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
         cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
-            Collections.<ContainerId> emptyList(), null, null, null, null);
+            Collections.<ContainerId> emptyList(), null, null,
+            NULL_UPDATE_REQUESTS);
 
 
     // All resources should be sent as headroom
     // All resources should be sent as headroom
     Assert.assertEquals(newResource, allocate.getResourceLimit());
     Assert.assertEquals(newResource, allocate.getResourceLimit());
@@ -3573,7 +3578,7 @@ public class TestCapacityScheduler {
       cs.allocate(appAttemptId3,
       cs.allocate(appAttemptId3,
           Collections.<ResourceRequest>singletonList(y1Req),
           Collections.<ResourceRequest>singletonList(y1Req),
           Collections.<ContainerId>emptyList(),
           Collections.<ContainerId>emptyList(),
-          null, null, null, null);
+          null, null, NULL_UPDATE_REQUESTS);
       CapacityScheduler.schedule(cs);
       CapacityScheduler.schedule(cs);
     }
     }
     assertEquals("Y1 Used Resource should be 4 GB", 4 * GB,
     assertEquals("Y1 Used Resource should be 4 GB", 4 * GB,
@@ -3587,7 +3592,7 @@ public class TestCapacityScheduler {
       cs.allocate(appAttemptId1,
       cs.allocate(appAttemptId1,
           Collections.<ResourceRequest>singletonList(x1Req),
           Collections.<ResourceRequest>singletonList(x1Req),
           Collections.<ContainerId>emptyList(),
           Collections.<ContainerId>emptyList(),
-          null, null, null, null);
+          null, null, NULL_UPDATE_REQUESTS);
       CapacityScheduler.schedule(cs);
       CapacityScheduler.schedule(cs);
     }
     }
     assertEquals("X1 Used Resource should be 7 GB", 7 * GB,
     assertEquals("X1 Used Resource should be 7 GB", 7 * GB,
@@ -3600,7 +3605,7 @@ public class TestCapacityScheduler {
     cs.allocate(appAttemptId2,
     cs.allocate(appAttemptId2,
         Collections.<ResourceRequest>singletonList(x2Req),
         Collections.<ResourceRequest>singletonList(x2Req),
         Collections.<ContainerId>emptyList(),
         Collections.<ContainerId>emptyList(),
-        null, null, null, null);
+        null, null, NULL_UPDATE_REQUESTS);
     CapacityScheduler.schedule(cs);
     CapacityScheduler.schedule(cs);
     assertEquals("X2 Used Resource should be 0", 0,
     assertEquals("X2 Used Resource should be 0", 0,
         cs.getQueue("x2").getUsedResources().getMemorySize());
         cs.getQueue("x2").getUsedResources().getMemorySize());
@@ -3612,7 +3617,7 @@ public class TestCapacityScheduler {
     cs.allocate(appAttemptId1,
     cs.allocate(appAttemptId1,
         Collections.<ResourceRequest>singletonList(x1Req),
         Collections.<ResourceRequest>singletonList(x1Req),
         Collections.<ContainerId>emptyList(),
         Collections.<ContainerId>emptyList(),
-        null, null, null, null);
+        null, null, NULL_UPDATE_REQUESTS);
     CapacityScheduler.schedule(cs);
     CapacityScheduler.schedule(cs);
     assertEquals("X1 Used Resource should be 7 GB", 7 * GB,
     assertEquals("X1 Used Resource should be 7 GB", 7 * GB,
         cs.getQueue("x1").getUsedResources().getMemorySize());
         cs.getQueue("x1").getUsedResources().getMemorySize());
@@ -3626,7 +3631,7 @@ public class TestCapacityScheduler {
       cs.allocate(appAttemptId3,
       cs.allocate(appAttemptId3,
           Collections.<ResourceRequest>singletonList(y1Req),
           Collections.<ResourceRequest>singletonList(y1Req),
           Collections.<ContainerId>emptyList(),
           Collections.<ContainerId>emptyList(),
-          null, null, null, null);
+          null, null, NULL_UPDATE_REQUESTS);
       CapacityScheduler.schedule(cs);
       CapacityScheduler.schedule(cs);
     }
     }
     assertEquals("P2 Used Resource should be 8 GB", 8 * GB,
     assertEquals("P2 Used Resource should be 8 GB", 8 * GB,
@@ -3685,7 +3690,7 @@ public class TestCapacityScheduler {
     //This will allocate for app1
     //This will allocate for app1
     cs.allocate(appAttemptId1, Collections.<ResourceRequest>singletonList(r1),
     cs.allocate(appAttemptId1, Collections.<ResourceRequest>singletonList(r1),
         Collections.<ContainerId>emptyList(),
         Collections.<ContainerId>emptyList(),
-        null, null, null, null).getContainers().size();
+        null, null, NULL_UPDATE_REQUESTS).getContainers().size();
     CapacityScheduler.schedule(cs);
     CapacityScheduler.schedule(cs);
     ResourceRequest r2 = null;
     ResourceRequest r2 = null;
     for (int i =0; i < 13; i++) {
     for (int i =0; i < 13; i++) {
@@ -3694,7 +3699,7 @@ public class TestCapacityScheduler {
       cs.allocate(appAttemptId2,
       cs.allocate(appAttemptId2,
           Collections.<ResourceRequest>singletonList(r2),
           Collections.<ResourceRequest>singletonList(r2),
           Collections.<ContainerId>emptyList(),
           Collections.<ContainerId>emptyList(),
-          null, null, null, null);
+          null, null, NULL_UPDATE_REQUESTS);
       CapacityScheduler.schedule(cs);
       CapacityScheduler.schedule(cs);
     }
     }
     assertEquals("A Used Resource should be 2 GB", 2 * GB,
     assertEquals("A Used Resource should be 2 GB", 2 * GB,
@@ -3707,11 +3712,11 @@ public class TestCapacityScheduler {
         ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
         ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
     cs.allocate(appAttemptId1, Collections.<ResourceRequest>singletonList(r1),
     cs.allocate(appAttemptId1, Collections.<ResourceRequest>singletonList(r1),
         Collections.<ContainerId>emptyList(),
         Collections.<ContainerId>emptyList(),
-        null, null, null, null).getContainers().size();
+        null, null, NULL_UPDATE_REQUESTS).getContainers().size();
     CapacityScheduler.schedule(cs);
     CapacityScheduler.schedule(cs);
 
 
     cs.allocate(appAttemptId2, Collections.<ResourceRequest>singletonList(r2),
     cs.allocate(appAttemptId2, Collections.<ResourceRequest>singletonList(r2),
-        Collections.<ContainerId>emptyList(), null, null, null, null);
+        Collections.<ContainerId>emptyList(), null, null, NULL_UPDATE_REQUESTS);
     CapacityScheduler.schedule(cs);
     CapacityScheduler.schedule(cs);
     //Check blocked Resource
     //Check blocked Resource
     assertEquals("A Used Resource should be 2 GB", 2 * GB,
     assertEquals("A Used Resource should be 2 GB", 2 * GB,

+ 4 - 1
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

@@ -56,6 +56,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preempti
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
+
+import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -280,7 +282,8 @@ public class TestChildQueueOrder {
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
     ContainerId containerId = BuilderUtils.newContainerId(appAttemptId, 1);
     Container container=TestUtils.getMockContainer(containerId, 
     Container container=TestUtils.getMockContainer(containerId, 
         node_0.getNodeID(), Resources.createResource(1*GB), priority);
         node_0.getNodeID(), Resources.createResource(1*GB), priority);
-    RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
+    RMContainer rmContainer = new RMContainerImpl(container,
+        SchedulerRequestKey.extractFrom(container), appAttemptId,
         node_0.getNodeID(), "user", rmContext);
         node_0.getNodeID(), "user", rmContext);
 
 
     // Assign {1,2,3,4} 1GB containers respectively to queues
     // Assign {1,2,3,4} 1GB containers respectively to queues

+ 6 - 3
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

@@ -921,13 +921,15 @@ public class TestReservations {
     Container container = TestUtils.getMockContainer(containerId,
     Container container = TestUtils.getMockContainer(containerId,
         node_1.getNodeID(), Resources.createResource(2*GB),
         node_1.getNodeID(), Resources.createResource(2*GB),
         priorityMap.getPriority());
         priorityMap.getPriority());
-    RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
+    RMContainer rmContainer = new RMContainerImpl(container,
+        SchedulerRequestKey.extractFrom(container), appAttemptId,
         node_1.getNodeID(), "user", rmContext);
         node_1.getNodeID(), "user", rmContext);
 
 
     Container container_1 = TestUtils.getMockContainer(containerId,
     Container container_1 = TestUtils.getMockContainer(containerId,
         node_0.getNodeID(), Resources.createResource(1*GB),
         node_0.getNodeID(), Resources.createResource(1*GB),
         priorityMap.getPriority());
         priorityMap.getPriority());
-    RMContainer rmContainer_1 = new RMContainerImpl(container_1, appAttemptId,
+    RMContainer rmContainer_1 = new RMContainerImpl(container_1,
+        SchedulerRequestKey.extractFrom(container_1), appAttemptId,
         node_0.getNodeID(), "user", rmContext);
         node_0.getNodeID(), "user", rmContext);
 
 
     // no reserved containers
     // no reserved containers
@@ -994,7 +996,8 @@ public class TestReservations {
     Container container = TestUtils.getMockContainer(containerId,
     Container container = TestUtils.getMockContainer(containerId,
         node_1.getNodeID(), Resources.createResource(2*GB),
         node_1.getNodeID(), Resources.createResource(2*GB),
         priorityMap.getPriority());
         priorityMap.getPriority());
-    RMContainer rmContainer = new RMContainerImpl(container, appAttemptId,
+    RMContainer rmContainer = new RMContainerImpl(container,
+        SchedulerRequestKey.extractFrom(container), appAttemptId,
         node_1.getNodeID(), "user", rmContext);
         node_1.getNodeID(), "user", rmContext);
 
 
     // nothing reserved
     // nothing reserved

+ 11 - 3
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

@@ -42,6 +42,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptMetrics;
+
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
@@ -72,6 +75,8 @@ public class FairSchedulerTestBase {
   public static final float TEST_RESERVATION_THRESHOLD = 0.09f;
   public static final float TEST_RESERVATION_THRESHOLD = 0.09f;
   private static final int SLEEP_DURATION = 10;
   private static final int SLEEP_DURATION = 10;
   private static final int SLEEP_RETRIES = 1000;
   private static final int SLEEP_RETRIES = 1000;
+  final static ContainerUpdates NULL_UPDATE_REQUESTS =
+      new ContainerUpdates();
 
 
   /**
   /**
    * The list of nodes added to the cluster using the {@link #addNode} method.
    * The list of nodes added to the cluster using the {@link #addNode} method.
@@ -183,7 +188,8 @@ public class FairSchedulerTestBase {
     resourceManager.getRMContext().getRMApps()
     resourceManager.getRMContext().getRMApps()
         .put(id.getApplicationId(), rmApp);
         .put(id.getApplicationId(), rmApp);
 
 
-    scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null, null, null);
+    scheduler.allocate(id, ask, new ArrayList<ContainerId>(),
+        null, null, NULL_UPDATE_REQUESTS);
     return id;
     return id;
   }
   }
   
   
@@ -209,7 +215,8 @@ public class FairSchedulerTestBase {
     resourceManager.getRMContext().getRMApps()
     resourceManager.getRMContext().getRMApps()
         .put(id.getApplicationId(), rmApp);
         .put(id.getApplicationId(), rmApp);
 
 
-    scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null, null, null);
+    scheduler.allocate(id, ask, new ArrayList<ContainerId>(),
+        null, null, NULL_UPDATE_REQUESTS);
     return id;
     return id;
   }
   }
 
 
@@ -231,7 +238,8 @@ public class FairSchedulerTestBase {
       ResourceRequest request, ApplicationAttemptId attId) {
       ResourceRequest request, ApplicationAttemptId attId) {
     List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
     List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
     ask.add(request);
     ask.add(request);
-    scheduler.allocate(attId, ask,  new ArrayList<ContainerId>(), null, null, null, null);
+    scheduler.allocate(attId, ask,  new ArrayList<ContainerId>(),
+        null, null, NULL_UPDATE_REQUESTS);
   }
   }
 
 
   protected void createApplicationWithAMResource(ApplicationAttemptId attId,
   protected void createApplicationWithAMResource(ApplicationAttemptId attId,

+ 5 - 4
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

@@ -119,7 +119,8 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
     List<ResourceRequest> ask = new ArrayList<>();
     List<ResourceRequest> ask = new ArrayList<>();
     ask.add(createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true));
     ask.add(createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true));
     scheduler.allocate(
     scheduler.allocate(
-        appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
+        appAttemptId, ask, new ArrayList<ContainerId>(),
+        null, null, NULL_UPDATE_REQUESTS);
     FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
     FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
 
 
     triggerSchedulingAttempt();
     triggerSchedulingAttempt();
@@ -157,7 +158,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
         createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
         createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
     ask.add(request);
     ask.add(request);
     scheduler.allocate(appAttemptId, ask,
     scheduler.allocate(appAttemptId, ask,
-        new ArrayList<ContainerId>(), null, null, null, null);
+        new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS);
     triggerSchedulingAttempt();
     triggerSchedulingAttempt();
 
 
     FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
     FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
@@ -169,7 +170,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
     ask.clear();
     ask.clear();
     ask.add(request);
     ask.add(request);
     scheduler.allocate(appAttemptId, ask,
     scheduler.allocate(appAttemptId, ask,
-        new ArrayList<ContainerId>(), null, null, null, null);
+        new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS);
     triggerSchedulingAttempt();
     triggerSchedulingAttempt();
 
 
     checkAppConsumption(app, Resources.createResource(2048,2));
     checkAppConsumption(app, Resources.createResource(2048,2));
@@ -335,7 +336,7 @@ public class TestContinuousScheduling extends FairSchedulerTestBase {
     ask1.add(request1);
     ask1.add(request1);
     ask1.add(request2);
     ask1.add(request2);
     scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null,
     scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null,
-        null, null);
+        NULL_UPDATE_REQUESTS);
 
 
     NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
     NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
     scheduler.handle(nodeEvent1);
     scheduler.handle(nodeEvent1);

+ 21 - 12
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

@@ -94,6 +94,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
+
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
@@ -127,6 +130,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
   private final int GB = 1024;
   private final int GB = 1024;
   private final static String ALLOC_FILE =
   private final static String ALLOC_FILE =
       new File(TEST_DIR, "test-queues").getAbsolutePath();
       new File(TEST_DIR, "test-queues").getAbsolutePath();
+  private final static ContainerUpdates NULL_UPDATE_REQUESTS =
+      new ContainerUpdates();
 
 
   @Before
   @Before
   public void setUp() throws IOException {
   public void setUp() throws IOException {
@@ -1260,7 +1265,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     asks.add(createResourceRequest(2048, node2.getRackName(), 1, 1, false));
     asks.add(createResourceRequest(2048, node2.getRackName(), 1, 1, false));
 
 
     scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
     scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
-            null, null, null);
+            null, NULL_UPDATE_REQUESTS);
 
 
     ApplicationAttemptId attId = createSchedulingRequest(2048, "queue1", "user1", 1);
     ApplicationAttemptId attId = createSchedulingRequest(2048, "queue1", "user1", 1);
     scheduler.update();
     scheduler.update();
@@ -2114,7 +2119,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     ResourceRequest request1 =
     ResourceRequest request1 =
         createResourceRequest(minReqSize * 2, ResourceRequest.ANY, 1, 1, true);
         createResourceRequest(minReqSize * 2, ResourceRequest.ANY, 1, 1, true);
     ask1.add(request1);
     ask1.add(request1);
-    scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null, null, null);
+    scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(),
+        null, null, NULL_UPDATE_REQUESTS);
 
 
     // Second ask, queue2 requests 1 large.
     // Second ask, queue2 requests 1 large.
     List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
     List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
@@ -2124,7 +2130,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
             ResourceRequest.ANY, 1, 1, false);
             ResourceRequest.ANY, 1, 1, false);
     ask2.add(request2);
     ask2.add(request2);
     ask2.add(request3);
     ask2.add(request3);
-    scheduler.allocate(id21, ask2, new ArrayList<ContainerId>(), null, null, null, null);
+    scheduler.allocate(id21, ask2, new ArrayList<ContainerId>(),
+        null, null, NULL_UPDATE_REQUESTS);
 
 
     // Third ask, queue2 requests 2 small (minReqSize).
     // Third ask, queue2 requests 2 small (minReqSize).
     List<ResourceRequest> ask3 = new ArrayList<ResourceRequest>();
     List<ResourceRequest> ask3 = new ArrayList<ResourceRequest>();
@@ -2134,7 +2141,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
             ResourceRequest.ANY, 2, 2, true);
             ResourceRequest.ANY, 2, 2, true);
     ask3.add(request4);
     ask3.add(request4);
     ask3.add(request5);
     ask3.add(request5);
-    scheduler.allocate(id22, ask3, new ArrayList<ContainerId>(), null, null, null, null);
+    scheduler.allocate(id22, ask3, new ArrayList<ContainerId>(),
+        null, null, NULL_UPDATE_REQUESTS);
 
 
     scheduler.update();
     scheduler.update();
 
 
@@ -2661,7 +2669,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         app1.getLiveContainers().iterator().next().getContainerId();
         app1.getLiveContainers().iterator().next().getContainerId();
     scheduler.allocate(app1.getApplicationAttemptId(),
     scheduler.allocate(app1.getApplicationAttemptId(),
         new ArrayList<ResourceRequest>(),
         new ArrayList<ResourceRequest>(),
-        Arrays.asList(containerId), null, null, null, null);
+        Arrays.asList(containerId), null, null, new ContainerUpdates());
 
 
     // Trigger allocation for app2
     // Trigger allocation for app2
     scheduler.handle(updateEvent);
     scheduler.handle(updateEvent);
@@ -2747,7 +2755,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     asks.add(createResourceRequest(1024, ResourceRequest.ANY, 1, 2, true));
     asks.add(createResourceRequest(1024, ResourceRequest.ANY, 1, 2, true));
 
 
     scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
     scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
-        null, null, null);
+        null, NULL_UPDATE_REQUESTS);
     
     
     // node 1 checks in
     // node 1 checks in
     scheduler.update();
     scheduler.update();
@@ -3193,7 +3201,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         createResourceRequest(1024, node1.getHostName(), 1, 0, true),
         createResourceRequest(1024, node1.getHostName(), 1, 0, true),
         createResourceRequest(1024, "rack1", 1, 0, true),
         createResourceRequest(1024, "rack1", 1, 0, true),
         createResourceRequest(1024, ResourceRequest.ANY, 1, 1, true));
         createResourceRequest(1024, ResourceRequest.ANY, 1, 1, true));
-    scheduler.allocate(attId1, update, new ArrayList<ContainerId>(), null, null, null, null);
+    scheduler.allocate(attId1, update, new ArrayList<ContainerId>(),
+        null, null, NULL_UPDATE_REQUESTS);
     
     
     // then node2 should get the container
     // then node2 should get the container
     scheduler.handle(node2UpdateEvent);
     scheduler.handle(node2UpdateEvent);
@@ -4330,7 +4339,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
 
 
     ask1.add(request1);
     ask1.add(request1);
     scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null,
     scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null,
-        null, null, null);
+        null, NULL_UPDATE_REQUESTS);
 
 
     String hostName = "127.0.0.1";
     String hostName = "127.0.0.1";
     RMNode node1 = MockNodes.newNodeInfo(1,
     RMNode node1 = MockNodes.newNodeInfo(1,
@@ -4406,11 +4415,11 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     // Verify the blacklist can be updated independent of requesting containers
     // Verify the blacklist can be updated independent of requesting containers
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(),
         Collections.<ContainerId>emptyList(),
-        Collections.singletonList(host), null, null, null);
+        Collections.singletonList(host), null, NULL_UPDATE_REQUESTS);
     assertTrue(app.isPlaceBlacklisted(host));
     assertTrue(app.isPlaceBlacklisted(host));
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
     scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
         Collections.<ContainerId>emptyList(), null,
         Collections.<ContainerId>emptyList(), null,
-        Collections.singletonList(host), null, null);
+        Collections.singletonList(host), NULL_UPDATE_REQUESTS);
     assertFalse(scheduler.getSchedulerApp(appAttemptId)
     assertFalse(scheduler.getSchedulerApp(appAttemptId)
         .isPlaceBlacklisted(host));
         .isPlaceBlacklisted(host));
 
 
@@ -4420,7 +4429,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     // Verify a container does not actually get placed on the blacklisted host
     // Verify a container does not actually get placed on the blacklisted host
     scheduler.allocate(appAttemptId, update,
     scheduler.allocate(appAttemptId, update,
         Collections.<ContainerId>emptyList(),
         Collections.<ContainerId>emptyList(),
-        Collections.singletonList(host), null, null, null);
+        Collections.singletonList(host), null, NULL_UPDATE_REQUESTS);
     assertTrue(app.isPlaceBlacklisted(host));
     assertTrue(app.isPlaceBlacklisted(host));
     scheduler.update();
     scheduler.update();
     scheduler.handle(updateEvent);
     scheduler.handle(updateEvent);
@@ -4430,7 +4439,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     // Verify a container gets placed on the empty blacklist
     // Verify a container gets placed on the empty blacklist
     scheduler.allocate(appAttemptId, update,
     scheduler.allocate(appAttemptId, update,
         Collections.<ContainerId>emptyList(), null,
         Collections.<ContainerId>emptyList(), null,
-        Collections.singletonList(host), null, null);
+        Collections.singletonList(host), NULL_UPDATE_REQUESTS);
     assertFalse(app.isPlaceBlacklisted(host));
     assertFalse(app.isPlaceBlacklisted(host));
     createSchedulingRequest(GB, "root.default", "user", 1);
     createSchedulingRequest(GB, "root.default", "user", 1);
     scheduler.update();
     scheduler.update();

+ 31 - 15
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

@@ -83,6 +83,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
@@ -119,7 +120,10 @@ public class TestFifoScheduler {
   private static Configuration conf;
   private static Configuration conf;
   private static final RecordFactory recordFactory = 
   private static final RecordFactory recordFactory = 
       RecordFactoryProvider.getRecordFactory(null);
       RecordFactoryProvider.getRecordFactory(null);
-  
+
+  private final static ContainerUpdates NULL_UPDATE_REQUESTS =
+      new ContainerUpdates();
+
   @Before
   @Before
   public void setUp() throws Exception {
   public void setUp() throws Exception {
     conf = new Configuration();
     conf = new Configuration();
@@ -274,7 +278,8 @@ public class TestFifoScheduler {
     ask.add(nodeLocal);
     ask.add(nodeLocal);
     ask.add(rackLocal);
     ask.add(rackLocal);
     ask.add(any);
     ask.add(any);
-    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
+    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(),
+        null, null, NULL_UPDATE_REQUESTS);
 
 
     NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0);
     NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0);
 
 
@@ -368,7 +373,8 @@ public class TestFifoScheduler {
     ask.add(nodeLocal);
     ask.add(nodeLocal);
     ask.add(rackLocal);
     ask.add(rackLocal);
     ask.add(any);
     ask.add(any);
-    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null, null, null);
+    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(),
+        null, null, NULL_UPDATE_REQUESTS);
 
 
     // Before the node update event, there are one local request
     // Before the node update event, there are one local request
     Assert.assertEquals(1, nodeLocal.getNumContainers());
     Assert.assertEquals(1, nodeLocal.getNumContainers());
@@ -944,7 +950,7 @@ public class TestFifoScheduler {
         ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1,
         ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1,
         RMNodeLabelsManager.NO_LABEL));
         RMNodeLabelsManager.NO_LABEL));
     fs.allocate(appAttemptId1, ask1, emptyId,
     fs.allocate(appAttemptId1, ask1, emptyId,
-        Collections.singletonList(host_1_0), null, null, null);
+        Collections.singletonList(host_1_0), null, NULL_UPDATE_REQUESTS);
 
 
     // Trigger container assignment
     // Trigger container assignment
     fs.handle(new NodeUpdateSchedulerEvent(n3));
     fs.handle(new NodeUpdateSchedulerEvent(n3));
@@ -952,14 +958,16 @@ public class TestFifoScheduler {
     // Get the allocation for the application and verify no allocation on
     // Get the allocation for the application and verify no allocation on
     // blacklist node
     // blacklist node
     Allocation allocation1 =
     Allocation allocation1 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+            null, null, NULL_UPDATE_REQUESTS);
 
 
     Assert.assertEquals("allocation1", 0, allocation1.getContainers().size());
     Assert.assertEquals("allocation1", 0, allocation1.getContainers().size());
 
 
     // verify host_1_1 can get allocated as not in blacklist
     // verify host_1_1 can get allocated as not in blacklist
     fs.handle(new NodeUpdateSchedulerEvent(n4));
     fs.handle(new NodeUpdateSchedulerEvent(n4));
     Allocation allocation2 =
     Allocation allocation2 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+            null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("allocation2", 1, allocation2.getContainers().size());
     Assert.assertEquals("allocation2", 1, allocation2.getContainers().size());
     List<Container> containerList = allocation2.getContainers();
     List<Container> containerList = allocation2.getContainers();
     for (Container container : containerList) {
     for (Container container : containerList) {
@@ -974,29 +982,33 @@ public class TestFifoScheduler {
     ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
     ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
         ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
         ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
     fs.allocate(appAttemptId1, ask2, emptyId,
     fs.allocate(appAttemptId1, ask2, emptyId,
-        Collections.singletonList("rack0"), null, null, null);
+        Collections.singletonList("rack0"), null, NULL_UPDATE_REQUESTS);
 
 
     // verify n1 is not qualified to be allocated
     // verify n1 is not qualified to be allocated
     fs.handle(new NodeUpdateSchedulerEvent(n1));
     fs.handle(new NodeUpdateSchedulerEvent(n1));
     Allocation allocation3 =
     Allocation allocation3 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+            null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("allocation3", 0, allocation3.getContainers().size());
     Assert.assertEquals("allocation3", 0, allocation3.getContainers().size());
 
 
     // verify n2 is not qualified to be allocated
     // verify n2 is not qualified to be allocated
     fs.handle(new NodeUpdateSchedulerEvent(n2));
     fs.handle(new NodeUpdateSchedulerEvent(n2));
     Allocation allocation4 =
     Allocation allocation4 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+            null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("allocation4", 0, allocation4.getContainers().size());
     Assert.assertEquals("allocation4", 0, allocation4.getContainers().size());
 
 
     // verify n3 is not qualified to be allocated
     // verify n3 is not qualified to be allocated
     fs.handle(new NodeUpdateSchedulerEvent(n3));
     fs.handle(new NodeUpdateSchedulerEvent(n3));
     Allocation allocation5 =
     Allocation allocation5 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+            null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("allocation5", 0, allocation5.getContainers().size());
     Assert.assertEquals("allocation5", 0, allocation5.getContainers().size());
 
 
     fs.handle(new NodeUpdateSchedulerEvent(n4));
     fs.handle(new NodeUpdateSchedulerEvent(n4));
     Allocation allocation6 =
     Allocation allocation6 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+            null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("allocation6", 1, allocation6.getContainers().size());
     Assert.assertEquals("allocation6", 1, allocation6.getContainers().size());
 
 
     containerList = allocation6.getContainers();
     containerList = allocation6.getContainers();
@@ -1055,25 +1067,29 @@ public class TestFifoScheduler {
     List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
     List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
     ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
     ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
         ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
         ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
-    fs.allocate(appAttemptId1, ask1, emptyId, null, null, null, null);
+    fs.allocate(appAttemptId1, ask1, emptyId,
+        null, null, NULL_UPDATE_REQUESTS);
 
 
     // Ask for a 2 GB container for app 2
     // Ask for a 2 GB container for app 2
     List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
     List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
     ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
     ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
         ResourceRequest.ANY, BuilderUtils.newResource(2 * GB, 1), 1));
         ResourceRequest.ANY, BuilderUtils.newResource(2 * GB, 1), 1));
-    fs.allocate(appAttemptId2, ask2, emptyId, null, null, null, null);
+    fs.allocate(appAttemptId2, ask2, emptyId,
+        null, null, NULL_UPDATE_REQUESTS);
 
 
     // Trigger container assignment
     // Trigger container assignment
     fs.handle(new NodeUpdateSchedulerEvent(n1));
     fs.handle(new NodeUpdateSchedulerEvent(n1));
 
 
     // Get the allocation for the applications and verify headroom
     // Get the allocation for the applications and verify headroom
     Allocation allocation1 =
     Allocation allocation1 =
-        fs.allocate(appAttemptId1, emptyAsk, emptyId, null, null, null, null);
+        fs.allocate(appAttemptId1, emptyAsk, emptyId,
+            null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("Allocation headroom", 1 * GB, allocation1
     Assert.assertEquals("Allocation headroom", 1 * GB, allocation1
         .getResourceLimit().getMemorySize());
         .getResourceLimit().getMemorySize());
 
 
     Allocation allocation2 =
     Allocation allocation2 =
-        fs.allocate(appAttemptId2, emptyAsk, emptyId, null, null, null, null);
+        fs.allocate(appAttemptId2, emptyAsk, emptyId,
+            null, null, NULL_UPDATE_REQUESTS);
     Assert.assertEquals("Allocation headroom", 1 * GB, allocation2
     Assert.assertEquals("Allocation headroom", 1 * GB, allocation2
         .getResourceLimit().getMemorySize());
         .getResourceLimit().getMemorySize());