瀏覽代碼

YARN-5221. Expose UpdateResourceRequest API to allow AM to request for change in container properties. (asuresh)

(cherry picked from commit d6d9cff21b7b6141ed88359652cf22e8973c0661)
(cherry picked from commit b279f42d79175bef6529dc1ac4216198a3aaee4d)
Arun Suresh 8 年之前
父節點
當前提交
979b29a03c
共有 72 個文件被更改,包括 2059 次插入1058 次删除
  1. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
  2. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
  3. 3 3
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ResourceSchedulerWrapper.java
  4. 19 39
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
  5. 40 30
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
  6. 22 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
  7. 0 117
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceChangeRequest.java
  8. 45 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerUpdateType.java
  9. 119 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/UpdateContainerError.java
  10. 171 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/UpdateContainerRequest.java
  11. 118 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/UpdatedContainer.java
  12. 1 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  13. 25 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
  14. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  15. 6 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
  16. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
  17. 44 35
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
  18. 35 20
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java
  19. 7 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
  20. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
  21. 40 111
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
  22. 139 53
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
  23. 13 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java
  24. 0 141
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceChangeRequestPBImpl.java
  25. 90 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
  26. 125 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/UpdateContainerErrorPBImpl.java
  27. 166 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/UpdateContainerRequestPBImpl.java
  28. 117 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/UpdatedContainerPBImpl.java
  29. 24 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java
  30. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/yarn_security_token.proto
  31. 12 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
  32. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
  33. 12 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java
  34. 13 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NMContainerStatusPBImpl.java
  35. 9 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
  36. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
  37. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestProtocolRecords.java
  38. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestRegisterNodeManagerRequest.java
  39. 9 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  40. 12 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
  41. 40 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
  42. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
  43. 22 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
  44. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
  45. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
  46. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
  47. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
  48. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
  49. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
  50. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
  51. 40 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
  52. 181 157
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
  53. 7 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
  54. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java
  55. 8 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
  56. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
  57. 4 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
  58. 3 3
      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
  59. 3 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
  60. 33 29
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
  61. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
  62. 5 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java
  63. 38 48
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java
  64. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
  65. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
  66. 26 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
  67. 7 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
  68. 81 53
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java
  69. 50 26
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestIncreaseAllocationExpirer.java
  70. 9 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java
  71. 0 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYarnClusterNodeUtilization.java
  72. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/proto/test_token.proto

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -253,8 +254,7 @@ public class TestLocalContainerAllocator {
           Resources.none(), null, 1, null,
           Collections.<NMToken>emptyList(),
           yarnToken,
-          Collections.<Container>emptyList(),
-          Collections.<Container>emptyList());
+          Collections.<UpdatedContainer>emptyList());
       response.setApplicationPriority(Priority.newInstance(0));
       return response;
     }

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

