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

hadoop git commit: Commit YARN-3866 addendum patches to branch

Repository: hadoop
Updated Branches:
  refs/heads/YARN-3866 [created] cac909035


Commit YARN-3866 addendum patches to branch


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

Branch: refs/heads/YARN-3866
Commit: cac90903598001cd49a070898cec0ec53cb48205
Parents: 747be79
Author: Wangda Tan <wa...@apache.org>
Authored: Thu Dec 29 15:22:33 2016 -0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Thu Dec 29 15:22:33 2016 -0800

----------------------------------------------------------------------
 .../api/protocolrecords/AllocateRequest.java    |  50 +++++-
 .../api/protocolrecords/AllocateResponse.java   |  45 +++++
 .../api/records/ContainerResourceDecrease.java  |  72 ++++++++
 .../api/records/ContainerResourceIncrease.java  |  77 +++++++++
 .../ContainerResourceIncreaseRequest.java       |  73 ++++++++
 .../src/main/proto/yarn_protos.proto            |  19 +-
 .../src/main/proto/yarn_service_protos.proto    |  11 +-
 .../yarn/client/api/impl/AMRMClientImpl.java    |   2 +-
 .../impl/pb/AllocateRequestPBImpl.java          |  91 ++++++++++
 .../impl/pb/AllocateResponsePBImpl.java         |  12 ++
 .../pb/ContainerResourceDecreasePBImpl.java     | 137 +++++++++++++++
 .../pb/ContainerResourceIncreasePBImpl.java     | 172 +++++++++++++++++++
 .../ContainerResourceIncreaseRequestPBImpl.java | 141 +++++++++++++++
 .../hadoop/yarn/api/TestPBImplRecords.java      |   2 +
 .../yarn/server/resourcemanager/MockAM.java     |   2 +-
 15 files changed, 896 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
index f7ce127..dd0d9c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateRequest.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.api.protocolrecords;
 
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -26,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 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.ContainerResourceIncreaseRequest;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
@@ -67,7 +69,34 @@ public abstract class AllocateRequest {
       List<ContainerId> containersToBeReleased,
       ResourceBlacklistRequest resourceBlacklistRequest) {
     return newInstance(responseID, appProgress, resourceAsk,
-        containersToBeReleased, resourceBlacklistRequest, null);
+        containersToBeReleased, null, resourceBlacklistRequest);
+  }
+
+  /**
+   * Use {@link AllocateRequest#newInstance(int, float, List, List,
+   * ResourceBlacklistRequest, List)} instead
+   * @param responseID responseId
+   * @param appProgress appProgress
+   * @param resourceAsk resourceAsk
+   * @param containersToBeReleased containersToBeReleased
+   * @param resourceBlacklistRequest resourceBlacklistRequest
+   * @param increaseRequests increaseRequests
+   * @return AllocateRequest
+   */
+  @Deprecated
+  public static AllocateRequest newInstance(int responseID, float appProgress,
+      List<ResourceRequest> resourceAsk,
+      List<ContainerId> containersToBeReleased,
+      ResourceBlacklistRequest resourceBlacklistRequest,
+      List<ContainerResourceIncreaseRequest> increaseRequests) {
+    AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
+    allocateRequest.setResponseId(responseID);
+    allocateRequest.setProgress(appProgress);
+    allocateRequest.setAskList(resourceAsk);
+    allocateRequest.setReleaseList(containersToBeReleased);
+    allocateRequest.setResourceBlacklistRequest(resourceBlacklistRequest);
+    allocateRequest.setIncreaseRequests(increaseRequests);
+    return allocateRequest;
   }
   
   @Public