@@ -97,7 +97,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
@@ -106,6 +105,7 @@ import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
@@ -1701,8 +1701,8 @@ public class TestRMContainerAllocator {
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
         List<ContainerId> release, List<String> blacklistAdditions,
         List<String> blacklistRemovals,
-        List<ContainerResourceChangeRequest> increaseRequests,
-        List<ContainerResourceChangeRequest> decreaseRequests) {
+        List<UpdateContainerRequest> increaseRequests,
+        List<UpdateContainerRequest> decreaseRequests) {
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       for (ResourceRequest req : ask) {
         ResourceRequest reqCopy = ResourceRequest.newInstance(req
@@ -1748,8 +1748,8 @@ public class TestRMContainerAllocator {
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
         List<ContainerId> release, List<String> blacklistAdditions,
         List<String> blacklistRemovals,
-        List<ContainerResourceChangeRequest> increaseRequest,
-        List<ContainerResourceChangeRequest> decreaseRequests) {
+        List<UpdateContainerRequest> increaseRequest,
+        List<UpdateContainerRequest> decreaseRequests) {
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       for (ResourceRequest req : ask) {
         ResourceRequest reqCopy = ResourceRequest.newInstance(req

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

@@ -51,7 +51,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -60,6 +59,7 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 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.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
@@ -207,8 +207,8 @@ public class ResourceSchedulerWrapper
   public Allocation allocate(ApplicationAttemptId attemptId,
       List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
       List<String> strings, List<String> strings2,
-      List<ContainerResourceChangeRequest> increaseRequests,
-      List<ContainerResourceChangeRequest> decreaseRequests) {
+      List<UpdateContainerRequest> increaseRequests,
+      List<UpdateContainerRequest> decreaseRequests) {
     if (metricsON) {
       final Timer.Context context = schedulerAllocateTimer.time();
       Allocation allocation = null;

+ 19 - 39
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java

@@ -27,8 +27,8 @@ import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
@@ -48,13 +48,8 @@ import org.apache.hadoop.yarn.util.Records;
  *     A list of unused {@link Container} which are being returned.
  *   </li>
  *   <li>
- *     A list of {@link ContainerResourceChangeRequest} to inform
- *     the <code>ResourceManager</code> about the resource increase
- *     requirements of running containers.
- *   </li>
- *   <li>
- *     A list of {@link ContainerResourceChangeRequest} to inform
- *     the <code>ResourceManager</code> about the resource decrease
+ *     A list of {@link UpdateContainerRequest} to inform
+ *     the <code>ResourceManager</code> about the change in
  *     requirements of running containers.
  *   </li>
  * </ul>
@@ -72,25 +67,23 @@ public abstract class AllocateRequest {
       List<ContainerId> containersToBeReleased,
       ResourceBlacklistRequest resourceBlacklistRequest) {
     return newInstance(responseID, appProgress, resourceAsk,
-        containersToBeReleased, resourceBlacklistRequest, null, null);
+        containersToBeReleased, resourceBlacklistRequest, null);
   }
   
   @Public
-  @Stable
+  @Unstable
   public static AllocateRequest newInstance(int responseID, float appProgress,
       List<ResourceRequest> resourceAsk,
       List<ContainerId> containersToBeReleased,
       ResourceBlacklistRequest resourceBlacklistRequest,
-      List<ContainerResourceChangeRequest> increaseRequests,
-      List<ContainerResourceChangeRequest> decreaseRequests) {
+      List<UpdateContainerRequest> updateRequests) {
     AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
     allocateRequest.setResponseId(responseID);
     allocateRequest.setProgress(appProgress);
     allocateRequest.setAskList(resourceAsk);
     allocateRequest.setReleaseList(containersToBeReleased);
     allocateRequest.setResourceBlacklistRequest(resourceBlacklistRequest);
-    allocateRequest.setIncreaseRequests(increaseRequests);
-    allocateRequest.setDecreaseRequests(decreaseRequests);
+    allocateRequest.setUpdateRequests(updateRequests);
     return allocateRequest;
   }
   
@@ -197,38 +190,25 @@ public abstract class AllocateRequest {
       ResourceBlacklistRequest resourceBlacklistRequest);
   
   /**
-   * Get the list of container resource increase requests being sent by the
-   * <code>ApplicationMaster</code>.
-   */
-  @Public
-  @Unstable
-  public abstract List<ContainerResourceChangeRequest> getIncreaseRequests();
-
-  /**
-   * Set the list of container resource increase requests to inform the
-   * <code>ResourceManager</code> about the containers whose resources need
-   * to be increased.
-   */
-  @Public
-  @Unstable
-  public abstract void setIncreaseRequests(
-      List<ContainerResourceChangeRequest> increaseRequests);
-
-  /**
-   * Get the list of container resource decrease requests being sent by the
+   * Get the list of container update requests being sent by the
    * <code>ApplicationMaster</code>.
+   * @return list of {@link UpdateContainerRequest}
+   *         being sent by the
+   *         <code>ApplicationMaster</code>.
    */
   @Public
   @Unstable
-  public abstract List<ContainerResourceChangeRequest> getDecreaseRequests();
+  public abstract List<UpdateContainerRequest> getUpdateRequests();
 
   /**
-   * Set the list of container resource decrease requests to inform the
-   * <code>ResourceManager</code> about the containers whose resources need
-   * to be decreased.
+   * Set the list of container update requests to inform the
+   * <code>ResourceManager</code> about the containers that need to be
+   * updated.
+   * @param updateRequests list of <code>UpdateContainerRequest</code> for
+   *                       containers to be updated
    */
   @Public
   @Unstable
-  public abstract void setDecreaseRequests(
-      List<ContainerResourceChangeRequest> decreaseRequests);
+  public abstract void setUpdateRequests(
+      List<UpdateContainerRequest> updateRequests);
 }

+ 40 - 30
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.api.protocolrecords;
 
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -35,6 +36,8 @@ import org.apache.hadoop.yarn.api.records.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.UpdateContainerError;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
@@ -95,19 +98,17 @@ public abstract class AllocateResponse {
   }
 
   @Public
-  @Stable
+  @Unstable
   public static AllocateResponse newInstance(int responseId,
       List<ContainerStatus> completedContainers,
       List<Container> allocatedContainers, List<NodeReport> updatedNodes,
       Resource availResources, AMCommand command, int numClusterNodes,
       PreemptionMessage preempt, List<NMToken> nmTokens,
-      List<Container> increasedContainers,
-      List<Container> decreasedContainers) {
+      List<UpdatedContainer> updatedContainers) {
     AllocateResponse response = newInstance(responseId, completedContainers,
         allocatedContainers, updatedNodes, availResources, command,
         numClusterNodes, preempt, nmTokens);
-    response.setIncreasedContainers(increasedContainers);
-    response.setDecreasedContainers(decreasedContainers);
+    response.setUpdatedContainers(updatedContainers);
     return response;
   }
 
@@ -118,12 +119,11 @@ public abstract class AllocateResponse {
       List<Container> allocatedContainers, List<NodeReport> updatedNodes,
       Resource availResources, AMCommand command, int numClusterNodes,
       PreemptionMessage preempt, List<NMToken> nmTokens, Token amRMToken,
-      List<Container> increasedContainers,
-      List<Container> decreasedContainers) {
+      List<UpdatedContainer> updatedContainers) {
     AllocateResponse response =
         newInstance(responseId, completedContainers, allocatedContainers,
           updatedNodes, availResources, command, numClusterNodes, preempt,
-          nmTokens, increasedContainers, decreasedContainers);
+          nmTokens, updatedContainers);
     response.setAMRMToken(amRMToken);
     return response;
   }
@@ -270,38 +270,23 @@ public abstract class AllocateResponse {
   public abstract void setNMTokens(List<NMToken> nmTokens);
   
   /**
-   * Get the list of newly increased containers by
-   * <code>ResourceManager</code>.
-   */
-  @Public
-  @Unstable
-  public abstract List<Container> getIncreasedContainers();
-
-  /**
-   * Set the list of newly increased containers by
-   * <code>ResourceManager</code>.
-   */
-  @Private
-  @Unstable
-  public abstract void setIncreasedContainers(
-      List<Container> increasedContainers);
-
-  /**
-   * Get the list of newly decreased containers by
+   * Get the list of newly updated containers by
    * <code>ResourceManager</code>.
    */
   @Public
   @Unstable
-  public abstract List<Container> getDecreasedContainers();
+  public abstract List<UpdatedContainer> getUpdatedContainers();
 
   /**
-   * Set the list of newly decreased containers by
+   * Set the list of newly updated containers by
    * <code>ResourceManager</code>.
+   *
+   * @param updatedContainers List of Updated Containers.
    */
   @Private
   @Unstable
-  public abstract void setDecreasedContainers(
-      List<Container> decreasedContainers);
+  public abstract void setUpdatedContainers(
+      List<UpdatedContainer> updatedContainers);
 
   /**
    * The AMRMToken that belong to this attempt
@@ -328,4 +313,29 @@ public abstract class AllocateResponse {
   @Private
   @Unstable
   public abstract void setApplicationPriority(Priority priority);
+
+  /**
+   * Get the list of container update errors to inform the
+   * Application Master about the container updates that could not be
+   * satisfied due to error.
+   *
+   * @return List of Update Container Errors.
+   */
+  @Public
+  @Unstable
+  public List<UpdateContainerError> getUpdateErrors() {
+    return new ArrayList<>();
+  }
+
+  /**
+   * Set the list of container update errors to inform the
+   * Application Master about the container updates that could not be
+   * satisfied due to error.
+   * @param updateErrors list of <code>UpdateContainerError</code> for
+   *                       containers updates requests that were in error
+   */
+  @Public
+  @Unstable
+  public void setUpdateErrors(List<UpdateContainerError> updateErrors) {
+  }
 }

+ 22 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java

@@ -163,4 +163,26 @@ public abstract class Container implements Comparable<Container> {
   @Private
   @Unstable
   public abstract void setContainerToken(Token containerToken);
+
+  /**
+   * Get the version of this container. The version will be incremented when
+   * a container is updated.
+   *
+   * @return version of this container.
+   */
+  @Private
+  @Unstable
+  public int getVersion() {
+    return 0;
+  }
+
+  /**
+   * Set the version of this container.
+   * @param version of this container.
+   */
+  @Private
+  @Unstable
+  public void setVersion(int version) {
+    throw new UnsupportedOperationException();
+  }
 }

+ 0 - 117
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceChangeRequest.java

@@ -1,117 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api.records;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
-import org.apache.hadoop.yarn.util.Records;
-
-/**
- * {@code ContainerResourceChangeRequest} represents the request made by an
- * application to the {@code ResourceManager} to change resource allocation of
- * a running {@code Container}.
- * <p>
- * It includes:
- * <ul>
- *   <li>{@link ContainerId} for the container.</li>
- *   <li>
- *     {@link Resource} capability of the container after the resource change
- *     is completed.
- *   </li>
- * </ul>
- *
- * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
- */
-@Public
-@Unstable
-public abstract class ContainerResourceChangeRequest {
-
-  @Public
-  @Unstable
-  public static ContainerResourceChangeRequest newInstance(
-      ContainerId existingContainerId, Resource targetCapability) {
-    ContainerResourceChangeRequest context = Records
-        .newRecord(ContainerResourceChangeRequest.class);
-    context.setContainerId(existingContainerId);
-    context.setCapability(targetCapability);
-    return context;
-  }
-
-  /**
-   * Get the <code>ContainerId</code> of the container.
-   * @return <code>ContainerId</code> of the container
-   */
-  @Public
-  @Unstable
-  public abstract ContainerId getContainerId();
-
-  /**
-   * Set the <code>ContainerId</code> of the container.
-   * @param containerId <code>ContainerId</code> of the container
-   */
-  @Public
-  @Unstable
-  public abstract void setContainerId(ContainerId containerId);
-
-  /**
-   * Get the <code>Resource</code> capability of the container.
-   * @return <code>Resource</code> capability of the container
-   */
-  @Public
-  @Unstable
-  public abstract Resource getCapability();
-
-  /**
-   * Set the <code>Resource</code> capability of the container.
-   * @param capability <code>Resource</code> capability of the container
-   */
-  @Public
-  @Unstable
-  public abstract void setCapability(Resource capability);
-
-  @Override
-  public int hashCode() {
-    return getCapability().hashCode() + getContainerId().hashCode();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other instanceof ContainerResourceChangeRequest) {
-      ContainerResourceChangeRequest ctx =
-          (ContainerResourceChangeRequest) other;
-
-      if (getContainerId() == null && ctx.getContainerId() != null) {
-        return false;
-      } else if (!getContainerId().equals(ctx.getContainerId())) {
-        return false;
-      }
-
-      if (getCapability() == null && ctx.getCapability() != null) {
-        return false;
-      } else if (!getCapability().equals(ctx.getCapability())) {
-        return false;
-      }
-
-      return true;
-    } else {
-      return false;
-    }
-  }
-}

+ 45 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerUpdateType.java

@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Encodes the type of Container Update.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public enum ContainerUpdateType {
+
+  /**
+   * Resource increase.
+   */
+  INCREASE_RESOURCE,
+
+  /**
+   * Resource decrease.
+   */
+  DECREASE_RESOURCE,
+
+  /**
+   * Execution Type change.
+   */
+  UPDATE_EXECUTION_TYPE
+}

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

@@ -0,0 +1,119 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@code UpdateContainerError} is used by the Scheduler to notify the
+ * ApplicationMaster of an UpdateContainerRequest it cannot satisfy due to
+ * an error in the request. It includes the update request as well as
+ * a reason for why the request was not satisfiable.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class UpdateContainerError {
+
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public static UpdateContainerError newInstance(String reason,
+      UpdateContainerRequest updateContainerRequest) {
+    UpdateContainerError error = Records.newRecord(UpdateContainerError.class);
+    error.setReason(reason);
+    error.setUpdateContainerRequest(updateContainerRequest);
+    return error;
+  }
+
+  /**
+   * Get reason why the update request was not satisfiable.
+   * @return Reason
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract String getReason();
+
+  /**
+   * Set reason why the update request was not satisfiable.
+   * @param reason Reason
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract void setReason(String reason);
+
+  /**
+   * Get the {@code UpdateContainerRequest} that was not satisfiable.
+   * @return UpdateContainerRequest
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract UpdateContainerRequest getUpdateContainerRequest();
+
+  /**
+   * Set the {@code UpdateContainerRequest} that was not satisfiable.
+   * @param updateContainerRequest Update Container Request
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract void setUpdateContainerRequest(
+      UpdateContainerRequest updateContainerRequest);
+
+  @Override
+  public int hashCode() {
+    final int prime = 2153;
+    int result = 2459;
+    String reason = getReason();
+    UpdateContainerRequest updateReq = getUpdateContainerRequest();
+    result = prime * result + ((reason == null) ? 0 : reason.hashCode());
+    result = prime * result + ((updateReq == null) ? 0 : updateReq.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    UpdateContainerError other = (UpdateContainerError) obj;
+    String reason = getReason();
+    if (reason == null) {
+      if (other.getReason() != null) {
+        return false;
+      }
+    } else if (!reason.equals(other.getReason())) {
+      return false;
+    }
+    UpdateContainerRequest req = getUpdateContainerRequest();
+    if (req == null) {
+      if (other.getUpdateContainerRequest() != null) {
+        return false;
+      }
+    } else if (!req.equals(other.getUpdateContainerRequest())) {
+      return false;
+    }
+    return true;
+  }
+}

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

@@ -0,0 +1,171 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * {@code UpdateContainerRequest} represents the request made by an
+ * application to the {@code ResourceManager} to update an attribute of a
+ * {@code Container} such as its Resource allocation or (@code ExecutionType}
+ * <p>
+ * It includes:
+ * <ul>
+ *   <li>version for the container.</li>
+ *   <li>{@link ContainerId} for the container.</li>
+ *   <li>
+ *     {@link Resource} capability of the container after the update request
+ *     is completed.
+ *   </li>
+ * </ul>
+ *
+ * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class UpdateContainerRequest {
+
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public static UpdateContainerRequest newInstance(int version,
+      ContainerId containerId, ContainerUpdateType updateType,
+      Resource targetCapability) {
+    UpdateContainerRequest request =
+        Records.newRecord(UpdateContainerRequest.class);
+    request.setContainerVersion(version);
+    request.setContainerId(containerId);
+    request.setContainerUpdateType(updateType);
+    request.setCapability(targetCapability);
+    return request;
+  }
+
+  /**
+   * Get the <code>ContainerId</code> of the container.
+   * @return <code>ContainerId</code> of the container
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract int getContainerVersion();
+
+  /**
+   * Set the current version of the container.
+   * @param containerVersion of the container
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract void setContainerVersion(int containerVersion);
+
+  /**
+   * Get the <code>ContainerUpdateType</code> of the container.
+   * @return <code>ContainerUpdateType</code> of the container.
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract ContainerUpdateType getContainerUpdateType();
+
+  /**
+   * Set the <code>ContainerUpdateType</code> of the container.
+   * @param updateType of the Container
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract void setContainerUpdateType(ContainerUpdateType updateType);
+
+  /**
+   * Get the <code>ContainerId</code> of the container.
+   * @return <code>ContainerId</code> of the container
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract ContainerId getContainerId();
+
+  /**
+   * Set the <code>ContainerId</code> of the container.
+   * @param containerId <code>ContainerId</code> of the container
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract void setContainerId(ContainerId containerId);
+
+  /**
+   * Get the <code>Resource</code> capability of the container.
+   * @return <code>Resource</code> capability of the container
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract Resource getCapability();
+
+  /**
+   * Set the <code>Resource</code> capability of the container.
+   * @param capability <code>Resource</code> capability of the container
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public abstract void setCapability(Resource capability);
+
+  @Override
+  public int hashCode() {
+    final int prime = 2153;
+    int result = 2459;
+    ContainerId cId = getContainerId();
+    Resource capability = getCapability();
+    result =
+        prime * result + ((capability == null) ? 0 : capability.hashCode());
+    result = prime * result + ((cId == null) ? 0 : cId.hashCode());
+    result = prime * result + getContainerVersion();
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    UpdateContainerRequest other = (UpdateContainerRequest) obj;
+    Resource capability = getCapability();
+    if (capability == null) {
+      if (other.getCapability() != null) {
+        return false;
+      }
+    } else if (!capability.equals(other.getCapability())) {
+      return false;
+    }
+    ContainerId cId = getContainerId();
+    if (cId == null) {
+      if (other.getContainerId() != null) {
+        return false;
+      }
+    } else if (!cId.equals(other.getContainerId())) {
+      return false;
+    }
+    if (getContainerVersion() != other.getContainerVersion()) {
+      return false;
+    }
+    return true;
+  }
+}

+ 118 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/UpdatedContainer.java

@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * An object that encapsulates an updated container and the
+ * type of Update.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class UpdatedContainer {
+
+  /**
+   * Static Factory method.
+   *
+   * @param updateType ContainerUpdateType
+   * @param container Container
+   * @return UpdatedContainer
+   */
+  @InterfaceAudience.Public
+  @InterfaceStability.Unstable
+  public static UpdatedContainer newInstance(ContainerUpdateType updateType,
+      Container container) {
+    UpdatedContainer updatedContainer =
+        Records.newRecord(UpdatedContainer.class);
+    updatedContainer.setUpdateType(updateType);
+    updatedContainer.setContainer(container);
+    return updatedContainer;
+  }
+
+  /**
+   * Get the <code>ContainerUpdateType</code>.
+   * @return ContainerUpdateType
+   */
+  public abstract ContainerUpdateType getUpdateType();
+
+  /**
+   * Set the <code>ContainerUpdateType</code>.
+   * @param updateType ContainerUpdateType
+   */
+  public abstract void setUpdateType(ContainerUpdateType updateType);
+
+  /**
+   * Get the <code>Container</code>.
+   * @return Container
+   */
+  public abstract Container getContainer();
+
+  /**
+   * Set the <code>Container</code>.
+   * @param container Container
+   */
+  public  abstract void setContainer(Container container);
+
+  @Override
+  public int hashCode() {
+    final int prime = 2153;
+    int result = 2459;
+    ContainerUpdateType updateType = getUpdateType();
+    Container container = getContainer();
+    result = prime * result + ((updateType == null) ? 0 :
+        updateType.hashCode());
+    result = prime * result + ((container == null) ? 0 : container.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    UpdatedContainer other = (UpdatedContainer) obj;
+    ContainerUpdateType updateType = getUpdateType();
+    if (updateType == null) {
+      if (other.getUpdateType() != null) {
+        return false;
+      }
+    } else if (updateType != other.getUpdateType()) {
+      return false;
+    }
+    Container container = getContainer();
+    if (container == null) {
+      if (other.getContainer() != null) {
+        return false;
+      }
+    } else if (!container.equals(other.getContainer())) {
+      return false;
+    }
+    return true;
+  }
+
+}

+ 1 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto

@@ -91,6 +91,7 @@ message ContainerProto {
   optional ResourceProto resource = 4;
   optional PriorityProto priority = 5;
   optional hadoop.common.TokenProto container_token = 6;
+  optional int32 version = 9 [default = 0];
 }
 
 message ContainerReportProto {
@@ -520,10 +521,6 @@ enum ContainerExitStatusProto {
   DISKS_FAILED = -101;
 }
 
-message ContainerResourceChangeRequestProto {
-  optional ContainerIdProto container_id = 1;
-  optional ResourceProto capability = 2;
-} 
 
 ////////////////////////////////////////////////////////////////////////
 ////// From common//////////////////////////////////////////////////////

+ 25 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto

@@ -60,14 +60,30 @@ message FinishApplicationMasterResponseProto {
   optional bool isUnregistered = 1 [default = false];
 }
 
+enum ContainerUpdateTypeProto {
+  INCREASE_RESOURCE = 0;
+  DECREASE_RESOURCE = 1;
+}
+
+message UpdateContainerRequestProto {
+  required int32 container_version = 1;
+  required ContainerIdProto container_id = 2;
+  required ContainerUpdateTypeProto update_type = 3;
+  optional ResourceProto capability = 4;
+}
+
+message UpdateContainerErrorProto {
+  optional string reason = 1;
+  optional UpdateContainerRequestProto update_request = 2;
+}
+
 message AllocateRequestProto {
   repeated ResourceRequestProto ask = 1;
   repeated ContainerIdProto release = 2;
   optional ResourceBlacklistRequestProto blacklist_request = 3;
   optional int32 response_id = 4;
   optional float progress = 5;
-  repeated ContainerResourceChangeRequestProto increase_request = 6;
-  repeated ContainerResourceChangeRequestProto decrease_request = 7;
+  repeated UpdateContainerRequestProto update_requests = 6;
 }
 
 message NMTokenProto {
@@ -75,6 +91,11 @@ message NMTokenProto {
   optional hadoop.common.TokenProto token = 2;
 }
 
+message UpdatedContainerProto {
+  required ContainerUpdateTypeProto update_type = 1;
+  required ContainerProto container = 2;
+}
+
 message AllocateResponseProto {
   optional AMCommandProto a_m_command = 1;
   optional int32 response_id = 2;
@@ -85,10 +106,10 @@ message AllocateResponseProto {
   optional int32 num_cluster_nodes = 7;
   optional PreemptionMessageProto preempt = 8;
   repeated NMTokenProto nm_tokens = 9;
-  repeated ContainerProto increased_containers = 10;
-  repeated ContainerProto decreased_containers = 11;
+  repeated UpdatedContainerProto updated_containers = 10;
   optional hadoop.common.TokenProto am_rm_token = 12;
   optional PriorityProto application_priority = 13;
+  repeated UpdateContainerErrorProto update_errors = 15;
 }
 
 enum SchedulerResourceTypes {

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java

@@ -87,6 +87,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
@@ -842,7 +843,8 @@ public class ApplicationMaster {
     }
 
     @Override
-    public void onContainersResourceChanged(List<Container> containers) {}
+    public void onContainersUpdated(
+        List<UpdatedContainer> containers) {}
 
     @Override
     public void onShutdownRequest() {

+ 6 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java

@@ -29,6 +29,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
@@ -39,6 +40,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl;
@@ -61,7 +63,7 @@ import com.google.common.annotations.VisibleForTesting;
  *     [run tasks on the containers]
  *   }
  *
- *   public void onContainersResourceChanged(List<Container> containers) {
+ *   public void onContainersUpdated(List<Container> containers) {
  *     [determine if resource allocation of containers have been increased in
  *      the ResourceManager, and if so, inform the NodeManagers to increase the
  *      resource monitor/enforcement on the containers]
@@ -374,8 +376,9 @@ extends AbstractService {
      * Called when the ResourceManager responds to a heartbeat with containers
      * whose resource allocation has been changed.
      */
-    public abstract void onContainersResourceChanged(
-        List<Container> containers);
+    @Public
+    @Unstable
+    public abstract void onContainersUpdated(List<UpdatedContainer> containers);
 
     /**
      * Called when the ResourceManager wants the ApplicationMaster to shutdown

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
@@ -338,12 +339,11 @@ extends AMRMClientAsync<T> {
           if (handler instanceof AMRMClientAsync.AbstractCallbackHandler) {
             // RM side of the implementation guarantees that there are
             // no duplications between increased and decreased containers
-            List<Container> changed = new ArrayList<>();
-            changed.addAll(response.getIncreasedContainers());
-            changed.addAll(response.getDecreasedContainers());
+            List<UpdatedContainer> changed = new ArrayList<>();
+            changed.addAll(response.getUpdatedContainers());
             if (!changed.isEmpty()) {
               ((AMRMClientAsync.AbstractCallbackHandler) handler)
-                  .onContainersResourceChanged(changed);
+                  .onContainersUpdated(changed);
             }
           }
 

+ 44 - 35
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java

@@ -52,8 +52,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterReque
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -61,6 +61,8 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
@@ -257,33 +259,10 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
         new HashMap<>();
     try {
       synchronized (this) {
-        askList = new ArrayList<ResourceRequest>(ask.size());
-        for(ResourceRequest r : ask) {
-          // create a copy of ResourceRequest as we might change it while the 
-          // RPC layer is using it to send info across
-          askList.add(ResourceRequest.newInstance(r.getPriority(),
-              r.getResourceName(), r.getCapability(), r.getNumContainers(),
-              r.getRelaxLocality(), r.getNodeLabelExpression()));
-        }
-        List<ContainerResourceChangeRequest> increaseList = new ArrayList<>();
-        List<ContainerResourceChangeRequest> decreaseList = new ArrayList<>();
+        askList = cloneAsks();
         // Save the current change for recovery
         oldChange.putAll(change);
-        for (Map.Entry<ContainerId, SimpleEntry<Container, Resource>> entry :
-            change.entrySet()) {
-          Container container = entry.getValue().getKey();
-          Resource original = container.getResource();
-          Resource target = entry.getValue().getValue();
-          if (Resources.fitsIn(target, original)) {
-            // This is a decrease request
-            decreaseList.add(ContainerResourceChangeRequest.newInstance(
-                container.getId(), target));
-          } else {
-            // This is an increase request
-            increaseList.add(ContainerResourceChangeRequest.newInstance(
-                container.getId(), target));
-          }
-        }
+        List<UpdateContainerRequest> updateList = createUpdateList();
         releaseList = new ArrayList<ContainerId>(release);
         // optimistically clear this collection assuming no RPC failure
         ask.clear();
@@ -299,8 +278,7 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
         
         allocateRequest =
             AllocateRequest.newInstance(lastResponseId, progressIndicator,
-                askList, releaseList, blacklistRequest,
-                    increaseList, decreaseList);
+                askList, releaseList, blacklistRequest, updateList);
         // clear blacklistAdditions and blacklistRemovals before
         // unsynchronized part
         blacklistAdditions.clear();
@@ -350,9 +328,8 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
         if (!pendingChange.isEmpty()) {
           List<ContainerStatus> completed =
               allocateResponse.getCompletedContainersStatuses();
-          List<Container> changed = new ArrayList<>();
-          changed.addAll(allocateResponse.getIncreasedContainers());
-          changed.addAll(allocateResponse.getDecreasedContainers());
+          List<UpdatedContainer> changed = new ArrayList<>();
+          changed.addAll(allocateResponse.getUpdatedContainers());
           // remove all pending change requests that belong to the completed
           // containers
           for (ContainerStatus status : completed) {
@@ -409,6 +386,38 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
     return allocateResponse;
   }
 
+  private List<UpdateContainerRequest> createUpdateList() {
+    List<UpdateContainerRequest> updateList = new ArrayList<>();
+    for (Map.Entry<ContainerId, SimpleEntry<Container, Resource>> entry :
+        change.entrySet()) {
+      Resource targetCapability = entry.getValue().getValue();
+      Resource currCapability = entry.getValue().getKey().getResource();
+      int version = entry.getValue().getKey().getVersion();
+      ContainerUpdateType updateType =
+          ContainerUpdateType.INCREASE_RESOURCE;
+      if (Resources.fitsIn(targetCapability, currCapability)) {
+        updateType = ContainerUpdateType.DECREASE_RESOURCE;
+      }
+      updateList.add(
+          UpdateContainerRequest.newInstance(version, entry.getKey(),
+              updateType, targetCapability));
+    }
+    return updateList;
+  }
+
+  private List<ResourceRequest> cloneAsks() {
+    List<ResourceRequest> askList = new ArrayList<ResourceRequest>(ask.size());
+    for(ResourceRequest r : ask) {
+      // create a copy of ResourceRequest as we might change it while the
+      // RPC layer is using it to send info across
+      ResourceRequest rr = ResourceRequest.newInstance(r.getPriority(),
+          r.getResourceName(), r.getCapability(), r.getNumContainers(),
+          r.getRelaxLocality(), r.getNodeLabelExpression());
+      askList.add(rr);
+    }
+    return askList;
+  }
+
   protected void removePendingReleaseRequests(
       List<ContainerStatus> completedContainersStatuses) {
     for (ContainerStatus containerStatus : completedContainersStatuses) {
@@ -417,16 +426,16 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
   }
 
   protected void removePendingChangeRequests(
-      List<Container> changedContainers) {
-    for (Container changedContainer : changedContainers) {
-      ContainerId containerId = changedContainer.getId();
+      List<UpdatedContainer> changedContainers) {
+    for (UpdatedContainer changedContainer : changedContainers) {
+      ContainerId containerId = changedContainer.getContainer().getId();
       if (pendingChange.get(containerId) == null) {
         continue;
       }
       if (LOG.isDebugEnabled()) {
         LOG.debug("RM has confirmed changed resource allocation for "
             + "container " + containerId + ". Current resource allocation:"
-            + changedContainer.getResource()
+            + changedContainer.getContainer().getResource()
             + ". Remove pending change request:"
             + pendingChange.get(containerId).getValue());
       }

+ 35 - 20
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java

@@ -45,9 +45,11 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
@@ -89,20 +91,21 @@ public class TestAMRMClientAsync {
     TestCallbackHandler callbackHandler = new TestCallbackHandler();
     final AMRMClient<ContainerRequest> client = mock(AMRMClientImpl.class);
     final AtomicInteger secondHeartbeatSync = new AtomicInteger(0);
-    when(client.allocate(anyFloat())).thenReturn(response1).thenAnswer(new Answer<AllocateResponse>() {
-      @Override
-      public AllocateResponse answer(InvocationOnMock invocation)
-          throws Throwable {
-        secondHeartbeatSync.incrementAndGet();
-        while (heartbeatBlock.get()) {
-          synchronized (heartbeatBlock) {
-            heartbeatBlock.wait();
+    when(client.allocate(anyFloat())).thenReturn(response1).thenAnswer(
+        new Answer<AllocateResponse>() {
+          @Override
+          public AllocateResponse answer(InvocationOnMock invocation)
+              throws Throwable {
+            secondHeartbeatSync.incrementAndGet();
+            while (heartbeatBlock.get()) {
+              synchronized (heartbeatBlock) {
+                heartbeatBlock.wait();
+              }
+            }
+            secondHeartbeatSync.incrementAndGet();
+            return response2;
           }
-        }
-        secondHeartbeatSync.incrementAndGet();
-        return response2;
-      }
-    }).thenReturn(response3).thenReturn(emptyResponse);
+        }).thenReturn(response3).thenReturn(emptyResponse);
     when(client.registerApplicationMaster(anyString(), anyInt(), anyString()))
       .thenReturn(null);
     when(client.getAvailableResources()).thenAnswer(new Answer<Resource>() {
@@ -410,10 +413,21 @@ public class TestAMRMClientAsync {
       List<ContainerStatus> completed, List<Container> allocated,
       List<Container> increased, List<Container> decreased,
       List<NMToken> nmTokens) {
+    List<UpdatedContainer> updatedContainers = new ArrayList<>();
+    for (Container c : increased) {
+      updatedContainers.add(
+          UpdatedContainer.newInstance(
+              ContainerUpdateType.INCREASE_RESOURCE, c));
+    }
+    for (Container c : decreased) {
+      updatedContainers.add(
+          UpdatedContainer.newInstance(
+              ContainerUpdateType.DECREASE_RESOURCE, c));
+    }
     AllocateResponse response =
         AllocateResponse.newInstance(0, completed, allocated,
             new ArrayList<NodeReport>(), null, null, 1, null, nmTokens,
-            increased, decreased);
+            updatedContainers);
     return response;
   }
 
@@ -429,7 +443,7 @@ public class TestAMRMClientAsync {
       extends AMRMClientAsync.AbstractCallbackHandler {
     private volatile List<ContainerStatus> completedContainers;
     private volatile List<Container> allocatedContainers;
-    private final List<Container> changedContainers = new ArrayList<>();
+    private final List<UpdatedContainer> changedContainers = new ArrayList<>();
     Exception savedException = null;
     volatile boolean reboot = false;
     Object notifier = new Object();
@@ -448,8 +462,8 @@ public class TestAMRMClientAsync {
       return ret;
     }
 
-    public List<Container> takeChangedContainers() {
-      List<Container> ret = null;
+    public List<UpdatedContainer> takeChangedContainers() {
+      List<UpdatedContainer> ret = null;
       synchronized (changedContainers) {
         if (!changedContainers.isEmpty()) {
           ret = new ArrayList<>(changedContainers);
@@ -488,8 +502,8 @@ public class TestAMRMClientAsync {
     }
 
     @Override
-    public void onContainersResourceChanged(
-        List<Container> changed) {
+    public void onContainersUpdated(
+        List<UpdatedContainer> changed) {
       synchronized (changedContainers) {
         changedContainers.clear();
         changedContainers.addAll(changed);
@@ -564,7 +578,8 @@ public class TestAMRMClientAsync {
     public void onContainersAllocated(List<Container> containers) {}
 
     @Override
-    public void onContainersResourceChanged(List<Container> containers) {}
+    public void onContainersUpdated(
+        List<UpdatedContainer> containers) {}
 
     @Override
     public void onShutdownRequest() {}

+ 7 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java

@@ -70,6 +70,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
@@ -881,22 +882,16 @@ public class TestAMRMClient {
     AllocateResponse allocResponse = amClient.allocate(0.1f);
     Assert.assertEquals(0, amClientImpl.change.size());
     // we should get decrease confirmation right away
-    List<Container> decreasedContainers =
-        allocResponse.getDecreasedContainers();
-    List<Container> increasedContainers =
-        allocResponse.getIncreasedContainers();
-    Assert.assertEquals(1, decreasedContainers.size());
-    Assert.assertEquals(0, increasedContainers.size());
+    List<UpdatedContainer> updatedContainers =
+        allocResponse.getUpdatedContainers();
+    Assert.assertEquals(1, updatedContainers.size());
     // we should get increase allocation after the next NM's heartbeat to RM
     sleep(150);
     // get allocations
     allocResponse = amClient.allocate(0.1f);
-    decreasedContainers =
-        allocResponse.getDecreasedContainers();
-    increasedContainers =
-        allocResponse.getIncreasedContainers();
-    Assert.assertEquals(1, increasedContainers.size());
-    Assert.assertEquals(0, decreasedContainers.size());
+    updatedContainers =
+        allocResponse.getUpdatedContainers();
+    Assert.assertEquals(1, updatedContainers.size());
   }
 
   private void testAllocation(final AMRMClientImpl<ContainerRequest> amClient)

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

@@ -38,12 +38,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 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.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -249,7 +249,7 @@ public class TestAMRMClientOnRMRestart {
     // new NM to represent NM re-register
     nm1 = new MockNM("h1:1234", 10240, rm2.getResourceTrackerService());
     NMContainerStatus containerReport =
-        NMContainerStatus.newInstance(containerId, ContainerState.RUNNING,
+        NMContainerStatus.newInstance(containerId, 0, ContainerState.RUNNING,
             Resource.newInstance(1024, 1), "recover container", 0,
             Priority.newInstance(0), 0);
     nm1.registerNode(Collections.singletonList(containerReport),
@@ -386,7 +386,7 @@ public class TestAMRMClientOnRMRestart {
 
     ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
     NMContainerStatus containerReport =
-        NMContainerStatus.newInstance(containerId, ContainerState.RUNNING,
+        NMContainerStatus.newInstance(containerId, 0, ContainerState.RUNNING,
             Resource.newInstance(1024, 1), "recover container", 0,
             Priority.newInstance(0), 0);
     nm1.registerNode(Arrays.asList(containerReport), null);
@@ -549,8 +549,8 @@ public class TestAMRMClientOnRMRestart {
 
     List<ResourceRequest> lastAsk = null;
     List<ContainerId> lastRelease = null;
-    List<ContainerResourceChangeRequest> lastIncrease = null;
-    List<ContainerResourceChangeRequest> lastDecrease = null;
+    List<UpdateContainerRequest> lastIncrease = null;
+    List<UpdateContainerRequest> lastDecrease = null;
     List<String> lastBlacklistAdditions;
     List<String> lastBlacklistRemovals;
 
@@ -561,8 +561,8 @@ public class TestAMRMClientOnRMRestart {
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
         List<ContainerId> release, List<String> blacklistAdditions,
         List<String> blacklistRemovals,
-        List<ContainerResourceChangeRequest> increaseRequests,
-        List<ContainerResourceChangeRequest> decreaseRequests) {
+        List<UpdateContainerRequest> increaseRequests,
+        List<UpdateContainerRequest> decreaseRequests) {
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       for (ResourceRequest req : ask) {
         ResourceRequest reqCopy =

+ 40 - 111
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java

@@ -27,17 +27,17 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceChangeRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceBlacklistRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.UpdateContainerRequestPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.UpdateContainerRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProtoOrBuilder;
 
@@ -52,8 +52,7 @@ public class AllocateRequestPBImpl extends AllocateRequest {
 
   private List<ResourceRequest> ask = null;
   private List<ContainerId> release = null;
-  private List<ContainerResourceChangeRequest> increaseRequests = null;
-  private List<ContainerResourceChangeRequest> decreaseRequests = null;
+  private List<UpdateContainerRequest> updateRequests = null;
   private ResourceBlacklistRequest blacklistRequest = null;
   
   public AllocateRequestPBImpl() {
@@ -99,11 +98,8 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     if (this.release != null) {
       addReleasesToProto();
     }
-    if (this.increaseRequests != null) {
-      addIncreaseRequestsToProto();
-    }
-    if (this.decreaseRequests != null) {
-      addDecreaseRequestsToProto();
+    if (this.updateRequests != null) {
+      addUpdateRequestsToProto();
     }
     if (this.blacklistRequest != null) {
       builder.setBlacklistRequest(convertToProtoFormat(this.blacklistRequest));
@@ -166,37 +162,19 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   }
   
   @Override
-  public List<ContainerResourceChangeRequest> getIncreaseRequests() {
-    initIncreaseRequests();
-    return this.increaseRequests;
+  public List<UpdateContainerRequest> getUpdateRequests() {
+    initUpdateRequests();
+    return this.updateRequests;
   }
 
   @Override
-  public void setIncreaseRequests(
-      List<ContainerResourceChangeRequest> increaseRequests) {
-    if (increaseRequests == null) {
+  public void setUpdateRequests(List<UpdateContainerRequest> updateRequests) {
+    if (updateRequests == null) {
       return;
     }
-    initIncreaseRequests();
-    this.increaseRequests.clear();
-    this.increaseRequests.addAll(increaseRequests);
-  }
-
-  @Override
-  public List<ContainerResourceChangeRequest> getDecreaseRequests() {
-    initDecreaseRequests();
-    return this.decreaseRequests;
-  }
-
-  @Override
-  public void setDecreaseRequests(
-          List<ContainerResourceChangeRequest> decreaseRequests) {
-    if (decreaseRequests == null) {
-      return;
-    }
-    initDecreaseRequests();
-    this.decreaseRequests.clear();
-    this.decreaseRequests.addAll(decreaseRequests);
+    initUpdateRequests();
+    this.updateRequests.clear();
+    this.updateRequests.addAll(updateRequests);
   }
 
   @Override
@@ -239,7 +217,8 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     builder.clearAsk();
     if (ask == null)
       return;
-    Iterable<ResourceRequestProto> iterable = new Iterable<ResourceRequestProto>() {
+    Iterable<ResourceRequestProto> iterable =
+        new Iterable<ResourceRequestProto>() {
       @Override
       public Iterator<ResourceRequestProto> iterator() {
         return new Iterator<ResourceRequestProto>() {
@@ -268,84 +247,34 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     builder.addAllAsk(iterable);
   }
   
-  private void initIncreaseRequests() {
-    if (this.increaseRequests != null) {
+  private void initUpdateRequests() {
+    if (this.updateRequests != null) {
       return;
     }
     AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
-    List<ContainerResourceChangeRequestProto> list =
-        p.getIncreaseRequestList();
-    this.increaseRequests = new ArrayList<ContainerResourceChangeRequest>();
+    List<UpdateContainerRequestProto> list =
+        p.getUpdateRequestsList();
+    this.updateRequests = new ArrayList<>();
 
-    for (ContainerResourceChangeRequestProto c : list) {
-      this.increaseRequests.add(convertFromProtoFormat(c));
-    }
-  }
-
-  private void initDecreaseRequests() {
-    if (this.decreaseRequests != null) {
-      return;
-    }
-    AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
-    List<ContainerResourceChangeRequestProto> list =
-            p.getDecreaseRequestList();
-    this.decreaseRequests = new ArrayList<>();
-
-    for (ContainerResourceChangeRequestProto c : list) {
-      this.decreaseRequests.add(convertFromProtoFormat(c));
-    }
-  }
-
-  private void addIncreaseRequestsToProto() {
-    maybeInitBuilder();
-    builder.clearIncreaseRequest();
-    if (increaseRequests == null) {
-      return;
+    for (UpdateContainerRequestProto c : list) {
+      this.updateRequests.add(convertFromProtoFormat(c));
     }
-    Iterable<ContainerResourceChangeRequestProto> iterable =
-        new Iterable<ContainerResourceChangeRequestProto>() {
-          @Override
-          public Iterator<ContainerResourceChangeRequestProto> iterator() {
-            return new Iterator<ContainerResourceChangeRequestProto>() {
-
-              Iterator<ContainerResourceChangeRequest> iter =
-                  increaseRequests.iterator();
-
-              @Override
-              public boolean hasNext() {
-                return iter.hasNext();
-              }
-
-              @Override
-              public ContainerResourceChangeRequestProto next() {
-                return convertToProtoFormat(iter.next());
-              }
-
-              @Override
-              public void remove() {
-                throw new UnsupportedOperationException();
-              }
-            };
-
-          }
-        };
-    builder.addAllIncreaseRequest(iterable);
   }
 
-  private void addDecreaseRequestsToProto() {
+  private void addUpdateRequestsToProto() {
     maybeInitBuilder();
-    builder.clearDecreaseRequest();
-    if (decreaseRequests == null) {
+    builder.clearUpdateRequests();
+    if (updateRequests == null) {
       return;
     }
-    Iterable<ContainerResourceChangeRequestProto> iterable =
-        new Iterable<ContainerResourceChangeRequestProto>() {
+    Iterable<UpdateContainerRequestProto> iterable =
+        new Iterable<UpdateContainerRequestProto>() {
           @Override
-          public Iterator<ContainerResourceChangeRequestProto> iterator() {
-            return new Iterator<ContainerResourceChangeRequestProto>() {
+          public Iterator<UpdateContainerRequestProto> iterator() {
+            return new Iterator<UpdateContainerRequestProto>() {
 
-              Iterator<ContainerResourceChangeRequest> iter =
-                      decreaseRequests.iterator();
+              private Iterator<UpdateContainerRequest> iter =
+                  updateRequests.iterator();
 
               @Override
               public boolean hasNext() {
@@ -353,7 +282,7 @@ public class AllocateRequestPBImpl extends AllocateRequest {
               }
 
               @Override
-              public ContainerResourceChangeRequestProto next() {
+              public UpdateContainerRequestProto next() {
                 return convertToProtoFormat(iter.next());
               }
 
@@ -365,7 +294,7 @@ public class AllocateRequestPBImpl extends AllocateRequest {
 
           }
         };
-    builder.addAllDecreaseRequest(iterable);
+    builder.addAllUpdateRequests(iterable);
   }
 
   @Override
@@ -438,14 +367,14 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     return ((ResourceRequestPBImpl)t).getProto();
   }
   
-  private ContainerResourceChangeRequestPBImpl convertFromProtoFormat(
-      ContainerResourceChangeRequestProto p) {
-    return new ContainerResourceChangeRequestPBImpl(p);
+  private UpdateContainerRequestPBImpl convertFromProtoFormat(
+      UpdateContainerRequestProto p) {
+    return new UpdateContainerRequestPBImpl(p);
   }
 
-  private ContainerResourceChangeRequestProto convertToProtoFormat(
-      ContainerResourceChangeRequest t) {
-    return ((ContainerResourceChangeRequestPBImpl) t).getProto();
+  private UpdateContainerRequestProto convertToProtoFormat(
+      UpdateContainerRequest t) {
+    return ((UpdateContainerRequestPBImpl) t).getProto();
   }
 
   private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {

+ 139 - 53
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java

@@ -36,6 +36,8 @@ import org.apache.hadoop.yarn.api.records.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.UpdateContainerError;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
@@ -45,12 +47,14 @@ import org.apache.hadoop.yarn.api.records.impl.pb.PriorityPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.UpdatedContainerPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionMessageProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.NMTokenProto;
@@ -69,10 +73,10 @@ public class AllocateResponsePBImpl extends AllocateResponse {
   private List<Container> allocatedContainers = null;
   private List<NMToken> nmTokens = null;
   private List<ContainerStatus> completedContainersStatuses = null;
-  private List<Container> increasedContainers = null;
-  private List<Container> decreasedContainers = null;
+  private List<UpdatedContainer> updatedContainers = null;
 
   private List<NodeReport> updatedNodes = null;
+  private List<UpdateContainerError> updateErrors = null;
   private PreemptionMessage preempt;
   private Token amrmToken = null;
   private Priority appPriority = null;
@@ -143,17 +147,17 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     if (this.preempt != null) {
       builder.setPreempt(convertToProtoFormat(this.preempt));
     }
-    if (this.increasedContainers != null) {
-      builder.clearIncreasedContainers();
-      Iterable<ContainerProto> iterable =
-          getContainerProtoIterable(this.increasedContainers);
-      builder.addAllIncreasedContainers(iterable);
+    if (this.updatedContainers != null) {
+      builder.clearUpdatedContainers();
+      Iterable<YarnServiceProtos.UpdatedContainerProto> iterable =
+          getUpdatedContainerProtoIterable(this.updatedContainers);
+      builder.addAllUpdatedContainers(iterable);
     }
-    if (this.decreasedContainers != null) {
-      builder.clearDecreasedContainers();
-      Iterable<ContainerProto> iterable =
-          getContainerProtoIterable(this.decreasedContainers);
-      builder.addAllDecreasedContainers(iterable);
+    if (this.updateErrors != null) {
+      builder.clearUpdateErrors();
+      Iterable<YarnServiceProtos.UpdateContainerErrorProto> iterable =
+          getUpdateErrorsIterable(this.updateErrors);
+      builder.addAllUpdateErrors(iterable);
     }
     if (this.amrmToken != null) {
       builder.setAmRmToken(convertToProtoFormat(this.amrmToken));
@@ -248,49 +252,52 @@ public class AllocateResponsePBImpl extends AllocateResponse {
   }
 
   @Override
-  public synchronized List<Container> getAllocatedContainers() {
-    initLocalNewContainerList();
-    return this.allocatedContainers;
+  public synchronized List<UpdateContainerError> getUpdateErrors() {
+    initLocalUpdateErrorsList();
+    return this.updateErrors;
   }
 
   @Override
-  public synchronized void setAllocatedContainers(
-      final List<Container> containers) {
-    if (containers == null)
+  public synchronized void setUpdateErrors(
+      List<UpdateContainerError> updateErrors) {
+    if (updateErrors == null) {
+      this.updateErrors.clear();
       return;
-    // this looks like a bug because it results in append and not set
-    initLocalNewContainerList();
-    allocatedContainers.addAll(containers);
+    }
+    this.updateErrors = new ArrayList<>(
+        updateErrors.size());
+    this.updateErrors.addAll(updateErrors);
   }
 
   @Override
-  public synchronized List<Container> getIncreasedContainers() {
-    initLocalIncreasedContainerList();
-    return this.increasedContainers;
+  public synchronized List<Container> getAllocatedContainers() {
+    initLocalNewContainerList();
+    return this.allocatedContainers;
   }
 
   @Override
-  public synchronized void setIncreasedContainers(
+  public synchronized void setAllocatedContainers(
       final List<Container> containers) {
     if (containers == null)
       return;
-    initLocalIncreasedContainerList();
-    increasedContainers.addAll(containers);
+    // this looks like a bug because it results in append and not set
+    initLocalNewContainerList();
+    allocatedContainers.addAll(containers);
   }
 
   @Override
-  public synchronized List<Container> getDecreasedContainers() {
-    initLocalDecreasedContainerList();
-    return this.decreasedContainers;
+  public synchronized List<UpdatedContainer> getUpdatedContainers() {
+    initLocalUpdatedContainerList();
+    return this.updatedContainers;
   }
 
   @Override
-  public synchronized void setDecreasedContainers(
-      final List<Container> containers) {
+  public synchronized void setUpdatedContainers(
+      final List<UpdatedContainer> containers) {
     if (containers == null)
       return;
-    initLocalDecreasedContainerList();
-    decreasedContainers.addAll(containers);
+    initLocalUpdatedContainerList();
+    updatedContainers.addAll(containers);
   }
 
   //// Finished containers
@@ -406,29 +413,17 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     this.appPriority = priority;
   }
 
-  private synchronized void initLocalIncreasedContainerList() {
-    if (this.increasedContainers != null) {
+  private synchronized void initLocalUpdatedContainerList() {
+    if (this.updatedContainers != null) {
       return;
     }
     AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
-    List<ContainerProto> list = p.getIncreasedContainersList();
-    increasedContainers = new ArrayList<>();
+    List<YarnServiceProtos.UpdatedContainerProto> list =
+        p.getUpdatedContainersList();
+    updatedContainers = new ArrayList<>();
 
-    for (ContainerProto c : list) {
-      increasedContainers.add(convertFromProtoFormat(c));
-    }
-  }
-
-  private synchronized void initLocalDecreasedContainerList() {
-    if (this.decreasedContainers != null) {
-      return;
-    }
-    AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
-    List<ContainerProto> list = p.getDecreasedContainersList();
-    decreasedContainers = new ArrayList<>();
-
-    for (ContainerProto c : list) {
-      decreasedContainers.add(convertFromProtoFormat(c));
+    for (YarnServiceProtos.UpdatedContainerProto c : list) {
+      updatedContainers.add(convertFromProtoFormat(c));
     }
   }
 
@@ -474,6 +469,53 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     }
   }
 
+  private synchronized void initLocalUpdateErrorsList() {
+    if (updateErrors != null) {
+      return;
+    }
+    AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<YarnServiceProtos.UpdateContainerErrorProto> list =
+        p.getUpdateErrorsList();
+    this.updateErrors = new ArrayList<UpdateContainerError>();
+    for (YarnServiceProtos.UpdateContainerErrorProto t : list) {
+      updateErrors.add(ProtoUtils.convertFromProtoFormat(t));
+    }
+  }
+
+  private synchronized Iterable<YarnServiceProtos.UpdateContainerErrorProto>
+      getUpdateErrorsIterable(
+      final List<UpdateContainerError> updateErrorsList) {
+    maybeInitBuilder();
+    return new Iterable<YarnServiceProtos.UpdateContainerErrorProto>() {
+      @Override
+      public synchronized Iterator<YarnServiceProtos
+          .UpdateContainerErrorProto> iterator() {
+        return new Iterator<YarnServiceProtos.UpdateContainerErrorProto>() {
+
+          private Iterator<UpdateContainerError> iter =
+              updateErrorsList.iterator();
+
+          @Override
+          public synchronized boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public synchronized YarnServiceProtos.UpdateContainerErrorProto
+              next() {
+            return ProtoUtils.convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public synchronized void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+
+      }
+    };
+  }
+
   private synchronized Iterable<ContainerProto> getContainerProtoIterable(
       final List<Container> newContainersList) {
     maybeInitBuilder();
@@ -505,6 +547,40 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     };
   }
 
+  private synchronized Iterable<YarnServiceProtos.UpdatedContainerProto>
+        getUpdatedContainerProtoIterable(
+      final List<UpdatedContainer> newUpdatedContainersList) {
+    maybeInitBuilder();
+    return new Iterable<YarnServiceProtos.UpdatedContainerProto>() {
+      @Override
+      public synchronized Iterator<YarnServiceProtos.UpdatedContainerProto>
+          iterator() {
+        return new Iterator<YarnServiceProtos.UpdatedContainerProto>() {
+
+          private Iterator<UpdatedContainer> iter =
+              newUpdatedContainersList.iterator();
+
+          @Override
+          public synchronized boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public synchronized YarnServiceProtos.UpdatedContainerProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public synchronized void remove() {
+            throw new UnsupportedOperationException();
+
+          }
+        };
+
+      }
+    };
+  }
+
   private synchronized Iterable<NMTokenProto> getTokenProtoIterable(
       final List<NMToken> nmTokenList) {
     maybeInitBuilder();
@@ -631,6 +707,16 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     return ((ContainerPBImpl)t).getProto();
   }
 
+  private synchronized UpdatedContainerPBImpl convertFromProtoFormat(
+      YarnServiceProtos.UpdatedContainerProto p) {
+    return new UpdatedContainerPBImpl(p);
+  }
+
+  private synchronized YarnServiceProtos.UpdatedContainerProto
+      convertToProtoFormat(UpdatedContainer t) {
+    return ((UpdatedContainerPBImpl)t).getProto();
+  }
+
   private synchronized ContainerStatusPBImpl convertFromProtoFormat(
       ContainerStatusProto p) {
     return new ContainerStatusPBImpl(p);

+ 13 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerPBImpl.java

@@ -248,6 +248,18 @@ public class ContainerPBImpl extends Container {
     this.containerToken = containerToken;
   }
 
+  @Override
+  public int getVersion() {
+    ContainerProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getVersion();
+  }
+
+  @Override
+  public void setVersion(int version) {
+    maybeInitBuilder();
+    builder.setVersion(version);
+  }
+
   private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
     return new ContainerIdPBImpl(p);
   }
@@ -292,6 +304,7 @@ public class ContainerPBImpl extends Container {
     StringBuilder sb = new StringBuilder();
     sb.append("Container: [");
     sb.append("ContainerId: ").append(getId()).append(", ");
+    sb.append("Version: ").append(getVersion()).append(", ");
     sb.append("NodeId: ").append(getNodeId()).append(", ");
     sb.append("NodeHttpAddress: ").append(getNodeHttpAddress()).append(", ");
     sb.append("Resource: ").append(getResource()).append(", ");

+ 0 - 141
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceChangeRequestPBImpl.java

@@ -1,141 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api.records.impl.pb;
-
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
-
-
-public class ContainerResourceChangeRequestPBImpl extends
-    ContainerResourceChangeRequest {
-  ContainerResourceChangeRequestProto proto =
-      ContainerResourceChangeRequestProto.getDefaultInstance();
-  ContainerResourceChangeRequestProto.Builder builder = null;
-  boolean viaProto = false;
-
-  private ContainerId existingContainerId = null;
-  private Resource targetCapability = null;
-
-  public ContainerResourceChangeRequestPBImpl() {
-    builder = ContainerResourceChangeRequestProto.newBuilder();
-  }
-
-  public ContainerResourceChangeRequestPBImpl(
-      ContainerResourceChangeRequestProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-
-  public ContainerResourceChangeRequestProto getProto() {
-    mergeLocalToProto();
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
-  }
-
-  @Override
-  public ContainerId getContainerId() {
-    ContainerResourceChangeRequestProtoOrBuilder p = viaProto ? proto
-        : builder;
-    if (this.existingContainerId != null) {
-      return this.existingContainerId;
-    }
-    if (p.hasContainerId()) {
-      this.existingContainerId = convertFromProtoFormat(p.getContainerId());
-    }
-    return this.existingContainerId;
-  }
-
-  @Override
-  public void setContainerId(ContainerId existingContainerId) {
-    maybeInitBuilder();
-    if (existingContainerId == null) {
-      builder.clearContainerId();
-    }
-    this.existingContainerId = existingContainerId;
-  }
-
-  @Override
-  public Resource getCapability() {
-    ContainerResourceChangeRequestProtoOrBuilder p = viaProto ? proto
-        : builder;
-    if (this.targetCapability != null) {
-      return this.targetCapability;
-    }
-    if (p.hasCapability()) {
-      this.targetCapability = convertFromProtoFormat(p.getCapability());
-    }
-    return this.targetCapability;
-  }
-
-  @Override
-  public void setCapability(Resource targetCapability) {
-    maybeInitBuilder();
-    if (targetCapability == null) {
-      builder.clearCapability();
-    }
-    this.targetCapability = targetCapability;
-  }
-
-  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
-    return new ContainerIdPBImpl(p);
-  }
-
-  private ContainerIdProto convertToProtoFormat(ContainerId t) {
-    return ((ContainerIdPBImpl) t).getProto();
-  }
-
-  private Resource convertFromProtoFormat(ResourceProto p) {
-    return new ResourcePBImpl(p);
-  }
-
-  private ResourceProto convertToProtoFormat(Resource t) {
-    return ((ResourcePBImpl) t).getProto();
-  }
-
-  private void mergeLocalToProto() {
-    if (viaProto) {
-      maybeInitBuilder();
-    }
-    mergeLocalToBuilder();
-    proto = builder.build();
-    viaProto = true;
-  }
-
-  private void maybeInitBuilder() {
-    if (viaProto || builder == null) {
-      builder = ContainerResourceChangeRequestProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-
-  private void mergeLocalToBuilder() {
-    if (this.existingContainerId != null) {
-      builder.setContainerId(convertToProtoFormat(this.existingContainerId));
-    }
-    if (this.targetCapability != null) {
-      builder.setCapability(convertToProtoFormat(this.targetCapability));
-    }
-  }
-}

+ 90 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java

@@ -26,7 +26,10 @@ import org.apache.hadoop.yarn.api.protocolrecords.ApplicationsRequestScope;
 import org.apache.hadoop.yarn.api.records.AMCommand;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@@ -36,11 +39,16 @@ import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.UpdateContainerError;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.hadoop.yarn.proto.YarnProtos.AMCommandProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAccessTypeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationResourceUsageReportProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.FinalApplicationStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceTypeProto;
@@ -51,10 +59,12 @@ import org.apache.hadoop.yarn.proto.YarnProtos.NodeStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueACLProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestInterpreterProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationAttemptStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerTypeProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerUpdateTypeProto;
 import org.apache.hadoop.yarn.server.api.ContainerType;
 
 import com.google.protobuf.ByteString;
@@ -282,4 +292,84 @@ public class ProtoUtils {
   public static ContainerType convertFromProtoFormat(ContainerTypeProto e) {
     return ContainerType.valueOf(e.name());
   }
+
+  /*
+   * ContainerUpdateType
+   */
+  public static ContainerUpdateTypeProto convertToProtoFormat(
+      ContainerUpdateType e) {
+    return ContainerUpdateTypeProto.valueOf(e.name());
+  }
+  public static ContainerUpdateType convertFromProtoFormat(
+      ContainerUpdateTypeProto e) {
+    return ContainerUpdateType.valueOf(e.name());
+  }
+
+  /*
+   * Resource
+   */
+  public static synchronized ResourceProto convertToProtoFormat(Resource r) {
+    return ((ResourcePBImpl) r).getProto();
+  }
+
+  public static Resource convertFromProtoFormat(ResourceProto resource) {
+    return new ResourcePBImpl(resource);
+  }
+
+  /*
+   * Container
+   */
+  public static YarnProtos.ContainerProto convertToProtoFormat(
+      Container t) {
+    return ((ContainerPBImpl)t).getProto();
+  }
+
+  public static ContainerPBImpl convertFromProtoFormat(
+      YarnProtos.ContainerProto t) {
+    return new ContainerPBImpl(t);
+  }
+
+  public static ContainerStatusPBImpl convertFromProtoFormat(
+      YarnProtos.ContainerStatusProto p) {
+    return new ContainerStatusPBImpl(p);
+  }
+
+  /*
+   * ContainerId
+   */
+  public static ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
+    return new ContainerIdPBImpl(p);
+  }
+
+  public static ContainerIdProto convertToProtoFormat(ContainerId t) {
+    return ((ContainerIdPBImpl) t).getProto();
+  }
+
+  /*
+   * UpdateContainerRequest
+   */
+  public static UpdateContainerRequestPBImpl convertFromProtoFormat(
+      YarnServiceProtos.UpdateContainerRequestProto p) {
+    return new UpdateContainerRequestPBImpl(p);
+  }
+
+  public static YarnServiceProtos.UpdateContainerRequestProto
+      convertToProtoFormat(UpdateContainerRequest t) {
+    return ((UpdateContainerRequestPBImpl) t).getProto();
+  }
+
+  /*
+   * UpdateContainerError
+   */
+  public static UpdateContainerErrorPBImpl convertFromProtoFormat(
+      YarnServiceProtos.UpdateContainerErrorProto p) {
+    return new UpdateContainerErrorPBImpl(p);
+  }
+
+  public static YarnServiceProtos.UpdateContainerErrorProto
+      convertToProtoFormat(UpdateContainerError t) {
+    return ((UpdateContainerErrorPBImpl) t).getProto();
+  }
 }
+
+

+ 125 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/UpdateContainerErrorPBImpl.java

@@ -0,0 +1,125 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.UpdateContainerError;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+
+/**
+ * Implementation of <code>UpdateContainerError</code>.
+ */
+public class UpdateContainerErrorPBImpl extends UpdateContainerError {
+  private YarnServiceProtos.UpdateContainerErrorProto proto =
+      YarnServiceProtos.UpdateContainerErrorProto.getDefaultInstance();
+  private YarnServiceProtos.UpdateContainerErrorProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private String reason = null;
+  private UpdateContainerRequest updateRequest = null;
+
+  public UpdateContainerErrorPBImpl() {
+    builder = YarnServiceProtos.UpdateContainerErrorProto.newBuilder();
+  }
+
+  public UpdateContainerErrorPBImpl(YarnServiceProtos
+      .UpdateContainerErrorProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public YarnServiceProtos.UpdateContainerErrorProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public String getReason() {
+    YarnServiceProtos.UpdateContainerErrorProtoOrBuilder p = viaProto ? proto
+        : builder;
+    if (this.reason != null) {
+      return this.reason;
+    }
+    if (p.hasReason()) {
+      this.reason = p.getReason();
+    }
+    return this.reason;
+  }
+
+  @Override
+  public void setReason(String reason) {
+    maybeInitBuilder();
+    if (reason == null) {
+      builder.clearReason();
+    }
+    this.reason = reason;
+  }
+
+  @Override
+  public UpdateContainerRequest getUpdateContainerRequest() {
+    YarnServiceProtos.UpdateContainerErrorProtoOrBuilder p = viaProto ? proto
+        : builder;
+    if (this.updateRequest != null) {
+      return this.updateRequest;
+    }
+    if (p.hasUpdateRequest()) {
+      this.updateRequest =
+          ProtoUtils.convertFromProtoFormat(p.getUpdateRequest());
+    }
+    return this.updateRequest;
+  }
+
+  @Override
+  public void setUpdateContainerRequest(
+      UpdateContainerRequest updateContainerRequest) {
+    maybeInitBuilder();
+    if (updateContainerRequest == null) {
+      builder.clearUpdateRequest();
+    }
+    this.updateRequest = updateContainerRequest;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = YarnServiceProtos.UpdateContainerErrorProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reason != null) {
+      builder.setReason(this.reason);
+    }
+    if (this.updateRequest != null) {
+      builder.setUpdateRequest(
+          ProtoUtils.convertToProtoFormat(this.updateRequest));
+    }
+  }
+}

+ 166 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/UpdateContainerRequestPBImpl.java

@@ -0,0 +1,166 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+
+/**
+ * Implementation of <code>UpdateContainerRequest</code>.
+ */
+public class UpdateContainerRequestPBImpl extends UpdateContainerRequest {
+  private YarnServiceProtos.UpdateContainerRequestProto proto =
+      YarnServiceProtos.UpdateContainerRequestProto.getDefaultInstance();
+  private YarnServiceProtos.UpdateContainerRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private ContainerId existingContainerId = null;
+  private Resource targetCapability = null;
+
+  public UpdateContainerRequestPBImpl() {
+    builder = YarnServiceProtos.UpdateContainerRequestProto.newBuilder();
+  }
+
+  public UpdateContainerRequestPBImpl(YarnServiceProtos
+      .UpdateContainerRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public YarnServiceProtos.UpdateContainerRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int getContainerVersion() {
+    YarnServiceProtos.UpdateContainerRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (!p.hasContainerVersion()) {
+      return 0;
+    }
+    return p.getContainerVersion();
+  }
+
+  @Override
+  public void setContainerVersion(int containerVersion) {
+    maybeInitBuilder();
+    builder.setContainerVersion(containerVersion);
+  }
+
+  @Override
+  public ContainerId getContainerId() {
+    YarnServiceProtos.UpdateContainerRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (this.existingContainerId != null) {
+      return this.existingContainerId;
+    }
+    if (p.hasContainerId()) {
+      this.existingContainerId =
+          ProtoUtils.convertFromProtoFormat(p.getContainerId());
+    }
+    return this.existingContainerId;
+  }
+
+  @Override
+  public void setContainerId(ContainerId containerId) {
+    maybeInitBuilder();
+    if (containerId == null) {
+      builder.clearContainerId();
+    }
+    this.existingContainerId = containerId;
+  }
+
+  @Override
+  public Resource getCapability() {
+    YarnServiceProtos.UpdateContainerRequestProtoOrBuilder p = viaProto ? proto
+        : builder;
+    if (this.targetCapability != null) {
+      return this.targetCapability;
+    }
+    if (p.hasCapability()) {
+      this.targetCapability =
+          ProtoUtils.convertFromProtoFormat(p.getCapability());
+    }
+    return this.targetCapability;
+  }
+
+  @Override
+  public void setCapability(Resource capability) {
+    maybeInitBuilder();
+    if (capability == null) {
+      builder.clearCapability();
+    }
+    this.targetCapability = capability;
+  }
+
+  @Override
+  public ContainerUpdateType getContainerUpdateType() {
+    YarnServiceProtos.UpdateContainerRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (!p.hasUpdateType()) {
+      return null;
+    }
+    return ProtoUtils.convertFromProtoFormat(p.getUpdateType());
+  }
+
+  @Override
+  public void setContainerUpdateType(ContainerUpdateType updateType) {
+    maybeInitBuilder();
+    if (updateType == null) {
+      builder.clearUpdateType();
+      return;
+    }
+    builder.setUpdateType(ProtoUtils.convertToProtoFormat(updateType));
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = YarnServiceProtos.UpdateContainerRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.existingContainerId != null) {
+      builder.setContainerId(
+          ProtoUtils.convertToProtoFormat(this.existingContainerId));
+    }
+    if (this.targetCapability != null) {
+      builder.setCapability(
+          ProtoUtils.convertToProtoFormat(this.targetCapability));
+    }
+  }
+
+
+}

+ 117 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/UpdatedContainerPBImpl.java

@@ -0,0 +1,117 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
+
+/**
+ * Implementation of <code>UpdatedContainer</code>.
+ */
+public class UpdatedContainerPBImpl extends UpdatedContainer {
+  private YarnServiceProtos.UpdatedContainerProto proto =
+      YarnServiceProtos.UpdatedContainerProto.getDefaultInstance();
+  private YarnServiceProtos.UpdatedContainerProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private Container container = null;
+
+  public UpdatedContainerPBImpl() {
+    builder = YarnServiceProtos.UpdatedContainerProto.newBuilder();
+  }
+
+  public UpdatedContainerPBImpl(YarnServiceProtos.UpdatedContainerProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.container != null) {
+      builder.setContainer(ProtoUtils.convertToProtoFormat(this.container));
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = YarnServiceProtos.UpdatedContainerProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  public YarnServiceProtos.UpdatedContainerProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public ContainerUpdateType getUpdateType() {
+    YarnServiceProtos.UpdatedContainerProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (!p.hasUpdateType()) {
+      return null;
+    }
+    return ProtoUtils.convertFromProtoFormat(p.getUpdateType());
+  }
+
+  @Override
+  public void setUpdateType(ContainerUpdateType updateType) {
+    maybeInitBuilder();
+    if (updateType == null) {
+      builder.clearUpdateType();
+      return;
+    }
+    builder.setUpdateType(ProtoUtils.convertToProtoFormat(updateType));
+  }
+
+  @Override
+  public Container getContainer() {
+    YarnServiceProtos.UpdatedContainerProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (this.container != null) {
+      return this.container;
+    }
+    if (!p.hasContainer()) {
+      return null;
+    }
+    this.container = ProtoUtils.convertFromProtoFormat(p.getContainer());
+    return this.container;
+  }
+
+  @Override
+  public void setContainer(Container container) {
+    maybeInitBuilder();
+    if (container == null) {
+      builder.clearContainer();
+    }
+    this.container = container;
+  }
+}

+ 24 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ContainerTokenIdentifier.java

@@ -85,11 +85,22 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
       long rmIdentifier, Priority priority, long creationTime,
       LogAggregationContext logAggregationContext, String nodeLabelExpression,
       ContainerType containerType) {
+    this(containerID, 0, hostName, appSubmitter, r, expiryTimeStamp,
+        masterKeyId, rmIdentifier, priority, creationTime,
+        logAggregationContext, nodeLabelExpression, containerType);
+  }
+
+  public ContainerTokenIdentifier(ContainerId containerID, int containerVersion,
+      String hostName, String appSubmitter, Resource r, long expiryTimeStamp,
+      int masterKeyId, long rmIdentifier, Priority priority, long creationTime,
+      LogAggregationContext logAggregationContext, String nodeLabelExpression,
+      ContainerType containerType) {
     ContainerTokenIdentifierProto.Builder builder =
         ContainerTokenIdentifierProto.newBuilder();
     if (containerID != null) {
       builder.setContainerId(((ContainerIdPBImpl)containerID).getProto());
     }
+    builder.setVersion(containerVersion);
     builder.setNmHostAddr(hostName);
     builder.setAppSubmitter(appSubmitter);
     if (r != null) {
@@ -171,7 +182,7 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
   }
 
   /**
-   * Get the ContainerType of container to allocate
+   * Get the ContainerType of container to allocate.
    * @return ContainerType
    */
   public ContainerType getContainerType(){
@@ -217,7 +228,18 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
     return UserGroupInformation.createRemoteUser(
         containerId);
   }
-  
+
+  /**
+   * Get the Container version
+   * @return container version
+   */
+  public int getVersion() {
+    if (proto.hasVersion()) {
+      return proto.getVersion();
+    } else {
+      return 0;
+    }
+  }
   /**
    * Get the node-label-expression in the original ResourceRequest
    */

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/proto/yarn_security_token.proto

@@ -51,6 +51,7 @@ message ContainerTokenIdentifierProto {
   optional LogAggregationContextProto logAggregationContext = 10;
   optional string nodeLabelExpression = 11;
   optional ContainerTypeProto containerType = 12;
+  optional int32 version = 14 [default = 0];
 }
 
 message ClientToAMTokenIdentifierProto {

+ 12 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java

@@ -119,7 +119,6 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
@@ -151,6 +150,9 @@ import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
+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.YarnClusterMetrics;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptReportPBImpl;
@@ -162,7 +164,6 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerLaunchContextPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerReportPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceChangeRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
@@ -184,6 +185,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.StrictPreemptionContractPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.URLPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.UpdateContainerRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.YarnClusterMetricsPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptReportProto;
@@ -195,7 +197,6 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerReportProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceChangeRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
@@ -240,6 +241,8 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.Repla
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ReplaceLabelsOnNodeResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeResourceRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeResourceResponseProto;
+
+import org.apache.hadoop.yarn.proto.YarnServiceProtos;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterRequestProto;
@@ -478,7 +481,8 @@ public class TestPBImplRecords {
     generateByNewInstance(ContainerLaunchContext.class);
     generateByNewInstance(ApplicationSubmissionContext.class);
     generateByNewInstance(ContainerReport.class);
-    generateByNewInstance(ContainerResourceChangeRequest.class);
+    generateByNewInstance(UpdateContainerRequest.class);
+    generateByNewInstance(UpdateContainerError.class);
     generateByNewInstance(IncreaseContainersResourceRequest.class);
     generateByNewInstance(IncreaseContainersResourceResponse.class);
     generateByNewInstance(ContainerStatus.class);
@@ -490,6 +494,7 @@ public class TestPBImplRecords {
     generateByNewInstance(PreemptionMessage.class);
     generateByNewInstance(StartContainerRequest.class);
     generateByNewInstance(NodeLabel.class);
+    generateByNewInstance(UpdatedContainer.class);
     // genByNewInstance does not apply to QueueInfo, cause
     // it is recursive(has sub queues)
     typeValueCache.put(QueueInfo.class, QueueInfo.newInstance("root", 1.0f,
@@ -981,9 +986,9 @@ public class TestPBImplRecords {
   }
 
   @Test
-  public void testContainerResourceChangeRequestPBImpl() throws Exception {
-    validatePBImplRecord(ContainerResourceChangeRequestPBImpl.class,
-        ContainerResourceChangeRequestProto.class);
+  public void testUpdateContainerRequestPBImpl() throws Exception {
+    validatePBImplRecord(UpdateContainerRequestPBImpl.class,
+        YarnServiceProtos.UpdateContainerRequestProto.class);
   }
 
   @Test

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java

@@ -152,7 +152,7 @@ public class TestYARNTokenIdentifier {
     long creationTime = 1000;
     
     ContainerTokenIdentifier token = new ContainerTokenIdentifier(
-        containerID, hostName, appSubmitter, r, expiryTimeStamp, 
+        containerID, hostName, appSubmitter, r, expiryTimeStamp,
         masterKeyId, rmIdentifier, priority, creationTime);
     
     ContainerTokenIdentifier anotherToken = new ContainerTokenIdentifier();
@@ -385,7 +385,7 @@ public class TestYARNTokenIdentifier {
         anotherToken.getContainerType());
 
     token =
-        new ContainerTokenIdentifier(containerID, hostName, appSubmitter, r,
+        new ContainerTokenIdentifier(containerID, 0, hostName, appSubmitter, r,
             expiryTimeStamp, masterKeyId, rmIdentifier, priority, creationTime,
             null, CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK);
 

+ 12 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NMContainerStatus.java

@@ -35,21 +35,22 @@ public abstract class NMContainerStatus {
   
   // Used by tests only
   public static NMContainerStatus newInstance(ContainerId containerId,
-      ContainerState containerState, Resource allocatedResource,
+      int version, ContainerState containerState, Resource allocatedResource,
       String diagnostics, int containerExitStatus, Priority priority,
       long creationTime) {
-    return newInstance(containerId, containerState, allocatedResource,
+    return newInstance(containerId, version, containerState, allocatedResource,
         diagnostics, containerExitStatus, priority, creationTime,
         CommonNodeLabelsManager.NO_LABEL);
   }
 
   public static NMContainerStatus newInstance(ContainerId containerId,
-      ContainerState containerState, Resource allocatedResource,
+      int version, ContainerState containerState, Resource allocatedResource,
       String diagnostics, int containerExitStatus, Priority priority,
       long creationTime, String nodeLabelExpression) {
     NMContainerStatus status =
         Records.newRecord(NMContainerStatus.class);
     status.setContainerId(containerId);
+    status.setVersion(version);
     status.setContainerState(containerState);
     status.setAllocatedResource(allocatedResource);
     status.setDiagnostics(diagnostics);
@@ -125,4 +126,12 @@ public abstract class NMContainerStatus {
 
   public abstract void setNodeLabelExpression(
       String nodeLabelExpression);
+
+  public int getVersion() {
+    return 0;
+  }
+
+  public void setVersion(int version) {
+
+  }
 }

+ 13 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NMContainerStatusPBImpl.java

@@ -83,6 +83,7 @@ public class NMContainerStatusPBImpl extends NMContainerStatus {
     StringBuilder sb = new StringBuilder();
     sb.append("[").append(getContainerId()).append(", ")
         .append("CreateTime: ").append(getCreationTime()).append(", ")
+        .append("Version: ").append(getVersion()).append(", ")
         .append("State: ").append(getContainerState()).append(", ")
         .append("Capability: ").append(getAllocatedResource()).append(", ")
         .append("Diagnostics: ").append(getDiagnostics()).append(", ")
@@ -184,6 +185,18 @@ public class NMContainerStatusPBImpl extends NMContainerStatus {
     builder.setContainerExitStatus(containerExitStatus);
   }
 
+  @Override
+  public int getVersion() {
+    NMContainerStatusProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getVersion();
+  }
+
+  @Override
+  public void setVersion(int version) {
+    maybeInitBuilder();
+    builder.setVersion(version);
+  }
+
   @Override
   public Priority getPriority() {
     NMContainerStatusProtoOrBuilder p = viaProto ? proto : builder;

+ 9 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java

@@ -62,9 +62,11 @@ import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.yarn.server.api.ContainerType;
 
 /**
  * Builder utilities to construct various objects.
@@ -154,12 +156,14 @@ public class BuilderUtils {
     return cId;
   }
 
-  public static Token newContainerToken(ContainerId cId, String host,
-      int port, String user, Resource r, long expiryTime, int masterKeyId,
-      byte[] password, long rmIdentifier) throws IOException {
+  public static Token newContainerToken(ContainerId cId, int containerVersion,
+      String host, int port, String user, Resource r, long expiryTime,
+      int masterKeyId, byte[] password, long rmIdentifier) throws IOException {
     ContainerTokenIdentifier identifier =
-        new ContainerTokenIdentifier(cId, host + ":" + port, user, r,
-          expiryTime, masterKeyId, rmIdentifier, Priority.newInstance(0), 0);
+        new ContainerTokenIdentifier(cId, containerVersion, host + ":" + port,
+            user, r, expiryTime, masterKeyId, rmIdentifier,
+            Priority.newInstance(0), 0, null, CommonNodeLabelsManager.NO_LABEL,
+            ContainerType.TASK);
     return newContainerToken(BuilderUtils.newNodeId(host, port), password,
         identifier);
   }

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto

@@ -101,6 +101,7 @@ message NMContainerStatusProto {
   optional int32 container_exit_status = 6;
   optional int64 creation_time = 7;
   optional string nodeLabelExpression = 8;
+  optional int32 version = 9;
 }
 
 message SCMUploaderNotifyRequestProto {

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestProtocolRecords.java

@@ -55,7 +55,7 @@ public class TestProtocolRecords {
     Resource resource = Resource.newInstance(1000, 200);
 
     NMContainerStatus report =
-        NMContainerStatus.newInstance(containerId,
+        NMContainerStatus.newInstance(containerId, 0,
           ContainerState.COMPLETE, resource, "diagnostics",
           ContainerExitStatus.ABORTED, Priority.newInstance(10), 1234);
     NMContainerStatus reportProto =
@@ -79,7 +79,7 @@ public class TestProtocolRecords {
     ContainerId containerId = ContainerId.newContainerId(attemptId, 1);
 
     NMContainerStatus containerReport =
-        NMContainerStatus.newInstance(containerId,
+        NMContainerStatus.newInstance(containerId, 0,
           ContainerState.RUNNING, Resource.newInstance(1024, 1), "diagnostics",
           0, Priority.newInstance(10), 1234);
     List<NMContainerStatus> reports = Arrays.asList(containerReport);

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestRegisterNodeManagerRequest.java

@@ -40,7 +40,7 @@ public class TestRegisterNodeManagerRequest {
           "version", Arrays.asList(NMContainerStatus.newInstance(
             ContainerId.newContainerId(
               ApplicationAttemptId.newInstance(
-                ApplicationId.newInstance(1234L, 1), 1), 1),
+                ApplicationId.newInstance(1234L, 1), 1), 1), 0,
             ContainerState.RUNNING, Resource.newInstance(1024, 1), "good", -1,
             Priority.newInstance(0), 1234)), Arrays.asList(
             ApplicationId.newInstance(1234L, 1),

+ 9 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java

@@ -345,8 +345,7 @@ public class ContainerManagerImpl extends CompositeService implements
           YarnServerSecurityUtils.parseCredentials(launchContext);
       Container container = new ContainerImpl(getConfig(), dispatcher,
           req.getContainerLaunchContext(),
-          credentials, metrics, token, rcs.getStatus(), rcs.getExitCode(),
-          rcs.getDiagnostics(), rcs.getKilled(), rcs.getCapability(), context);
+          credentials, metrics, token, context, rcs);
       context.getContainers().put(containerId, container);
       dispatcher.getEventHandler().handle(
           new ApplicationContainerInitEvent(container));
@@ -937,7 +936,8 @@ public class ContainerManagerImpl extends CompositeService implements
               logAggregationContext));
         }
 
-        this.context.getNMStateStore().storeContainer(containerId, request);
+        this.context.getNMStateStore().storeContainer(containerId,
+            containerTokenIdentifier.getVersion(), request);
         dispatcher.getEventHandler().handle(
           new ApplicationContainerInitEvent(container));
 
@@ -1021,7 +1021,8 @@ public class ContainerManagerImpl extends CompositeService implements
           // an updated NMToken.
           updateNMTokenIdentifier(nmTokenIdentifier);
           Resource resource = containerTokenIdentifier.getResource();
-          changeContainerResourceInternal(containerId, resource, true);
+          changeContainerResourceInternal(containerId,
+              containerTokenIdentifier.getVersion(), resource, true);
           successfullyIncreasedContainers.add(containerId);
         } catch (YarnException | InvalidToken e) {
           failedContainers.put(containerId, SerializedException.newInstance(e));
@@ -1035,8 +1036,8 @@ public class ContainerManagerImpl extends CompositeService implements
   }
 
   @SuppressWarnings("unchecked")
-  private void changeContainerResourceInternal(
-      ContainerId containerId, Resource targetResource, boolean increase)
+  private void changeContainerResourceInternal(ContainerId containerId,
+      int containerVersion, Resource targetResource, boolean increase)
           throws YarnException, IOException {
     Container container = context.getContainers().get(containerId);
     // Check container existence
@@ -1103,7 +1104,7 @@ public class ContainerManagerImpl extends CompositeService implements
       if (!serviceStopped) {
         // Persist container resource change for recovery
         this.context.getNMStateStore().storeContainerResourceChanged(
-            containerId, targetResource);
+            containerId, containerVersion, targetResource);
         getContainersMonitor().handle(
             new ChangeMonitoringContainerResourceEvent(
                 containerId, targetResource));
@@ -1338,7 +1339,7 @@ public class ContainerManagerImpl extends CompositeService implements
           : containersDecreasedEvent.getContainersToDecrease()) {
         try {
           changeContainerResourceInternal(container.getId(),
-              container.getResource(), false);
+              container.getVersion(), container.getResource(), false);
         } catch (YarnException e) {
           LOG.error("Unable to decrease container resource", e);
         } catch (IOException e) {

+ 12 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java

@@ -70,6 +70,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.Contai
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerStatus;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
@@ -78,7 +79,6 @@ import org.apache.hadoop.yarn.state.SingleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.util.Clock;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -95,6 +95,7 @@ public class ContainerImpl implements Container {
   private final ContainerId containerId;
   private volatile Resource resource;
   private final String user;
+  private int version;
   private int exitCode = ContainerExitStatus.INVALID;
   private final StringBuilder diagnostics;
   private boolean wasLaunched;
@@ -135,6 +136,7 @@ public class ContainerImpl implements Container {
     this.daemonConf = conf;
     this.dispatcher = dispatcher;
     this.stateStore = context.getNMStateStore();
+    this.version = containerTokenIdentifier.getVersion();
     this.launchContext = launchContext;
     this.containerTokenIdentifier = containerTokenIdentifier;
     this.containerId = containerTokenIdentifier.getContainerID();
@@ -155,22 +157,22 @@ public class ContainerImpl implements Container {
   public ContainerImpl(Configuration conf, Dispatcher dispatcher,
       ContainerLaunchContext launchContext, Credentials creds,
       NodeManagerMetrics metrics,
-      ContainerTokenIdentifier containerTokenIdentifier,
-      RecoveredContainerStatus recoveredStatus, int exitCode,
-      String diagnostics, boolean wasKilled, Resource recoveredCapability,
-      Context context) {
+      ContainerTokenIdentifier containerTokenIdentifier, Context context,
+      RecoveredContainerState rcs) {
     this(conf, dispatcher, launchContext, creds, metrics,
         containerTokenIdentifier, context);
-    this.recoveredStatus = recoveredStatus;
-    this.exitCode = exitCode;
-    this.recoveredAsKilled = wasKilled;
+    this.recoveredStatus = rcs.getStatus();
+    this.exitCode = rcs.getExitCode();
+    this.recoveredAsKilled = rcs.getKilled();
     this.diagnostics.append(diagnostics);
+    Resource recoveredCapability = rcs.getCapability();
     if (recoveredCapability != null
         && !this.resource.equals(recoveredCapability)) {
       // resource capability had been updated before NM was down
       this.resource = Resource.newInstance(recoveredCapability.getMemorySize(),
           recoveredCapability.getVirtualCores());
     }
+    this.version = rcs.getVersion();
   }
 
   private static final ContainerDiagnosticsUpdateTransition UPDATE_DIAGNOSTICS_TRANSITION =
@@ -446,8 +448,8 @@ public class ContainerImpl implements Container {
   public NMContainerStatus getNMContainerStatus() {
     this.readLock.lock();
     try {
-      return NMContainerStatus.newInstance(this.containerId, getCurrentState(),
-          getResource(), diagnostics.toString(), exitCode,
+      return NMContainerStatus.newInstance(this.containerId, this.version,
+          getCurrentState(), getResource(), diagnostics.toString(), exitCode,
           containerTokenIdentifier.getPriority(),
           containerTokenIdentifier.getCreationTime(),
           containerTokenIdentifier.getNodeLabelExpression());

+ 40 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java

@@ -104,6 +104,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
   private static final String CONTAINERS_KEY_PREFIX =
       "ContainerManager/containers/";
   private static final String CONTAINER_REQUEST_KEY_SUFFIX = "/request";
+  private static final String CONTAINER_VERSION_KEY_SUFFIX = "/version";
   private static final String CONTAINER_DIAGS_KEY_SUFFIX = "/diagnostics";
   private static final String CONTAINER_LAUNCHED_KEY_SUFFIX = "/launched";
   private static final String CONTAINER_RESOURCE_CHANGED_KEY_SUFFIX =
@@ -233,6 +234,8 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
       if (suffix.equals(CONTAINER_REQUEST_KEY_SUFFIX)) {
         rcs.startRequest = new StartContainerRequestPBImpl(
             StartContainerRequestProto.parseFrom(entry.getValue()));
+      } else if (suffix.equals(CONTAINER_VERSION_KEY_SUFFIX)) {
+        rcs.version = Integer.parseInt(asString(entry.getValue()));
       } else if (suffix.equals(CONTAINER_DIAGS_KEY_SUFFIX)) {
         rcs.diagnostics = asString(entry.getValue());
       } else if (suffix.equals(CONTAINER_LAUNCHED_KEY_SUFFIX)) {
@@ -255,13 +258,27 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
   }
 
   @Override
-  public void storeContainer(ContainerId containerId,
+  public void storeContainer(ContainerId containerId, int containerVersion,
       StartContainerRequest startRequest) throws IOException {
-    String key = CONTAINERS_KEY_PREFIX + containerId.toString()
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("storeContainer: containerId= " + containerId
+          + ", startRequest= " + startRequest);
+    }
+    String keyRequest = CONTAINERS_KEY_PREFIX + containerId.toString()
         + CONTAINER_REQUEST_KEY_SUFFIX;
+    String keyVersion = CONTAINERS_KEY_PREFIX + containerId.toString()
+        + CONTAINER_VERSION_KEY_SUFFIX;
     try {
-      db.put(bytes(key),
-        ((StartContainerRequestPBImpl) startRequest).getProto().toByteArray());
+      WriteBatch batch = db.createWriteBatch();
+      try {
+        batch.put(bytes(keyRequest),
+            ((StartContainerRequestPBImpl) startRequest)
+                .getProto().toByteArray());
+        batch.put(bytes(keyVersion), bytes(Integer.toString(containerVersion)));
+        db.write(batch);
+      } finally {
+        batch.close();
+      }
     } catch (DBException e) {
       throw new IOException(e);
     }
@@ -293,13 +310,27 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
 
   @Override
   public void storeContainerResourceChanged(ContainerId containerId,
-      Resource capability) throws IOException {
-    String key = CONTAINERS_KEY_PREFIX + containerId.toString()
+      int containerVersion, Resource capability) throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("storeContainerResourceChanged: containerId=" + containerId
+          + ", capability=" + capability);
+    }
+
+    String keyResChng = CONTAINERS_KEY_PREFIX + containerId.toString()
         + CONTAINER_RESOURCE_CHANGED_KEY_SUFFIX;
+    String keyVersion = CONTAINERS_KEY_PREFIX + containerId.toString()
+        + CONTAINER_VERSION_KEY_SUFFIX;
     try {
-      // New value will overwrite old values for the same key
-      db.put(bytes(key),
-          ((ResourcePBImpl) capability).getProto().toByteArray());
+      WriteBatch batch = db.createWriteBatch();
+      try {
+        // New value will overwrite old values for the same key
+        batch.put(bytes(keyResChng),
+            ((ResourcePBImpl) capability).getProto().toByteArray());
+        batch.put(bytes(keyVersion), bytes(Integer.toString(containerVersion)));
+        db.write(batch);
+      } finally {
+        batch.close();
+      }
     } catch (DBException e) {
       throw new IOException(e);
     }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java

@@ -70,7 +70,7 @@ public class NMNullStateStoreService extends NMStateStoreService {
   }
 
   @Override
-  public void storeContainer(ContainerId containerId,
+  public void storeContainer(ContainerId containerId, int version,
       StartContainerRequest startRequest) throws IOException {
   }
 
@@ -86,7 +86,7 @@ public class NMNullStateStoreService extends NMStateStoreService {
 
   @Override
   public void storeContainerResourceChanged(ContainerId containerId,
-      Resource capability) throws IOException {
+      int version, Resource capability) throws IOException {
   }
 
   @Override

+ 22 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java

@@ -72,6 +72,7 @@ public abstract class NMStateStoreService extends AbstractService {
     String diagnostics = "";
     StartContainerRequest startRequest;
     Resource capability;
+    int version;
 
     public RecoveredContainerStatus getStatus() {
       return status;
@@ -89,6 +90,10 @@ public abstract class NMStateStoreService extends AbstractService {
       return diagnostics;
     }
 
+    public int getVersion() {
+      return version;
+    }
+
     public StartContainerRequest getStartRequest() {
       return startRequest;
     }
@@ -96,6 +101,18 @@ public abstract class NMStateStoreService extends AbstractService {
     public Resource getCapability() {
       return capability;
     }
+
+    @Override
+    public String toString() {
+      return new StringBuffer("Status: ").append(getStatus())
+          .append(", Exit code: ").append(exitCode)
+          .append(", Version: ").append(version)
+          .append(", Killed: ").append(getKilled())
+          .append(", Diagnostics: ").append(getDiagnostics())
+          .append(", Capability: ").append(getCapability())
+          .append(", StartRequest: ").append(getStartRequest())
+          .toString();
+    }
   }
 
   public static class LocalResourceTrackerState {
@@ -263,11 +280,13 @@ public abstract class NMStateStoreService extends AbstractService {
   /**
    * Record a container start request
    * @param containerId the container ID
+   * @param containerVersion the container Version
    * @param startRequest the container start request
    * @throws IOException
    */
   public abstract void storeContainer(ContainerId containerId,
-      StartContainerRequest startRequest) throws IOException;
+      int containerVersion, StartContainerRequest startRequest)
+      throws IOException;
 
   /**
    * Record that a container has been launched
@@ -280,11 +299,12 @@ public abstract class NMStateStoreService extends AbstractService {
   /**
    * Record that a container resource has been changed
    * @param containerId the container ID
+   * @param containerVersion the container version
    * @param capability the container resource capability
    * @throws IOException
    */
   public abstract void storeContainerResourceChanged(ContainerId containerId,
-      Resource capability) throws IOException;
+      int containerVersion, Resource capability) throws IOException;
 
   /**
    * Record that a container has completed

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java

@@ -875,7 +875,7 @@ public class TestNodeManagerResync {
         ApplicationAttemptId.newInstance(applicationId, 1);
     ContainerId containerId = ContainerId.newContainerId(applicationAttemptId, id);
     NMContainerStatus containerReport =
-        NMContainerStatus.newInstance(containerId, containerState,
+        NMContainerStatus.newInstance(containerId, 0, containerState,
           Resource.newInstance(1024, 1), "recover container", 0,
           Priority.newInstance(10), 0);
     return containerReport;

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java

@@ -249,7 +249,7 @@ public class TestNodeStatusUpdater {
         String user = "testUser";
         ContainerTokenIdentifier containerToken = BuilderUtils
             .newContainerTokenIdentifier(BuilderUtils.newContainerToken(
-                firstContainerID, InetAddress.getByName("localhost")
+                firstContainerID, 0, InetAddress.getByName("localhost")
                     .getCanonicalHostName(), 1234, user, resource,
                 currentTime + 10000, 123, "password".getBytes(), currentTime));
         Context context = mock(Context.class);
@@ -290,7 +290,7 @@ public class TestNodeStatusUpdater {
         Resource resource = BuilderUtils.newResource(3, 1);
         ContainerTokenIdentifier containerToken = BuilderUtils
             .newContainerTokenIdentifier(BuilderUtils.newContainerToken(
-                secondContainerID, InetAddress.getByName("localhost")
+                secondContainerID, 0, InetAddress.getByName("localhost")
                     .getCanonicalHostName(), 1234, user, resource,
                 currentTime + 10000, 123, "password".getBytes(), currentTime));
         Context context = mock(Context.class);
@@ -1009,7 +1009,7 @@ public class TestNodeStatusUpdater {
   
     ContainerId cId = ContainerId.newContainerId(appAttemptId, 1);
     Token containerToken =
-        BuilderUtils.newContainerToken(cId, "anyHost", 1234, "anyUser",
+        BuilderUtils.newContainerToken(cId, 0, "anyHost", 1234, "anyUser",
             BuilderUtils.newResource(1024, 1), 0, 123,
             "password".getBytes(), 0);
     Container anyCompletedContainer = new ContainerImpl(conf, null,
@@ -1031,7 +1031,7 @@ public class TestNodeStatusUpdater {
     ContainerId runningContainerId =
         ContainerId.newContainerId(appAttemptId, 3);
     Token runningContainerToken =
-        BuilderUtils.newContainerToken(runningContainerId, "anyHost",
+        BuilderUtils.newContainerToken(runningContainerId, 0, "anyHost",
           1234, "anyUser", BuilderUtils.newResource(1024, 1), 0, 123,
           "password".getBytes(), 0);
     Container runningContainer =
@@ -1090,7 +1090,7 @@ public class TestNodeStatusUpdater {
         ApplicationAttemptId.newInstance(appId, 0);
     ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
     Token containerToken =
-        BuilderUtils.newContainerToken(containerId, "host", 1234, "user",
+        BuilderUtils.newContainerToken(containerId, 0, "host", 1234, "user",
             BuilderUtils.newResource(1024, 1), 0, 123,
             "password".getBytes(), 0);
     Container completedContainer = new ContainerImpl(conf, null,
@@ -1128,7 +1128,7 @@ public class TestNodeStatusUpdater {
 
     ContainerId cId = ContainerId.newContainerId(appAttemptId, 1);
     Token containerToken =
-        BuilderUtils.newContainerToken(cId, "anyHost", 1234, "anyUser",
+        BuilderUtils.newContainerToken(cId, 0, "anyHost", 1234, "anyUser",
             BuilderUtils.newResource(1024, 1), 0, 123,
             "password".getBytes(), 0);
     Container anyCompletedContainer = new ContainerImpl(conf, null,

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java

@@ -106,6 +106,7 @@ import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -297,8 +298,7 @@ public class MockResourceManagerFacade implements
         new ArrayList<ContainerStatus>(), containerList,
         new ArrayList<NodeReport>(), null, AMCommand.AM_RESYNC, 1, null,
         new ArrayList<NMToken>(),
-        new ArrayList<Container>(),
-        new ArrayList<Container>());
+        new ArrayList<UpdatedContainer>());
   }
 
   @Override

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java

@@ -122,9 +122,10 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
 
   @Override
   public synchronized void storeContainer(ContainerId containerId,
-      StartContainerRequest startRequest) throws IOException {
+      int version, StartContainerRequest startRequest) throws IOException {
     RecoveredContainerState rcs = new RecoveredContainerState();
     rcs.startRequest = startRequest;
+    rcs.version = version;
     containerStates.put(containerId, rcs);
   }
 
@@ -147,9 +148,11 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
 
   @Override
   public synchronized void storeContainerResourceChanged(
-      ContainerId containerId, Resource capability) throws IOException {
+      ContainerId containerId, int version, Resource capability)
+      throws IOException {
     RecoveredContainerState rcs = getRecoveredContainerState(containerId);
     rcs.capability = capability;
+    rcs.version = version;
   }
 
   @Override

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java

@@ -262,11 +262,12 @@ public class TestNMLeveldbStateStoreService {
         StartContainerRequest.newInstance(clc, containerToken);
 
     // store a container and verify recovered
-    stateStore.storeContainer(containerId, containerReq);
+    stateStore.storeContainer(containerId, 1, containerReq);
     restartStateStore();
     recoveredContainers = stateStore.loadContainersState();
     assertEquals(1, recoveredContainers.size());
     RecoveredContainerState rcs = recoveredContainers.get(0);
+    assertEquals(1, rcs.getVersion());
     assertEquals(RecoveredContainerStatus.REQUESTED, rcs.getStatus());
     assertEquals(ContainerExitStatus.INVALID, rcs.getExitCode());
     assertEquals(false, rcs.getKilled());
@@ -296,11 +297,13 @@ public class TestNMLeveldbStateStoreService {
     assertEquals(diags.toString(), rcs.getDiagnostics());
 
     // increase the container size, and verify recovered
-    stateStore.storeContainerResourceChanged(containerId, Resource.newInstance(2468, 4));
+    stateStore.storeContainerResourceChanged(containerId, 2,
+        Resource.newInstance(2468, 4));
     restartStateStore();
     recoveredContainers = stateStore.loadContainersState();
     assertEquals(1, recoveredContainers.size());
     rcs = recoveredContainers.get(0);
+    assertEquals(2, rcs.getVersion());
     assertEquals(RecoveredContainerStatus.LAUNCHED, rcs.getStatus());
     assertEquals(ContainerExitStatus.INVALID, rcs.getExitCode());
     assertEquals(false, rcs.getKilled());

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java

@@ -66,7 +66,7 @@ public class MockContainer implements Container {
     long currentTime = System.currentTimeMillis();
     this.containerTokenIdentifier =
         BuilderUtils.newContainerTokenIdentifier(BuilderUtils
-          .newContainerToken(id, "127.0.0.1", 1234, user,
+          .newContainerToken(id, 0, "127.0.0.1", 1234, user,
             BuilderUtils.newResource(1024, 1), currentTime + 10000, 123,
             "password".getBytes(), currentTime));
     this.state = ContainerState.NEW;

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java

@@ -212,9 +212,9 @@ public class TestNMWebServer {
           recordFactory.newRecordInstance(ContainerLaunchContext.class);
       long currentTime = System.currentTimeMillis();
       Token containerToken =
-          BuilderUtils.newContainerToken(containerId, "127.0.0.1", 1234, user,
-            BuilderUtils.newResource(1024, 1), currentTime + 10000L, 123,
-            "password".getBytes(), currentTime);
+          BuilderUtils.newContainerToken(containerId, 0, "127.0.0.1", 1234,
+              user, BuilderUtils.newResource(1024, 1), currentTime + 10000L,
+              123, "password".getBytes(), currentTime);
       Context context = mock(Context.class);
       Container container =
           new ContainerImpl(conf, dispatcher, launchContext,

+ 40 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -64,6 +65,9 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
+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.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
 import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
@@ -85,6 +89,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptS
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptStatusupdateEvent;
 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.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
@@ -471,15 +476,6 @@ public class ApplicationMasterService extends AbstractService implements
         throw e;
       }
 
-      try {
-        RMServerUtils.increaseDecreaseRequestSanityCheck(rmContext,
-            request.getIncreaseRequests(), request.getDecreaseRequests(),
-            maximumCapacity);
-      } catch (InvalidResourceRequestException e) {
-        LOG.warn(e);
-        throw e;
-      }
-
       // In the case of work-preserving AM restart, it's possible for the
       // AM to release containers from the earlier attempt.
       if (!app.getApplicationSubmissionContext()
@@ -487,11 +483,22 @@ public class ApplicationMasterService extends AbstractService implements
         try {
           RMServerUtils.validateContainerReleaseRequest(release, appAttemptId);
         } catch (InvalidContainerReleaseException e) {
-          LOG.warn("Invalid container release by application " + appAttemptId, e);
+          LOG.warn("Invalid container release by application " + appAttemptId,
+              e);
           throw e;
         }
       }
 
+      // Split Update Resource Requests into increase and decrease.
+      // No Exceptions are thrown here. All update errors are aggregated
+      // and returned to the AM.
+      List<UpdateContainerRequest> increaseResourceReqs = new ArrayList<>();
+      List<UpdateContainerRequest> decreaseResourceReqs = new ArrayList<>();
+      List<UpdateContainerError> updateContainerErrors =
+          RMServerUtils.validateAndSplitUpdateResourceRequests(rmContext,
+              request, maximumCapacity, increaseResourceReqs,
+              decreaseResourceReqs);
+
       // Send new requests to appAttempt.
       Allocation allocation;
       RMAppAttemptState state =
@@ -506,7 +513,7 @@ public class ApplicationMasterService extends AbstractService implements
         allocation =
             this.rScheduler.allocate(appAttemptId, ask, release,
                 blacklistAdditions, blacklistRemovals,
-                request.getIncreaseRequests(), request.getDecreaseRequests());
+                increaseResourceReqs, decreaseResourceReqs);
       }
 
       if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) {
@@ -521,6 +528,10 @@ public class ApplicationMasterService extends AbstractService implements
         allocateResponse.setNMTokens(allocation.getNMTokens());
       }
 
+      // Notify the AM of container update errors
+      if (!updateContainerErrors.isEmpty()) {
+        allocateResponse.setUpdateErrors(updateContainerErrors);
+      }
       // update the response with the deltas of node status changes
       List<RMNode> updatedNodes = new ArrayList<RMNode>();
       if(app.pullRMNodeUpdates(updatedNodes) > 0) {
@@ -554,8 +565,23 @@ public class ApplicationMasterService extends AbstractService implements
       allocateResponse.setAvailableResources(allocation.getResourceLimit());
       
       // Handling increased/decreased containers
-      allocateResponse.setIncreasedContainers(allocation.getIncreasedContainers());
-      allocateResponse.setDecreasedContainers(allocation.getDecreasedContainers());
+      List<UpdatedContainer> updatedContainers = new ArrayList<>();
+      if (allocation.getIncreasedContainers() != null) {
+        for (Container c : allocation.getIncreasedContainers()) {
+          updatedContainers.add(
+              UpdatedContainer.newInstance(
+                  ContainerUpdateType.INCREASE_RESOURCE, c));
+        }
+      }
+      if (allocation.getDecreasedContainers() != null) {
+        for (Container c : allocation.getDecreasedContainers()) {
+          updatedContainers.add(
+              UpdatedContainer.newInstance(
+                  ContainerUpdateType.DECREASE_RESOURCE, c));
+        }
+      }
+
+      allocateResponse.setUpdatedContainers(updatedContainers);
 
       allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
 
@@ -599,7 +625,7 @@ public class ApplicationMasterService extends AbstractService implements
       return allocateResponse;
     }    
   }
-  
+
   private PreemptionMessage generatePreemptionMessage(Allocation allocation){
     PreemptionMessage pMsg = null;
     // assemble strict preemption request

+ 181 - 157
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java

@@ -6,9 +6,9 @@
  * 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
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -33,28 +33,35 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 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.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.InvalidContainerReleaseException;
-import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException;
+import org.apache.hadoop.yarn.exceptions
+    .InvalidResourceBlacklistRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt
+    .RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler
     .SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
@@ -68,6 +75,18 @@ import org.apache.hadoop.yarn.util.resource.Resources;
  */
 public class RMServerUtils {
 
+  private static final String UPDATE_OUTSTANDING_ERROR =
+      "UPDATE_OUTSTANDING_ERROR";
+  private static final String INCORRECT_CONTAINER_VERSION_ERROR =
+      "INCORRECT_CONTAINER_VERSION_ERROR";
+  private static final String INVALID_CONTAINER_ID =
+      "INVALID_CONTAINER_ID";
+  private static final String RESOURCE_OUTSIDE_ALLOWED_RANGE =
+      "RESOURCE_OUTSIDE_ALLOWED_RANGE";
+
+  protected static final RecordFactory RECORD_FACTORY =
+      RecordFactoryProvider.getRecordFactory(null);
+
   public static List<RMNode> queryRMNodes(RMContext context,
       EnumSet<NodeState> acceptedStates) {
     // nodes contains nodes that are NEW, RUNNING OR UNHEALTHY
@@ -96,6 +115,78 @@ public class RMServerUtils {
     return results;
   }
 
+  /**
+   * 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
+   */
+  public static List<UpdateContainerError>
+      validateAndSplitUpdateResourceRequests(RMContext rmContext,
+      AllocateRequest request, Resource maximumAllocation,
+      List<UpdateContainerRequest> increaseResourceReqs,
+      List<UpdateContainerRequest> decreaseResourceReqs) {
+    List<UpdateContainerError> errors = new ArrayList<>();
+    Set<ContainerId> outstandingUpdate = new HashSet<>();
+    for (UpdateContainerRequest updateReq : request.getUpdateRequests()) {
+      RMContainer rmContainer = rmContext.getScheduler().getRMContainer(
+          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;
+      }
+      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());
+          } else {
+            msg = RESOURCE_OUTSIDE_ALLOWED_RANGE;
+          }
+        } 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;
+          }
+        }
+      }
+      if (msg != null) {
+        UpdateContainerError updateError = RECORD_FACTORY
+            .newRecordInstance(UpdateContainerError.class);
+        updateError.setReason(msg);
+        updateError.setUpdateContainerRequest(updateReq);
+        errors.add(updateError);
+      }
+    }
+    return errors;
+  }
+
   /**
    * Utility method to validate a list resource requests, by insuring that the
    * requested memory/vcore is non-negative and not greater than max
@@ -122,8 +213,6 @@ public class RMServerUtils {
    * the queue lock to make sure that the access to container resource is
    * atomic. Refer to LeafQueue.decreaseContainer() and
    * CapacityScheduelr.updateIncreaseRequests()
-   *
-   * 
    * <pre>
    * - Throw exception when any other error happens
    * </pre>
@@ -145,7 +234,7 @@ public class RMServerUtils {
     if (increase) {
       if (originalResource.getMemorySize() > targetResource.getMemorySize()
           || originalResource.getVirtualCores() > targetResource
-              .getVirtualCores()) {
+          .getVirtualCores()) {
         String msg =
             "Trying to increase a container, but target resource has some"
                 + " resource < original resource, target=" + targetResource
@@ -156,7 +245,7 @@ public class RMServerUtils {
     } else {
       if (originalResource.getMemorySize() < targetResource.getMemorySize()
           || originalResource.getVirtualCores() < targetResource
-              .getVirtualCores()) {
+          .getVirtualCores()) {
         String msg =
             "Trying to decrease a container, but target resource has "
                 + "some resource > original resource, target=" + targetResource
@@ -194,112 +283,46 @@ public class RMServerUtils {
       }
     }
   }
-  
-  /**
-   * Check if we have:
-   * - Request for same containerId and different target resource
-   * - If targetResources violates maximum/minimumAllocation
-   */
-  public static void increaseDecreaseRequestSanityCheck(RMContext rmContext,
-      List<ContainerResourceChangeRequest> incRequests,
-      List<ContainerResourceChangeRequest> decRequests,
-      Resource maximumAllocation) throws InvalidResourceRequestException {
-    checkDuplicatedIncreaseDecreaseRequest(incRequests, decRequests);
-    validateIncreaseDecreaseRequest(rmContext, incRequests, maximumAllocation,
-        true);
-    validateIncreaseDecreaseRequest(rmContext, decRequests, maximumAllocation,
-        false);
-  }
-  
-  private static void checkDuplicatedIncreaseDecreaseRequest(
-      List<ContainerResourceChangeRequest> incRequests,
-      List<ContainerResourceChangeRequest> decRequests)
-          throws InvalidResourceRequestException {
-    String msg = "There're multiple increase or decrease container requests "
-        + "for same containerId=";
-    Set<ContainerId> existedContainerIds = new HashSet<ContainerId>();
-    if (incRequests != null) {
-      for (ContainerResourceChangeRequest r : incRequests) {
-        if (!existedContainerIds.add(r.getContainerId())) {
-          throw new InvalidResourceRequestException(msg + r.getContainerId());
-        }
-      }
-    }
-    
-    if (decRequests != null) {
-      for (ContainerResourceChangeRequest r : decRequests) {
-        if (!existedContainerIds.add(r.getContainerId())) {
-          throw new InvalidResourceRequestException(msg + r.getContainerId());
-        }
-      }
-    }
-  }
 
   // Sanity check and normalize target resource
-  private static void validateIncreaseDecreaseRequest(RMContext rmContext,
-      List<ContainerResourceChangeRequest> requests, Resource maximumAllocation,
-      boolean increase)
-      throws InvalidResourceRequestException {
-    if (requests == null) {
-      return;
+  private static boolean validateIncreaseDecreaseRequest(RMContext rmContext,
+      UpdateContainerRequest request, Resource maximumAllocation,
+      boolean increase) {
+    if (request.getCapability().getMemorySize() < 0
+        || request.getCapability().getMemorySize() > maximumAllocation
+        .getMemorySize()) {
+      return false;
     }
-    for (ContainerResourceChangeRequest request : requests) {
-      if (request.getCapability().getMemorySize() < 0
-          || request.getCapability().getMemorySize() > maximumAllocation
-              .getMemorySize()) {
-        throw new InvalidResourceRequestException("Invalid "
-            + (increase ? "increase" : "decrease") + " request"
-            + ", requested memory < 0"
-            + ", or requested memory > max configured" + ", requestedMemory="
-            + request.getCapability().getMemorySize() + ", maxMemory="
-            + maximumAllocation.getMemorySize());
-      }
-      if (request.getCapability().getVirtualCores() < 0
-          || request.getCapability().getVirtualCores() > maximumAllocation
-              .getVirtualCores()) {
-        throw new InvalidResourceRequestException("Invalid "
-            + (increase ? "increase" : "decrease") + " request"
-            + ", requested virtual cores < 0"
-            + ", or requested virtual cores > max configured"
-            + ", requestedVirtualCores="
-            + request.getCapability().getVirtualCores() + ", maxVirtualCores="
-            + maximumAllocation.getVirtualCores());
-      }
-      ContainerId containerId = request.getContainerId();
-      ResourceScheduler scheduler = rmContext.getScheduler();
-      RMContainer rmContainer = scheduler.getRMContainer(containerId);
-      if (null == rmContainer) {
-        String msg =
-            "Failed to get rmContainer for "
-                + (increase ? "increase" : "decrease")
-                + " request, with container-id=" + containerId;
-        throw new InvalidResourceRequestException(msg);
-      }
-      ResourceCalculator rc = scheduler.getResourceCalculator();
-      Resource targetResource = Resources.normalize(rc, request.getCapability(),
-          scheduler.getMinimumResourceCapability(),
-          scheduler.getMaximumResourceCapability(),
-          scheduler.getMinimumResourceCapability());
-      // Update normalized target resource
-      request.setCapability(targetResource);
+    if (request.getCapability().getVirtualCores() < 0
+        || request.getCapability().getVirtualCores() > maximumAllocation
+        .getVirtualCores()) {
+      return false;
     }
+    ResourceScheduler scheduler = rmContext.getScheduler();
+    ResourceCalculator rc = scheduler.getResourceCalculator();
+    Resource targetResource = Resources.normalize(rc, request.getCapability(),
+        scheduler.getMinimumResourceCapability(),
+        scheduler.getMaximumResourceCapability(),
+        scheduler.getMinimumResourceCapability());
+    // Update normalized target resource
+    request.setCapability(targetResource);
+    return true;
   }
 
   /**
    * It will validate to make sure all the containers belong to correct
    * application attempt id. If not then it will throw
    * {@link InvalidContainerReleaseException}
-   * 
-   * @param containerReleaseList
-   *          containers to be released as requested by application master.
-   * @param appAttemptId
-   *          Application attempt Id
+   *
+   * @param containerReleaseList containers to be released as requested by
+   *                             application master.
+   * @param appAttemptId         Application attempt Id
    * @throws InvalidContainerReleaseException
    */
   public static void
       validateContainerReleaseRequest(List<ContainerId> containerReleaseList,
-          ApplicationAttemptId appAttemptId)
-          throws InvalidContainerReleaseException {
+      ApplicationAttemptId appAttemptId)
+      throws InvalidContainerReleaseException {
     for (ContainerId cId : containerReleaseList) {
       if (!appAttemptId.equals(cId.getApplicationAttemptId())) {
         throw new InvalidContainerReleaseException(
@@ -321,10 +344,11 @@ public class RMServerUtils {
   /**
    * Utility method to verify if the current user has access based on the
    * passed {@link AccessControlList}
+   *
    * @param authorizer the {@link AccessControlList} to check against
-   * @param method the method name to be logged
-   * @param module like AdminService or NodeLabelManager
-   * @param LOG the logger to use
+   * @param method     the method name to be logged
+   * @param module     like AdminService or NodeLabelManager
+   * @param LOG        the logger to use
    * @return {@link UserGroupInformation} of the current user
    * @throws IOException
    */
@@ -347,11 +371,11 @@ public class RMServerUtils {
           " to call '" + method + "'");
 
       RMAuditLogger.logFailure(user.getShortUserName(), method, "", module,
-        RMAuditLogger.AuditConstants.UNAUTHORIZED_USER);
+          RMAuditLogger.AuditConstants.UNAUTHORIZED_USER);
 
       throw new AccessControlException("User " + user.getShortUserName() +
-              " doesn't have permission" +
-              " to call '" + method + "'");
+          " doesn't have permission" +
+          " to call '" + method + "'");
     }
     if (LOG.isTraceEnabled()) {
       LOG.trace(method + " invoked by user " + user.getShortUserName());
@@ -362,56 +386,56 @@ public class RMServerUtils {
   public static YarnApplicationState createApplicationState(
       RMAppState rmAppState) {
     switch (rmAppState) {
-      case NEW:
-        return YarnApplicationState.NEW;
-      case NEW_SAVING:
-        return YarnApplicationState.NEW_SAVING;
-      case SUBMITTED:
-        return YarnApplicationState.SUBMITTED;
-      case ACCEPTED:
-        return YarnApplicationState.ACCEPTED;
-      case RUNNING:
-        return YarnApplicationState.RUNNING;
-      case FINISHING:
-      case FINISHED:
-        return YarnApplicationState.FINISHED;
-      case KILLED:
-        return YarnApplicationState.KILLED;
-      case FAILED:
-        return YarnApplicationState.FAILED;
-      default:
-        throw new YarnRuntimeException("Unknown state passed!");
-      }
+    case NEW:
+      return YarnApplicationState.NEW;
+    case NEW_SAVING:
+      return YarnApplicationState.NEW_SAVING;
+    case SUBMITTED:
+      return YarnApplicationState.SUBMITTED;
+    case ACCEPTED:
+      return YarnApplicationState.ACCEPTED;
+    case RUNNING:
+      return YarnApplicationState.RUNNING;
+    case FINISHING:
+    case FINISHED:
+      return YarnApplicationState.FINISHED;
+    case KILLED:
+      return YarnApplicationState.KILLED;
+    case FAILED:
+      return YarnApplicationState.FAILED;
+    default:
+      throw new YarnRuntimeException("Unknown state passed!");
+    }
   }
 
   public static YarnApplicationAttemptState createApplicationAttemptState(
       RMAppAttemptState rmAppAttemptState) {
     switch (rmAppAttemptState) {
-      case NEW:
-        return YarnApplicationAttemptState.NEW;
-      case SUBMITTED:
-        return YarnApplicationAttemptState.SUBMITTED;
-      case SCHEDULED:
-        return YarnApplicationAttemptState.SCHEDULED;
-      case ALLOCATED:
-        return YarnApplicationAttemptState.ALLOCATED;
-      case LAUNCHED:
-        return YarnApplicationAttemptState.LAUNCHED;
-      case ALLOCATED_SAVING:
-      case LAUNCHED_UNMANAGED_SAVING:
-        return YarnApplicationAttemptState.ALLOCATED_SAVING;
-      case RUNNING:
-        return YarnApplicationAttemptState.RUNNING;
-      case FINISHING:
-        return YarnApplicationAttemptState.FINISHING;
-      case FINISHED:
-        return YarnApplicationAttemptState.FINISHED;
-      case KILLED:
-        return YarnApplicationAttemptState.KILLED;
-      case FAILED:
-        return YarnApplicationAttemptState.FAILED;
-      default:
-        throw new YarnRuntimeException("Unknown state passed!");
+    case NEW:
+      return YarnApplicationAttemptState.NEW;
+    case SUBMITTED:
+      return YarnApplicationAttemptState.SUBMITTED;
+    case SCHEDULED:
+      return YarnApplicationAttemptState.SCHEDULED;
+    case ALLOCATED:
+      return YarnApplicationAttemptState.ALLOCATED;
+    case LAUNCHED:
+      return YarnApplicationAttemptState.LAUNCHED;
+    case ALLOCATED_SAVING:
+    case LAUNCHED_UNMANAGED_SAVING:
+      return YarnApplicationAttemptState.ALLOCATED_SAVING;
+    case RUNNING:
+      return YarnApplicationAttemptState.RUNNING;
+    case FINISHING:
+      return YarnApplicationAttemptState.FINISHING;
+    case FINISHED:
+      return YarnApplicationAttemptState.FINISHED;
+    case KILLED:
+      return YarnApplicationAttemptState.KILLED;
+    case FAILED:
+      return YarnApplicationAttemptState.FAILED;
+    default:
+      throw new YarnRuntimeException("Unknown state passed!");
     }
   }
 
@@ -420,13 +444,12 @@ public class RMServerUtils {
    * a return value when a valid report cannot be found.
    */
   public static final ApplicationResourceUsageReport
-    DUMMY_APPLICATION_RESOURCE_USAGE_REPORT =
+      DUMMY_APPLICATION_RESOURCE_USAGE_REPORT =
       BuilderUtils.newApplicationResourceUsageReport(-1, -1,
           Resources.createResource(-1, -1), Resources.createResource(-1, -1),
           Resources.createResource(-1, -1), 0, 0);
 
 
-
   /**
    * Find all configs whose name starts with
    * YarnConfiguration.RM_PROXY_USER_PREFIX, and add a record for each one by
@@ -438,7 +461,8 @@ public class RMServerUtils {
       String propName = entry.getKey();
       if (propName.startsWith(YarnConfiguration.RM_PROXY_USER_PREFIX)) {
         rmProxyUsers.put(ProxyUsers.CONF_HADOOP_PROXYUSER + "." +
-            propName.substring(YarnConfiguration.RM_PROXY_USER_PREFIX.length()),
+                propName.substring(YarnConfiguration.RM_PROXY_USER_PREFIX
+                    .length()),
             entry.getValue());
       }
     }

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

@@ -44,7 +44,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -52,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.ResourceOption;
 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.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -468,6 +468,7 @@ public abstract class AbstractYarnScheduler
         Container.newInstance(status.getContainerId(), node.getNodeID(),
           node.getHttpAddress(), status.getAllocatedResource(),
           status.getPriority(), null);
+    container.setVersion(status.getVersion());
     ApplicationAttemptId attemptId =
         container.getId().getApplicationAttemptId();
     RMContainer rmContainer =
@@ -590,7 +591,7 @@ public abstract class AbstractYarnScheduler
   }
 
   protected void decreaseContainers(
-      List<ContainerResourceChangeRequest> decreaseRequests,
+      List<UpdateContainerRequest> decreaseRequests,
       SchedulerApplicationAttempt attempt) {
     if (null == decreaseRequests || decreaseRequests.isEmpty()) {
       return;
@@ -841,7 +842,7 @@ public abstract class AbstractYarnScheduler
   /**
    * Sanity check increase/decrease request, and return
    * SchedulerContainerResourceChangeRequest according to given
-   * ContainerResourceChangeRequest.
+   * UpdateContainerRequest.
    * 
    * <pre>
    * - Returns non-null value means validation succeeded
@@ -849,7 +850,7 @@ public abstract class AbstractYarnScheduler
    * </pre>
    */
   private SchedContainerChangeRequest createSchedContainerChangeRequest(
-      ContainerResourceChangeRequest request, boolean increase)
+      UpdateContainerRequest request, boolean increase)
       throws YarnException {
     ContainerId containerId = request.getContainerId();
     RMContainer rmContainer = getRMContainer(containerId);
@@ -868,11 +869,11 @@ public abstract class AbstractYarnScheduler
 
   protected List<SchedContainerChangeRequest>
       createSchedContainerChangeRequests(
-          List<ContainerResourceChangeRequest> changeRequests,
+          List<UpdateContainerRequest> changeRequests,
           boolean increase) {
     List<SchedContainerChangeRequest> schedulerChangeRequests =
         new ArrayList<SchedContainerChangeRequest>();
-    for (ContainerResourceChangeRequest r : changeRequests) {
+    for (UpdateContainerRequest r : changeRequests) {
       SchedContainerChangeRequest sr = null;
       try {
         sr = createSchedContainerChangeRequest(r, increase);

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

@@ -27,7 +27,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 /**
- * This is ContainerResourceChangeRequest in scheduler side, it contains some
+ * This is UpdateContainerRequest in scheduler side, it contains some
  * pointers to runtime objects like RMContainer, SchedulerNode, etc. This will
  * be easier for scheduler making decision.
  */

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

@@ -496,6 +496,9 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
       boolean newContainer, boolean increasedContainer) {
     Container container = rmContainer.getContainer();
     ContainerType containerType = ContainerType.TASK;
+    if (!newContainer) {
+      container.setVersion(container.getVersion() + 1);
+    }
     // The working knowledge is that masterContainer for AM is null as it
     // itself is the master container.
     if (isWaitingForAMContainer()) {
@@ -504,10 +507,11 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     try {
       // create container token and NMToken altogether.
       container.setContainerToken(rmContext.getContainerTokenSecretManager()
-          .createContainerToken(container.getId(), container.getNodeId(),
-              getUser(), container.getResource(), container.getPriority(),
-              rmContainer.getCreationTime(), this.logAggregationContext,
-              rmContainer.getNodeLabelExpression(), containerType));
+          .createContainerToken(container.getId(), container.getVersion(),
+              container.getNodeId(), getUser(), container.getResource(),
+              container.getPriority(), rmContainer.getCreationTime(),
+              this.logAggregationContext, rmContainer.getNodeLabelExpression(),
+              containerType));
       NMToken nmToken =
           rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(),
               getApplicationAttemptId(), container);

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

@@ -35,7 +35,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -43,6 +42,7 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 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.event.EventHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -143,8 +143,8 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
   Allocation allocate(ApplicationAttemptId appAttemptId,
       List<ResourceRequest> ask, List<ContainerId> release,
       List<String> blacklistAdditions, List<String> blacklistRemovals,
-      List<ContainerResourceChangeRequest> increaseRequests,
-      List<ContainerResourceChangeRequest> decreaseRequests);
+      List<UpdateContainerRequest> increaseRequests,
+      List<UpdateContainerRequest> decreaseRequests);
 
   /**
    * Get node resource usage report.

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

@@ -37,7 +37,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -49,6 +48,7 @@ import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 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.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -902,7 +902,7 @@ public class CapacityScheduler extends
   //    SchedContainerChangeRequest
   // 2. Deadlock with the scheduling thread.
   private LeafQueue updateIncreaseRequests(
-      List<ContainerResourceChangeRequest> increaseRequests,
+      List<UpdateContainerRequest> increaseRequests,
       FiCaSchedulerApp app) {
     if (null == increaseRequests || increaseRequests.isEmpty()) {
       return null;
@@ -930,8 +930,8 @@ public class CapacityScheduler extends
   public Allocation allocate(ApplicationAttemptId applicationAttemptId,
       List<ResourceRequest> ask, List<ContainerId> release,
       List<String> blacklistAdditions, List<String> blacklistRemovals,
-      List<ContainerResourceChangeRequest> increaseRequests,
-      List<ContainerResourceChangeRequest> decreaseRequests) {
+      List<UpdateContainerRequest> increaseRequests,
+      List<UpdateContainerRequest> decreaseRequests) {
 
     FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {

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

@@ -40,7 +40,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -51,6 +50,7 @@ import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 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.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -955,8 +955,8 @@ public class FairScheduler extends
   public Allocation allocate(ApplicationAttemptId appAttemptId,
       List<ResourceRequest> ask, List<ContainerId> release,
       List<String> blacklistAdditions, List<String> blacklistRemovals,
-      List<ContainerResourceChangeRequest> increaseRequests,
-      List<ContainerResourceChangeRequest> decreaseRequests) {
+      List<UpdateContainerRequest> increaseRequests,
+      List<UpdateContainerRequest> decreaseRequests) {
 
     // Make sure this application exists
     FSAppAttempt application = getSchedulerApp(appAttemptId);

+ 3 - 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.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
@@ -52,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 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.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -316,8 +316,8 @@ public class FifoScheduler extends
   public Allocation allocate(ApplicationAttemptId applicationAttemptId,
       List<ResourceRequest> ask, List<ContainerId> release,
       List<String> blacklistAdditions, List<String> blacklistRemovals,
-      List<ContainerResourceChangeRequest> increaseRequests,
-      List<ContainerResourceChangeRequest> decreaseRequests) {
+      List<UpdateContainerRequest> increaseRequests,
+      List<UpdateContainerRequest> decreaseRequests) {
     FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {
       LOG.error("Calling allocate on removed " +

+ 33 - 29
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java

@@ -168,39 +168,43 @@ public class RMContainerTokenSecretManager extends
   /**
    * Helper function for creating ContainerTokens
    *
-   * @param containerId
-   * @param nodeId
-   * @param appSubmitter
-   * @param capability
-   * @param priority
-   * @param createTime
+   * @param containerId Container Id
+   * @param containerVersion Container Version
+   * @param nodeId Node Id
+   * @param appSubmitter App Submitter
+   * @param capability Capability
+   * @param priority Priority
+   * @param createTime Create Time
    * @return the container-token
    */
-  public Token createContainerToken(ContainerId containerId, NodeId nodeId,
-      String appSubmitter, Resource capability, Priority priority,
-      long createTime) {
-    return createContainerToken(containerId, nodeId, appSubmitter, capability,
-      priority, createTime, null, null, ContainerType.TASK);
+  public Token createContainerToken(ContainerId containerId,
+      int containerVersion, NodeId nodeId, String appSubmitter,
+      Resource capability, Priority priority, long createTime) {
+    return createContainerToken(containerId, containerVersion, nodeId,
+        appSubmitter, capability, priority, createTime,
+        null, null, ContainerType.TASK);
   }
 
   /**
    * Helper function for creating ContainerTokens
    *
-   * @param containerId
-   * @param nodeId
-   * @param appSubmitter
-   * @param capability
-   * @param priority
-   * @param createTime
-   * @param logAggregationContext
-   * @param nodeLabelExpression
-   * @param containerType
+   * @param containerId Container Id
+   * @param containerVersion Container version
+   * @param nodeId Node Id
+   * @param appSubmitter App Submitter
+   * @param capability Capability
+   * @param priority Priority
+   * @param createTime Create Time
+   * @param logAggregationContext Log Aggregation Context
+   * @param nodeLabelExpression Node Label Expression
+   * @param containerType Container Type
    * @return the container-token
    */
-  public Token createContainerToken(ContainerId containerId, NodeId nodeId,
-      String appSubmitter, Resource capability, Priority priority,
-      long createTime, LogAggregationContext logAggregationContext,
-      String nodeLabelExpression, ContainerType containerType) {
+  public Token createContainerToken(ContainerId containerId,
+      int containerVersion, NodeId nodeId, String appSubmitter,
+      Resource capability, Priority priority, long createTime,
+      LogAggregationContext logAggregationContext, String nodeLabelExpression,
+      ContainerType containerType) {
     byte[] password;
     ContainerTokenIdentifier tokenIdentifier;
     long expiryTimeStamp =
@@ -210,11 +214,11 @@ public class RMContainerTokenSecretManager extends
     this.readLock.lock();
     try {
       tokenIdentifier =
-          new ContainerTokenIdentifier(containerId, nodeId.toString(),
-            appSubmitter, capability, expiryTimeStamp, this.currentMasterKey
-              .getMasterKey().getKeyId(),
-            ResourceManager.getClusterTimeStamp(), priority, createTime,
-            logAggregationContext, nodeLabelExpression, containerType);
+          new ContainerTokenIdentifier(containerId, containerVersion,
+              nodeId.toString(), appSubmitter, capability, expiryTimeStamp,
+              this.currentMasterKey.getMasterKey().getKeyId(),
+              ResourceManager.getClusterTimeStamp(), priority, createTime,
+              logAggregationContext, nodeLabelExpression, containerType);
       password = this.createPassword(tokenIdentifier);
 
     } finally {

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java

@@ -34,11 +34,11 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -238,10 +238,9 @@ public class MockAM {
   }
   
   public AllocateResponse sendContainerResizingRequest(
-      List<ContainerResourceChangeRequest> increaseRequests,
-      List<ContainerResourceChangeRequest> decreaseRequests) throws Exception {
+      List<UpdateContainerRequest> updateRequests) throws Exception {
     final AllocateRequest req = AllocateRequest.newInstance(0, 0F, null, null,
-        null, increaseRequests, decreaseRequests);
+        null, updateRequests);
     return allocate(req);
   }
 

+ 5 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java

@@ -383,9 +383,10 @@ public class TestApplicationCleanup {
     // nm1/nm2 register to rm2, and do a heartbeat
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     nm1.registerNode(Arrays.asList(NMContainerStatus.newInstance(
-      ContainerId.newContainerId(am0.getApplicationAttemptId(), 1),
-      ContainerState.COMPLETE, Resource.newInstance(1024, 1), "", 0,
-      Priority.newInstance(0), 1234)), Arrays.asList(app0.getApplicationId()));
+        ContainerId.newContainerId(am0.getApplicationAttemptId(), 1), 0,
+        ContainerState.COMPLETE, Resource.newInstance(1024, 1), "", 0,
+        Priority.newInstance(0), 1234)),
+        Arrays.asList(app0.getApplicationId()));
     nm2.setResourceTrackerService(rm2.getResourceTrackerService());
     nm2.registerNode(Arrays.asList(app0.getApplicationId()));
 
@@ -595,7 +596,7 @@ public class TestApplicationCleanup {
       int memory) {
     ContainerId containerId = ContainerId.newContainerId(appAttemptId, id);
     NMContainerStatus containerReport =
-        NMContainerStatus.newInstance(containerId, containerState,
+        NMContainerStatus.newInstance(containerId, 0, containerState,
             Resource.newInstance(memory, 1), "recover container", 0,
             Priority.newInstance(0), 0);
     return containerReport;

+ 38 - 48
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterService.java

@@ -35,16 +35,16 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 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.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
 import org.apache.hadoop.yarn.exceptions.InvalidContainerReleaseException;
-import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -380,57 +380,47 @@ public class TestApplicationMasterService {
       
       // Ask for a normal increase should be successfull
       am1.sendContainerResizingRequest(Arrays.asList(
-              ContainerResourceChangeRequest.newInstance(
-                  ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
-                  Resources.createResource(2048))), null);
+              UpdateContainerRequest.newInstance(
+                  0, ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
+                  ContainerUpdateType.INCREASE_RESOURCE,
+                  Resources.createResource(2048))));
       
       // Target resource is negative, should fail
-      boolean exceptionCaught = false;
-      try {
-        am1.sendContainerResizingRequest(Arrays.asList(
-                ContainerResourceChangeRequest.newInstance(
-                    ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
-                    Resources.createResource(-1))), null);
-      } catch (InvalidResourceRequestException e) {
-        // This is expected
-        exceptionCaught = true;
-      }
-      Assert.assertTrue(exceptionCaught);
-      
+      AllocateResponse response =
+          am1.sendContainerResizingRequest(Arrays.asList(
+              UpdateContainerRequest.newInstance(0,
+                  ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
+                  ContainerUpdateType.INCREASE_RESOURCE,
+                  Resources.createResource(-1))));
+      Assert.assertEquals(1, response.getUpdateErrors().size());
+      Assert.assertEquals("RESOURCE_OUTSIDE_ALLOWED_RANGE",
+          response.getUpdateErrors().get(0).getReason());
+
       // Target resource is more than maxAllocation, should fail
-      try {
-        am1.sendContainerResizingRequest(Arrays.asList(
-                ContainerResourceChangeRequest.newInstance(
-                    ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
-                    Resources
-                        .add(registerResponse.getMaximumResourceCapability(),
-                            Resources.createResource(1)))), null);
-      } catch (InvalidResourceRequestException e) {
-        // This is expected
-        exceptionCaught = true;
-      }
+      response = am1.sendContainerResizingRequest(Arrays.asList(
+          UpdateContainerRequest.newInstance(0,
+              ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
+              ContainerUpdateType.INCREASE_RESOURCE,
+              Resources.add(
+                  registerResponse.getMaximumResourceCapability(),
+                  Resources.createResource(1)))));
+      Assert.assertEquals(1, response.getUpdateErrors().size());
+      Assert.assertEquals("RESOURCE_OUTSIDE_ALLOWED_RANGE",
+          response.getUpdateErrors().get(0).getReason());
 
-      Assert.assertTrue(exceptionCaught);
-      
       // Contains multiple increase/decrease requests for same contaienrId 
-      try {
-        am1.sendContainerResizingRequest(Arrays.asList(
-                ContainerResourceChangeRequest.newInstance(
-                    ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
-                    Resources
-                        .add(registerResponse.getMaximumResourceCapability(),
-                            Resources.createResource(1)))), Arrays.asList(
-                ContainerResourceChangeRequest.newInstance(
-                    ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
-                    Resources
-                        .add(registerResponse.getMaximumResourceCapability(),
-                            Resources.createResource(1)))));
-      } catch (InvalidResourceRequestException e) {
-        // This is expected
-        exceptionCaught = true;
-      }
-
-      Assert.assertTrue(exceptionCaught);
+      response = am1.sendContainerResizingRequest(Arrays.asList(
+          UpdateContainerRequest.newInstance(0,
+              ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
+              ContainerUpdateType.INCREASE_RESOURCE,
+              Resources.createResource(2048, 4)),
+          UpdateContainerRequest.newInstance(0,
+              ContainerId.newContainerId(attempt1.getAppAttemptId(), 1),
+              ContainerUpdateType.DECREASE_RESOURCE,
+              Resources.createResource(1024, 1))));
+      Assert.assertEquals(1, response.getUpdateErrors().size());
+      Assert.assertEquals("UPDATE_OUTSTANDING_ERROR",
+          response.getUpdateErrors().get(0).getReason());
     } finally {
       if (rm != null) {
         rm.close();

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

@@ -2025,7 +2025,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       String nodeLabelExpression) {
     ContainerId containerId = ContainerId.newContainerId(appAttemptId, id);
     NMContainerStatus containerReport =
-        NMContainerStatus.newInstance(containerId, containerState,
+        NMContainerStatus.newInstance(containerId, 0, containerState,
             Resource.newInstance(1024, 1), "recover container", 0,
             Priority.newInstance(0), 0, nodeLabelExpression);
     return containerReport;

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

@@ -933,7 +933,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
     NMContainerStatus report =
         NMContainerStatus.newInstance(
           ContainerId.newContainerId(
-            ApplicationAttemptId.newInstance(app.getApplicationId(), 2), 1),
+            ApplicationAttemptId.newInstance(app.getApplicationId(), 2), 1), 0,
           ContainerState.COMPLETE, Resource.newInstance(1024, 1),
           "Dummy Completed", 0, Priority.newInstance(10), 1234);
     rm.getResourceTrackerService().handleNMContainerStatus(report, null);
@@ -944,7 +944,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
         (RMAppAttemptImpl) app.getCurrentAppAttempt();
     currentAttempt.setMasterContainer(null);
     report = NMContainerStatus.newInstance(
-          ContainerId.newContainerId(currentAttempt.getAppAttemptId(), 0),
+          ContainerId.newContainerId(currentAttempt.getAppAttemptId(), 0), 0,
           ContainerState.COMPLETE, Resource.newInstance(1024, 1),
           "Dummy Completed", 0, Priority.newInstance(10), 1234);
     rm.getResourceTrackerService().handleNMContainerStatus(report, null);
@@ -956,7 +956,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
     // Case 2.1: AppAttemptId is null
     report = NMContainerStatus.newInstance(
           ContainerId.newContainerId(
-            ApplicationAttemptId.newInstance(app.getApplicationId(), 2), 1),
+            ApplicationAttemptId.newInstance(app.getApplicationId(), 2), 1), 0,
           ContainerState.COMPLETE, Resource.newInstance(1024, 1),
           "Dummy Completed", 0, Priority.newInstance(10), 1234);
     try {
@@ -971,7 +971,7 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
         (RMAppAttemptImpl) app.getCurrentAppAttempt();
     currentAttempt.setMasterContainer(null);
     report = NMContainerStatus.newInstance(
-      ContainerId.newContainerId(currentAttempt.getAppAttemptId(), 0),
+      ContainerId.newContainerId(currentAttempt.getAppAttemptId(), 0), 0,
       ContainerState.COMPLETE, Resource.newInstance(1024, 1),
       "Dummy Completed", 0, Priority.newInstance(10), 1234);
     try {

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

@@ -58,9 +58,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -70,6 +70,7 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 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.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -3229,9 +3230,10 @@ public class TestCapacityScheduler {
 
     // am1 asks to change its AM container from 1GB to 3GB
     am1.sendContainerResizingRequest(Arrays.asList(
-            ContainerResourceChangeRequest
-                .newInstance(containerId1, Resources.createResource(3 * GB))),
-        null);
+            UpdateContainerRequest
+                .newInstance(0, containerId1,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(3 * GB))));
     
     FiCaSchedulerApp app = getFiCaSchedulerApp(rm, app1.getApplicationId());
     
@@ -3243,11 +3245,14 @@ public class TestCapacityScheduler {
     
     // am1 asks to change containerId2 (2G -> 3G) and containerId3 (2G -> 5G)
     am1.sendContainerResizingRequest(Arrays.asList(
-            ContainerResourceChangeRequest
-                .newInstance(containerId2, Resources.createResource(3 * GB)),
-            ContainerResourceChangeRequest
-                .newInstance(containerId3, Resources.createResource(5 * GB))),
-        null);
+        UpdateContainerRequest
+                .newInstance(0, containerId2,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(3 * GB)),
+        UpdateContainerRequest
+                .newInstance(0, containerId3,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(5 * GB))));
     
     Assert.assertEquals(6 * GB,
         app.getAppAttemptResourceUsage().getPending().getMemorySize());
@@ -3258,13 +3263,18 @@ public class TestCapacityScheduler {
     // am1 asks to change containerId1 (1G->3G), containerId2 (2G -> 4G) and
     // containerId3 (2G -> 2G)
     am1.sendContainerResizingRequest(Arrays.asList(
-            ContainerResourceChangeRequest
-                .newInstance(containerId1, Resources.createResource(3 * GB)),
-            ContainerResourceChangeRequest
-                .newInstance(containerId2, Resources.createResource(4 * GB)),
-            ContainerResourceChangeRequest
-                .newInstance(containerId3, Resources.createResource(2 * GB))),
-        null);
+        UpdateContainerRequest
+                .newInstance(0, containerId1,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(3 * GB)),
+        UpdateContainerRequest
+                .newInstance(0, containerId2,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(4 * GB)),
+        UpdateContainerRequest
+                .newInstance(0, containerId3,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(2 * GB))));
     Assert.assertEquals(4 * GB,
         app.getAppAttemptResourceUsage().getPending().getMemorySize());
     checkPendingResource(rm, "a1", 4 * GB, null);

+ 7 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java

@@ -288,13 +288,14 @@ public class TestContainerAllocation {
 
         @Override
         public Token createContainerToken(ContainerId containerId,
-            NodeId nodeId, String appSubmitter, Resource capability,
-            Priority priority, long createTime,
-            LogAggregationContext logAggregationContext, String nodeLabelExp, ContainerType containerType) {
+            int containerVersion, NodeId nodeId, String appSubmitter,
+            Resource capability, Priority priority, long createTime,
+            LogAggregationContext logAggregationContext, String nodeLabelExp,
+            ContainerType containerType) {
           numRetries++;
-          return super.createContainerToken(containerId, nodeId, appSubmitter,
-              capability, priority, createTime, logAggregationContext,
-              nodeLabelExp, containerType);
+          return super.createContainerToken(containerId, containerVersion,
+              nodeId, appSubmitter, capability, priority, createTime,
+              logAggregationContext, nodeLabelExp, containerType);
         }
       };
     }

+ 81 - 53
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerResizing.java

@@ -30,10 +30,12 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
@@ -84,7 +86,7 @@ public class TestContainerResizing {
 
     @Override
     protected void decreaseContainers(
-        List<ContainerResourceChangeRequest> decreaseRequests,
+        List<UpdateContainerRequest> decreaseRequests,
         SchedulerApplicationAttempt attempt) {
       try {
         Thread.sleep(1000);
@@ -138,9 +140,10 @@ public class TestContainerResizing {
     sentRMContainerLaunched(rm1, containerId1);
     // am1 asks to change its AM container from 1GB to 3GB
     am1.sendContainerResizingRequest(Arrays.asList(
-            ContainerResourceChangeRequest
-                .newInstance(containerId1, Resources.createResource(3 * GB))),
-        null);
+            UpdateContainerRequest
+                .newInstance(0, containerId1,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(3 * GB))));
 
     FiCaSchedulerApp app = TestUtils.getFiCaSchedulerApp(
         rm1, app1.getApplicationId());
@@ -195,9 +198,11 @@ public class TestContainerResizing {
     sentRMContainerLaunched(rm1, containerId1);
 
     // am1 asks to change its AM container from 1GB to 3GB
-    AllocateResponse response = am1.sendContainerResizingRequest(null, Arrays
-        .asList(ContainerResourceChangeRequest
-            .newInstance(containerId1, Resources.createResource(1 * GB))));
+    AllocateResponse response = am1.sendContainerResizingRequest(Arrays
+        .asList(UpdateContainerRequest
+            .newInstance(0, containerId1,
+                ContainerUpdateType.DECREASE_RESOURCE,
+                Resources.createResource(1 * GB))));
 
     verifyContainerDecreased(response, containerId1, 1 * GB);
     checkUsedResource(rm1, "default", 1 * GB, null);
@@ -266,9 +271,10 @@ public class TestContainerResizing {
 
     // am1 asks to change its AM container from 1GB to 3GB
     am1.sendContainerResizingRequest(Arrays.asList(
-            ContainerResourceChangeRequest
-                .newInstance(containerId1, Resources.createResource(7 * GB))),
-        null);
+            UpdateContainerRequest
+                .newInstance(0, containerId1,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(7 * GB))));
 
     checkPendingResource(rm1, "default", 6 * GB, null);
     Assert.assertEquals(6 * GB,
@@ -367,9 +373,10 @@ public class TestContainerResizing {
     // am1 asks to change container2 from 2GB to 8GB, which will exceed user
     // limit
     am1.sendContainerResizingRequest(Arrays.asList(
-            ContainerResourceChangeRequest
-                .newInstance(containerId2, Resources.createResource(8 * GB))),
-        null);
+            UpdateContainerRequest
+                .newInstance(0, containerId2,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(8 * GB))));
 
     checkPendingResource(rm1, "default", 6 * GB, null);
     Assert.assertEquals(6 * GB,
@@ -447,9 +454,10 @@ public class TestContainerResizing {
 
     // am1 asks to change its AM container from 1GB to 3GB
     am1.sendContainerResizingRequest(Arrays.asList(
-            ContainerResourceChangeRequest
-                .newInstance(containerId1, Resources.createResource(7 * GB))),
-        null);
+            UpdateContainerRequest
+                .newInstance(0, containerId1,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(7 * GB))));
 
     checkPendingResource(rm1, "default", 6 * GB, null);
     Assert.assertEquals(6 * GB,
@@ -487,9 +495,10 @@ public class TestContainerResizing {
     // am1 asks to change its AM container from 1G to 1G (cancel the increase
     // request actually)
     am1.sendContainerResizingRequest(Arrays.asList(
-            ContainerResourceChangeRequest
-                .newInstance(containerId1, Resources.createResource(1 * GB))),
-        null);
+            UpdateContainerRequest
+                .newInstance(0, containerId1,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(1 * GB))));
     // Trigger a node heartbeat..
     cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
     
@@ -559,9 +568,10 @@ public class TestContainerResizing {
 
     // am1 asks to change its AM container from 2GB to 8GB
     am1.sendContainerResizingRequest(Arrays.asList(
-            ContainerResourceChangeRequest
-                .newInstance(containerId1, Resources.createResource(8 * GB))),
-        null);
+            UpdateContainerRequest
+                .newInstance(0, containerId1,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(8 * GB))));
 
     checkPendingResource(rm1, "default", 6 * GB, null);
     Assert.assertEquals(6 * GB,
@@ -597,9 +607,11 @@ public class TestContainerResizing {
     // request, make target_capacity=existing_capacity)
     am1.allocate(null, Arrays.asList(containerId2));
     // am1 asks to change its AM container from 2G to 1G (decrease)
-    am1.sendContainerResizingRequest(null, Arrays.asList(
-        ContainerResourceChangeRequest
-            .newInstance(containerId1, Resources.createResource(1 * GB))));
+    am1.sendContainerResizingRequest(Arrays.asList(
+        UpdateContainerRequest
+            .newInstance(0, containerId1,
+                ContainerUpdateType.INCREASE_RESOURCE,
+                Resources.createResource(1 * GB))));
     // Trigger a node heartbeat..
     cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
     
@@ -667,9 +679,10 @@ public class TestContainerResizing {
 
     // am1 asks to change its AM container from 2GB to 8GB
     am1.sendContainerResizingRequest(Arrays.asList(
-            ContainerResourceChangeRequest
-                .newInstance(containerId2, Resources.createResource(8 * GB))),
-        null);
+            UpdateContainerRequest
+                .newInstance(0, containerId2,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(8 * GB))));
 
     checkPendingResource(rm1, "default", 6 * GB, null);
     Assert.assertEquals(6 * GB,
@@ -765,9 +778,10 @@ public class TestContainerResizing {
 
     // am1 asks to change its AM container from 2GB to 8GB
     am1.sendContainerResizingRequest(Arrays.asList(
-            ContainerResourceChangeRequest
-                .newInstance(containerId2, Resources.createResource(8 * GB))),
-        null);
+            UpdateContainerRequest
+                .newInstance(0, containerId2,
+                    ContainerUpdateType.INCREASE_RESOURCE,
+                    Resources.createResource(8 * GB))));
 
     checkPendingResource(rm1, "default", 6 * GB, null);
     Assert.assertEquals(6 * GB,
@@ -883,14 +897,16 @@ public class TestContainerResizing {
     allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 4, 6);
 
     // am1 asks to change its container[2-7] from 1G to 2G
-    List<ContainerResourceChangeRequest> increaseRequests = new ArrayList<>();
+    List<UpdateContainerRequest> increaseRequests = new ArrayList<>();
     for (int cId = 2; cId <= 7; cId++) {
       ContainerId containerId =
           ContainerId.newContainerId(am1.getApplicationAttemptId(), cId);
-      increaseRequests.add(ContainerResourceChangeRequest
-          .newInstance(containerId, Resources.createResource(2 * GB)));
+      increaseRequests.add(UpdateContainerRequest
+          .newInstance(0, containerId,
+              ContainerUpdateType.INCREASE_RESOURCE,
+              Resources.createResource(2 * GB)));
     }
-    am1.sendContainerResizingRequest(increaseRequests, null);
+    am1.sendContainerResizingRequest(increaseRequests);
 
     checkPendingResource(rm1, "default", 6 * GB, null);
     Assert.assertEquals(6 * GB,
@@ -904,7 +920,7 @@ public class TestContainerResizing {
     // earlier allocated)
     cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
     AllocateResponse allocateResponse = am1.allocate(null, null);
-    Assert.assertEquals(3, allocateResponse.getIncreasedContainers().size());
+    Assert.assertEquals(3, allocateResponse.getUpdatedContainers().size());
     verifyContainerIncreased(allocateResponse,
         ContainerId.newContainerId(attemptId, 4), 2 * GB);
     verifyContainerIncreased(allocateResponse,
@@ -964,14 +980,16 @@ public class TestContainerResizing {
     allocateAndLaunchContainers(am1, nm1, rm1, 2, 1 * GB, 4, 6);
 
     // am1 asks to change its container[2-7] from 1G to 2G
-    List<ContainerResourceChangeRequest> increaseRequests = new ArrayList<>();
+    List<UpdateContainerRequest> increaseRequests = new ArrayList<>();
     for (int cId = 2; cId <= 7; cId++) {
       ContainerId containerId =
           ContainerId.newContainerId(am1.getApplicationAttemptId(), cId);
-      increaseRequests.add(ContainerResourceChangeRequest
-          .newInstance(containerId, Resources.createResource(2 * GB)));
+      increaseRequests.add(UpdateContainerRequest
+          .newInstance(0, containerId,
+              ContainerUpdateType.INCREASE_RESOURCE,
+              Resources.createResource(2 * GB)));
     }
-    am1.sendContainerResizingRequest(increaseRequests, null);
+    am1.sendContainerResizingRequest(increaseRequests);
 
     checkPendingResource(rm1, "default", 6 * GB, null);
     Assert.assertEquals(6 * GB,
@@ -985,7 +1003,7 @@ public class TestContainerResizing {
     // earlier allocated)
     cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
     AllocateResponse allocateResponse = am1.allocate(null, null);
-    Assert.assertEquals(3, allocateResponse.getIncreasedContainers().size());
+    Assert.assertEquals(3, allocateResponse.getUpdatedContainers().size());
     verifyContainerIncreased(allocateResponse,
         ContainerId.newContainerId(attemptId, 4), 2 * GB);
     verifyContainerIncreased(allocateResponse,
@@ -1046,9 +1064,11 @@ public class TestContainerResizing {
     nm.nodeHeartbeat(true);
     // *In the mean time*, am1 asks to decrease its AM container resource from
     // 3GB to 1GB
-    AllocateResponse response = am1.sendContainerResizingRequest(null,
-        Collections.singletonList(ContainerResourceChangeRequest
-            .newInstance(containerId1, Resources.createResource(GB))));
+    AllocateResponse response = am1.sendContainerResizingRequest(
+        Collections.singletonList(UpdateContainerRequest
+            .newInstance(0, containerId1,
+                ContainerUpdateType.DECREASE_RESOURCE,
+                Resources.createResource(GB))));
     // verify that the containe resource is decreased
     verifyContainerDecreased(response, containerId1, GB);
 
@@ -1077,12 +1097,16 @@ public class TestContainerResizing {
 
   private void verifyContainerIncreased(AllocateResponse response,
       ContainerId containerId, int mem) {
-    List<Container> increasedContainers = response.getIncreasedContainers();
+    List<UpdatedContainer> increasedContainers =
+        response.getUpdatedContainers();
     boolean found = false;
-    for (Container c : increasedContainers) {
-      if (c.getId().equals(containerId)) {
+    for (UpdatedContainer c : increasedContainers) {
+      if (c.getContainer().getId().equals(containerId)) {
         found = true;
-        Assert.assertEquals(mem, c.getResource().getMemorySize());
+        Assert.assertEquals(ContainerUpdateType.INCREASE_RESOURCE,
+            c.getUpdateType());
+        Assert.assertEquals(mem,
+            c.getContainer().getResource().getMemorySize());
       }
     }
     if (!found) {
@@ -1092,12 +1116,16 @@ public class TestContainerResizing {
 
   private void verifyContainerDecreased(AllocateResponse response,
       ContainerId containerId, int mem) {
-    List<Container> decreasedContainers = response.getDecreasedContainers();
+    List<UpdatedContainer> decreasedContainers =
+        response.getUpdatedContainers();
     boolean found = false;
-    for (Container c : decreasedContainers) {
-      if (c.getId().equals(containerId)) {
+    for (UpdatedContainer c : decreasedContainers) {
+      if (c.getContainer().getId().equals(containerId)) {
         found = true;
-        Assert.assertEquals(mem, c.getResource().getMemorySize());
+        Assert.assertEquals(ContainerUpdateType.DECREASE_RESOURCE,
+            c.getUpdateType());
+        Assert.assertEquals(mem,
+            c.getContainer().getResource().getMemorySize());
       }
     }
     if (!found) {

+ 50 - 26
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestIncreaseAllocationExpirer.java

@@ -21,10 +21,12 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.UpdateContainerError;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
@@ -109,8 +111,9 @@ public class TestIncreaseAllocationExpirer {
     rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
     // am1 asks to increase containerId2 from 1GB to 3GB
     am1.sendContainerResizingRequest(Collections.singletonList(
-        ContainerResourceChangeRequest.newInstance(
-            containerId2, Resources.createResource(3 * GB))), null);
+        UpdateContainerRequest.newInstance(0, containerId2,
+            ContainerUpdateType.INCREASE_RESOURCE,
+            Resources.createResource(3 * GB))));
     // Kick off scheduling and sleep for 1 second;
     nm1.nodeHeartbeat(true);
     Thread.sleep(1000);
@@ -180,8 +183,9 @@ public class TestIncreaseAllocationExpirer {
     rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
     // am1 asks to increase containerId2 from 1GB to 3GB
     am1.sendContainerResizingRequest(Collections.singletonList(
-        ContainerResourceChangeRequest.newInstance(
-            containerId2, Resources.createResource(3 * GB))), null);
+        UpdateContainerRequest.newInstance(0, containerId2,
+            ContainerUpdateType.INCREASE_RESOURCE,
+            Resources.createResource(3 * GB))));
     // Kick off scheduling and wait for 1 second;
     nm1.nodeHeartbeat(true);
     Thread.sleep(1000);
@@ -249,8 +253,9 @@ public class TestIncreaseAllocationExpirer {
     rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
     // am1 asks to change containerId2 from 1GB to 3GB
     am1.sendContainerResizingRequest(Collections.singletonList(
-        ContainerResourceChangeRequest.newInstance(
-            containerId2, Resources.createResource(3 * GB))), null);
+        UpdateContainerRequest.newInstance(0, containerId2,
+            ContainerUpdateType.INCREASE_RESOURCE,
+            Resources.createResource(3 * GB))));
     // Kick off scheduling and sleep for 1 second to
     // make sure the allocation is done
     nm1.nodeHeartbeat(true);
@@ -261,10 +266,23 @@ public class TestIncreaseAllocationExpirer {
     Resource resource1 = Resources.clone(
         rm1.getResourceScheduler().getRMContainer(containerId2)
             .getAllocatedResource());
+
+    // This should not work, since the container version is wrong
+    AllocateResponse response = am1.sendContainerResizingRequest(Collections
+        .singletonList(
+        UpdateContainerRequest.newInstance(0, containerId2,
+            ContainerUpdateType.INCREASE_RESOURCE,
+            Resources.createResource(5 * GB))));
+    List<UpdateContainerError> updateErrors = response.getUpdateErrors();
+    Assert.assertEquals(1, updateErrors.size());
+    Assert.assertEquals("INCORRECT_CONTAINER_VERSION_ERROR|0|1",
+        updateErrors.get(0).getReason());
+
     // am1 asks to change containerId2 from 3GB to 5GB
     am1.sendContainerResizingRequest(Collections.singletonList(
-        ContainerResourceChangeRequest.newInstance(
-            containerId2, Resources.createResource(5 * GB))), null);
+        UpdateContainerRequest.newInstance(1, containerId2,
+            ContainerUpdateType.INCREASE_RESOURCE,
+            Resources.createResource(5 * GB))));
     // Kick off scheduling and sleep for 1 second to
     // make sure the allocation is done
     nm1.nodeHeartbeat(true);
@@ -362,30 +380,36 @@ public class TestIncreaseAllocationExpirer {
     rm1.waitForState(nm1, containerId3, RMContainerState.RUNNING);
     rm1.waitForState(nm1, containerId4, RMContainerState.RUNNING);
     // am1 asks to change containerId2 and containerId3 from 1GB to 3GB
-    List<ContainerResourceChangeRequest> increaseRequests = new ArrayList<>();
-    increaseRequests.add(ContainerResourceChangeRequest.newInstance(
-        containerId2, Resources.createResource(6 * GB)));
-    increaseRequests.add(ContainerResourceChangeRequest.newInstance(
-        containerId3, Resources.createResource(6 * GB)));
-    increaseRequests.add(ContainerResourceChangeRequest.newInstance(
-        containerId4, Resources.createResource(6 * GB)));
-    am1.sendContainerResizingRequest(increaseRequests, null);
+    List<UpdateContainerRequest> increaseRequests = new ArrayList<>();
+    increaseRequests.add(UpdateContainerRequest.newInstance(0, containerId2,
+        ContainerUpdateType.INCREASE_RESOURCE,
+        Resources.createResource(6 * GB)));
+    increaseRequests.add(UpdateContainerRequest.newInstance(0, containerId3,
+        ContainerUpdateType.INCREASE_RESOURCE,
+        Resources.createResource(6 * GB)));
+    increaseRequests.add(UpdateContainerRequest.newInstance(0, containerId4,
+        ContainerUpdateType.INCREASE_RESOURCE,
+        Resources.createResource(6 * GB)));
+    am1.sendContainerResizingRequest(increaseRequests);
     nm1.nodeHeartbeat(true);
     Thread.sleep(1000);
     // Start container increase allocation expirer
     am1.allocate(null, null);
     // Decrease containers
-    List<ContainerResourceChangeRequest> decreaseRequests = new ArrayList<>();
-    decreaseRequests.add(ContainerResourceChangeRequest.newInstance(
-        containerId2, Resources.createResource(2 * GB)));
-    decreaseRequests.add(ContainerResourceChangeRequest.newInstance(
-        containerId3, Resources.createResource(4 * GB)));
-    decreaseRequests.add(ContainerResourceChangeRequest.newInstance(
-        containerId4, Resources.createResource(4 * GB)));
+    List<UpdateContainerRequest> decreaseRequests = new ArrayList<>();
+    decreaseRequests.add(UpdateContainerRequest.newInstance(1, containerId2,
+        ContainerUpdateType.INCREASE_RESOURCE,
+        Resources.createResource(2 * GB)));
+    decreaseRequests.add(UpdateContainerRequest.newInstance(1, containerId3,
+        ContainerUpdateType.INCREASE_RESOURCE,
+        Resources.createResource(4 * GB)));
+    decreaseRequests.add(UpdateContainerRequest.newInstance(1, containerId4,
+        ContainerUpdateType.INCREASE_RESOURCE,
+        Resources.createResource(4 * GB)));
     AllocateResponse response =
-        am1.sendContainerResizingRequest(null, decreaseRequests);
+        am1.sendContainerResizingRequest(decreaseRequests);
     // Verify containers are decreased in scheduler
-    Assert.assertEquals(3, response.getDecreasedContainers().size());
+    Assert.assertEquals(3, response.getUpdatedContainers().size());
     // Use the token for containerId4 on NM (6G). This should set the last
     // confirmed resource to 4G, and cancel the allocation expirer
     nm1.containerIncreaseStatus(getContainer(

+ 9 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java

@@ -220,7 +220,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     
     org.apache.hadoop.yarn.api.records.Token validContainerToken =
         containerTokenSecretManager.createContainerToken(validContainerId,
-            validNode, user, r, Priority.newInstance(10), 1234);
+            0, validNode, user, r, Priority.newInstance(10), 1234);
     ContainerTokenIdentifier identifier =
         BuilderUtils.newContainerTokenIdentifier(validContainerToken);
     Assert.assertEquals(Priority.newInstance(10), identifier.getPriority());
@@ -277,7 +277,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
         4 * 60 * 1000);
     validContainerToken =
         containerTokenSecretManager.createContainerToken(validContainerId,
-            validNode, user, r, Priority.newInstance(0), 0);
+            0, validNode, user, r, Priority.newInstance(0), 0);
     Assert.assertTrue(testStartContainer(rpc, validAppAttemptId, validNode,
       validContainerToken, validNMToken, false).isEmpty());
     Assert.assertTrue(nmTokenSecretManagerNM
@@ -293,7 +293,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
 
     org.apache.hadoop.yarn.api.records.Token validContainerToken2 =
         containerTokenSecretManager.createContainerToken(validContainerId2,
-            validNode, user, r, Priority.newInstance(0), 0);
+            0, validNode, user, r, Priority.newInstance(0), 0);
     
     org.apache.hadoop.yarn.api.records.Token validNMToken2 =
         nmTokenSecretManagerRM.createNMToken(validAppAttemptId2, validNode, user);
@@ -379,7 +379,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
           .createNMToken(validAppAttemptId, validNode, user);
     org.apache.hadoop.yarn.api.records.Token newContainerToken =
         containerTokenSecretManager.createContainerToken(
-          ContainerId.newContainerId(attempt2, 1), validNode, user, r,
+          ContainerId.newContainerId(attempt2, 1), 0, validNode, user, r,
             Priority.newInstance(0), 0);
     Assert.assertTrue(testStartContainer(rpc, attempt2, validNode,
       newContainerToken, attempt1NMToken, false).isEmpty());
@@ -639,7 +639,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     
     Token containerToken = 
         containerTokenSecretManager.createContainerToken(
-            cId, nodeId, user, r, Priority.newInstance(0), 0);
+            cId, 0, nodeId, user, r, Priority.newInstance(0), 0);
     
     ContainerTokenIdentifier containerTokenIdentifier = 
         getContainerTokenIdentifierFromToken(containerToken);
@@ -672,8 +672,8 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     ContainerId cId2 = ContainerId.newContainerId(appAttemptId, 1);
     // Creating modified containerToken
     Token containerToken2 =
-        tamperedContainerTokenSecretManager.createContainerToken(cId2, nodeId,
-            user, r, Priority.newInstance(0), 0);
+        tamperedContainerTokenSecretManager.createContainerToken(cId2, 0,
+            nodeId, user, r, Priority.newInstance(0), 0);
     
     StringBuilder sb = new StringBuilder("Given Container ");
     sb.append(cId2);
@@ -731,8 +731,8 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
             getContainerTokenSecretManager();
     Resource r = Resource.newInstance(1230, 2);
     Token containerToken =
-        containerTokenSecretManager.createContainerToken(cId, nodeId, user, r,
-            Priority.newInstance(0), 0);
+        containerTokenSecretManager.createContainerToken(cId, 0, nodeId, user,
+            r, Priority.newInstance(0), 0);
     
     ContainerTokenIdentifier containerTokenIdentifier =
         new ContainerTokenIdentifier();

+ 0 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYarnClusterNodeUtilization.java

@@ -198,8 +198,6 @@ public class TestMiniYarnClusterNodeUtilization {
   /**
    * Verify both the RMNode and SchedulerNode have been updated with the test
    * fixture utilization data.
-   * @param containersUtilization Utilization of the container.
-   * @param nodeUtilization Utilization of the node.
    */
   private void verifySimulatedUtilization() throws InterruptedException {
     ResourceManager rm = cluster.getResourceManager(0);

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/proto/test_token.proto

@@ -44,5 +44,6 @@ message ContainerTokenIdentifierForTestProto {
   optional int64 creationTime = 9;
   optional LogAggregationContextProto logAggregationContext = 10;
   optional string message = 11;
+  optional int32 version = 14 [default = 0];
 }