@@ -75,8 +104,8 @@ public abstract class AllocateRequest {
   public static AllocateRequest newInstance(int responseID, float appProgress,
       List<ResourceRequest> resourceAsk,
       List<ContainerId> containersToBeReleased,
-      ResourceBlacklistRequest resourceBlacklistRequest,
-      List<UpdateContainerRequest> updateRequests) {
+      List<UpdateContainerRequest> updateRequests,
+      ResourceBlacklistRequest resourceBlacklistRequest) {
     AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
     allocateRequest.setResponseId(responseID);
     allocateRequest.setProgress(appProgress);
@@ -188,6 +217,21 @@ public abstract class AllocateRequest {
   @Stable
   public abstract void setResourceBlacklistRequest(
       ResourceBlacklistRequest resourceBlacklistRequest);
+
+  /**
+   * Use {@link AllocateRequest#getUpdateRequests()} instead
+   * @return ContainerResourceIncreaseRequests
+   */
+  @Deprecated
+  public abstract List<ContainerResourceIncreaseRequest> getIncreaseRequests();
+
+  /**
+   * Use {@link AllocateRequest#setUpdateRequests(List)} instead
+   * @param increaseRequests increaseRequests
+   */
+  @Deprecated
+  public abstract void setIncreaseRequests(
+      List<ContainerResourceIncreaseRequest> increaseRequests);
   
   /**
    * Get the list of container update requests being sent by the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
index beb1fa1..f205053 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
@@ -29,6 +29,8 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.records.AMCommand;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -97,6 +99,35 @@ public abstract class AllocateResponse {
     return response;
   }
 
+  /**
+   * Use {@link AllocateResponse#newInstance(int, List, List, List, Resource,
+   * AMCommand, int, PreemptionMessage, List, Token, List)} instead
+   * @param responseId responseId
+   * @param completedContainers completedContainers
+   * @param allocatedContainers allocatedContainers
+   * @param updatedNodes updatedNodes
+   * @param availResources availResources
+   * @param command command
+   * @param numClusterNodes numClusterNodes
+   * @param preempt preempt
+   * @param nmTokens nmTokens
+   * @param increasedContainers increasedContainers
+   * @param decreasedContainers decreasedContainers
+   * @return AllocateResponse
+   */
+  @Deprecated
+  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<ContainerResourceIncrease> increasedContainers,
+      List<ContainerResourceDecrease> decreasedContainers) {
+    return newInstance(responseId, completedContainers, allocatedContainers,
+        updatedNodes, availResources, command, numClusterNodes, preempt,
+        nmTokens);
+  }
+
   @Public
   @Unstable
   public static AllocateResponse newInstance(int responseId,
@@ -338,4 +369,18 @@ public abstract class AllocateResponse {
   @Unstable
   public void setUpdateErrors(List<UpdateContainerError> updateErrors) {
   }
+
+  /**
+   * Use {@link AllocateResponse#getUpdatedContainers()} instead
+   * @return null
+   */
+  @Deprecated
+  public abstract List<ContainerResourceIncrease> getIncreasedContainers();
+
+  /**
+   * Use {@link AllocateResponse#getUpdatedContainers()} instead
+   * @return null
+   */
+  @Deprecated
+  public abstract List<ContainerResourceDecrease> getDecreasedContainers();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java
new file mode 100644
index 0000000..db138c3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java
@@ -0,0 +1,72 @@
+/**
+ * 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.yarn.util.Records;
+
+/**
+ * Use {@link UpdatedContainer} instead
+ */
+@Deprecated
+public abstract class ContainerResourceDecrease {
+  public static ContainerResourceDecrease newInstance(
+      ContainerId existingContainerId, Resource targetCapability) {
+    ContainerResourceDecrease context = Records
+        .newRecord(ContainerResourceDecrease.class);
+    context.setContainerId(existingContainerId);
+    context.setCapability(targetCapability);
+    return context;
+  }
+
+  public abstract ContainerId getContainerId();
+
+  public abstract void setContainerId(ContainerId containerId);
+
+  public abstract Resource getCapability();
+
+  public abstract void setCapability(Resource capability);
+
+  @Override
+  public int hashCode() {
+    return getCapability().hashCode() + getContainerId().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other instanceof ContainerResourceDecrease) {
+      ContainerResourceDecrease ctx = (ContainerResourceDecrease)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;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java
new file mode 100644
index 0000000..26d3555
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java
@@ -0,0 +1,77 @@
+/**
+ * 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.yarn.util.Records;
+
+/**
+ * Use {@link UpdatedContainer} instead
+ */
+@Deprecated
+public abstract class ContainerResourceIncrease {
+  public static ContainerResourceIncrease newInstance(
+      ContainerId existingContainerId, Resource targetCapability, Token token) {
+    ContainerResourceIncrease context = Records
+        .newRecord(ContainerResourceIncrease.class);
+    context.setContainerId(existingContainerId);
+    context.setCapability(targetCapability);
+    context.setContainerToken(token);
+    return context;
+  }
+
+  public abstract ContainerId getContainerId();
+
+  public abstract void setContainerId(ContainerId containerId);
+
+  public abstract Resource getCapability();
+
+  public abstract void setCapability(Resource capability);
+
+  public abstract Token getContainerToken();
+
+  public abstract void setContainerToken(Token token);
+
+  @Override
+  public int hashCode() {
+    return getCapability().hashCode() + getContainerId().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other instanceof ContainerResourceIncrease) {
+      ContainerResourceIncrease ctx = (ContainerResourceIncrease)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;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java
new file mode 100644
index 0000000..2eae157
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java
@@ -0,0 +1,73 @@
+/**
+ * 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.yarn.util.Records;
+
+/**
+ * Use {@link UpdateContainerRequest} instead
+ */
+@Deprecated
+public abstract class ContainerResourceIncreaseRequest {
+  public static ContainerResourceIncreaseRequest newInstance(
+      ContainerId existingContainerId, Resource targetCapability) {
+    ContainerResourceIncreaseRequest context = Records
+        .newRecord(ContainerResourceIncreaseRequest.class);
+    context.setContainerId(existingContainerId);
+    context.setCapability(targetCapability);
+    return context;
+  }
+
+  public abstract ContainerId getContainerId();
+
+  public abstract void setContainerId(ContainerId containerId);
+
+  public abstract Resource getCapability();
+
+  public abstract void setCapability(Resource capability);
+
+  @Override
+  public int hashCode() {
+    return getCapability().hashCode() + getContainerId().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other instanceof ContainerResourceIncreaseRequest) {
+      ContainerResourceIncreaseRequest ctx =
+          (ContainerResourceIncreaseRequest) 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;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index e5f33ac..cefaf55 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -442,7 +442,6 @@ enum SignalContainerCommandProto {
   FORCEFUL_SHUTDOWN = 3;
 }
 
-
 ////////////////////////////////////////////////////////////////////////
 ////// From reservation_protocol /////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////
@@ -541,3 +540,21 @@ message StringBytesMapProto {
   optional string key = 1;
   optional bytes value = 2;
 }
+
+///////////////////////////////////////////////////////////////////////
+/////// DEPRECATED ////////////////////////////////////////////////////
+message ContainerResourceIncreaseRequestProto {
+  optional ContainerIdProto container_id = 1;
+  optional ResourceProto capability = 2;
+}
+
+message ContainerResourceIncreaseProto {
+  optional ContainerIdProto container_id = 1;
+  optional ResourceProto capability = 2;
+  optional hadoop.common.TokenProto container_token = 3;
+}
+
+message ContainerResourceDecreaseProto {
+  optional ContainerIdProto container_id = 1;
+  optional ResourceProto capability = 2;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index c81ba42..d1a7204 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -83,7 +83,8 @@ message AllocateRequestProto {
   optional ResourceBlacklistRequestProto blacklist_request = 3;
   optional int32 response_id = 4;
   optional float progress = 5;
-  repeated UpdateContainerRequestProto update_requests = 6;
+  repeated ContainerResourceIncreaseRequestProto increase_request = 6;
+  repeated UpdateContainerRequestProto update_requests = 7;
 }
 
 message NMTokenProto {
@@ -106,9 +107,11 @@ message AllocateResponseProto {
   optional int32 num_cluster_nodes = 7;
   optional PreemptionMessageProto preempt = 8;
   repeated NMTokenProto nm_tokens = 9;
-  repeated UpdatedContainerProto updated_containers = 10;
-  optional hadoop.common.TokenProto am_rm_token = 12;
-  optional PriorityProto application_priority = 13;
+  repeated ContainerResourceIncreaseProto increased_containers = 10;
+  repeated ContainerResourceDecreaseProto decreased_containers = 11;
+  repeated UpdatedContainerProto updated_containers = 12;
+  optional hadoop.common.TokenProto am_rm_token = 13;
+  optional PriorityProto application_priority = 14;
   repeated UpdateContainerErrorProto update_errors = 15;
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
index b10f8de..8069413 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
@@ -278,7 +278,7 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
         
         allocateRequest =
             AllocateRequest.newInstance(lastResponseId, progressIndicator,
-                askList, releaseList, blacklistRequest, updateList);
+                askList, releaseList, updateList, blacklistRequest);
         // clear blacklistAdditions and blacklistRemovals before
         // unsynchronized part
         blacklistAdditions.clear();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
index 0f0f571..f441d69 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateRequestPBImpl.java
@@ -27,13 +27,16 @@ 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.ContainerResourceIncreaseRequest;
 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.ContainerResourceIncreaseRequestPBImpl;
 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;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
@@ -54,6 +57,10 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   private List<ContainerId> release = null;
   private List<UpdateContainerRequest> updateRequests = null;
   private ResourceBlacklistRequest blacklistRequest = null;
+
+  // This is deprecated, leave it here only to make unit test not break
+  @Deprecated
+  private List<ContainerResourceIncreaseRequest> deprecatedIncreaseReqs = null;
   
   public AllocateRequestPBImpl() {
     builder = AllocateRequestProto.newBuilder();
@@ -104,6 +111,9 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     if (this.blacklistRequest != null) {
       builder.setBlacklistRequest(convertToProtoFormat(this.blacklistRequest));
     }
+    if (this.deprecatedIncreaseReqs != null) {
+      addIncreaseRequestsToProto();
+    }
   }
 
   private void mergeLocalToProto() {
@@ -297,6 +307,56 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     builder.addAllUpdateRequests(iterable);
   }
 
+  private void initIncreaseRequests() {
+    if (this.deprecatedIncreaseReqs != null) {
+      return;
+    }
+    AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<YarnProtos.ContainerResourceIncreaseRequestProto> list =
+        p.getIncreaseRequestList();
+    this.deprecatedIncreaseReqs = new ArrayList<>();
+
+    for (YarnProtos.ContainerResourceIncreaseRequestProto c : list) {
+      this.deprecatedIncreaseReqs.add(convertFromProtoFormat(c));
+    }
+  }
+
+  private void addIncreaseRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearIncreaseRequest();
+    if (deprecatedIncreaseReqs == null) {
+      return;
+    }
+    Iterable<YarnProtos.ContainerResourceIncreaseRequestProto> iterable =
+        new Iterable<YarnProtos.ContainerResourceIncreaseRequestProto>() {
+          @Override
+          public Iterator<YarnProtos.ContainerResourceIncreaseRequestProto> iterator() {
+            return new Iterator<YarnProtos.ContainerResourceIncreaseRequestProto>() {
+
+              private Iterator<ContainerResourceIncreaseRequest> iter =
+                  deprecatedIncreaseReqs.iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iter.hasNext();
+              }
+
+              @Override
+              public YarnProtos.ContainerResourceIncreaseRequestProto next() {
+                return convertToProtoFormat(iter.next());
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+            };
+
+          }
+        };
+    builder.addAllIncreaseRequest(iterable);
+  }
+
   @Override
   public List<ContainerId> getReleaseList() {
     initReleases();
@@ -392,4 +452,35 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   private ResourceBlacklistRequestProto convertToProtoFormat(ResourceBlacklistRequest t) {
     return ((ResourceBlacklistRequestPBImpl)t).getProto();
   }
+
+  @Deprecated
+  @Override
+  public List<ContainerResourceIncreaseRequest> getIncreaseRequests() {
+    initIncreaseRequests();
+    return this.deprecatedIncreaseReqs;
+  }
+
+  @Deprecated
+  @Override
+  public void setIncreaseRequests(
+      List<ContainerResourceIncreaseRequest> increaseRequests) {
+    if (increaseRequests == null) {
+      return;
+    }
+    initIncreaseRequests();
+    this.deprecatedIncreaseReqs.clear();
+    this.deprecatedIncreaseReqs.addAll(increaseRequests);
+  }
+
+  private ContainerResourceIncreaseRequestPBImpl convertFromProtoFormat(
+      YarnProtos.ContainerResourceIncreaseRequestProto p) {
+    return new ContainerResourceIncreaseRequestPBImpl(p);
+
+  }
+
+  private YarnProtos.ContainerResourceIncreaseRequestProto convertToProtoFormat(
+      ContainerResourceIncreaseRequest t) {
+    return ((ContainerResourceIncreaseRequestPBImpl) t).getProto();
+
+  }
 }  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
index 8625e25..603997e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/AllocateResponsePBImpl.java
@@ -29,6 +29,8 @@ import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.AMCommand;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -270,6 +272,16 @@ public class AllocateResponsePBImpl extends AllocateResponse {
   }
 
   @Override
+  public List<ContainerResourceIncrease> getIncreasedContainers() {
+    return null;
+  }
+
+  @Override
+  public List<ContainerResourceDecrease> getDecreasedContainers() {
+    return null;
+  }
+
+  @Override
   public synchronized List<Container> getAllocatedContainers() {
     initLocalNewContainerList();
     return this.allocatedContainers;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java
new file mode 100644
index 0000000..9843085
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java
@@ -0,0 +1,137 @@
+/**
+ * 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.ContainerResourceDecrease;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+
+@Deprecated
+public class ContainerResourceDecreasePBImpl extends ContainerResourceDecrease {
+  ContainerResourceDecreaseProto proto = ContainerResourceDecreaseProto
+      .getDefaultInstance();
+  ContainerResourceDecreaseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ContainerId existingContainerId = null;
+  private Resource targetCapability = null;
+
+  public ContainerResourceDecreasePBImpl() {
+    builder = ContainerResourceDecreaseProto.newBuilder();
+  }
+
+  public ContainerResourceDecreasePBImpl(ContainerResourceDecreaseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ContainerResourceDecreaseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public ContainerId getContainerId() {
+    ContainerResourceDecreaseProtoOrBuilder 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() {
+    ContainerResourceDecreaseProtoOrBuilder 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 = ContainerResourceDecreaseProto.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));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java
new file mode 100644
index 0000000..f3155ef
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java
@@ -0,0 +1,172 @@
+/**
+ * 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.security.proto.SecurityProtos.TokenProto;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+
+@Deprecated
+public class ContainerResourceIncreasePBImpl extends ContainerResourceIncrease {
+  ContainerResourceIncreaseProto proto = ContainerResourceIncreaseProto
+      .getDefaultInstance();
+  ContainerResourceIncreaseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ContainerId existingContainerId = null;
+  private Resource targetCapability = null;
+  private Token token = null;
+
+  public ContainerResourceIncreasePBImpl() {
+    builder = ContainerResourceIncreaseProto.newBuilder();
+  }
+
+  public ContainerResourceIncreasePBImpl(ContainerResourceIncreaseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ContainerResourceIncreaseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public ContainerId getContainerId() {
+    ContainerResourceIncreaseProtoOrBuilder 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() {
+    ContainerResourceIncreaseProtoOrBuilder 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;
+  }
+
+  @Override
+  public Token getContainerToken() {
+    ContainerResourceIncreaseProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.token != null) {
+      return this.token;
+    }
+    if (p.hasContainerToken()) {
+      this.token = convertFromProtoFormat(p.getContainerToken());
+    }
+    return this.token;
+  }
+
+  @Override
+  public void setContainerToken(Token token) {
+    maybeInitBuilder();
+    if (token == null) {
+      builder.clearContainerToken();
+    }
+    this.token = token;
+  }
+
+  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 Token convertFromProtoFormat(TokenProto p) {
+    return new TokenPBImpl(p);
+  }
+
+  private TokenProto convertToProtoFormat(Token t) {
+    return ((TokenPBImpl) t).getProto();
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ContainerResourceIncreaseProto.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));
+    }
+    if (this.token != null) {
+      builder.setContainerToken(convertToProtoFormat(this.token));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java
new file mode 100644
index 0000000..1b56b45
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java
@@ -0,0 +1,141 @@
+/**
+ * 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.ContainerResourceIncreaseRequest;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+
+@Deprecated
+public class ContainerResourceIncreaseRequestPBImpl extends
+    ContainerResourceIncreaseRequest {
+  ContainerResourceIncreaseRequestProto proto =
+      ContainerResourceIncreaseRequestProto.getDefaultInstance();
+  ContainerResourceIncreaseRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private ContainerId existingContainerId = null;
+  private Resource targetCapability = null;
+
+  public ContainerResourceIncreaseRequestPBImpl() {
+    builder = ContainerResourceIncreaseRequestProto.newBuilder();
+  }
+
+  public ContainerResourceIncreaseRequestPBImpl(
+      ContainerResourceIncreaseRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ContainerResourceIncreaseRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public ContainerId getContainerId() {
+    ContainerResourceIncreaseRequestProtoOrBuilder 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() {
+    ContainerResourceIncreaseRequestProtoOrBuilder 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 = ContainerResourceIncreaseRequestProto.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));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 7e3abf2..f637ad6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -119,6 +119,7 @@ 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.ContainerResourceIncreaseRequest;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
@@ -511,6 +512,7 @@ public class TestPBImplRecords {
     generateByNewInstance(ResourceAllocationRequest.class);
     generateByNewInstance(ReservationAllocationState.class);
     generateByNewInstance(ResourceUtilization.class);
+    generateByNewInstance(ContainerResourceIncreaseRequest.class);
   }
 
   private class GetSetPair {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cac90903/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
index b87bf2b..49a6aec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
@@ -240,7 +240,7 @@ public class MockAM {
   public AllocateResponse sendContainerResizingRequest(
       List<UpdateContainerRequest> updateRequests) throws Exception {
     final AllocateRequest req = AllocateRequest.newInstance(0, 0F, null, null,
-        null, updateRequests);
+        updateRequests, null);
     return allocate(req);
   }
 


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