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 2015/09/23 22:32:53 UTC

[01/21] hadoop git commit: HDFS-9039. Separate client and server side methods of o.a.h.hdfs.NameNodeProxies. Contributed by Mingliang Liu.

Repository: hadoop
Updated Branches:
  refs/heads/YARN-1197 80c50a42f -> b3f6b641d (forced update)


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
index ccce736..c2d4d91 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
@@ -149,7 +149,7 @@ public class ConfiguredFailoverProxyProvider<T> extends
     if (current.namenode == null) {
       try {
         current.namenode = factory.createProxy(conf,
-            current.address, xface, ugi, false, fallbackToSimpleAuth);
+            current.address, xface, ugi, false, getFallbackToSimpleAuth());
       } catch (IOException e) {
         LOG.error("Failed to create RPC proxy to NameNode", e);
         throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.java
deleted file mode 100644
index 2842fb9..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.java
+++ /dev/null
@@ -1,80 +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.hdfs.server.namenode.ha;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URI;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.io.retry.FailoverProxyProvider;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.UserGroupInformation;
-
-import com.google.common.base.Preconditions;
-
-/**
- * A NNFailoverProxyProvider implementation which wrapps old implementations
- * directly implementing the {@link FailoverProxyProvider} interface.
- *
- * It is assumed that the old impelmentation is using logical URI.
- */
-public class WrappedFailoverProxyProvider<T> extends
-    AbstractNNFailoverProxyProvider<T> {
-  private final FailoverProxyProvider<T> proxyProvider;
-  
-  /**
-   * Wrap the given instance of an old FailoverProxyProvider.
-   */
-  public WrappedFailoverProxyProvider(FailoverProxyProvider<T> provider) {
-    proxyProvider = provider;
-  }
-    
-  @Override
-  public Class<T> getInterface() {
-    return proxyProvider.getInterface();
-  }
-
-  @Override
-  public synchronized ProxyInfo<T> getProxy() {
-    return proxyProvider.getProxy();
-  }
-
-  @Override
-  public void performFailover(T currentProxy) {
-    proxyProvider.performFailover(currentProxy);
-  }
-
-  /**
-   * Close the proxy,
-   */
-  @Override
-  public synchronized void close() throws IOException {
-    proxyProvider.close();
-  }
-
-  /**
-   * Assume logical URI is used for old proxy provider implementations.
-   */
-  @Override
-  public boolean useLogicalURI() {
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index 6ccd604..9c7a1ef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -60,7 +60,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.NameNodeProxies;
-import org.apache.hadoop.hdfs.NameNodeProxies.ProxyAndInfo;
+import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
index c27ead5..4af9c75 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
@@ -57,7 +57,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.NameNodeProxies;
+import org.apache.hadoop.hdfs.NameNodeProxiesClient;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
@@ -196,7 +196,7 @@ public class TestRetryCacheWithHA {
   private DFSClient genClientWithDummyHandler() throws IOException {
     URI nnUri = dfs.getUri();
     FailoverProxyProvider<ClientProtocol> failoverProxyProvider = 
-        NameNodeProxies.createFailoverProxyProvider(conf, 
+        NameNodeProxiesClient.createFailoverProxyProvider(conf,
             nnUri, ClientProtocol.class, true, null);
     InvocationHandler dummyHandler = new DummyRetryInvocationHandler(
         failoverProxyProvider, RetryPolicies


[12/21] hadoop git commit: YARN-3867. ContainerImpl changes to support container resizing. Contributed by Meng Ding

Posted by wa...@apache.org.
YARN-3867. ContainerImpl changes to support container resizing. Contributed by Meng Ding


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

Branch: refs/heads/YARN-1197
Commit: 5f5a968d65c44a831176764439e00db9203999ed
Parents: ffd820c
Author: Jian He <ji...@apache.org>
Authored: Tue Jul 28 13:51:23 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Sep 23 13:29:37 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../hadoop/yarn/server/utils/BuilderUtils.java  |  4 ++-
 .../containermanager/ContainerManagerImpl.java  |  7 ++--
 .../container/ChangeContainerResourceEvent.java | 36 -------------------
 .../containermanager/container/Container.java   |  2 ++
 .../container/ContainerEventType.java           |  4 ---
 .../container/ContainerImpl.java                | 16 ++++++---
 .../ChangeMonitoringContainerResourceEvent.java | 37 ++++++++++++++++++++
 .../monitor/ContainersMonitorEventType.java     |  3 +-
 .../nodemanager/metrics/NodeManagerMetrics.java | 11 ++++++
 .../nodemanager/TestNodeStatusUpdater.java      |  2 +-
 .../metrics/TestNodeManagerMetrics.java         | 18 +++++++---
 .../nodemanager/webapp/MockContainer.java       |  4 +++
 .../yarn/server/resourcemanager/MockNM.java     |  2 +-
 .../server/resourcemanager/NodeManager.java     |  2 +-
 .../resourcemanager/TestApplicationCleanup.java |  6 ++--
 .../attempt/TestRMAppAttemptTransitions.java    | 21 +++++++----
 .../capacity/TestCapacityScheduler.java         |  2 +-
 .../scheduler/fifo/TestFifoScheduler.java       |  4 +--
 .../security/TestAMRMTokens.java                |  3 +-
 20 files changed, 118 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 346fe85..309059f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -209,6 +209,9 @@ Release 2.8.0 - UNRELEASED
     YARN-1645. ContainerManager implementation to support container resizing.
     (Meng Ding & Wangda Tan via jianhe)
 
+    YARN-3867. ContainerImpl changes to support container resizing. (Meng Ding 
+    via jianhe)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
index a3bd6f8..475e9fe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
@@ -200,13 +200,15 @@ public class BuilderUtils {
   }
 
   public static ContainerStatus newContainerStatus(ContainerId containerId,
-      ContainerState containerState, String diagnostics, int exitStatus) {
+      ContainerState containerState, String diagnostics, int exitStatus,
+      Resource capability) {
     ContainerStatus containerStatus = recordFactory
       .newRecordInstance(ContainerStatus.class);
     containerStatus.setState(containerState);
     containerStatus.setContainerId(containerId);
     containerStatus.setDiagnostics(diagnostics);
     containerStatus.setExitStatus(exitStatus);
+    containerStatus.setCapability(capability);
     return containerStatus;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 890a4e4..4f2ccbe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -115,7 +115,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationInitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ChangeContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
@@ -130,6 +129,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.LogHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.NonAggregatingLogHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEventType;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ChangeMonitoringContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
@@ -1078,8 +1078,9 @@ public class ContainerManagerImpl extends CompositeService implements
     this.readLock.lock();
     try {
       if (!serviceStopped) {
-        dispatcher.getEventHandler().handle(new ChangeContainerResourceEvent(
-            containerId, targetResource));
+        getContainersMonitor().handle(
+            new ChangeMonitoringContainerResourceEvent(
+                containerId, targetResource));
       } else {
         throw new YarnException(
             "Unable to change container resource as the NodeManager is "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.java
deleted file mode 100644
index 3944a3d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
-
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.Resource;
-
-public class ChangeContainerResourceEvent extends ContainerEvent {
-
-  private Resource resource;
-
-  public ChangeContainerResourceEvent(ContainerId c, Resource resource) {
-    super(c, ContainerEventType.CHANGE_CONTAINER_RESOURCE);
-    this.resource = resource;
-  }
-
-  public Resource getResource() {
-    return this.resource;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
index 56b4fdd..1d2ec56 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
@@ -37,6 +37,8 @@ public interface Container extends EventHandler<ContainerEvent> {
 
   Resource getResource();
 
+  void setResource(Resource targetResource);
+
   ContainerTokenIdentifier getContainerTokenIdentifier();
 
   String getUser();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
index dc712bf..5622f8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
@@ -25,10 +25,6 @@ public enum ContainerEventType {
   KILL_CONTAINER,
   UPDATE_DIAGNOSTICS_MSG,
   CONTAINER_DONE,
-  CHANGE_CONTAINER_RESOURCE,
-
-  // Producer: ContainerMonitor
-  CONTAINER_RESOURCE_CHANGED,
 
   // DownloadManager
   CONTAINER_INITED,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 3c76596..5c61a92 100644
--- a/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
+++ b/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
@@ -79,6 +79,7 @@ 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;
 
 public class ContainerImpl implements Container {
 
@@ -91,7 +92,7 @@ public class ContainerImpl implements Container {
   private final ContainerLaunchContext launchContext;
   private final ContainerTokenIdentifier containerTokenIdentifier;
   private final ContainerId containerId;
-  private final Resource resource;
+  private volatile Resource resource;
   private final String user;
   private int exitCode = ContainerExitStatus.INVALID;
   private final StringBuilder diagnostics;
@@ -249,7 +250,7 @@ public class ContainerImpl implements Container {
         ContainerEventType.KILL_CONTAINER, new KillTransition())
     .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_FAILURE,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
-        new KilledExternallyTransition()) 
+        new KilledExternallyTransition())
 
     // From CONTAINER_EXITED_WITH_SUCCESS State
     .addTransition(ContainerState.EXITED_WITH_SUCCESS, ContainerState.DONE,
@@ -424,7 +425,7 @@ public class ContainerImpl implements Container {
     this.readLock.lock();
     try {
       return BuilderUtils.newContainerStatus(this.containerId,
-        getCurrentState(), diagnostics.toString(), exitCode);
+        getCurrentState(), diagnostics.toString(), exitCode, getResource());
     } finally {
       this.readLock.unlock();
     }
@@ -451,7 +452,14 @@ public class ContainerImpl implements Container {
 
   @Override
   public Resource getResource() {
-    return this.resource;
+    return Resources.clone(this.resource);
+  }
+
+  @Override
+  public void setResource(Resource targetResource) {
+    Resource currentResource = getResource();
+    this.resource = Resources.clone(targetResource);
+    this.metrics.changeContainer(currentResource, targetResource);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ChangeMonitoringContainerResourceEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ChangeMonitoringContainerResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ChangeMonitoringContainerResourceEvent.java
new file mode 100644
index 0000000..e0abbed
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ChangeMonitoringContainerResourceEvent.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+public class ChangeMonitoringContainerResourceEvent extends ContainersMonitorEvent {
+  private final Resource resource;
+
+  public ChangeMonitoringContainerResourceEvent(ContainerId containerId,
+      Resource resource) {
+    super(containerId,
+        ContainersMonitorEventType.CHANGE_MONITORING_CONTAINER_RESOURCE);
+    this.resource = resource;
+  }
+
+  public Resource getResource() {
+    return this.resource;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEventType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEventType.java
index be99651..2b31480 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorEventType.java
@@ -20,5 +20,6 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
 
 public enum ContainersMonitorEventType {
   START_MONITORING_CONTAINER,
-  STOP_MONITORING_CONTAINER
+  STOP_MONITORING_CONTAINER,
+  CHANGE_MONITORING_CONTAINER_RESOURCE
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
index 56797d1..a38d0b7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
@@ -133,6 +133,17 @@ public class NodeManagerMetrics {
     availableVCores.incr(res.getVirtualCores());
   }
 
+  public void changeContainer(Resource before, Resource now) {
+    int deltaMB = now.getMemory() - before.getMemory();
+    int deltaVCores = now.getVirtualCores() - before.getVirtualCores();
+    allocatedMB = allocatedMB + deltaMB;
+    allocatedGB.set((int)Math.ceil(allocatedMB/1024d));
+    availableMB = availableMB - deltaMB;
+    availableGB.set((int)Math.floor(availableMB/1024d));
+    allocatedVCores.incr(deltaVCores);
+    availableVCores.decr(deltaVCores);
+  }
+
   public void addResource(Resource res) {
     availableMB = availableMB + res.getMemory();
     availableGB.incr((int)Math.floor(availableMB/1024d));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
index 3c0368b..70a8f55 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
@@ -1662,7 +1662,7 @@ public class TestNodeStatusUpdater {
     ContainerStatus containerStatus =
         BuilderUtils.newContainerStatus(contaierId, containerState,
           "test_containerStatus: id=" + id + ", containerState: "
-              + containerState, 0);
+              + containerState, 0, Resource.newInstance(1024, 1));
     return containerStatus;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
index 4dc4648..c0210d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
@@ -38,7 +38,12 @@ public class TestNodeManagerMetrics {
     Resource resource = Records.newRecord(Resource.class);
     resource.setMemory(512); //512MiB
     resource.setVirtualCores(2);
-
+    Resource largerResource = Records.newRecord(Resource.class);
+    largerResource.setMemory(1024);
+    largerResource.setVirtualCores(2);
+    Resource smallerResource = Records.newRecord(Resource.class);
+    smallerResource.setMemory(256);
+    smallerResource.setVirtualCores(1);
 
     metrics.addResource(total);
 
@@ -65,15 +70,20 @@ public class TestNodeManagerMetrics {
     metrics.initingContainer();
     metrics.runningContainer();
 
+    // Increase resource for a container
+    metrics.changeContainer(resource, largerResource);
+    // Decrease resource for a container
+    metrics.changeContainer(resource, smallerResource);
+
     Assert.assertTrue(!metrics.containerLaunchDuration.changed());
     metrics.addContainerLaunchDuration(1);
     Assert.assertTrue(metrics.containerLaunchDuration.changed());
 
     // availableGB is expected to be floored,
     // while allocatedGB is expected to be ceiled.
-    // allocatedGB: 3.5GB allocated memory is shown as 4GB
-    // availableGB: 4.5GB available memory is shown as 4GB
-    checkMetrics(10, 1, 1, 1, 1, 1, 4, 7, 4, 14, 2);
+    // allocatedGB: 3.75GB allocated memory is shown as 4GB
+    // availableGB: 4.25GB available memory is shown as 4GB
+    checkMetrics(10, 1, 1, 1, 1, 1, 4, 7, 4, 13, 3);
   }
 
   private void checkMetrics(int launched, int completed, int failed, int killed,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
index b2ccb61..394a92c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
@@ -132,6 +132,10 @@ public class MockContainer implements Container {
   }
 
   @Override
+  public void setResource(Resource targetResource) {
+  }
+
+  @Override
   public ContainerTokenIdentifier getContainerTokenIdentifier() {
     return this.containerTokenIdentifier;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.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/MockNM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
index c917f79..4233cd4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
@@ -143,7 +143,7 @@ public class MockNM {
         new HashMap<ApplicationId, List<ContainerStatus>>(1);
     ContainerStatus containerStatus = BuilderUtils.newContainerStatus(
         BuilderUtils.newContainerId(attemptId, containerId), containerState,
-        "Success", 0);
+        "Success", 0, BuilderUtils.newResource(memory, vCores));
     ArrayList<ContainerStatus> containerStatusList =
         new ArrayList<ContainerStatus>(1);
     containerStatusList.add(containerStatus);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.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/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
index 5b7735e..b4ebf92 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
@@ -193,7 +193,7 @@ public class NodeManager implements ContainerManagementProtocol {
 
       ContainerStatus containerStatus =
           BuilderUtils.newContainerStatus(container.getId(),
-            ContainerState.NEW, "", -1000);
+            ContainerState.NEW, "", -1000, container.getResource());
       applicationContainers.add(container);
       containerStatusMap.put(container, containerStatus);
       Resources.subtractFrom(available, tokenId.getResource());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.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/TestApplicationCleanup.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java
index 6e08aeb..3fa377e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java
@@ -231,7 +231,8 @@ public class TestApplicationCleanup {
     ArrayList<ContainerStatus> containerStatusList =
         new ArrayList<ContainerStatus>();
     containerStatusList.add(BuilderUtils.newContainerStatus(conts.get(0)
-      .getId(), ContainerState.RUNNING, "nothing", 0));
+      .getId(), ContainerState.RUNNING, "nothing", 0,
+          conts.get(0).getResource()));
     containerStatuses.put(app.getApplicationId(), containerStatusList);
 
     NodeHeartbeatResponse resp = nm1.nodeHeartbeat(containerStatuses, true);
@@ -244,7 +245,8 @@ public class TestApplicationCleanup {
     containerStatuses.clear();
     containerStatusList.clear();
     containerStatusList.add(BuilderUtils.newContainerStatus(conts.get(0)
-      .getId(), ContainerState.RUNNING, "nothing", 0));
+      .getId(), ContainerState.RUNNING, "nothing", 0,
+          conts.get(0).getResource()));
     containerStatuses.put(app.getApplicationId(), containerStatusList);
 
     resp = nm1.nodeHeartbeat(containerStatuses, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
index c8b6bd0..10ec453 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
@@ -956,7 +956,8 @@ public class TestRMAppAttemptTransitions {
     int exitCode = 123;
     ContainerStatus cs =
         BuilderUtils.newContainerStatus(amContainer.getId(),
-          ContainerState.COMPLETE, containerDiagMsg, exitCode);
+          ContainerState.COMPLETE, containerDiagMsg, exitCode,
+          amContainer.getResource());
     NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
       applicationAttempt.getAppAttemptId(), cs, anyNodeId));
@@ -980,7 +981,8 @@ public class TestRMAppAttemptTransitions {
     String containerDiagMsg = "some error";
     int exitCode = 123;
     ContainerStatus cs = BuilderUtils.newContainerStatus(amContainer.getId(),
-        ContainerState.COMPLETE, containerDiagMsg, exitCode);
+        ContainerState.COMPLETE, containerDiagMsg, exitCode,
+            amContainer.getResource());
     ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
     NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
@@ -992,7 +994,8 @@ public class TestRMAppAttemptTransitions {
       applicationAttempt.getAppAttemptState());
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
       applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
-        amContainer.getId(), ContainerState.COMPLETE, "", 0), anyNodeId));
+        amContainer.getId(), ContainerState.COMPLETE, "", 0,
+            amContainer.getResource()), anyNodeId));
     applicationAttempt.handle(new RMAppAttemptEvent(
       applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
     assertEquals(RMAppAttemptState.FINAL_SAVING,
@@ -1030,7 +1033,8 @@ public class TestRMAppAttemptTransitions {
     NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
       applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
-        amContainer.getId(), ContainerState.COMPLETE, "", 0), anyNodeId));
+        amContainer.getId(), ContainerState.COMPLETE, "", 0,
+            amContainer.getResource()), anyNodeId));
     applicationAttempt.handle(new RMAppAttemptEvent(
       applicationAttempt.getAppAttemptId(), RMAppAttemptEventType.EXPIRE));
     assertEquals(RMAppAttemptState.FINAL_SAVING,
@@ -1207,7 +1211,8 @@ public class TestRMAppAttemptTransitions {
             BuilderUtils.newContainerStatus(
                 BuilderUtils.newContainerId(
                     applicationAttempt.getAppAttemptId(), 42),
-                ContainerState.COMPLETE, "", 0), anyNodeId));
+                ContainerState.COMPLETE, "", 0,
+                    amContainer.getResource()), anyNodeId));
     testAppAttemptFinishingState(amContainer, finalStatus, trackingUrl,
         diagnostics);
   }
@@ -1227,7 +1232,8 @@ public class TestRMAppAttemptTransitions {
         new RMAppAttemptContainerFinishedEvent(
             applicationAttempt.getAppAttemptId(),
             BuilderUtils.newContainerStatus(amContainer.getId(),
-                ContainerState.COMPLETE, "", 0), anyNodeId));
+                ContainerState.COMPLETE, "", 0,
+                    amContainer.getResource()), anyNodeId));
     testAppAttemptFinishedState(amContainer, finalStatus, trackingUrl,
         diagnostics, 0, false);
   }
@@ -1256,7 +1262,8 @@ public class TestRMAppAttemptTransitions {
     NodeId anyNodeId = NodeId.newInstance("host", 1234);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
       applicationAttempt.getAppAttemptId(), BuilderUtils.newContainerStatus(
-        amContainer.getId(), ContainerState.COMPLETE, "", 0), anyNodeId));
+        amContainer.getId(), ContainerState.COMPLETE, "", 0,
+            amContainer.getResource()), anyNodeId));
     assertEquals(RMAppAttemptState.FINAL_SAVING,
       applicationAttempt.getAppAttemptState());
     // send attempt_saved

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index fb7fce4..88c1444 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -870,7 +870,7 @@ public class TestCapacityScheduler {
     
     // Check container can complete successfully in case of resource over-commitment.
     ContainerStatus containerStatus = BuilderUtils.newContainerStatus(
-        c1.getId(), ContainerState.COMPLETE, "", 0);
+        c1.getId(), ContainerState.COMPLETE, "", 0, c1.getResource());
     nm1.containerStatus(containerStatus);
     int waitCount = 0;
     while (attempt1.getJustFinishedContainers().size() < 1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
index 5b5c5ed..1353bdd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
@@ -746,7 +746,7 @@ public class TestFifoScheduler {
     Assert.assertEquals(GB, c1.getResource().getMemory());
     ContainerStatus containerStatus =
         BuilderUtils.newContainerStatus(c1.getId(), ContainerState.COMPLETE,
-            "", 0);
+            "", 0, c1.getResource());
     nm1.containerStatus(containerStatus);
     int waitCount = 0;
     while (attempt1.getJustFinishedContainers().size() < 1 && waitCount++ != 20) {
@@ -1141,7 +1141,7 @@ public class TestFifoScheduler {
     // over-commitment.
     ContainerStatus containerStatus =
         BuilderUtils.newContainerStatus(c1.getId(), ContainerState.COMPLETE,
-            "", 0);
+            "", 0, c1.getResource());
     nm1.containerStatus(containerStatus);
     int waitCount = 0;
     while (attempt1.getJustFinishedContainers().size() < 1 && waitCount++ != 20) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5a968d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.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/security/TestAMRMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java
index 5dfd092..4488ad6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestAMRMTokens.java
@@ -171,7 +171,8 @@ public class TestAMRMTokens {
       ContainerStatus containerStatus =
           BuilderUtils.newContainerStatus(attempt.getMasterContainer().getId(),
               ContainerState.COMPLETE,
-              "AM Container Finished", 0);
+              "AM Container Finished", 0,
+              attempt.getMasterContainer().getResource());
       rm.getRMContext()
           .getDispatcher()
           .getEventHandler()


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

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


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

Branch: refs/heads/YARN-1197
Commit: 89cab1ba5f0671f8ef30dbe7432079c18362b434
Parents: c57eac5
Author: Jian He <ji...@apache.org>
Authored: Tue Sep 15 10:21:39 2015 +0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Sep 23 13:29:38 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
index 1a3829e..e6aebb4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
@@ -98,6 +98,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
@@ -1634,8 +1635,10 @@ public class TestRMContainerAllocator {
     @Override
     public synchronized Allocation allocate(
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
-        List<ContainerId> release, 
-        List<String> blacklistAdditions, List<String> blacklistRemovals) {
+        List<ContainerId> release, List<String> blacklistAdditions,
+        List<String> blacklistRemovals,
+        List<ContainerResourceChangeRequest> increaseRequests,
+        List<ContainerResourceChangeRequest> decreaseRequests) {
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       for (ResourceRequest req : ask) {
         ResourceRequest reqCopy = ResourceRequest.newInstance(req
@@ -1649,8 +1652,8 @@ public class TestRMContainerAllocator {
       lastBlacklistAdditions = blacklistAdditions;
       lastBlacklistRemovals = blacklistRemovals;
       return super.allocate(
-          applicationAttemptId, askCopy, release, 
-          blacklistAdditions, blacklistRemovals);
+          applicationAttemptId, askCopy, release, blacklistAdditions,
+          blacklistRemovals, increaseRequests, decreaseRequests);
     }
   }
 
@@ -1670,8 +1673,10 @@ public class TestRMContainerAllocator {
     @Override
     public synchronized Allocation allocate(
         ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
-        List<ContainerId> release,
-        List<String> blacklistAdditions, List<String> blacklistRemovals) {
+        List<ContainerId> release, List<String> blacklistAdditions,
+        List<String> blacklistRemovals,
+        List<ContainerResourceChangeRequest> increaseRequest,
+        List<ContainerResourceChangeRequest> decreaseRequests) {
       List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
       for (ResourceRequest req : ask) {
         ResourceRequest reqCopy = ResourceRequest.newInstance(req
@@ -1682,7 +1687,7 @@ public class TestRMContainerAllocator {
       SecurityUtil.setTokenServiceUseIp(false);
       Allocation normalAlloc = super.allocate(
           applicationAttemptId, askCopy, release,
-          blacklistAdditions, blacklistRemovals);
+          blacklistAdditions, blacklistRemovals, null, null);
       List<Container> containers = normalAlloc.getContainers();
       if(containers.size() > 0) {
         // allocate excess container

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index 7e774c5..248cdc6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -452,6 +452,8 @@ public class ResourceTrackerService extends AbstractService implements
             getResponseId() + 1, NodeAction.NORMAL, null, null, null, null,
             nextHeartBeatInterval);
     rmNode.updateNodeHeartbeatResponseForCleanup(nodeHeartBeatResponse);
+    rmNode.updateNodeHeartbeatResponseForContainersDecreasing(
+        nodeHeartBeatResponse);
 
     populateKeys(request, nodeHeartBeatResponse);
 
@@ -464,8 +466,9 @@ public class ResourceTrackerService extends AbstractService implements
     // 4. Send status to RMNode, saving the latest response.
     RMNodeStatusEvent nodeStatusEvent =
         new RMNodeStatusEvent(nodeId, remoteNodeStatus.getNodeHealthStatus(),
-          remoteNodeStatus.getContainersStatuses(),
-          remoteNodeStatus.getKeepAliveApplications(), nodeHeartBeatResponse);
+            remoteNodeStatus.getContainersStatuses(),
+            remoteNodeStatus.getKeepAliveApplications(), nodeHeartBeatResponse,
+            remoteNodeStatus.getIncreasedContainers());
     if (request.getLogAggregationReportsForApps() != null
         && !request.getLogAggregationReportsForApps().isEmpty()) {
       nodeStatusEvent.setLogAggregationReportsForApps(request

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
index 21d79ee..dc0d9ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java
@@ -82,4 +82,8 @@ public interface RMContainer extends EventHandler<RMContainerEvent> {
   String getNodeHttpAddress();
   
   String getNodeLabelExpression();
+  
+  boolean hasIncreaseReservation();
+  
+  void cancelIncreaseReservation();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerChangeResourceEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerChangeResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerChangeResourceEvent.java
new file mode 100644
index 0000000..920cfdb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerChangeResourceEvent.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+public class RMContainerChangeResourceEvent extends RMContainerEvent {
+  
+  final Resource targetResource;
+  final boolean increase;
+
+  public RMContainerChangeResourceEvent(ContainerId containerId,
+      Resource targetResource, boolean increase) {
+    super(containerId, RMContainerEventType.CHANGE_RESOURCE);
+
+    this.targetResource = targetResource;
+    this.increase = increase;
+  }
+  
+  public Resource getTargetResource() {
+    return targetResource;
+  }
+  
+  public boolean isIncrease() {
+    return increase;
+  }
+}

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerUpdatesAcquiredEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerUpdatesAcquiredEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerUpdatesAcquiredEvent.java
new file mode 100644
index 0000000..0dccc5f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerUpdatesAcquiredEvent.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.rmcontainer;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+
+public class RMContainerUpdatesAcquiredEvent extends RMContainerEvent  {
+  private final boolean increasedContainer;
+  
+  public RMContainerUpdatesAcquiredEvent(ContainerId containerId,
+      boolean increasedContainer) {
+    super(containerId, RMContainerEventType.ACQUIRE_UPDATED_CONTAINER);
+    this.increasedContainer = increasedContainer; 
+  }
+  
+  public boolean isIncreasedContainer() {
+    return increasedContainer;
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeDecreaseContainerEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeDecreaseContainerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeDecreaseContainerEvent.java
new file mode 100644
index 0000000..62925ad
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeDecreaseContainerEvent.java
@@ -0,0 +1,39 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.resourcemanager.rmnode;
+
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.NodeId;
+
+public class RMNodeDecreaseContainerEvent extends RMNodeEvent {
+  final List<Container> toBeDecreasedContainers;
+
+  public RMNodeDecreaseContainerEvent(NodeId nodeId,
+      List<Container> toBeDecreasedContainers) {
+    super(nodeId, RMNodeEventType.DECREASE_CONTAINER);
+    
+    this.toBeDecreasedContainers = toBeDecreasedContainers;
+  }
+  
+  public List<Container> getToBeDecreasedContainers() {
+    return toBeDecreasedContainers;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
index ad36036..abe8544 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeEventType.java
@@ -42,6 +42,7 @@ public enum RMNodeEventType {
   // Source: Container
   CONTAINER_ALLOCATED,
   CLEANUP_CONTAINER,
+  DECREASE_CONTAINER,
 
   // Source: RMAppAttempt
   FINISHED_CONTAINERS_PULLED_BY_AM,

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

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


[02/21] hadoop git commit: HDFS-9039. Separate client and server side methods of o.a.h.hdfs.NameNodeProxies. Contributed by Mingliang Liu.

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index b38ec00..8f87895 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -94,6 +94,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
@@ -313,14 +314,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     int numResponseToDrop = conf.getInt(
         DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
         DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
-    NameNodeProxies.ProxyAndInfo<ClientProtocol> proxyInfo = null;
+    ProxyAndInfo<ClientProtocol> proxyInfo = null;
     AtomicBoolean nnFallbackToSimpleAuth = new AtomicBoolean(false);
     if (numResponseToDrop > 0) {
       // This case is used for testing.
       LOG.warn(DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
           + " is set to " + numResponseToDrop
           + ", this hacked client will proactively drop responses");
-      proxyInfo = NameNodeProxies.createProxyWithLossyRetryHandler(conf,
+      proxyInfo = NameNodeProxiesClient.createProxyWithLossyRetryHandler(conf,
           nameNodeUri, ClientProtocol.class, numResponseToDrop,
           nnFallbackToSimpleAuth);
     }
@@ -336,8 +337,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     } else {
       Preconditions.checkArgument(nameNodeUri != null,
           "null URI");
-      proxyInfo = NameNodeProxies.createProxy(conf, nameNodeUri,
-          ClientProtocol.class, nnFallbackToSimpleAuth);
+      proxyInfo = NameNodeProxiesClient.createProxyWithClientProtocol(conf,
+          nameNodeUri, nnFallbackToSimpleAuth);
       this.dtService = proxyInfo.getDelegationTokenService();
       this.namenode = proxyInfo.getProxy();
     }
@@ -780,8 +781,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
             "a failover proxy provider configured.");
       }
       
-      NameNodeProxies.ProxyAndInfo<ClientProtocol> info =
-        NameNodeProxies.createProxy(conf, uri, ClientProtocol.class);
+      ProxyAndInfo<ClientProtocol> info =
+        NameNodeProxiesClient.createProxyWithClientProtocol(conf, uri, null);
       assert info.getDelegationTokenService().equals(token.getService()) :
         "Returned service '" + info.getDelegationTokenService().toString() +
         "' doesn't match expected service '" +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
index 686a0b7..ff409c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.NameNodeProxies.ProxyAndInfo;
+import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -220,9 +220,9 @@ public class HAUtil {
   public static boolean useLogicalUri(Configuration conf, URI nameNodeUri) 
       throws IOException {
     // Create the proxy provider. Actual proxy is not created.
-    AbstractNNFailoverProxyProvider<ClientProtocol> provider = NameNodeProxies
+    AbstractNNFailoverProxyProvider<ClientProtocol> provider = NameNodeProxiesClient
         .createFailoverProxyProvider(conf, nameNodeUri, ClientProtocol.class,
-        false, null);
+            false, null);
 
     // No need to use logical URI since failover is not configured.
     if (provider == null) {
@@ -336,8 +336,7 @@ public class HAUtil {
     List<ProxyAndInfo<T>> proxies = new ArrayList<ProxyAndInfo<T>>(
         nnAddresses.size());
     for (InetSocketAddress nnAddress : nnAddresses.values()) {
-      NameNodeProxies.ProxyAndInfo<T> proxyInfo = null;
-      proxyInfo = NameNodeProxies.createNonHAProxy(conf,
+      ProxyAndInfo<T> proxyInfo = NameNodeProxies.createNonHAProxy(conf,
           nnAddress, xface,
           UserGroupInformation.getCurrentUser(), false);
       proxies.add(proxyInfo);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
index 80efa19..61d701d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/NameNodeProxies.java
@@ -18,9 +18,6 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.Proxy;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.util.HashMap;
@@ -32,31 +29,19 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
-import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.JournalProtocolTranslatorPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
 import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.namenode.ha.AbstractNNFailoverProxyProvider;
-import org.apache.hadoop.hdfs.server.namenode.ha.WrappedFailoverProxyProvider;
 import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider;
-import org.apache.hadoop.io.retry.FailoverProxyProvider;
-import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
-import org.apache.hadoop.io.retry.RetryUtils;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RefreshCallQueueProtocol;
@@ -75,9 +60,6 @@ import org.apache.hadoop.tools.GetUserMappingsProtocol;
 import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB;
 import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolPB;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
 /**
  * Create proxy objects to communicate with a remote NN. All remote access to an
  * NN should be funneled through this class. Most of the time you'll want to use
@@ -90,37 +72,6 @@ public class NameNodeProxies {
   private static final Log LOG = LogFactory.getLog(NameNodeProxies.class);
 
   /**
-   * Wrapper for a client proxy as well as its associated service ID.
-   * This is simply used as a tuple-like return type for
-   * {@link NameNodeProxies#createProxy} and
-   * {@link NameNodeProxies#createNonHAProxy}.
-   */
-  public static class ProxyAndInfo<PROXYTYPE> {
-    private final PROXYTYPE proxy;
-    private final Text dtService;
-    private final InetSocketAddress address;
-    
-    public ProxyAndInfo(PROXYTYPE proxy, Text dtService,
-        InetSocketAddress address) {
-      this.proxy = proxy;
-      this.dtService = dtService;
-      this.address = address;
-    }
-    
-    public PROXYTYPE getProxy() {
-      return proxy;
-    }
-    
-    public Text getDelegationTokenService() {
-      return dtService;
-    }
-
-    public InetSocketAddress getAddress() {
-      return address;
-    }
-  }
-
-  /**
    * Creates the namenode proxy with the passed protocol. This will handle
    * creation of either HA- or non-HA-enabled proxy objects, depending upon
    * if the provided URI is a configured logical URI.
@@ -160,103 +111,16 @@ public class NameNodeProxies {
       URI nameNodeUri, Class<T> xface, AtomicBoolean fallbackToSimpleAuth)
       throws IOException {
     AbstractNNFailoverProxyProvider<T> failoverProxyProvider =
-        createFailoverProxyProvider(conf, nameNodeUri, xface, true,
-          fallbackToSimpleAuth);
-  
+        NameNodeProxiesClient.createFailoverProxyProvider(conf, nameNodeUri,
+            xface, true, fallbackToSimpleAuth);
+
     if (failoverProxyProvider == null) {
-      // Non-HA case
       return createNonHAProxy(conf, DFSUtilClient.getNNAddress(nameNodeUri),
           xface, UserGroupInformation.getCurrentUser(), true,
           fallbackToSimpleAuth);
     } else {
-      // HA case
-      DfsClientConf config = new DfsClientConf(conf);
-      T proxy = (T) RetryProxy.create(xface, failoverProxyProvider,
-          RetryPolicies.failoverOnNetworkException(
-              RetryPolicies.TRY_ONCE_THEN_FAIL, config.getMaxFailoverAttempts(),
-              config.getMaxRetryAttempts(), config.getFailoverSleepBaseMillis(),
-              config.getFailoverSleepMaxMillis()));
-
-      Text dtService;
-      if (failoverProxyProvider.useLogicalURI()) {
-        dtService = HAUtilClient.buildTokenServiceForLogicalUri(nameNodeUri,
-                                                                HdfsConstants.HDFS_URI_SCHEME);
-      } else {
-        dtService = SecurityUtil.buildTokenService(
-            DFSUtilClient.getNNAddress(nameNodeUri));
-      }
-      return new ProxyAndInfo<T>(proxy, dtService,
-          DFSUtilClient.getNNAddress(nameNodeUri));
-    }
-  }
-  
-  /**
-   * Generate a dummy namenode proxy instance that utilizes our hacked
-   * {@link LossyRetryInvocationHandler}. Proxy instance generated using this
-   * method will proactively drop RPC responses. Currently this method only
-   * support HA setup. null will be returned if the given configuration is not 
-   * for HA.
-   * 
-   * @param config the configuration containing the required IPC
-   *        properties, client failover configurations, etc.
-   * @param nameNodeUri the URI pointing either to a specific NameNode
-   *        or to a logical nameservice.
-   * @param xface the IPC interface which should be created
-   * @param numResponseToDrop The number of responses to drop for each RPC call
-   * @param fallbackToSimpleAuth set to true or false during calls to indicate if
-   *   a secure client falls back to simple auth
-   * @return an object containing both the proxy and the associated
-   *         delegation token service it corresponds to. Will return null of the
-   *         given configuration does not support HA.
-   * @throws IOException if there is an error creating the proxy
-   */
-  @SuppressWarnings("unchecked")
-  public static <T> ProxyAndInfo<T> createProxyWithLossyRetryHandler(
-      Configuration config, URI nameNodeUri, Class<T> xface,
-      int numResponseToDrop, AtomicBoolean fallbackToSimpleAuth)
-      throws IOException {
-    Preconditions.checkArgument(numResponseToDrop > 0);
-    AbstractNNFailoverProxyProvider<T> failoverProxyProvider =
-        createFailoverProxyProvider(config, nameNodeUri, xface, true,
-          fallbackToSimpleAuth);
-
-    if (failoverProxyProvider != null) { // HA case
-      int delay = config.getInt(
-          HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_KEY,
-          HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT);
-      int maxCap = config.getInt(
-          HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY,
-          HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT);
-      int maxFailoverAttempts = config.getInt(
-          HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY,
-          HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_DEFAULT);
-      int maxRetryAttempts = config.getInt(
-          HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY,
-          HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_DEFAULT);
-      InvocationHandler dummyHandler = new LossyRetryInvocationHandler<T>(
-              numResponseToDrop, failoverProxyProvider,
-              RetryPolicies.failoverOnNetworkException(
-                  RetryPolicies.TRY_ONCE_THEN_FAIL, maxFailoverAttempts, 
-                  Math.max(numResponseToDrop + 1, maxRetryAttempts), delay, 
-                  maxCap));
-      
-      T proxy = (T) Proxy.newProxyInstance(
-          failoverProxyProvider.getInterface().getClassLoader(),
-          new Class[] { xface }, dummyHandler);
-      Text dtService;
-      if (failoverProxyProvider.useLogicalURI()) {
-        dtService = HAUtilClient.buildTokenServiceForLogicalUri(nameNodeUri,
-                                                                HdfsConstants.HDFS_URI_SCHEME);
-      } else {
-        dtService = SecurityUtil.buildTokenService(
-            DFSUtilClient.getNNAddress(nameNodeUri));
-      }
-      return new ProxyAndInfo<T>(proxy, dtService,
-          DFSUtilClient.getNNAddress(nameNodeUri));
-    } else {
-      LOG.warn("Currently creating proxy using " +
-      		"LossyRetryInvocationHandler requires NN HA setup");
-      return null;
+      return NameNodeProxiesClient.createHAProxy(conf, nameNodeUri, xface,
+          failoverProxyProvider);
     }
   }
 
@@ -303,8 +167,8 @@ public class NameNodeProxies {
   
     T proxy;
     if (xface == ClientProtocol.class) {
-      proxy = (T) createNNProxyWithClientProtocol(nnAddr, conf, ugi,
-          withRetries, fallbackToSimpleAuth);
+      proxy = (T) NameNodeProxiesClient.createNonHAProxyWithClientProtocol(
+          nnAddr, conf, ugi, withRetries, fallbackToSimpleAuth);
     } else if (xface == JournalProtocol.class) {
       proxy = (T) createNNProxyWithJournalProtocol(nnAddr, conf, ugi);
     } else if (xface == NamenodeProtocol.class) {
@@ -390,45 +254,6 @@ public class NameNodeProxies {
       return new NamenodeProtocolTranslatorPB(proxy);
     }
   }
-  
-  private static ClientProtocol createNNProxyWithClientProtocol(
-      InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
-      boolean withRetries, AtomicBoolean fallbackToSimpleAuth)
-      throws IOException {
-    RPC.setProtocolEngine(conf, ClientNamenodeProtocolPB.class, ProtobufRpcEngine.class);
-
-    final RetryPolicy defaultPolicy = 
-        RetryUtils.getDefaultRetryPolicy(
-            conf, 
-            HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY, 
-            HdfsClientConfigKeys.Retry.POLICY_ENABLED_DEFAULT, 
-            HdfsClientConfigKeys.Retry.POLICY_SPEC_KEY,
-            HdfsClientConfigKeys.Retry.POLICY_SPEC_DEFAULT,
-            SafeModeException.class.getName());
-    
-    final long version = RPC.getProtocolVersion(ClientNamenodeProtocolPB.class);
-    ClientNamenodeProtocolPB proxy = RPC.getProtocolProxy(
-        ClientNamenodeProtocolPB.class, version, address, ugi, conf,
-        NetUtils.getDefaultSocketFactory(conf),
-        org.apache.hadoop.ipc.Client.getTimeout(conf), defaultPolicy,
-        fallbackToSimpleAuth).getProxy();
-
-    if (withRetries) { // create the proxy with retries
-
-      Map<String, RetryPolicy> methodNameToPolicyMap 
-                 = new HashMap<String, RetryPolicy>();
-      ClientProtocol translatorProxy =
-        new ClientNamenodeProtocolTranslatorPB(proxy);
-      return (ClientProtocol) RetryProxy.create(
-          ClientProtocol.class,
-          new DefaultFailoverProxyProvider<ClientProtocol>(
-              ClientProtocol.class, translatorProxy),
-          methodNameToPolicyMap,
-          defaultPolicy);
-    } else {
-      return new ClientNamenodeProtocolTranslatorPB(proxy);
-    }
-  }
 
   private static Object createNameNodeProxy(InetSocketAddress address,
       Configuration conf, UserGroupInformation ugi, Class<?> xface,
@@ -439,88 +264,4 @@ public class NameNodeProxies {
     return proxy;
   }
 
-  /** Gets the configured Failover proxy provider's class */
-  @VisibleForTesting
-  public static <T> Class<FailoverProxyProvider<T>> getFailoverProxyProviderClass(
-      Configuration conf, URI nameNodeUri) throws IOException {
-    if (nameNodeUri == null) {
-      return null;
-    }
-    String host = nameNodeUri.getHost();
-    String configKey = HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX
-        + "." + host;
-    try {
-      @SuppressWarnings("unchecked")
-      Class<FailoverProxyProvider<T>> ret = (Class<FailoverProxyProvider<T>>) conf
-          .getClass(configKey, null, FailoverProxyProvider.class);
-      return ret;
-    } catch (RuntimeException e) {
-      if (e.getCause() instanceof ClassNotFoundException) {
-        throw new IOException("Could not load failover proxy provider class "
-            + conf.get(configKey) + " which is configured for authority "
-            + nameNodeUri, e);
-      } else {
-        throw e;
-      }
-    }
-  }
-
-  /** Creates the Failover proxy provider instance*/
-  @VisibleForTesting
-  public static <T> AbstractNNFailoverProxyProvider<T> createFailoverProxyProvider(
-      Configuration conf, URI nameNodeUri, Class<T> xface, boolean checkPort,
-      AtomicBoolean fallbackToSimpleAuth) throws IOException {
-    Class<FailoverProxyProvider<T>> failoverProxyProviderClass = null;
-    AbstractNNFailoverProxyProvider<T> providerNN;
-    Preconditions.checkArgument(
-        xface.isAssignableFrom(NamenodeProtocols.class),
-        "Interface %s is not a NameNode protocol", xface);
-    try {
-      // Obtain the class of the proxy provider
-      failoverProxyProviderClass = getFailoverProxyProviderClass(conf,
-          nameNodeUri);
-      if (failoverProxyProviderClass == null) {
-        return null;
-      }
-      // Create a proxy provider instance.
-      Constructor<FailoverProxyProvider<T>> ctor = failoverProxyProviderClass
-          .getConstructor(Configuration.class, URI.class, Class.class);
-      FailoverProxyProvider<T> provider = ctor.newInstance(conf, nameNodeUri,
-          xface);
-
-      // If the proxy provider is of an old implementation, wrap it.
-      if (!(provider instanceof AbstractNNFailoverProxyProvider)) {
-        providerNN = new WrappedFailoverProxyProvider<T>(provider);
-      } else {
-        providerNN = (AbstractNNFailoverProxyProvider<T>)provider;
-      }
-    } catch (Exception e) {
-      String message = "Couldn't create proxy provider " + failoverProxyProviderClass;
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(message, e);
-      }
-      if (e.getCause() instanceof IOException) {
-        throw (IOException) e.getCause();
-      } else {
-        throw new IOException(message, e);
-      }
-    }
-
-    // Check the port in the URI, if it is logical.
-    if (checkPort && providerNN.useLogicalURI()) {
-      int port = nameNodeUri.getPort();
-      if (port > 0 &&
-          port != HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT) {
-        // Throwing here without any cleanup is fine since we have not
-        // actually created the underlying proxies yet.
-        throw new IOException("Port " + port + " specified in URI "
-            + nameNodeUri + " but host '" + nameNodeUri.getHost()
-            + "' is a logical (HA) namenode"
-            + " and does not use port information.");
-      }
-    }
-    providerNN.setFallbackToSimpleAuth(fallbackToSimpleAuth);
-    return providerNN;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
deleted file mode 100644
index e7ce44b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
+++ /dev/null
@@ -1,46 +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.hdfs.protocolPB;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.security.KerberosInfo;
-import org.apache.hadoop.security.token.TokenInfo;
-
-
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-@KerberosInfo(
-    serverPrincipal = HdfsClientConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
-@TokenInfo(DelegationTokenSelector.class)
-@ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME,
-    protocolVersion = 1)
-/**
- * Protocol that a clients use to communicate with the NameNode.
- * 
- * Note: This extends the protocolbuffer service based interface to
- * add annotations required for security.
- */
-public interface ClientNamenodeProtocolPB extends 
-  ClientNamenodeProtocol.BlockingInterface {
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
deleted file mode 100644
index f4ce46d..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ /dev/null
@@ -1,1531 +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.hdfs.protocolPB;
-
-import java.io.Closeable;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.EnumSet;
-import java.util.List;
-
-import com.google.common.collect.Lists;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.crypto.CryptoProtocolVersion;
-import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
-import org.apache.hadoop.fs.CacheFlag;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FsServerDefaults;
-import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.XAttr;
-import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.inotify.EventBatchList;
-import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
-import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
-import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.EncryptionZone;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
-import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
-import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.AclProtos.ModifyAclEntriesRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclEntriesRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveDefaultAclRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.AclProtos.SetAclRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DisallowSnapshotRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetCurrentEditLogTxidRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatusRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
-import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
-import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.ProtobufHelper;
-import org.apache.hadoop.ipc.ProtocolMetaInterface;
-import org.apache.hadoop.ipc.ProtocolTranslator;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RpcClientUtil;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
-import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
-import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
-import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
-import org.apache.hadoop.security.token.Token;
-
-import com.google.protobuf.ByteString;
-import com.google.protobuf.ServiceException;
-
-import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
-import static org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos
-    .EncryptionZoneProto;
-
-/**
- * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
- * while translating from the parameter types used in ClientProtocol to the
- * new PB types.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ClientNamenodeProtocolTranslatorPB implements
-    ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
-  final private ClientNamenodeProtocolPB rpcProxy;
-
-  static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST = 
-  GetServerDefaultsRequestProto.newBuilder().build();
-
-  private final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST =
-  GetFsStatusRequestProto.newBuilder().build();
-
-  private final static SaveNamespaceRequestProto VOID_SAVE_NAMESPACE_REQUEST =
-  SaveNamespaceRequestProto.newBuilder().build();
-
-  private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST = 
-  RollEditsRequestProto.getDefaultInstance();
-
-  private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST =
-  RefreshNodesRequestProto.newBuilder().build();
-
-  private final static FinalizeUpgradeRequestProto
-  VOID_FINALIZE_UPGRADE_REQUEST =
-      FinalizeUpgradeRequestProto.newBuilder().build();
-
-  private final static GetDataEncryptionKeyRequestProto
-  VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
-      GetDataEncryptionKeyRequestProto.newBuilder().build();
-
-  private final static GetStoragePoliciesRequestProto
-  VOID_GET_STORAGE_POLICIES_REQUEST =
-      GetStoragePoliciesRequestProto.newBuilder().build();
-
-  public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
-    rpcProxy = proxy;
-  }
-  
-  @Override
-  public void close() {
-    RPC.stopProxy(rpcProxy);
-  }
-
-  @Override
-  public LocatedBlocks getBlockLocations(String src, long offset, long length)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto
-        .newBuilder()
-        .setSrc(src)
-        .setOffset(offset)
-        .setLength(length)
-        .build();
-    try {
-      GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
-          req);
-      return resp.hasLocations() ?
-        PBHelperClient.convert(resp.getLocations()) : null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public FsServerDefaults getServerDefaults() throws IOException {
-    GetServerDefaultsRequestProto req = VOID_GET_SERVER_DEFAULT_REQUEST;
-    try {
-      return PBHelperClient
-          .convert(rpcProxy.getServerDefaults(null, req).getServerDefaults());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public HdfsFileStatus create(String src, FsPermission masked,
-      String clientName, EnumSetWritable<CreateFlag> flag,
-      boolean createParent, short replication, long blockSize, 
-      CryptoProtocolVersion[] supportedVersions)
-      throws AccessControlException, AlreadyBeingCreatedException,
-      DSQuotaExceededException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    CreateRequestProto.Builder builder = CreateRequestProto.newBuilder()
-        .setSrc(src)
-        .setMasked(PBHelperClient.convert(masked))
-        .setClientName(clientName)
-        .setCreateFlag(PBHelperClient.convertCreateFlag(flag))
-        .setCreateParent(createParent)
-        .setReplication(replication)
-        .setBlockSize(blockSize);
-    builder.addAllCryptoProtocolVersion(PBHelperClient.convert(supportedVersions));
-    CreateRequestProto req = builder.build();
-    try {
-      CreateResponseProto res = rpcProxy.create(null, req);
-      return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-  }
-
-  @Override
-  public boolean truncate(String src, long newLength, String clientName)
-      throws IOException, UnresolvedLinkException {
-    TruncateRequestProto req = TruncateRequestProto.newBuilder()
-        .setSrc(src)
-        .setNewLength(newLength)
-        .setClientName(clientName)
-        .build();
-    try {
-      return rpcProxy.truncate(null, req).getResult();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public LastBlockWithStatus append(String src, String clientName,
-      EnumSetWritable<CreateFlag> flag) throws AccessControlException,
-      DSQuotaExceededException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src)
-        .setClientName(clientName).setFlag(PBHelperClient.convertCreateFlag(flag))
-        .build();
-    try {
-      AppendResponseProto res = rpcProxy.append(null, req);
-      LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
-          .convert(res.getBlock()) : null;
-      HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
-          : null;
-      return new LastBlockWithStatus(lastBlock, stat);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public boolean setReplication(String src, short replication)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder()
-        .setSrc(src)
-        .setReplication(replication)
-        .build();
-    try {
-      return rpcProxy.setReplication(null, req).getResult();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void setPermission(String src, FsPermission permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
-        .setSrc(src)
-        .setPermission(PBHelperClient.convert(permission))
-        .build();
-    try {
-      rpcProxy.setPermission(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void setOwner(String src, String username, String groupname)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder()
-        .setSrc(src);
-    if (username != null)
-        req.setUsername(username);
-    if (groupname != null)
-        req.setGroupname(groupname);
-    try {
-      rpcProxy.setOwner(null, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void abandonBlock(ExtendedBlock b, long fileId, String src,
-      String holder) throws AccessControlException, FileNotFoundException,
-        UnresolvedLinkException, IOException {
-    AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
-        .setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder)
-            .setFileId(fileId).build();
-    try {
-      rpcProxy.abandonBlock(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-  
-  @Override
-  public LocatedBlock addBlock(String src, String clientName,
-      ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
-      String[] favoredNodes)
-      throws AccessControlException, FileNotFoundException,
-      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder()
-        .setSrc(src).setClientName(clientName).setFileId(fileId);
-    if (previous != null) 
-      req.setPrevious(PBHelperClient.convert(previous));
-    if (excludeNodes != null)
-      req.addAllExcludeNodes(PBHelperClient.convert(excludeNodes));
-    if (favoredNodes != null) {
-      req.addAllFavoredNodes(Arrays.asList(favoredNodes));
-    }
-    try {
-      return PBHelperClient.convert(rpcProxy.addBlock(null, req.build()).getBlock());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public LocatedBlock getAdditionalDatanode(String src, long fileId,
-      ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs,
-      DatanodeInfo[] excludes,
-      int numAdditionalNodes, String clientName) throws AccessControlException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto
-        .newBuilder()
-        .setSrc(src)
-        .setFileId(fileId)
-        .setBlk(PBHelperClient.convert(blk))
-        .addAllExistings(PBHelperClient.convert(existings))
-        .addAllExistingStorageUuids(Arrays.asList(existingStorageIDs))
-        .addAllExcludes(PBHelperClient.convert(excludes))
-        .setNumAdditionalNodes(numAdditionalNodes)
-        .setClientName(clientName)
-        .build();
-    try {
-      return PBHelperClient.convert(rpcProxy.getAdditionalDatanode(null, req)
-          .getBlock());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public boolean complete(String src, String clientName,
-                          ExtendedBlock last, long fileId)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder()
-        .setSrc(src)
-        .setClientName(clientName)
-        .setFileId(fileId);
-    if (last != null)
-      req.setLast(PBHelperClient.convert(last));
-    try {
-      return rpcProxy.complete(null, req.build()).getResult();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
-    ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
-        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks)))
-        .build();
-    try {
-      rpcProxy.reportBadBlocks(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public boolean rename(String src, String dst) throws UnresolvedLinkException,
-      IOException {
-    RenameRequestProto req = RenameRequestProto.newBuilder()
-        .setSrc(src)
-        .setDst(dst).build();
-    try {
-      return rpcProxy.rename(null, req).getResult();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-  
-
-  @Override
-  public void rename2(String src, String dst, Rename... options)
-      throws AccessControlException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    boolean overwrite = false;
-    if (options != null) {
-      for (Rename option : options) {
-        if (option == Rename.OVERWRITE) {
-          overwrite = true;
-        }
-      }
-    }
-    Rename2RequestProto req = Rename2RequestProto.newBuilder().
-        setSrc(src).
-        setDst(dst).setOverwriteDest(overwrite).
-        build();
-    try {
-      rpcProxy.rename2(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-  }
-
-  @Override
-  public void concat(String trg, String[] srcs) throws IOException,
-      UnresolvedLinkException {
-    ConcatRequestProto req = ConcatRequestProto.newBuilder().
-        setTrg(trg).
-        addAllSrcs(Arrays.asList(srcs)).build();
-    try {
-      rpcProxy.concat(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-
-  @Override
-  public boolean delete(String src, boolean recursive)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src).setRecursive(recursive).build();
-    try {
-      return rpcProxy.delete(null, req).getResult();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
-      throws AccessControlException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
-        .setSrc(src)
-        .setMasked(PBHelperClient.convert(masked))
-        .setCreateParent(createParent).build();
-
-    try {
-      return rpcProxy.mkdirs(null, req).getResult();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public DirectoryListing getListing(String src, byte[] startAfter,
-      boolean needLocation) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    GetListingRequestProto req = GetListingRequestProto.newBuilder()
-        .setSrc(src)
-        .setStartAfter(ByteString.copyFrom(startAfter))
-        .setNeedLocation(needLocation).build();
-    try {
-      GetListingResponseProto result = rpcProxy.getListing(null, req);
-      
-      if (result.hasDirList()) {
-        return PBHelperClient.convert(result.getDirList());
-      }
-      return null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void renewLease(String clientName) throws AccessControlException,
-      IOException {
-    RenewLeaseRequestProto req = RenewLeaseRequestProto.newBuilder()
-        .setClientName(clientName).build();
-    try {
-      rpcProxy.renewLease(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public boolean recoverLease(String src, String clientName)
-      throws IOException {
-    RecoverLeaseRequestProto req = RecoverLeaseRequestProto.newBuilder()
-        .setSrc(src)
-        .setClientName(clientName).build();
-    try {
-      return rpcProxy.recoverLease(null, req).getResult();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }  
-  }
-
-  @Override
-  public long[] getStats() throws IOException {
-    try {
-      return PBHelperClient.convert(rpcProxy.getFsStats(null,
-          VOID_GET_FSSTATUS_REQUEST));
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
-      throws IOException {
-    GetDatanodeReportRequestProto req = GetDatanodeReportRequestProto
-        .newBuilder()
-        .setType(PBHelperClient.convert(type)).build();
-    try {
-      return PBHelperClient.convert(
-          rpcProxy.getDatanodeReport(null, req).getDiList());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public DatanodeStorageReport[] getDatanodeStorageReport(DatanodeReportType type)
-      throws IOException {
-    final GetDatanodeStorageReportRequestProto req
-        = GetDatanodeStorageReportRequestProto.newBuilder()
-            .setType(PBHelperClient.convert(type)).build();
-    try {
-      return PBHelperClient.convertDatanodeStorageReports(
-          rpcProxy.getDatanodeStorageReport(null, req).getDatanodeStorageReportsList());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public long getPreferredBlockSize(String filename) throws IOException,
-      UnresolvedLinkException {
-    GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto
-        .newBuilder()
-        .setFilename(filename)
-        .build();
-    try {
-      return rpcProxy.getPreferredBlockSize(null, req).getBsize();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException {
-    SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder()
-        .setAction(PBHelperClient.convert(action)).setChecked(isChecked).build();
-    try {
-      return rpcProxy.setSafeMode(null, req).getResult();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
-    try {
-      SaveNamespaceRequestProto req = SaveNamespaceRequestProto.newBuilder()
-          .setTimeWindow(timeWindow).setTxGap(txGap).build();
-      return rpcProxy.saveNamespace(null, req).getSaved();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-  
-  @Override
-  public long rollEdits() throws AccessControlException, IOException {
-    try {
-      RollEditsResponseProto resp = rpcProxy.rollEdits(null,
-          VOID_ROLLEDITS_REQUEST);
-      return resp.getNewSegmentTxId();
-    } catch (ServiceException se) {
-      throw ProtobufHelper.getRemoteException(se);
-    }
-  }
-
-  @Override
-  public boolean restoreFailedStorage(String arg) 
-      throws AccessControlException, IOException{
-    RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto
-        .newBuilder()
-        .setArg(arg).build();
-    try {
-      return rpcProxy.restoreFailedStorage(null, req).getResult();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void refreshNodes() throws IOException {
-    try {
-      rpcProxy.refreshNodes(null, VOID_REFRESH_NODES_REQUEST);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void finalizeUpgrade() throws IOException {
-    try {
-      rpcProxy.finalizeUpgrade(null, VOID_FINALIZE_UPGRADE_REQUEST);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
-    final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
-        .setAction(PBHelperClient.convert(action)).build();
-    try {
-      final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
-      if (proto.hasRollingUpgradeInfo()) {
-        return PBHelperClient.convert(proto.getRollingUpgradeInfo());
-      }
-      return null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
-      throws IOException {
-    ListCorruptFileBlocksRequestProto.Builder req = 
-        ListCorruptFileBlocksRequestProto.newBuilder().setPath(path);   
-    if (cookie != null) 
-      req.setCookie(cookie);
-    try {
-      return PBHelperClient.convert(
-          rpcProxy.listCorruptFileBlocks(null, req.build()).getCorrupt());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void metaSave(String filename) throws IOException {
-    MetaSaveRequestProto req = MetaSaveRequestProto.newBuilder()
-        .setFilename(filename).build();
-    try {
-      rpcProxy.metaSave(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-  }
-
-  @Override
-  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
-        .setSrc(src).build();
-    try {
-      GetFileInfoResponseProto res = rpcProxy.getFileInfo(null, req);
-      return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public HdfsFileStatus getFileLinkInfo(String src)
-      throws AccessControlException, UnresolvedLinkException, IOException {
-    GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
-        .setSrc(src).build();
-    try {
-      GetFileLinkInfoResponseProto result = rpcProxy.getFileLinkInfo(null, req);
-      return result.hasFs() ?  
-          PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) : null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public ContentSummary getContentSummary(String path)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    GetContentSummaryRequestProto req = GetContentSummaryRequestProto
-        .newBuilder()
-        .setPath(path)
-        .build();
-    try {
-      return PBHelperClient.convert(rpcProxy.getContentSummary(null, req)
-          .getSummary());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
-                       StorageType type)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    final SetQuotaRequestProto.Builder builder
-        = SetQuotaRequestProto.newBuilder()
-        .setPath(path)
-        .setNamespaceQuota(namespaceQuota)
-        .setStoragespaceQuota(storagespaceQuota);
-    if (type != null) {
-      builder.setStorageType(PBHelperClient.convertStorageType(type));
-    }
-    final SetQuotaRequestProto req = builder.build();
-    try {
-      rpcProxy.setQuota(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void fsync(String src, long fileId, String client,
-                    long lastBlockLength)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src)
-        .setClient(client).setLastBlockLength(lastBlockLength)
-            .setFileId(fileId).build();
-    try {
-      rpcProxy.fsync(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void setTimes(String src, long mtime, long atime)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
-    SetTimesRequestProto req = SetTimesRequestProto.newBuilder()
-        .setSrc(src)
-        .setMtime(mtime)
-        .setAtime(atime)
-        .build();
-    try {
-      rpcProxy.setTimes(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void createSymlink(String target, String link, FsPermission dirPerm,
-      boolean createParent) throws AccessControlException,
-      FileAlreadyExistsException, FileNotFoundException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
-    CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder()
-        .setTarget(target)
-        .setLink(link)
-        .setDirPerm(PBHelperClient.convert(dirPerm))
-        .setCreateParent(createParent)
-        .build();
-    try {
-      rpcProxy.createSymlink(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public String getLinkTarget(String path) throws AccessControlException,
-      FileNotFoundException, IOException {
-    GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder()
-        .setPath(path).build();
-    try {
-      GetLinkTargetResponseProto rsp = rpcProxy.getLinkTarget(null, req);
-      return rsp.hasTargetPath() ? rsp.getTargetPath() : null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
-      String clientName) throws IOException {
-    UpdateBlockForPipelineRequestProto req = UpdateBlockForPipelineRequestProto
-        .newBuilder()
-        .setBlock(PBHelperClient.convert(block))
-        .setClientName(clientName)
-        .build();
-    try {
-      return PBHelperClient.convert(
-          rpcProxy.updateBlockForPipeline(null, req).getBlock());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void updatePipeline(String clientName, ExtendedBlock oldBlock,
-      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
-    UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
-        .setClientName(clientName)
-        .setOldBlock(PBHelperClient.convert(oldBlock))
-        .setNewBlock(PBHelperClient.convert(newBlock))
-        .addAllNewNodes(Arrays.asList(PBHelperClient.convert(newNodes)))
-        .addAllStorageIDs(storageIDs == null ? null : Arrays.asList(storageIDs))
-        .build();
-    try {
-      rpcProxy.updatePipeline(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
-      throws IOException {
-    GetDelegationTokenRequestProto req = GetDelegationTokenRequestProto
-        .newBuilder()
-        .setRenewer(renewer == null ? "" : renewer.toString())
-        .build();
-    try {
-      GetDelegationTokenResponseProto resp = rpcProxy.getDelegationToken(null, req);
-      return resp.hasToken() ? PBHelperClient.convertDelegationToken(resp.getToken())
-          : null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException {
-    RenewDelegationTokenRequestProto req = RenewDelegationTokenRequestProto.newBuilder().
-        setToken(PBHelperClient.convert(token)).
-        build();
-    try {
-      return rpcProxy.renewDelegationToken(null, req).getNewExpiryTime();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException {
-    CancelDelegationTokenRequestProto req = CancelDelegationTokenRequestProto
-        .newBuilder()
-        .setToken(PBHelperClient.convert(token))
-        .build();
-    try {
-      rpcProxy.cancelDelegationToken(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void setBalancerBandwidth(long bandwidth) throws IOException {
-    SetBalancerBandwidthRequestProto req = SetBalancerBandwidthRequestProto.newBuilder()
-        .setBandwidth(bandwidth)
-        .build();
-    try {
-      rpcProxy.setBalancerBandwidth(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public boolean isMethodSupported(String methodName) throws IOException {
-    return RpcClientUtil.isMethodSupported(rpcProxy,
-        ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
-        RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
-  }
-  
-  @Override
-  public DataEncryptionKey getDataEncryptionKey() throws IOException {
-    try {
-      GetDataEncryptionKeyResponseProto rsp = rpcProxy.getDataEncryptionKey(
-          null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST);
-     return rsp.hasDataEncryptionKey() ? 
-          PBHelperClient.convert(rsp.getDataEncryptionKey()) : null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-  
-
-  @Override
-  public boolean isFileClosed(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
-    IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
-        .setSrc(src).build();
-    try {
-      return rpcProxy.isFileClosed(null, req).getResult();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public Object getUnderlyingProxyObject() {
-    return rpcProxy;
-  }
-
-  @Override
-  public String createSnapshot(String snapshotRoot, String snapshotName)
-      throws IOException {
-    final CreateSnapshotRequestProto.Builder builder
-        = CreateSnapshotRequestProto.newBuilder().setSnapshotRoot(snapshotRoot);
-    if (snapshotName != null) {
-      builder.setSnapshotName(snapshotName);
-    }
-    final CreateSnapshotRequestProto req = builder.build();
-    try {
-      return rpcProxy.createSnapshot(null, req).getSnapshotPath();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-  
-  @Override
-  public void deleteSnapshot(String snapshotRoot, String snapshotName)
-      throws IOException {
-    DeleteSnapshotRequestProto req = DeleteSnapshotRequestProto.newBuilder()
-        .setSnapshotRoot(snapshotRoot).setSnapshotName(snapshotName).build();
-    try {
-      rpcProxy.deleteSnapshot(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-  
-  @Override
-  public void allowSnapshot(String snapshotRoot) throws IOException {
-    AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder()
-        .setSnapshotRoot(snapshotRoot).build();
-    try {
-      rpcProxy.allowSnapshot(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void disallowSnapshot(String snapshotRoot) throws IOException {
-    DisallowSnapshotRequestProto req = DisallowSnapshotRequestProto
-        .newBuilder().setSnapshotRoot(snapshotRoot).build();
-    try {
-      rpcProxy.disallowSnapshot(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
-      String snapshotNewName) throws IOException {
-    RenameSnapshotRequestProto req = RenameSnapshotRequestProto.newBuilder()
-        .setSnapshotRoot(snapshotRoot).setSnapshotOldName(snapshotOldName)
-        .setSnapshotNewName(snapshotNewName).build();
-    try {
-      rpcProxy.renameSnapshot(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
-      throws IOException {
-    GetSnapshottableDirListingRequestProto req = 
-        GetSnapshottableDirListingRequestProto.newBuilder().build();
-    try {
-      GetSnapshottableDirListingResponseProto result = rpcProxy
-          .getSnapshottableDirListing(null, req);
-      
-      if (result.hasSnapshottableDirList()) {
-        return PBHelperClient.convert(result.getSnapshottableDirList());
-      }
-      return null;
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
-      String fromSnapshot, String toSnapshot) throws IOException {
-    GetSnapshotDiffReportRequestProto req = GetSnapshotDiffReportRequestProto
-        .newBuilder().setSnapshotRoot(snapshotRoot)
-        .setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build();
-    try {
-      GetSnapshotDiffReportResponseProto result = 
-          rpcProxy.getSnapshotDiffReport(null, req);
-    
-      return PBHelperClient.convert(result.getDiffReport());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public long addCacheDirective(CacheDirectiveInfo directive,
-      EnumSet<CacheFlag> flags) throws IOException {
-    try {
-      AddCacheDirectiveRequestProto.Builder builder =
-          AddCacheDirectiveRequestProto.newBuilder().
-              setInfo(PBHelperClient.convert(directive));
-      if (!flags.isEmpty()) {
-        builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags));
-      }
-      return rpcProxy.addCacheDirective(null, builder.build()).getId();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void modifyCacheDirective(CacheDirectiveInfo directive,
-      EnumSet<CacheFlag> flags) throws IOException {
-    try {
-      ModifyCacheDirectiveRequestProto.Builder builder =
-          ModifyCacheDirectiveRequestProto.newBuilder().
-              setInfo(PBHelperClient.convert(directive));
-      if (!flags.isEmpty()) {
-        builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags));
-      }
-      rpcProxy.modifyCacheDirective(null, builder.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void removeCacheDirective(long id)
-      throws IOException {
-    try {
-      rpcProxy.removeCacheDirective(null,
-          RemoveCacheDirectiveRequestProto.newBuilder().
-              setId(id).build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  private static class BatchedCacheEntries
-      implements BatchedEntries<CacheDirectiveEntry> {
-    private final ListCacheDirectivesResponseProto response;
-
-    BatchedCacheEntries(
-        ListCacheDirectivesResponseProto response) {
-      this.response = response;
-    }
-
-    @Override
-    public CacheDirectiveEntry get(int i) {
-      return PBHelperClient.convert(response.getElements(i));
-    }
-
-    @Override
-    public int size() {
-      return response.getElementsCount();
-    }
-    
-    @Override
-    public boolean hasMore() {
-      return response.getHasMore();
-    }
-  }
-
-  @Override
-  public BatchedEntries<CacheDirectiveEntry>
-      listCacheDirectives(long prevId,
-          CacheDirectiveInfo filter) throws IOException {
-    if (filter == null) {
-      filter = new CacheDirectiveInfo.Builder().build();
-    }
-    try {
-      return new BatchedCacheEntries(
-        rpcProxy.listCacheDirectives(null,
-          ListCacheDirectivesRequestProto.newBuilder().
-            setPrevId(prevId).
-            setFilter(PBHelperClient.convert(filter)).
-            build()));
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void addCachePool(CachePoolInfo info) throws IOException {
-    AddCachePoolRequestProto.Builder builder = 
-        AddCachePoolRequestProto.newBuilder();
-    builder.setInfo(PBHelperClient.convert(info));
-    try {
-      rpcProxy.addCachePool(null, builder.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void modifyCachePool(CachePoolInfo req) throws IOException {
-    ModifyCachePoolRequestProto.Builder builder = 
-        ModifyCachePoolRequestProto.newBuilder();
-    builder.setInfo(PBHelperClient.convert(req));
-    try {
-      rpcProxy.modifyCachePool(null, builder.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void removeCachePool(String cachePoolName) throws IOException {
-    try {
-      rpcProxy.removeCachePool(null, 
-          RemoveCachePoolRequestProto.newBuilder().
-            setPoolName(cachePoolName).build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  private static class BatchedCachePoolEntries
-    implements BatchedEntries<CachePoolEntry> {
-      private final ListCachePoolsResponseProto proto;
-    
-    public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) {
-      this.proto = proto;
-    }
-      
-    @Override
-    public CachePoolEntry get(int i) {
-      CachePoolEntryProto elem = proto.getEntries(i);
-      return PBHelperClient.convert(elem);
-    }
-
-    @Override
-    public int size() {
-      return proto.getEntriesCount();
-    }
-    
-    @Override
-    public boolean hasMore() {
-      return proto.getHasMore();
-    }
-  }
-
-  @Override
-  public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
-      throws IOException {
-    try {
-      return new BatchedCachePoolEntries(
-        rpcProxy.listCachePools(null,
-          ListCachePoolsRequestProto.newBuilder().
-            setPrevPoolName(prevKey).build()));
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void modifyAclEntries(String src, List<AclEntry> aclSpec)
-      throws IOException {
-    ModifyAclEntriesRequestProto req = ModifyAclEntriesRequestProto
-        .newBuilder().setSrc(src)
-        .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build();
-    try {
-      rpcProxy.modifyAclEntries(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void removeAclEntries(String src, List<AclEntry> aclSpec)
-      throws IOException {
-    RemoveAclEntriesRequestProto req = RemoveAclEntriesRequestProto
-        .newBuilder().setSrc(src)
-        .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build();
-    try {
-      rpcProxy.removeAclEntries(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void removeDefaultAcl(String src) throws IOException {
-    RemoveDefaultAclRequestProto req = RemoveDefaultAclRequestProto
-        .newBuilder().setSrc(src).build();
-    try {
-      rpcProxy.removeDefaultAcl(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void removeAcl(String src) throws IOException {
-    RemoveAclRequestProto req = RemoveAclRequestProto.newBuilder()
-        .setSrc(src).build();
-    try {
-      rpcProxy.removeAcl(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
-    SetAclRequestProto req = SetAclRequestProto.newBuilder()
-        .setSrc(src)
-        .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec))
-        .build();
-    try {
-      rpcProxy.setAcl(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public AclStatus getAclStatus(String src) throws IOException {
-    GetAclStatusRequestProto req = GetAclStatusRequestProto.newBuilder()
-        .setSrc(src).build();
-    try {
-      return PBHelperClient.convert(rpcProxy.getAclStatus(null, req));
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void createEncryptionZone(String src, String keyName)
-    throws IOException {
-    final CreateEncryptionZoneRequestProto.Builder builder =
-      CreateEncryptionZoneRequestProto.newBuilder();
-    builder.setSrc(src);
-    if (keyName != null && !keyName.isEmpty()) {
-      builder.setKeyName(keyName);
-    }
-    CreateEncryptionZoneRequestProto req = builder.build();
-    try {
-      rpcProxy.createEncryptionZone(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public EncryptionZone getEZForPath(String src)
-      throws IOException {
-    final GetEZForPathRequestProto.Builder builder =
-        GetEZForPathRequestProto.newBuilder();
-    builder.setSrc(src);
-    final GetEZForPathRequestProto req = builder.build();
-    try {
-      final EncryptionZonesProtos.GetEZForPathResponseProto response =
-          rpcProxy.getEZForPath(null, req);
-      if (response.hasZone()) {
-        return PBHelperClient.convert(response.getZone());
-      } else {
-        return null;
-      }
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public BatchedEntries<EncryptionZone> listEncryptionZones(long id)
-      throws IOException {
-    final ListEncryptionZonesRequestProto req =
-      ListEncryptionZonesRequestProto.newBuilder()
-          .setId(id)
-          .build();
-    try {
-      EncryptionZonesProtos.ListEncryptionZonesResponseProto response =
-          rpcProxy.listEncryptionZones(null, req);
-      List<EncryptionZone> elements =
-          Lists.newArrayListWithCapacity(response.getZonesCount());
-      for (EncryptionZoneProto p : response.getZonesList()) {
-        elements.add(PBHelperClient.convert(p));
-      }
-      return new BatchedListEntries<EncryptionZone>(elements,
-          response.getHasMore());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
-      throws IOException {
-    SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder()
-        .setSrc(src)
-        .setXAttr(PBHelperClient.convertXAttrProto(xAttr))
-        .setFlag(PBHelperClient.convert(flag))
-        .build();
-    try {
-      rpcProxy.setXAttr(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-  
-  @Override
-  public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
-      throws IOException {
-    GetXAttrsRequestProto.Builder builder = GetXAttrsRequestProto.newBuilder();
-    builder.setSrc(src);
-    if (xAttrs != null) {
-      builder.addAllXAttrs(PBHelperClient.convertXAttrProto(xAttrs));
-    }
-    GetXAttrsRequestProto req = builder.build();
-    try {
-      return PBHelperClient.convert(rpcProxy.getXAttrs(null, req));
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-  
-  @Override
-  public List<XAttr> listXAttrs(String src)
-      throws IOException {
-    ListXAttrsRequestProto.Builder builder = ListXAttrsRequestProto.newBuilder();
-    builder.setSrc(src);
-    ListXAttrsRequestProto req = builder.build();
-    try {
-      return PBHelperClient.convert(rpcProxy.listXAttrs(null, req));
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void removeXAttr(String src, XAttr xAttr) throws IOException {
-    RemoveXAttrRequestProto req = RemoveXAttrRequestProto
-        .newBuilder().setSrc(src)
-        .setXAttr(PBHelperClient.convertXAttrProto(xAttr)).build();
-    try {
-      rpcProxy.removeXAttr(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void checkAccess(String path, FsAction mode) throws IOException {
-    CheckAccessRequestProto req = CheckAccessRequestProto.newBuilder()
-        .setPath(path).setMode(PBHelperClient.convert(mode)).build();
-    try {
-      rpcProxy.checkAccess(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void setStoragePolicy(String src, String policyName)
-      throws IOException {
-    SetStoragePolicyRequestProto req = SetStoragePolicyRequestProto
-        .newBuilder().setSrc(src).setPolicyName(policyName).build();
-    try {
-      rpcProxy.setStoragePolicy(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
-    GetStoragePolicyRequestProto request = GetStoragePolicyRequestProto
-        .newBuilder().setPath(path).build();
-    try {
-      return PBHelperClient.convert(rpcProxy.getStoragePolicy(null, request)
-          .getStoragePolicy());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
-    try {
-      GetStoragePoliciesResponseProto response = rpcProxy
-          .getStoragePolicies(null, VOID_GET_STORAGE_POLICIES_REQUEST);
-      return PBHelperClient.convertStoragePolicies(response.getPoliciesList());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  public long getCurrentEditLogTxid() throws IOException {
-    GetCurrentEditLogTxidRequestProto req = GetCurrentEditLogTxidRequestProto
-        .getDefaultInstance();
-    try {
-      return rpcProxy.getCurrentEditLogTxid(null, req).getTxid();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public EventBatchList getEditsFromTxid(long txid) throws IOException {
-    GetEditsFromTxidRequestProto req = GetEditsFromTxidRequestProto.newBuilder()
-        .setTxid(txid).build();
-    try {
-      return PBHelperClient.convert(rpcProxy.getEditsFromTxid(null, req));
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 3de4513..7c08f71 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -348,7 +348,7 @@ public class PBHelper {
         new RecoveringBlock(block, locs, PBHelperClient.convert(b.getTruncateBlock())) :
         new RecoveringBlock(block, locs, b.getNewGenStamp());
   }
-  
+
   public static ReplicaState convert(ReplicaStateProto state) {
     switch (state) {
     case RBW:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
deleted file mode 100644
index a0aa10b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
+++ /dev/null
@@ -1,51 +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.hdfs.server.namenode.ha;
-
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.retry.FailoverProxyProvider;
-
-public abstract class AbstractNNFailoverProxyProvider<T> implements
-   FailoverProxyProvider <T> {
-
-  protected AtomicBoolean fallbackToSimpleAuth;
-
-  /**
-   * Inquire whether logical HA URI is used for the implementation. If it is
-   * used, a special token handling may be needed to make sure a token acquired 
-   * from a node in the HA pair can be used against the other node. 
-   *
-   * @return true if logical HA URI is used. false, if not used.
-   */
-  public abstract boolean useLogicalURI(); 
-
-  /**
-   * Set for tracking if a secure client falls back to simple auth.  This method
-   * is synchronized only to stifle a Findbugs warning.
-   *
-   * @param fallbackToSimpleAuth - set to true or false during this method to
-   *   indicate if a secure client falls back to simple auth
-   */
-  public synchronized void setFallbackToSimpleAuth(
-      AtomicBoolean fallbackToSimpleAuth) {
-    this.fallbackToSimpleAuth = fallbackToSimpleAuth;
-  }
-}


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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java
new file mode 100644
index 0000000..ea109fd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+/**
+ * This is ContainerResourceChangeRequest in scheduler side, it contains some
+ * pointers to runtime objects like RMContainer, SchedulerNode, etc. This will
+ * be easier for scheduler making decision.
+ */
+public class SchedContainerChangeRequest implements
+    Comparable<SchedContainerChangeRequest> {
+  RMContainer rmContainer;
+  Resource targetCapacity;
+  SchedulerNode schedulerNode;
+  Resource deltaCapacity;
+
+  public SchedContainerChangeRequest(SchedulerNode schedulerNode,
+      RMContainer rmContainer, Resource targetCapacity) {
+    this.rmContainer = rmContainer;
+    this.targetCapacity = targetCapacity;
+    this.schedulerNode = schedulerNode;
+    deltaCapacity = Resources.subtract(targetCapacity,
+        rmContainer.getAllocatedResource());
+  }
+  
+  public NodeId getNodeId() {
+    return this.rmContainer.getAllocatedNode();
+  }
+
+  public RMContainer getRMContainer() {
+    return this.rmContainer;
+  }
+
+  public Resource getTargetCapacity() {
+    return this.targetCapacity;
+  }
+
+  /**
+   * Delta capacity = before - target, so if it is a decrease request, delta
+   * capacity will be negative
+   */
+  public Resource getDeltaCapacity() {
+    return deltaCapacity;
+  }
+  
+  public Priority getPriority() {
+    return rmContainer.getContainer().getPriority();
+  }
+  
+  public ContainerId getContainerId() {
+    return rmContainer.getContainerId();
+  }
+  
+  public String getNodePartition() {
+    return schedulerNode.getPartition();
+  }
+  
+  public SchedulerNode getSchedulerNode() {
+    return schedulerNode;
+  }
+  
+  @Override
+  public int hashCode() {
+    return (getContainerId().hashCode() << 16) + targetCapacity.hashCode();
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof SchedContainerChangeRequest)) {
+      return false;
+    }
+    return compareTo((SchedContainerChangeRequest)other) == 0;
+  }
+
+  @Override
+  public int compareTo(SchedContainerChangeRequest other) {
+    if (other == null) {
+      return -1;
+    }
+    
+    int rc = getPriority().compareTo(other.getPriority());
+    if (0 != rc) {
+      return rc;
+    }
+    
+    return getContainerId().compareTo(other.getContainerId());
+  }
+  
+  @Override
+  public String toString() {
+    return "<container=" + getContainerId() + ", targetCapacity="
+        + targetCapacity + ", delta=" + deltaCapacity + ", node="
+        + getNodeId().toString() + ">";
+  }
+}

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index f03663a..f3d3906 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -157,6 +157,37 @@ public abstract class SchedulerNode {
         + getUsedResource() + " used and " + getAvailableResource()
         + " available after allocation");
   }
+  
+  private synchronized void changeContainerResource(ContainerId containerId,
+      Resource deltaResource, boolean increase) {
+    if (increase) {
+      deductAvailableResource(deltaResource);
+    } else {
+      addAvailableResource(deltaResource);
+    }
+
+    LOG.info((increase ? "Increased" : "Decreased") + " container "
+        + containerId + " of capacity " + deltaResource + " on host "
+        + rmNode.getNodeAddress() + ", which has " + numContainers
+        + " containers, " + getUsedResource() + " used and "
+        + getAvailableResource() + " available after allocation");
+  }
+  
+  /**
+   * The Scheduler increased container
+   */
+  public synchronized void increaseContainer(ContainerId containerId,
+      Resource deltaResource) {
+    changeContainerResource(containerId, deltaResource, true);
+  }
+  
+  /**
+   * The Scheduler decreased container
+   */
+  public synchronized void decreaseContainer(ContainerId containerId,
+      Resource deltaResource) {
+    changeContainerResource(containerId, deltaResource, false);
+  }
 
   /**
    * Get available resources on the node.

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java
index 928437f..68f6f12 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSAssignment.java
@@ -41,6 +41,7 @@ public class CSAssignment {
   private final boolean skipped;
   private boolean fulfilledReservation;
   private final AssignmentInformation assignmentInformation;
+  private boolean increaseAllocation;
 
   public CSAssignment(Resource resource, NodeType type) {
     this(resource, type, null, null, false, false);
@@ -138,4 +139,12 @@ public class CSAssignment {
   public AssignmentInformation getAssignmentInformation() {
     return this.assignmentInformation;
   }
+  
+  public boolean isIncreasedAllocation() {
+    return increaseAllocation;
+  }
+
+  public void setIncreasedAllocation(boolean flag) {
+    increaseAllocation = flag;
+  }
 }
\ No newline at end of file

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

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


[08/21] hadoop git commit: YARN-1449. AM-NM protocol changes to support container resizing. Contributed by Meng Ding & Wangda Tan)

Posted by wa...@apache.org.
YARN-1449. AM-NM protocol changes to support container resizing. Contributed by Meng Ding & Wangda Tan)


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

Branch: refs/heads/YARN-1197
Commit: 83a18add10ee937a04e833a66e0a4642e776e510
Parents: dfe2cb8
Author: Jian He <ji...@apache.org>
Authored: Tue Jul 14 16:06:25 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Sep 23 13:29:36 2015 -0700

----------------------------------------------------------------------
 .../v2/app/launcher/TestContainerLauncher.java  |  11 +
 .../app/launcher/TestContainerLauncherImpl.java |   9 +
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../yarn/api/ContainerManagementProtocol.java   |  30 ++-
 .../IncreaseContainersResourceRequest.java      |  75 ++++++
 .../IncreaseContainersResourceResponse.java     |  93 +++++++
 .../proto/containermanagement_protocol.proto    |   1 +
 ...ContainerManagementProtocolPBClientImpl.java |  20 ++
 ...ontainerManagementProtocolPBServiceImpl.java |  22 ++
 ...IncreaseContainersResourceRequestPBImpl.java | 170 +++++++++++++
 ...ncreaseContainersResourceResponsePBImpl.java | 241 +++++++++++++++++++
 .../hadoop/yarn/TestContainerLaunchRPC.java     |   8 +
 .../yarn/TestContainerResourceIncreaseRPC.java  | 162 +++++++++++++
 .../java/org/apache/hadoop/yarn/TestRPC.java    |   8 +
 .../hadoop/yarn/api/TestPBImplRecords.java      |  20 ++
 .../containermanager/ContainerManagerImpl.java  |  13 +
 .../server/resourcemanager/NodeManager.java     |  11 +-
 .../resourcemanager/TestAMAuthorization.java    |   8 +
 .../TestApplicationMasterLauncher.java          |   9 +
 19 files changed, 910 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
index 41ee65d..6c3a4d6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
@@ -30,6 +30,8 @@ import java.util.Map;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.junit.Assert;
 
 import org.apache.commons.logging.Log;
@@ -449,5 +451,14 @@ public class TestContainerLauncher {
           "Dummy function cause"));
       throw new IOException(e);
     }
+
+    @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(
+        IncreaseContainersResourceRequest request) throws IOException,
+        IOException {
+      Exception e = new Exception("Dummy function", new Exception(
+          "Dummy function cause"));
+      throw new IOException(e);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
index 184f1b2..610448c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
@@ -46,6 +46,8 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher.EventType;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
@@ -454,6 +456,13 @@ public class TestContainerLauncherImpl {
     }
 
     @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(
+        IncreaseContainersResourceRequest request) throws YarnException,
+        IOException {
+      return null;
+    }
+
+    @Override
     public void close() throws IOException {
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 7e4edce..bf6d9c4 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -203,6 +203,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3866. AM-RM protocol changes to support container resizing. (Meng Ding
     via jianhe)
 
+    YARN-1449. AM-NM protocol changes to support container resizing.
+    (Meng Ding & Wangda Tan via jianhe)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
index 7aa43df..43e1d4c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
@@ -22,6 +22,9 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@@ -38,9 +41,9 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 
 /**
  * <p>The protocol between an <code>ApplicationMaster</code> and a 
- * <code>NodeManager</code> to start/stop containers and to get status
- * of running containers.</p>
- * 
+ * <code>NodeManager</code> to start/stop and increase resource of containers
+ * and to get status of running containers.</p>
+ *
  * <p>If security is enabled the <code>NodeManager</code> verifies that the
  * <code>ApplicationMaster</code> has truly been allocated the container
  * by the <code>ResourceManager</code> and also verifies all interactions such 
@@ -170,4 +173,25 @@ public interface ContainerManagementProtocol {
   GetContainerStatusesResponse getContainerStatuses(
       GetContainerStatusesRequest request) throws YarnException,
       IOException;
+
+  /**
+   * <p>
+   * The API used by the <code>ApplicationMaster</code> to request for
+   * resource increase of running containers on the <code>NodeManager</code>.
+   * </p>
+   *
+   * @param request
+   *         request to increase resource of a list of containers
+   * @return response which includes a list of containerIds of containers
+   *         whose resource has been successfully increased and a
+   *         containerId-to-exception map for failed requests.
+   *
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Public
+  @Unstable
+  IncreaseContainersResourceResponse increaseContainersResource(
+      IncreaseContainersResourceRequest request) throws YarnException,
+      IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceRequest.java
new file mode 100644
index 0000000..1fe8e94
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceRequest.java
@@ -0,0 +1,75 @@
+/**
+ * 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.protocolrecords;
+
+import java.util.List;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>The request sent by <code>Application Master</code> to the
+ * <code>Node Manager</code> to change the resource quota of a container.</p>
+ *
+ * @see ContainerManagementProtocol#increaseContainersResource(IncreaseContainersResourceRequest)
+ */
+@Public
+@Unstable
+public abstract class IncreaseContainersResourceRequest {
+  @Public
+  @Unstable
+  public static IncreaseContainersResourceRequest newInstance(
+      List<Token> containersToIncrease) {
+    IncreaseContainersResourceRequest request =
+        Records.newRecord(IncreaseContainersResourceRequest.class);
+    request.setContainersToIncrease(containersToIncrease);
+    return request;
+  }
+
+  /**
+   * Get a list of container tokens to be used for authorization during
+   * container resource increase.
+   * <p>
+   * Note: {@link NMToken} will be used for authenticating communication with
+   * {@code NodeManager}.
+   * @return the list of container tokens to be used for authorization during
+   * container resource increase.
+   * @see NMToken
+   */
+  @Public
+  @Unstable
+  public abstract List<Token> getContainersToIncrease();
+
+  /**
+   * Set container tokens to be used during container resource increase.
+   * The token is acquired from
+   * <code>AllocateResponse.getIncreasedContainers</code>.
+   * The token contains the container id and resource capability required for
+   * container resource increase.
+   * @param containersToIncrease the list of container tokens to be used
+   *                             for container resource increase.
+   */
+  @Public
+  @Unstable
+  public abstract void setContainersToIncrease(
+      List<Token> containersToIncrease);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceResponse.java
new file mode 100644
index 0000000..aeb1e83
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/IncreaseContainersResourceResponse.java
@@ -0,0 +1,93 @@
+/**
+ * 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.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * <p>
+ * The response sent by the <code>NodeManager</code> to the
+ * <code>ApplicationMaster</code> when asked to increase container resource.
+ * </p>
+ *
+ * @see ContainerManagementProtocol#increaseContainersResource(IncreaseContainersResourceRequest)
+ */
+@Public
+@Unstable
+public abstract class IncreaseContainersResourceResponse {
+
+  @Private
+  @Unstable
+  public static IncreaseContainersResourceResponse newInstance(
+      List<ContainerId> successfullyIncreasedContainers,
+      Map<ContainerId, SerializedException> failedRequests) {
+    IncreaseContainersResourceResponse response =
+        Records.newRecord(IncreaseContainersResourceResponse.class);
+    response.setSuccessfullyIncreasedContainers(
+        successfullyIncreasedContainers);
+    response.setFailedRequests(failedRequests);
+    return response;
+  }
+
+  /**
+   * Get the list of containerIds of containers whose resource
+   * have been successfully increased.
+   *
+   * @return the list of containerIds of containers whose resource have
+   * been successfully increased.
+   */
+  @Public
+  @Unstable
+  public abstract List<ContainerId> getSuccessfullyIncreasedContainers();
+
+  /**
+   * Set the list of containerIds of containers whose resource have
+   * been successfully increased.
+   */
+  @Private
+  @Unstable
+  public abstract void setSuccessfullyIncreasedContainers(
+      List<ContainerId> succeedIncreasedContainers);
+
+  /**
+   * Get the containerId-to-exception map in which the exception indicates
+   * error from each container for failed requests.
+   */
+  @Public
+  @Unstable
+  public abstract Map<ContainerId, SerializedException> getFailedRequests();
+
+  /**
+   * Set the containerId-to-exception map in which the exception indicates
+   * error from each container for failed requests.
+   */
+  @Private
+  @Unstable
+  public abstract void setFailedRequests(
+      Map<ContainerId, SerializedException> failedRequests);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
index 7b1647b..f06f6cb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
@@ -34,4 +34,5 @@ service ContainerManagementProtocolService {
   rpc startContainers(StartContainersRequestProto) returns (StartContainersResponseProto);
   rpc stopContainers(StopContainersRequestProto) returns (StopContainersResponseProto);
   rpc getContainerStatuses(GetContainerStatusesRequestProto) returns (GetContainerStatusesResponseProto);
+  rpc increaseContainersResource(IncreaseContainersResourceRequestProto) returns (IncreaseContainersResourceResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
index 15397e3..ce18bde 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
@@ -30,12 +30,16 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl;
@@ -48,6 +52,7 @@ import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto;
 
 import com.google.protobuf.ServiceException;
 
@@ -128,4 +133,19 @@ public class ContainerManagementProtocolPBClientImpl implements ContainerManagem
       return null;
     }
   }
+
+  @Override
+  public IncreaseContainersResourceResponse increaseContainersResource(
+      IncreaseContainersResourceRequest request) throws YarnException,
+      IOException {
+    IncreaseContainersResourceRequestProto requestProto =
+        ((IncreaseContainersResourceRequestPBImpl)request).getProto();
+    try {
+      return new IncreaseContainersResourceResponsePBImpl(
+          proxy.increaseContainersResource(null, requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
index 2d33e69..7626441 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
@@ -23,9 +23,12 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl;
@@ -33,6 +36,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRespons
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
@@ -94,4 +99,21 @@ public class ContainerManagementProtocolPBServiceImpl implements ContainerManage
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public IncreaseContainersResourceResponseProto increaseContainersResource(
+      RpcController controller, IncreaseContainersResourceRequestProto proto)
+      throws ServiceException {
+    IncreaseContainersResourceRequestPBImpl request =
+        new IncreaseContainersResourceRequestPBImpl(proto);
+    try {
+      IncreaseContainersResourceResponse response =
+          real.increaseContainersResource(request);
+      return ((IncreaseContainersResourceResponsePBImpl)response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceRequestPBImpl.java
new file mode 100644
index 0000000..7417051
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceRequestPBImpl.java
@@ -0,0 +1,170 @@
+/**
+ * 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.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+@Private
+@Unstable
+public class IncreaseContainersResourceRequestPBImpl extends
+    IncreaseContainersResourceRequest {
+  IncreaseContainersResourceRequestProto proto =
+      IncreaseContainersResourceRequestProto.getDefaultInstance();
+  IncreaseContainersResourceRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private List<Token> containersToIncrease = null;
+
+  public IncreaseContainersResourceRequestPBImpl() {
+    builder = IncreaseContainersResourceRequestProto.newBuilder();
+  }
+
+  public IncreaseContainersResourceRequestPBImpl(
+      IncreaseContainersResourceRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public IncreaseContainersResourceRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.containersToIncrease != null) {
+      addIncreaseContainersToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = IncreaseContainersResourceRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public List<Token> getContainersToIncrease() {
+    if (containersToIncrease != null) {
+      return containersToIncrease;
+    }
+    IncreaseContainersResourceRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<TokenProto> list = p.getIncreaseContainersList();
+    containersToIncrease = new ArrayList<>();
+    for (TokenProto c : list) {
+      containersToIncrease.add(convertFromProtoFormat(c));
+    }
+    return containersToIncrease;
+  }
+
+  @Override
+  public void setContainersToIncrease(List<Token> containersToIncrease) {
+    maybeInitBuilder();
+    if (containersToIncrease == null) {
+      builder.clearIncreaseContainers();
+    }
+    this.containersToIncrease = containersToIncrease;
+  }
+
+  private void addIncreaseContainersToProto() {
+    maybeInitBuilder();
+    builder.clearIncreaseContainers();
+    if (this.containersToIncrease == null) {
+      return;
+    }
+    Iterable<TokenProto> iterable = new Iterable<TokenProto>() {
+      @Override
+      public Iterator<TokenProto> iterator() {
+        return new Iterator<TokenProto>() {
+          Iterator<Token> iter = containersToIncrease.iterator();
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public TokenProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+    builder.addAllIncreaseContainers(iterable);
+  }
+
+  private Token convertFromProtoFormat(TokenProto p) {
+    return new TokenPBImpl(p);
+  }
+
+  private TokenProto convertToProtoFormat(Token t) {
+    return ((TokenPBImpl) t).getProto();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceResponsePBImpl.java
new file mode 100644
index 0000000..15062e1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/IncreaseContainersResourceResponsePBImpl.java
@@ -0,0 +1,241 @@
+/**
+ * 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.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerExceptionMapProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+@Private
+@Unstable
+public class IncreaseContainersResourceResponsePBImpl extends
+    IncreaseContainersResourceResponse {
+  IncreaseContainersResourceResponseProto proto =
+      IncreaseContainersResourceResponseProto.getDefaultInstance();
+  IncreaseContainersResourceResponseProto.Builder builder = null;
+  boolean viaProto = false;
+  private List<ContainerId> succeededRequests = null;
+  private Map<ContainerId, SerializedException> failedRequests = null;
+
+  public IncreaseContainersResourceResponsePBImpl() {
+    builder = IncreaseContainersResourceResponseProto.newBuilder();
+  }
+
+  public IncreaseContainersResourceResponsePBImpl(
+      IncreaseContainersResourceResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public IncreaseContainersResourceResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.succeededRequests != null) {
+      addSucceededRequestsToProto();
+    }
+    if (this.failedRequests != null) {
+      addFailedRequestsToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = IncreaseContainersResourceResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public List<ContainerId> getSuccessfullyIncreasedContainers() {
+    initSucceededRequests();
+    return this.succeededRequests;
+  }
+
+  @Override
+  public void setSuccessfullyIncreasedContainers(
+      List<ContainerId> succeededRequests) {
+    maybeInitBuilder();
+    if (succeededRequests == null) {
+      builder.clearSucceededRequests();
+    }
+    this.succeededRequests = succeededRequests;
+  }
+
+  private void initSucceededRequests() {
+    if (this.succeededRequests != null) {
+      return;
+    }
+    IncreaseContainersResourceResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<ContainerIdProto> list = p.getSucceededRequestsList();
+    this.succeededRequests = new ArrayList<ContainerId>();
+    for (ContainerIdProto c : list) {
+      this.succeededRequests.add(convertFromProtoFormat(c));
+    }
+  }
+
+  private void addSucceededRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearSucceededRequests();
+    if (this.succeededRequests == null) {
+      return;
+    }
+    Iterable<ContainerIdProto> iterable = new Iterable<ContainerIdProto>() {
+      @Override
+      public Iterator<ContainerIdProto> iterator() {
+        return new Iterator<ContainerIdProto>() {
+          Iterator<ContainerId> iter = succeededRequests.iterator();
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public ContainerIdProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+    builder.addAllSucceededRequests(iterable);
+  }
+
+  @Override
+  public Map<ContainerId, SerializedException> getFailedRequests() {
+    initFailedRequests();
+    return this.failedRequests;
+  }
+
+  @Override
+  public void setFailedRequests(
+      Map<ContainerId, SerializedException> failedRequests) {
+    maybeInitBuilder();
+    if (failedRequests == null) {
+      builder.clearFailedRequests();
+    }
+    this.failedRequests = failedRequests;
+  }
+
+  private void initFailedRequests() {
+    if (this.failedRequests != null) {
+      return;
+    }
+    IncreaseContainersResourceResponseProtoOrBuilder
+        p = viaProto ? proto : builder;
+    List<ContainerExceptionMapProto> protoList = p.getFailedRequestsList();
+    this.failedRequests = new HashMap<ContainerId, SerializedException>();
+    for (ContainerExceptionMapProto ce : protoList) {
+      this.failedRequests.put(convertFromProtoFormat(ce.getContainerId()),
+          convertFromProtoFormat(ce.getException()));
+    }
+  }
+
+  private void addFailedRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearFailedRequests();
+    if (this.failedRequests == null) {
+      return;
+    }
+    List<ContainerExceptionMapProto> protoList =
+        new ArrayList<ContainerExceptionMapProto>();
+
+    for (Map.Entry<ContainerId, SerializedException> entry : this.failedRequests
+        .entrySet()) {
+      protoList.add(ContainerExceptionMapProto.newBuilder()
+          .setContainerId(convertToProtoFormat(entry.getKey()))
+          .setException(convertToProtoFormat(entry.getValue())).build());
+    }
+    builder.addAllFailedRequests(protoList);
+  }
+
+  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
+    return new ContainerIdPBImpl(p);
+  }
+
+  private ContainerIdProto convertToProtoFormat(ContainerId t) {
+    return ((ContainerIdPBImpl) t).getProto();
+  }
+
+  private SerializedExceptionPBImpl convertFromProtoFormat(
+          SerializedExceptionProto p) {
+    return new SerializedExceptionPBImpl(p);
+  }
+
+  private SerializedExceptionProto convertToProtoFormat(SerializedException t) {
+    return ((SerializedExceptionPBImpl) t).getProto();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
index e2071dd..0a19783 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@@ -166,5 +168,11 @@ public class TestContainerLaunchRPC {
           GetContainerStatusesResponse.newInstance(list, null);
       return null;
     }
+
+    @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(
+        IncreaseContainersResourceRequest request) throws YarnException, IOException {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
new file mode 100644
index 0000000..50ff1e0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerResourceIncreaseRPC.java
@@ -0,0 +1,162 @@
+/**
+ * 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;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketTimeoutException;
+import java.util.ArrayList;
+import java.util.List;
+
+/*
+ * Test that the container resource increase rpc times out properly.
+ * This is used by AM to increase container resource.
+ */
+public class TestContainerResourceIncreaseRPC {
+
+  static final Log LOG = LogFactory.getLog(
+      TestContainerResourceIncreaseRPC.class);
+
+  @Test
+  public void testHadoopProtoRPCTimeout() throws Exception {
+    testRPCTimeout(HadoopYarnProtoRPC.class.getName());
+  }
+
+  private void testRPCTimeout(String rpcClass) throws Exception {
+    Configuration conf = new Configuration();
+    // set timeout low for the test
+    conf.setInt("yarn.rpc.nm-command-timeout", 3000);
+    conf.set(YarnConfiguration.IPC_RPC_IMPL, rpcClass);
+    YarnRPC rpc = YarnRPC.create(conf);
+    String bindAddr = "localhost:0";
+    InetSocketAddress addr = NetUtils.createSocketAddr(bindAddr);
+    Server server = rpc.getServer(ContainerManagementProtocol.class,
+        new DummyContainerManager(), addr, conf, null, 1);
+    server.start();
+    try {
+      ContainerManagementProtocol proxy =
+          (ContainerManagementProtocol) rpc.getProxy(
+              ContainerManagementProtocol.class,
+                  server.getListenerAddress(), conf);
+      ApplicationId applicationId = ApplicationId.newInstance(0, 0);
+      ApplicationAttemptId applicationAttemptId =
+          ApplicationAttemptId.newInstance(applicationId, 0);
+      ContainerId containerId =
+          ContainerId.newContainerId(applicationAttemptId, 100);
+      NodeId nodeId = NodeId.newInstance("localhost", 1234);
+      Resource resource = Resource.newInstance(1234, 2);
+      ContainerTokenIdentifier containerTokenIdentifier =
+          new ContainerTokenIdentifier(containerId, "localhost", "user",
+              resource, System.currentTimeMillis() + 10000, 42, 42,
+                  Priority.newInstance(0), 0);
+      Token containerToken =
+          TestRPC.newContainerToken(nodeId, "password".getBytes(),
+              containerTokenIdentifier);
+      // Construct container resource increase request,
+      List<Token> increaseTokens = new ArrayList<>();
+      increaseTokens.add(containerToken);
+      IncreaseContainersResourceRequest increaseRequest =
+          IncreaseContainersResourceRequest
+              .newInstance(increaseTokens);
+      try {
+        proxy.increaseContainersResource(increaseRequest);
+      } catch (Exception e) {
+        LOG.info(StringUtils.stringifyException(e));
+        Assert.assertEquals("Error, exception is not: "
+            + SocketTimeoutException.class.getName(),
+            SocketTimeoutException.class.getName(), e.getClass().getName());
+        return;
+      }
+    } finally {
+      server.stop();
+    }
+    Assert.fail("timeout exception should have occurred!");
+  }
+
+  public class DummyContainerManager implements ContainerManagementProtocol {
+
+    @Override
+    public StartContainersResponse startContainers(
+        StartContainersRequest requests) throws YarnException, IOException {
+      Exception e = new Exception("Dummy function", new Exception(
+          "Dummy function cause"));
+      throw new YarnException(e);
+    }
+
+    @Override
+    public StopContainersResponse
+    stopContainers(StopContainersRequest requests) throws YarnException,
+        IOException {
+      Exception e = new Exception("Dummy function", new Exception(
+          "Dummy function cause"));
+      throw new YarnException(e);
+    }
+
+    @Override
+    public GetContainerStatusesResponse getContainerStatuses(
+        GetContainerStatusesRequest request) throws YarnException, IOException {
+      Exception e = new Exception("Dummy function", new Exception(
+          "Dummy function cause"));
+      throw new YarnException(e);
+    }
+
+    @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(
+        IncreaseContainersResourceRequest request) throws YarnException, IOException {
+      try {
+        // make the thread sleep to look like its not going to respond
+        Thread.sleep(10000);
+      } catch (Exception e) {
+        LOG.error(e);
+        throw new YarnException(e);
+      }
+      throw new YarnException("Shouldn't happen!!");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
index 39e6162..e718661 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
@@ -33,6 +33,8 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
@@ -219,6 +221,12 @@ public class TestRPC {
           new Exception(EXCEPTION_CAUSE));
       throw new YarnException(e);
     }
+
+    @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(
+        IncreaseContainersResourceRequest request) throws YarnException, IOException {
+      return null;
+    }
   }
 
   public static ContainerTokenIdentifier newContainerTokenIdentifier(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/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 0979c75..5f707b5 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
@@ -44,6 +44,8 @@ import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenReque
 import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenResponseProto;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenRequestPBImpl;
@@ -101,6 +103,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequest
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.IncreaseContainersResourceResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -278,6 +282,8 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionReque
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersResponseProto;
@@ -466,6 +472,8 @@ public class TestPBImplRecords {
     generateByNewInstance(ApplicationSubmissionContext.class);
     generateByNewInstance(ContainerReport.class);
     generateByNewInstance(ContainerResourceChangeRequest.class);
+    generateByNewInstance(IncreaseContainersResourceRequest.class);
+    generateByNewInstance(IncreaseContainersResourceResponse.class);
     generateByNewInstance(ContainerStatus.class);
     generateByNewInstance(PreemptionContainer.class);
     generateByNewInstance(PreemptionResourceRequest.class);
@@ -871,6 +879,18 @@ public class TestPBImplRecords {
   }
 
   @Test
+  public void testIncreaseContainersResourceRequestPBImpl() throws Exception {
+    validatePBImplRecord(IncreaseContainersResourceRequestPBImpl.class,
+        IncreaseContainersResourceRequestProto.class);
+  }
+
+  @Test
+  public void testIncreaseContainersResourceResponsePBImpl() throws Exception {
+    validatePBImplRecord(IncreaseContainersResourceResponsePBImpl.class,
+        IncreaseContainersResourceResponseProto.class);
+  }
+
+  @Test
   public void testSubmitApplicationRequestPBImpl() throws Exception {
     validatePBImplRecord(SubmitApplicationRequestPBImpl.class,
         SubmitApplicationRequestProto.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index a658e53..ba1aec2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -58,6 +58,8 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -952,6 +954,17 @@ public class ContainerManagerImpl extends CompositeService implements
     return containerTokenIdentifier;
   }
 
+  /**
+   * Increase resource of a list of containers on this NodeManager.
+   */
+  @Override
+  public IncreaseContainersResourceResponse increaseContainersResource(
+      IncreaseContainersResourceRequest requests)
+      throws YarnException, IOException {
+    // To be implemented in YARN-1645
+    return null;
+  }
+
   @Private
   @VisibleForTesting
   protected void updateNMTokenIdentifier(NMTokenIdentifier nmTokenIdentifier)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.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/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
index d8d474e..5b7735e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
@@ -25,6 +25,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.junit.Assert;
 
 import org.apache.commons.logging.Log;
@@ -295,7 +297,14 @@ public class NodeManager implements ContainerManagementProtocol {
     return GetContainerStatusesResponse.newInstance(statuses, null);
   }
 
-  public static org.apache.hadoop.yarn.server.api.records.NodeStatus 
+  @Override
+  public IncreaseContainersResourceResponse increaseContainersResource(
+      IncreaseContainersResourceRequest request)
+          throws YarnException, IOException {
+    return null;
+  }
+
+  public static org.apache.hadoop.yarn.server.api.records.NodeStatus
   createNodeStatus(NodeId nodeId, List<ContainerStatus> containers) {
     RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
     org.apache.hadoop.yarn.server.api.records.NodeStatus nodeStatus = 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.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/TestAMAuthorization.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
index c7f0d0a..2787f1e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
@@ -40,6 +40,8 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
@@ -122,6 +124,12 @@ public class TestAMAuthorization {
       return GetContainerStatusesResponse.newInstance(null, null);
     }
 
+    @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(IncreaseContainersResourceRequest request)
+        throws YarnException {
+      return IncreaseContainersResourceResponse.newInstance(null, null);
+    }
+
     public Credentials getContainerCredentials() throws IOException {
       Credentials credentials = new Credentials();
       DataInputByteBuffer buf = new DataInputByteBuffer();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/83a18add/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.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/TestApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
index d4f8e93..2760705 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
@@ -32,6 +32,8 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@@ -126,6 +128,13 @@ public class TestApplicationMasterLauncher {
         GetContainerStatusesRequest request) throws YarnException {
       return null;
     }
+
+    @Override
+    public IncreaseContainersResourceResponse increaseContainersResource(
+        IncreaseContainersResourceRequest request)
+            throws YarnException {
+      return null;
+    }
   }
 
   @Test


[21/21] hadoop git commit: YARN-4171. Fix findbugs warnings in YARN-1197 branch. Contributed by Wangda Tan

Posted by wa...@apache.org.
YARN-4171. Fix findbugs warnings in YARN-1197 branch. Contributed by Wangda Tan


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

Branch: refs/heads/YARN-1197
Commit: b3f6b641dccb0d59df78855e2951d2cae7dff8ad
Parents: 89cab1b
Author: Jian He <ji...@apache.org>
Authored: Fri Sep 18 16:42:26 2015 +0800
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Sep 23 13:29:38 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 2 ++
 .../yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java     | 1 +
 .../resourcemanager/scheduler/SchedulerApplicationAttempt.java    | 3 ---
 3 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3f6b641/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index c27c897..d1e6bc7 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -900,6 +900,8 @@ Release 2.8.0 - UNRELEASED
     YARN-4188. Make MoveApplicationAcrossQueues abstract, newInstance static.
     (Giovanni Matteo Fumarola via cdouglas)
 
+    YARN-4171. Fix findbugs warnings in YARN-1197 branch. (Wangda Tan via jianhe)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3f6b641/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
index 8323f3c..0dbea1a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeStatusEvent.java
@@ -93,6 +93,7 @@ public class RMNodeStatusEvent extends RMNodeEvent {
     this.logAggregationReportsForApps = logAggregationReportsForApps;
   }
   
+  @SuppressWarnings("unchecked")
   public List<Container> getNMReportedIncreasedContainers() {
     return nmReportedIncreasedContainers == null ? Collections.EMPTY_LIST
         : nmReportedIncreasedContainers;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3f6b641/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index f064e97..005fa71 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -478,9 +478,6 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     ContainerType containerType = ContainerType.TASK;
     // The working knowledge is that masterContainer for AM is null as it
     // itself is the master container.
-    RMAppAttempt appAttempt = rmContext.getRMApps()
-        .get(container.getId().getApplicationAttemptId().getApplicationId())
-        .getCurrentAppAttempt();
     if (isWaitingForAMContainer(getApplicationId())) {
       containerType = ContainerType.APPLICATION_MASTER;
     }


[03/21] hadoop git commit: HDFS-9039. Separate client and server side methods of o.a.h.hdfs.NameNodeProxies. Contributed by Mingliang Liu.

Posted by wa...@apache.org.
HDFS-9039. Separate client and server side methods of o.a.h.hdfs.NameNodeProxies. Contributed by Mingliang Liu.


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

Branch: refs/heads/YARN-1197
Commit: 63d9f1596c92206cce3b72e3214d2fb5f6242b90
Parents: cc2b473
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Sep 22 20:52:37 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Sep 22 20:52:37 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/NameNodeProxiesClient.java      |  366 +++++
 .../protocolPB/ClientNamenodeProtocolPB.java    |   46 +
 .../ClientNamenodeProtocolTranslatorPB.java     | 1531 ++++++++++++++++++
 .../ha/AbstractNNFailoverProxyProvider.java     |   55 +
 .../ha/WrappedFailoverProxyProvider.java        |   71 +
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |    3 +
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |   13 +-
 .../java/org/apache/hadoop/hdfs/HAUtil.java     |    9 +-
 .../org/apache/hadoop/hdfs/NameNodeProxies.java |  275 +---
 .../protocolPB/ClientNamenodeProtocolPB.java    |   46 -
 .../ClientNamenodeProtocolTranslatorPB.java     | 1531 ------------------
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |    2 +-
 .../ha/AbstractNNFailoverProxyProvider.java     |   51 -
 .../ha/ConfiguredFailoverProxyProvider.java     |    2 +-
 .../ha/WrappedFailoverProxyProvider.java        |   80 -
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |    2 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |    4 +-
 17 files changed, 2096 insertions(+), 1991 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java
new file mode 100644
index 0000000..223c40d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java
@@ -0,0 +1,366 @@
+/**
+ * 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.hdfs;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Proxy;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.server.namenode.ha.AbstractNNFailoverProxyProvider;
+import org.apache.hadoop.hdfs.server.namenode.ha.WrappedFailoverProxyProvider;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.DefaultFailoverProxyProvider;
+import org.apache.hadoop.io.retry.FailoverProxyProvider;
+import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.io.retry.RetryUtils;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * Create proxy objects with {@link ClientProtocol} to communicate with a remote
+ * NN. Generally use {@link NameNodeProxiesClient#createProxyWithClientProtocol(
+ * Configuration, URI, AtomicBoolean)}, which will create either an HA- or
+ * non-HA-enabled client proxy as appropriate.
+ *
+ * For creating proxy objects with other protocols, please see
+ * {@link NameNodeProxies#createProxy(Configuration, URI, Class)}.
+ */
+@InterfaceAudience.Private
+public class NameNodeProxiesClient {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      NameNodeProxiesClient.class);
+
+  /**
+   * Wrapper for a client proxy as well as its associated service ID.
+   * This is simply used as a tuple-like return type for created NN proxy.
+   */
+  public static class ProxyAndInfo<PROXYTYPE> {
+    private final PROXYTYPE proxy;
+    private final Text dtService;
+    private final InetSocketAddress address;
+
+    public ProxyAndInfo(PROXYTYPE proxy, Text dtService,
+                        InetSocketAddress address) {
+      this.proxy = proxy;
+      this.dtService = dtService;
+      this.address = address;
+    }
+
+    public PROXYTYPE getProxy() {
+      return proxy;
+    }
+
+    public Text getDelegationTokenService() {
+      return dtService;
+    }
+
+    public InetSocketAddress getAddress() {
+      return address;
+    }
+  }
+
+  /**
+   * Creates the namenode proxy with the ClientProtocol. This will handle
+   * creation of either HA- or non-HA-enabled proxy objects, depending upon
+   * if the provided URI is a configured logical URI.
+   *
+   * @param conf the configuration containing the required IPC
+   *        properties, client failover configurations, etc.
+   * @param nameNodeUri the URI pointing either to a specific NameNode
+   *        or to a logical nameservice.
+   * @param fallbackToSimpleAuth set to true or false during calls to indicate
+   *        if a secure client falls back to simple auth
+   * @return an object containing both the proxy and the associated
+   *         delegation token service it corresponds to
+   * @throws IOException if there is an error creating the proxy
+   * @see {@link NameNodeProxies#createProxy(Configuration, URI, Class)}.
+   */
+  public static ProxyAndInfo<ClientProtocol> createProxyWithClientProtocol(
+      Configuration conf, URI nameNodeUri, AtomicBoolean fallbackToSimpleAuth)
+      throws IOException {
+    AbstractNNFailoverProxyProvider<ClientProtocol> failoverProxyProvider =
+        createFailoverProxyProvider(conf, nameNodeUri, ClientProtocol.class,
+            true, fallbackToSimpleAuth);
+
+    if (failoverProxyProvider == null) {
+      InetSocketAddress nnAddr = DFSUtilClient.getNNAddress(nameNodeUri);
+      Text dtService = SecurityUtil.buildTokenService(nnAddr);
+      ClientProtocol proxy = createNonHAProxyWithClientProtocol(nnAddr, conf,
+          UserGroupInformation.getCurrentUser(), true, fallbackToSimpleAuth);
+      return new ProxyAndInfo<>(proxy, dtService, nnAddr);
+    } else {
+      return createHAProxy(conf, nameNodeUri, ClientProtocol.class,
+          failoverProxyProvider);
+    }
+  }
+
+  /**
+   * Generate a dummy namenode proxy instance that utilizes our hacked
+   * {@link LossyRetryInvocationHandler}. Proxy instance generated using this
+   * method will proactively drop RPC responses. Currently this method only
+   * support HA setup. null will be returned if the given configuration is not
+   * for HA.
+   *
+   * @param config the configuration containing the required IPC
+   *        properties, client failover configurations, etc.
+   * @param nameNodeUri the URI pointing either to a specific NameNode
+   *        or to a logical nameservice.
+   * @param xface the IPC interface which should be created
+   * @param numResponseToDrop The number of responses to drop for each RPC call
+   * @param fallbackToSimpleAuth set to true or false during calls to indicate
+   *        if a secure client falls back to simple auth
+   * @return an object containing both the proxy and the associated
+   *         delegation token service it corresponds to. Will return null of the
+   *         given configuration does not support HA.
+   * @throws IOException if there is an error creating the proxy
+   */
+  public static <T> ProxyAndInfo<T> createProxyWithLossyRetryHandler(
+      Configuration config, URI nameNodeUri, Class<T> xface,
+      int numResponseToDrop, AtomicBoolean fallbackToSimpleAuth)
+      throws IOException {
+    Preconditions.checkArgument(numResponseToDrop > 0);
+    AbstractNNFailoverProxyProvider<T> failoverProxyProvider =
+        createFailoverProxyProvider(config, nameNodeUri, xface, true,
+            fallbackToSimpleAuth);
+
+    if (failoverProxyProvider != null) { // HA case
+      int delay = config.getInt(
+          HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_KEY,
+          HdfsClientConfigKeys.Failover.SLEEPTIME_BASE_DEFAULT);
+      int maxCap = config.getInt(
+          HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_KEY,
+          HdfsClientConfigKeys.Failover.SLEEPTIME_MAX_DEFAULT);
+      int maxFailoverAttempts = config.getInt(
+          HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY,
+          HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_DEFAULT);
+      int maxRetryAttempts = config.getInt(
+          HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY,
+          HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_DEFAULT);
+      InvocationHandler dummyHandler = new LossyRetryInvocationHandler<>(
+              numResponseToDrop, failoverProxyProvider,
+              RetryPolicies.failoverOnNetworkException(
+                  RetryPolicies.TRY_ONCE_THEN_FAIL, maxFailoverAttempts,
+                  Math.max(numResponseToDrop + 1, maxRetryAttempts), delay,
+                  maxCap));
+
+      @SuppressWarnings("unchecked")
+      T proxy = (T) Proxy.newProxyInstance(
+          failoverProxyProvider.getInterface().getClassLoader(),
+          new Class[]{xface}, dummyHandler);
+      Text dtService;
+      if (failoverProxyProvider.useLogicalURI()) {
+        dtService = HAUtilClient.buildTokenServiceForLogicalUri(nameNodeUri,
+            HdfsConstants.HDFS_URI_SCHEME);
+      } else {
+        dtService = SecurityUtil.buildTokenService(
+            DFSUtilClient.getNNAddress(nameNodeUri));
+      }
+      return new ProxyAndInfo<>(proxy, dtService,
+          DFSUtilClient.getNNAddress(nameNodeUri));
+    } else {
+      LOG.warn("Currently creating proxy using " +
+          "LossyRetryInvocationHandler requires NN HA setup");
+      return null;
+    }
+  }
+
+  /** Creates the Failover proxy provider instance*/
+  @VisibleForTesting
+  public static <T> AbstractNNFailoverProxyProvider<T> createFailoverProxyProvider(
+      Configuration conf, URI nameNodeUri, Class<T> xface, boolean checkPort,
+      AtomicBoolean fallbackToSimpleAuth) throws IOException {
+    Class<FailoverProxyProvider<T>> failoverProxyProviderClass = null;
+    AbstractNNFailoverProxyProvider<T> providerNN;
+    try {
+      // Obtain the class of the proxy provider
+      failoverProxyProviderClass = getFailoverProxyProviderClass(conf,
+          nameNodeUri);
+      if (failoverProxyProviderClass == null) {
+        return null;
+      }
+      // Create a proxy provider instance.
+      Constructor<FailoverProxyProvider<T>> ctor = failoverProxyProviderClass
+          .getConstructor(Configuration.class, URI.class, Class.class);
+      FailoverProxyProvider<T> provider = ctor.newInstance(conf, nameNodeUri,
+          xface);
+
+      // If the proxy provider is of an old implementation, wrap it.
+      if (!(provider instanceof AbstractNNFailoverProxyProvider)) {
+        providerNN = new WrappedFailoverProxyProvider<>(provider);
+      } else {
+        providerNN = (AbstractNNFailoverProxyProvider<T>)provider;
+      }
+    } catch (Exception e) {
+      final String message = "Couldn't create proxy provider " +
+          failoverProxyProviderClass;
+      LOG.debug(message, e);
+      if (e.getCause() instanceof IOException) {
+        throw (IOException) e.getCause();
+      } else {
+        throw new IOException(message, e);
+      }
+    }
+
+    // Check the port in the URI, if it is logical.
+    if (checkPort && providerNN.useLogicalURI()) {
+      int port = nameNodeUri.getPort();
+      if (port > 0 &&
+          port != HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT) {
+        // Throwing here without any cleanup is fine since we have not
+        // actually created the underlying proxies yet.
+        throw new IOException("Port " + port + " specified in URI "
+            + nameNodeUri + " but host '" + nameNodeUri.getHost()
+            + "' is a logical (HA) namenode"
+            + " and does not use port information.");
+      }
+    }
+    providerNN.setFallbackToSimpleAuth(fallbackToSimpleAuth);
+    return providerNN;
+  }
+
+  /** Gets the configured Failover proxy provider's class */
+  @VisibleForTesting
+  public static <T> Class<FailoverProxyProvider<T>> getFailoverProxyProviderClass(
+      Configuration conf, URI nameNodeUri) throws IOException {
+    if (nameNodeUri == null) {
+      return null;
+    }
+    String host = nameNodeUri.getHost();
+    String configKey = HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX
+        + "." + host;
+    try {
+      @SuppressWarnings("unchecked")
+      Class<FailoverProxyProvider<T>> ret = (Class<FailoverProxyProvider<T>>)
+          conf.getClass(configKey, null, FailoverProxyProvider.class);
+      return ret;
+    } catch (RuntimeException e) {
+      if (e.getCause() instanceof ClassNotFoundException) {
+        throw new IOException("Could not load failover proxy provider class "
+            + conf.get(configKey) + " which is configured for authority "
+            + nameNodeUri, e);
+      } else {
+        throw e;
+      }
+    }
+  }
+
+  /**
+   * Creates an explicitly HA-enabled proxy object.
+   *
+   * @param conf the configuration object
+   * @param nameNodeUri the URI pointing either to a specific NameNode or to a
+   *        logical nameservice.
+   * @param xface the IPC interface which should be created
+   * @param failoverProxyProvider Failover proxy provider
+   * @return an object containing both the proxy and the associated
+   *         delegation token service it corresponds to
+   * @throws IOException
+   */
+  @SuppressWarnings("unchecked")
+  public static <T> ProxyAndInfo<T> createHAProxy(
+      Configuration conf, URI nameNodeUri, Class<T> xface,
+      AbstractNNFailoverProxyProvider<T> failoverProxyProvider)
+      throws IOException {
+    Preconditions.checkNotNull(failoverProxyProvider);
+    // HA case
+    DfsClientConf config = new DfsClientConf(conf);
+    T proxy = (T) RetryProxy.create(xface, failoverProxyProvider,
+        RetryPolicies.failoverOnNetworkException(
+            RetryPolicies.TRY_ONCE_THEN_FAIL, config.getMaxFailoverAttempts(),
+            config.getMaxRetryAttempts(), config.getFailoverSleepBaseMillis(),
+            config.getFailoverSleepMaxMillis()));
+
+    Text dtService;
+    if (failoverProxyProvider.useLogicalURI()) {
+      dtService = HAUtilClient.buildTokenServiceForLogicalUri(nameNodeUri,
+          HdfsConstants.HDFS_URI_SCHEME);
+    } else {
+      dtService = SecurityUtil.buildTokenService(
+          DFSUtilClient.getNNAddress(nameNodeUri));
+    }
+    return new ProxyAndInfo<>(proxy, dtService,
+        DFSUtilClient.getNNAddress(nameNodeUri));
+  }
+
+  public static ClientProtocol createNonHAProxyWithClientProtocol(
+      InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
+      boolean withRetries, AtomicBoolean fallbackToSimpleAuth)
+      throws IOException {
+    RPC.setProtocolEngine(conf, ClientNamenodeProtocolPB.class,
+        ProtobufRpcEngine.class);
+
+    final RetryPolicy defaultPolicy =
+        RetryUtils.getDefaultRetryPolicy(
+            conf,
+            HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY,
+            HdfsClientConfigKeys.Retry.POLICY_ENABLED_DEFAULT,
+            HdfsClientConfigKeys.Retry.POLICY_SPEC_KEY,
+            HdfsClientConfigKeys.Retry.POLICY_SPEC_DEFAULT,
+            SafeModeException.class.getName());
+
+    final long version = RPC.getProtocolVersion(ClientNamenodeProtocolPB.class);
+    ClientNamenodeProtocolPB proxy = RPC.getProtocolProxy(
+        ClientNamenodeProtocolPB.class, version, address, ugi, conf,
+        NetUtils.getDefaultSocketFactory(conf),
+        org.apache.hadoop.ipc.Client.getTimeout(conf), defaultPolicy,
+        fallbackToSimpleAuth).getProxy();
+
+    if (withRetries) { // create the proxy with retries
+      Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<>();
+      ClientProtocol translatorProxy =
+          new ClientNamenodeProtocolTranslatorPB(proxy);
+      return (ClientProtocol) RetryProxy.create(
+          ClientProtocol.class,
+          new DefaultFailoverProxyProvider<>(ClientProtocol.class,
+              translatorProxy),
+          methodNameToPolicyMap,
+          defaultPolicy);
+    } else {
+      return new ClientNamenodeProtocolTranslatorPB(proxy);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
new file mode 100644
index 0000000..e7ce44b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
@@ -0,0 +1,46 @@
+/**
+ * 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.hdfs.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.token.TokenInfo;
+
+
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+@KerberosInfo(
+    serverPrincipal = HdfsClientConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
+@TokenInfo(DelegationTokenSelector.class)
+@ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME,
+    protocolVersion = 1)
+/**
+ * Protocol that a clients use to communicate with the NameNode.
+ * 
+ * Note: This extends the protocolbuffer service based interface to
+ * add annotations required for security.
+ */
+public interface ClientNamenodeProtocolPB extends 
+  ClientNamenodeProtocol.BlockingInterface {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
new file mode 100644
index 0000000..f4ce46d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -0,0 +1,1531 @@
+/**
+ * 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.hdfs.protocolPB;
+
+import java.io.Closeable;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.inotify.EventBatchList;
+import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.ModifyAclEntriesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclEntriesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveDefaultAclRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.AclProtos.SetAclRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DisallowSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetCurrentEditLogTxidRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatusRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.ProtobufHelper;
+import org.apache.hadoop.ipc.ProtocolMetaInterface;
+import org.apache.hadoop.ipc.ProtocolTranslator;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RpcClientUtil;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
+import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
+import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
+import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
+import org.apache.hadoop.security.token.Token;
+
+import com.google.protobuf.ByteString;
+import com.google.protobuf.ServiceException;
+
+import static org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
+import static org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos
+    .EncryptionZoneProto;
+
+/**
+ * This class forwards NN's ClientProtocol calls as RPC calls to the NN server
+ * while translating from the parameter types used in ClientProtocol to the
+ * new PB types.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class ClientNamenodeProtocolTranslatorPB implements
+    ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
+  final private ClientNamenodeProtocolPB rpcProxy;
+
+  static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST = 
+  GetServerDefaultsRequestProto.newBuilder().build();
+
+  private final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST =
+  GetFsStatusRequestProto.newBuilder().build();
+
+  private final static SaveNamespaceRequestProto VOID_SAVE_NAMESPACE_REQUEST =
+  SaveNamespaceRequestProto.newBuilder().build();
+
+  private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST = 
+  RollEditsRequestProto.getDefaultInstance();
+
+  private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST =
+  RefreshNodesRequestProto.newBuilder().build();
+
+  private final static FinalizeUpgradeRequestProto
+  VOID_FINALIZE_UPGRADE_REQUEST =
+      FinalizeUpgradeRequestProto.newBuilder().build();
+
+  private final static GetDataEncryptionKeyRequestProto
+  VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
+      GetDataEncryptionKeyRequestProto.newBuilder().build();
+
+  private final static GetStoragePoliciesRequestProto
+  VOID_GET_STORAGE_POLICIES_REQUEST =
+      GetStoragePoliciesRequestProto.newBuilder().build();
+
+  public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
+    rpcProxy = proxy;
+  }
+  
+  @Override
+  public void close() {
+    RPC.stopProxy(rpcProxy);
+  }
+
+  @Override
+  public LocatedBlocks getBlockLocations(String src, long offset, long length)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto
+        .newBuilder()
+        .setSrc(src)
+        .setOffset(offset)
+        .setLength(length)
+        .build();
+    try {
+      GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
+          req);
+      return resp.hasLocations() ?
+        PBHelperClient.convert(resp.getLocations()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public FsServerDefaults getServerDefaults() throws IOException {
+    GetServerDefaultsRequestProto req = VOID_GET_SERVER_DEFAULT_REQUEST;
+    try {
+      return PBHelperClient
+          .convert(rpcProxy.getServerDefaults(null, req).getServerDefaults());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public HdfsFileStatus create(String src, FsPermission masked,
+      String clientName, EnumSetWritable<CreateFlag> flag,
+      boolean createParent, short replication, long blockSize, 
+      CryptoProtocolVersion[] supportedVersions)
+      throws AccessControlException, AlreadyBeingCreatedException,
+      DSQuotaExceededException, FileAlreadyExistsException,
+      FileNotFoundException, NSQuotaExceededException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    CreateRequestProto.Builder builder = CreateRequestProto.newBuilder()
+        .setSrc(src)
+        .setMasked(PBHelperClient.convert(masked))
+        .setClientName(clientName)
+        .setCreateFlag(PBHelperClient.convertCreateFlag(flag))
+        .setCreateParent(createParent)
+        .setReplication(replication)
+        .setBlockSize(blockSize);
+    builder.addAllCryptoProtocolVersion(PBHelperClient.convert(supportedVersions));
+    CreateRequestProto req = builder.build();
+    try {
+      CreateResponseProto res = rpcProxy.create(null, req);
+      return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+  }
+
+  @Override
+  public boolean truncate(String src, long newLength, String clientName)
+      throws IOException, UnresolvedLinkException {
+    TruncateRequestProto req = TruncateRequestProto.newBuilder()
+        .setSrc(src)
+        .setNewLength(newLength)
+        .setClientName(clientName)
+        .build();
+    try {
+      return rpcProxy.truncate(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public LastBlockWithStatus append(String src, String clientName,
+      EnumSetWritable<CreateFlag> flag) throws AccessControlException,
+      DSQuotaExceededException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src)
+        .setClientName(clientName).setFlag(PBHelperClient.convertCreateFlag(flag))
+        .build();
+    try {
+      AppendResponseProto res = rpcProxy.append(null, req);
+      LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
+          .convert(res.getBlock()) : null;
+      HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
+          : null;
+      return new LastBlockWithStatus(lastBlock, stat);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean setReplication(String src, short replication)
+      throws AccessControlException, DSQuotaExceededException,
+      FileNotFoundException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder()
+        .setSrc(src)
+        .setReplication(replication)
+        .build();
+    try {
+      return rpcProxy.setReplication(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setPermission(String src, FsPermission permission)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
+        .setSrc(src)
+        .setPermission(PBHelperClient.convert(permission))
+        .build();
+    try {
+      rpcProxy.setPermission(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setOwner(String src, String username, String groupname)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder()
+        .setSrc(src);
+    if (username != null)
+        req.setUsername(username);
+    if (groupname != null)
+        req.setGroupname(groupname);
+    try {
+      rpcProxy.setOwner(null, req.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void abandonBlock(ExtendedBlock b, long fileId, String src,
+      String holder) throws AccessControlException, FileNotFoundException,
+        UnresolvedLinkException, IOException {
+    AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
+        .setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder)
+            .setFileId(fileId).build();
+    try {
+      rpcProxy.abandonBlock(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public LocatedBlock addBlock(String src, String clientName,
+      ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
+      String[] favoredNodes)
+      throws AccessControlException, FileNotFoundException,
+      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder()
+        .setSrc(src).setClientName(clientName).setFileId(fileId);
+    if (previous != null) 
+      req.setPrevious(PBHelperClient.convert(previous));
+    if (excludeNodes != null)
+      req.addAllExcludeNodes(PBHelperClient.convert(excludeNodes));
+    if (favoredNodes != null) {
+      req.addAllFavoredNodes(Arrays.asList(favoredNodes));
+    }
+    try {
+      return PBHelperClient.convert(rpcProxy.addBlock(null, req.build()).getBlock());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public LocatedBlock getAdditionalDatanode(String src, long fileId,
+      ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs,
+      DatanodeInfo[] excludes,
+      int numAdditionalNodes, String clientName) throws AccessControlException,
+      FileNotFoundException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto
+        .newBuilder()
+        .setSrc(src)
+        .setFileId(fileId)
+        .setBlk(PBHelperClient.convert(blk))
+        .addAllExistings(PBHelperClient.convert(existings))
+        .addAllExistingStorageUuids(Arrays.asList(existingStorageIDs))
+        .addAllExcludes(PBHelperClient.convert(excludes))
+        .setNumAdditionalNodes(numAdditionalNodes)
+        .setClientName(clientName)
+        .build();
+    try {
+      return PBHelperClient.convert(rpcProxy.getAdditionalDatanode(null, req)
+          .getBlock());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean complete(String src, String clientName,
+                          ExtendedBlock last, long fileId)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder()
+        .setSrc(src)
+        .setClientName(clientName)
+        .setFileId(fileId);
+    if (last != null)
+      req.setLast(PBHelperClient.convert(last));
+    try {
+      return rpcProxy.complete(null, req.build()).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+    ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
+        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks)))
+        .build();
+    try {
+      rpcProxy.reportBadBlocks(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean rename(String src, String dst) throws UnresolvedLinkException,
+      IOException {
+    RenameRequestProto req = RenameRequestProto.newBuilder()
+        .setSrc(src)
+        .setDst(dst).build();
+    try {
+      return rpcProxy.rename(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+
+  @Override
+  public void rename2(String src, String dst, Rename... options)
+      throws AccessControlException, DSQuotaExceededException,
+      FileAlreadyExistsException, FileNotFoundException,
+      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    boolean overwrite = false;
+    if (options != null) {
+      for (Rename option : options) {
+        if (option == Rename.OVERWRITE) {
+          overwrite = true;
+        }
+      }
+    }
+    Rename2RequestProto req = Rename2RequestProto.newBuilder().
+        setSrc(src).
+        setDst(dst).setOverwriteDest(overwrite).
+        build();
+    try {
+      rpcProxy.rename2(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+  }
+
+  @Override
+  public void concat(String trg, String[] srcs) throws IOException,
+      UnresolvedLinkException {
+    ConcatRequestProto req = ConcatRequestProto.newBuilder().
+        setTrg(trg).
+        addAllSrcs(Arrays.asList(srcs)).build();
+    try {
+      rpcProxy.concat(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+
+  @Override
+  public boolean delete(String src, boolean recursive)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException {
+    DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src).setRecursive(recursive).build();
+    try {
+      return rpcProxy.delete(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
+      throws AccessControlException, FileAlreadyExistsException,
+      FileNotFoundException, NSQuotaExceededException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
+        .setSrc(src)
+        .setMasked(PBHelperClient.convert(masked))
+        .setCreateParent(createParent).build();
+
+    try {
+      return rpcProxy.mkdirs(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public DirectoryListing getListing(String src, byte[] startAfter,
+      boolean needLocation) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    GetListingRequestProto req = GetListingRequestProto.newBuilder()
+        .setSrc(src)
+        .setStartAfter(ByteString.copyFrom(startAfter))
+        .setNeedLocation(needLocation).build();
+    try {
+      GetListingResponseProto result = rpcProxy.getListing(null, req);
+      
+      if (result.hasDirList()) {
+        return PBHelperClient.convert(result.getDirList());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void renewLease(String clientName) throws AccessControlException,
+      IOException {
+    RenewLeaseRequestProto req = RenewLeaseRequestProto.newBuilder()
+        .setClientName(clientName).build();
+    try {
+      rpcProxy.renewLease(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean recoverLease(String src, String clientName)
+      throws IOException {
+    RecoverLeaseRequestProto req = RecoverLeaseRequestProto.newBuilder()
+        .setSrc(src)
+        .setClientName(clientName).build();
+    try {
+      return rpcProxy.recoverLease(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }  
+  }
+
+  @Override
+  public long[] getStats() throws IOException {
+    try {
+      return PBHelperClient.convert(rpcProxy.getFsStats(null,
+          VOID_GET_FSSTATUS_REQUEST));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
+      throws IOException {
+    GetDatanodeReportRequestProto req = GetDatanodeReportRequestProto
+        .newBuilder()
+        .setType(PBHelperClient.convert(type)).build();
+    try {
+      return PBHelperClient.convert(
+          rpcProxy.getDatanodeReport(null, req).getDiList());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public DatanodeStorageReport[] getDatanodeStorageReport(DatanodeReportType type)
+      throws IOException {
+    final GetDatanodeStorageReportRequestProto req
+        = GetDatanodeStorageReportRequestProto.newBuilder()
+            .setType(PBHelperClient.convert(type)).build();
+    try {
+      return PBHelperClient.convertDatanodeStorageReports(
+          rpcProxy.getDatanodeStorageReport(null, req).getDatanodeStorageReportsList());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public long getPreferredBlockSize(String filename) throws IOException,
+      UnresolvedLinkException {
+    GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto
+        .newBuilder()
+        .setFilename(filename)
+        .build();
+    try {
+      return rpcProxy.getPreferredBlockSize(null, req).getBsize();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException {
+    SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder()
+        .setAction(PBHelperClient.convert(action)).setChecked(isChecked).build();
+    try {
+      return rpcProxy.setSafeMode(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean saveNamespace(long timeWindow, long txGap) throws IOException {
+    try {
+      SaveNamespaceRequestProto req = SaveNamespaceRequestProto.newBuilder()
+          .setTimeWindow(timeWindow).setTxGap(txGap).build();
+      return rpcProxy.saveNamespace(null, req).getSaved();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public long rollEdits() throws AccessControlException, IOException {
+    try {
+      RollEditsResponseProto resp = rpcProxy.rollEdits(null,
+          VOID_ROLLEDITS_REQUEST);
+      return resp.getNewSegmentTxId();
+    } catch (ServiceException se) {
+      throw ProtobufHelper.getRemoteException(se);
+    }
+  }
+
+  @Override
+  public boolean restoreFailedStorage(String arg) 
+      throws AccessControlException, IOException{
+    RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto
+        .newBuilder()
+        .setArg(arg).build();
+    try {
+      return rpcProxy.restoreFailedStorage(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void refreshNodes() throws IOException {
+    try {
+      rpcProxy.refreshNodes(null, VOID_REFRESH_NODES_REQUEST);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void finalizeUpgrade() throws IOException {
+    try {
+      rpcProxy.finalizeUpgrade(null, VOID_FINALIZE_UPGRADE_REQUEST);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
+    final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
+        .setAction(PBHelperClient.convert(action)).build();
+    try {
+      final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
+      if (proto.hasRollingUpgradeInfo()) {
+        return PBHelperClient.convert(proto.getRollingUpgradeInfo());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+      throws IOException {
+    ListCorruptFileBlocksRequestProto.Builder req = 
+        ListCorruptFileBlocksRequestProto.newBuilder().setPath(path);   
+    if (cookie != null) 
+      req.setCookie(cookie);
+    try {
+      return PBHelperClient.convert(
+          rpcProxy.listCorruptFileBlocks(null, req.build()).getCorrupt());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void metaSave(String filename) throws IOException {
+    MetaSaveRequestProto req = MetaSaveRequestProto.newBuilder()
+        .setFilename(filename).build();
+    try {
+      rpcProxy.metaSave(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+
+  }
+
+  @Override
+  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      GetFileInfoResponseProto res = rpcProxy.getFileInfo(null, req);
+      return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public HdfsFileStatus getFileLinkInfo(String src)
+      throws AccessControlException, UnresolvedLinkException, IOException {
+    GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      GetFileLinkInfoResponseProto result = rpcProxy.getFileLinkInfo(null, req);
+      return result.hasFs() ?  
+          PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public ContentSummary getContentSummary(String path)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    GetContentSummaryRequestProto req = GetContentSummaryRequestProto
+        .newBuilder()
+        .setPath(path)
+        .build();
+    try {
+      return PBHelperClient.convert(rpcProxy.getContentSummary(null, req)
+          .getSummary());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
+                       StorageType type)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    final SetQuotaRequestProto.Builder builder
+        = SetQuotaRequestProto.newBuilder()
+        .setPath(path)
+        .setNamespaceQuota(namespaceQuota)
+        .setStoragespaceQuota(storagespaceQuota);
+    if (type != null) {
+      builder.setStorageType(PBHelperClient.convertStorageType(type));
+    }
+    final SetQuotaRequestProto req = builder.build();
+    try {
+      rpcProxy.setQuota(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void fsync(String src, long fileId, String client,
+                    long lastBlockLength)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src)
+        .setClient(client).setLastBlockLength(lastBlockLength)
+            .setFileId(fileId).build();
+    try {
+      rpcProxy.fsync(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setTimes(String src, long mtime, long atime)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException {
+    SetTimesRequestProto req = SetTimesRequestProto.newBuilder()
+        .setSrc(src)
+        .setMtime(mtime)
+        .setAtime(atime)
+        .build();
+    try {
+      rpcProxy.setTimes(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void createSymlink(String target, String link, FsPermission dirPerm,
+      boolean createParent) throws AccessControlException,
+      FileAlreadyExistsException, FileNotFoundException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      IOException {
+    CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder()
+        .setTarget(target)
+        .setLink(link)
+        .setDirPerm(PBHelperClient.convert(dirPerm))
+        .setCreateParent(createParent)
+        .build();
+    try {
+      rpcProxy.createSymlink(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public String getLinkTarget(String path) throws AccessControlException,
+      FileNotFoundException, IOException {
+    GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder()
+        .setPath(path).build();
+    try {
+      GetLinkTargetResponseProto rsp = rpcProxy.getLinkTarget(null, req);
+      return rsp.hasTargetPath() ? rsp.getTargetPath() : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
+      String clientName) throws IOException {
+    UpdateBlockForPipelineRequestProto req = UpdateBlockForPipelineRequestProto
+        .newBuilder()
+        .setBlock(PBHelperClient.convert(block))
+        .setClientName(clientName)
+        .build();
+    try {
+      return PBHelperClient.convert(
+          rpcProxy.updateBlockForPipeline(null, req).getBlock());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void updatePipeline(String clientName, ExtendedBlock oldBlock,
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
+    UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
+        .setClientName(clientName)
+        .setOldBlock(PBHelperClient.convert(oldBlock))
+        .setNewBlock(PBHelperClient.convert(newBlock))
+        .addAllNewNodes(Arrays.asList(PBHelperClient.convert(newNodes)))
+        .addAllStorageIDs(storageIDs == null ? null : Arrays.asList(storageIDs))
+        .build();
+    try {
+      rpcProxy.updatePipeline(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
+      throws IOException {
+    GetDelegationTokenRequestProto req = GetDelegationTokenRequestProto
+        .newBuilder()
+        .setRenewer(renewer == null ? "" : renewer.toString())
+        .build();
+    try {
+      GetDelegationTokenResponseProto resp = rpcProxy.getDelegationToken(null, req);
+      return resp.hasToken() ? PBHelperClient.convertDelegationToken(resp.getToken())
+          : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException {
+    RenewDelegationTokenRequestProto req = RenewDelegationTokenRequestProto.newBuilder().
+        setToken(PBHelperClient.convert(token)).
+        build();
+    try {
+      return rpcProxy.renewDelegationToken(null, req).getNewExpiryTime();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException {
+    CancelDelegationTokenRequestProto req = CancelDelegationTokenRequestProto
+        .newBuilder()
+        .setToken(PBHelperClient.convert(token))
+        .build();
+    try {
+      rpcProxy.cancelDelegationToken(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setBalancerBandwidth(long bandwidth) throws IOException {
+    SetBalancerBandwidthRequestProto req = SetBalancerBandwidthRequestProto.newBuilder()
+        .setBandwidth(bandwidth)
+        .build();
+    try {
+      rpcProxy.setBalancerBandwidth(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public boolean isMethodSupported(String methodName) throws IOException {
+    return RpcClientUtil.isMethodSupported(rpcProxy,
+        ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+        RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
+  }
+  
+  @Override
+  public DataEncryptionKey getDataEncryptionKey() throws IOException {
+    try {
+      GetDataEncryptionKeyResponseProto rsp = rpcProxy.getDataEncryptionKey(
+          null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST);
+     return rsp.hasDataEncryptionKey() ? 
+          PBHelperClient.convert(rsp.getDataEncryptionKey()) : null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+
+  @Override
+  public boolean isFileClosed(String src) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException {
+    IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      return rpcProxy.isFileClosed(null, req).getResult();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public Object getUnderlyingProxyObject() {
+    return rpcProxy;
+  }
+
+  @Override
+  public String createSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    final CreateSnapshotRequestProto.Builder builder
+        = CreateSnapshotRequestProto.newBuilder().setSnapshotRoot(snapshotRoot);
+    if (snapshotName != null) {
+      builder.setSnapshotName(snapshotName);
+    }
+    final CreateSnapshotRequestProto req = builder.build();
+    try {
+      return rpcProxy.createSnapshot(null, req).getSnapshotPath();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public void deleteSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException {
+    DeleteSnapshotRequestProto req = DeleteSnapshotRequestProto.newBuilder()
+        .setSnapshotRoot(snapshotRoot).setSnapshotName(snapshotName).build();
+    try {
+      rpcProxy.deleteSnapshot(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public void allowSnapshot(String snapshotRoot) throws IOException {
+    AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder()
+        .setSnapshotRoot(snapshotRoot).build();
+    try {
+      rpcProxy.allowSnapshot(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void disallowSnapshot(String snapshotRoot) throws IOException {
+    DisallowSnapshotRequestProto req = DisallowSnapshotRequestProto
+        .newBuilder().setSnapshotRoot(snapshotRoot).build();
+    try {
+      rpcProxy.disallowSnapshot(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+      String snapshotNewName) throws IOException {
+    RenameSnapshotRequestProto req = RenameSnapshotRequestProto.newBuilder()
+        .setSnapshotRoot(snapshotRoot).setSnapshotOldName(snapshotOldName)
+        .setSnapshotNewName(snapshotNewName).build();
+    try {
+      rpcProxy.renameSnapshot(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException {
+    GetSnapshottableDirListingRequestProto req = 
+        GetSnapshottableDirListingRequestProto.newBuilder().build();
+    try {
+      GetSnapshottableDirListingResponseProto result = rpcProxy
+          .getSnapshottableDirListing(null, req);
+      
+      if (result.hasSnapshottableDirList()) {
+        return PBHelperClient.convert(result.getSnapshottableDirList());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+      String fromSnapshot, String toSnapshot) throws IOException {
+    GetSnapshotDiffReportRequestProto req = GetSnapshotDiffReportRequestProto
+        .newBuilder().setSnapshotRoot(snapshotRoot)
+        .setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build();
+    try {
+      GetSnapshotDiffReportResponseProto result = 
+          rpcProxy.getSnapshotDiffReport(null, req);
+    
+      return PBHelperClient.convert(result.getDiffReport());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public long addCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
+    try {
+      AddCacheDirectiveRequestProto.Builder builder =
+          AddCacheDirectiveRequestProto.newBuilder().
+              setInfo(PBHelperClient.convert(directive));
+      if (!flags.isEmpty()) {
+        builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags));
+      }
+      return rpcProxy.addCacheDirective(null, builder.build()).getId();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException {
+    try {
+      ModifyCacheDirectiveRequestProto.Builder builder =
+          ModifyCacheDirectiveRequestProto.newBuilder().
+              setInfo(PBHelperClient.convert(directive));
+      if (!flags.isEmpty()) {
+        builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags));
+      }
+      rpcProxy.modifyCacheDirective(null, builder.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeCacheDirective(long id)
+      throws IOException {
+    try {
+      rpcProxy.removeCacheDirective(null,
+          RemoveCacheDirectiveRequestProto.newBuilder().
+              setId(id).build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  private static class BatchedCacheEntries
+      implements BatchedEntries<CacheDirectiveEntry> {
+    private final ListCacheDirectivesResponseProto response;
+
+    BatchedCacheEntries(
+        ListCacheDirectivesResponseProto response) {
+      this.response = response;
+    }
+
+    @Override
+    public CacheDirectiveEntry get(int i) {
+      return PBHelperClient.convert(response.getElements(i));
+    }
+
+    @Override
+    public int size() {
+      return response.getElementsCount();
+    }
+    
+    @Override
+    public boolean hasMore() {
+      return response.getHasMore();
+    }
+  }
+
+  @Override
+  public BatchedEntries<CacheDirectiveEntry>
+      listCacheDirectives(long prevId,
+          CacheDirectiveInfo filter) throws IOException {
+    if (filter == null) {
+      filter = new CacheDirectiveInfo.Builder().build();
+    }
+    try {
+      return new BatchedCacheEntries(
+        rpcProxy.listCacheDirectives(null,
+          ListCacheDirectivesRequestProto.newBuilder().
+            setPrevId(prevId).
+            setFilter(PBHelperClient.convert(filter)).
+            build()));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void addCachePool(CachePoolInfo info) throws IOException {
+    AddCachePoolRequestProto.Builder builder = 
+        AddCachePoolRequestProto.newBuilder();
+    builder.setInfo(PBHelperClient.convert(info));
+    try {
+      rpcProxy.addCachePool(null, builder.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void modifyCachePool(CachePoolInfo req) throws IOException {
+    ModifyCachePoolRequestProto.Builder builder = 
+        ModifyCachePoolRequestProto.newBuilder();
+    builder.setInfo(PBHelperClient.convert(req));
+    try {
+      rpcProxy.modifyCachePool(null, builder.build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeCachePool(String cachePoolName) throws IOException {
+    try {
+      rpcProxy.removeCachePool(null, 
+          RemoveCachePoolRequestProto.newBuilder().
+            setPoolName(cachePoolName).build());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  private static class BatchedCachePoolEntries
+    implements BatchedEntries<CachePoolEntry> {
+      private final ListCachePoolsResponseProto proto;
+    
+    public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) {
+      this.proto = proto;
+    }
+      
+    @Override
+    public CachePoolEntry get(int i) {
+      CachePoolEntryProto elem = proto.getEntries(i);
+      return PBHelperClient.convert(elem);
+    }
+
+    @Override
+    public int size() {
+      return proto.getEntriesCount();
+    }
+    
+    @Override
+    public boolean hasMore() {
+      return proto.getHasMore();
+    }
+  }
+
+  @Override
+  public BatchedEntries<CachePoolEntry> listCachePools(String prevKey)
+      throws IOException {
+    try {
+      return new BatchedCachePoolEntries(
+        rpcProxy.listCachePools(null,
+          ListCachePoolsRequestProto.newBuilder().
+            setPrevPoolName(prevKey).build()));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void modifyAclEntries(String src, List<AclEntry> aclSpec)
+      throws IOException {
+    ModifyAclEntriesRequestProto req = ModifyAclEntriesRequestProto
+        .newBuilder().setSrc(src)
+        .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build();
+    try {
+      rpcProxy.modifyAclEntries(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeAclEntries(String src, List<AclEntry> aclSpec)
+      throws IOException {
+    RemoveAclEntriesRequestProto req = RemoveAclEntriesRequestProto
+        .newBuilder().setSrc(src)
+        .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build();
+    try {
+      rpcProxy.removeAclEntries(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeDefaultAcl(String src) throws IOException {
+    RemoveDefaultAclRequestProto req = RemoveDefaultAclRequestProto
+        .newBuilder().setSrc(src).build();
+    try {
+      rpcProxy.removeDefaultAcl(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeAcl(String src) throws IOException {
+    RemoveAclRequestProto req = RemoveAclRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      rpcProxy.removeAcl(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
+    SetAclRequestProto req = SetAclRequestProto.newBuilder()
+        .setSrc(src)
+        .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec))
+        .build();
+    try {
+      rpcProxy.setAcl(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public AclStatus getAclStatus(String src) throws IOException {
+    GetAclStatusRequestProto req = GetAclStatusRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      return PBHelperClient.convert(rpcProxy.getAclStatus(null, req));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void createEncryptionZone(String src, String keyName)
+    throws IOException {
+    final CreateEncryptionZoneRequestProto.Builder builder =
+      CreateEncryptionZoneRequestProto.newBuilder();
+    builder.setSrc(src);
+    if (keyName != null && !keyName.isEmpty()) {
+      builder.setKeyName(keyName);
+    }
+    CreateEncryptionZoneRequestProto req = builder.build();
+    try {
+      rpcProxy.createEncryptionZone(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public EncryptionZone getEZForPath(String src)
+      throws IOException {
+    final GetEZForPathRequestProto.Builder builder =
+        GetEZForPathRequestProto.newBuilder();
+    builder.setSrc(src);
+    final GetEZForPathRequestProto req = builder.build();
+    try {
+      final EncryptionZonesProtos.GetEZForPathResponseProto response =
+          rpcProxy.getEZForPath(null, req);
+      if (response.hasZone()) {
+        return PBHelperClient.convert(response.getZone());
+      } else {
+        return null;
+      }
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public BatchedEntries<EncryptionZone> listEncryptionZones(long id)
+      throws IOException {
+    final ListEncryptionZonesRequestProto req =
+      ListEncryptionZonesRequestProto.newBuilder()
+          .setId(id)
+          .build();
+    try {
+      EncryptionZonesProtos.ListEncryptionZonesResponseProto response =
+          rpcProxy.listEncryptionZones(null, req);
+      List<EncryptionZone> elements =
+          Lists.newArrayListWithCapacity(response.getZonesCount());
+      for (EncryptionZoneProto p : response.getZonesList()) {
+        elements.add(PBHelperClient.convert(p));
+      }
+      return new BatchedListEntries<EncryptionZone>(elements,
+          response.getHasMore());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
+      throws IOException {
+    SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder()
+        .setSrc(src)
+        .setXAttr(PBHelperClient.convertXAttrProto(xAttr))
+        .setFlag(PBHelperClient.convert(flag))
+        .build();
+    try {
+      rpcProxy.setXAttr(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
+      throws IOException {
+    GetXAttrsRequestProto.Builder builder = GetXAttrsRequestProto.newBuilder();
+    builder.setSrc(src);
+    if (xAttrs != null) {
+      builder.addAllXAttrs(PBHelperClient.convertXAttrProto(xAttrs));
+    }
+    GetXAttrsRequestProto req = builder.build();
+    try {
+      return PBHelperClient.convert(rpcProxy.getXAttrs(null, req));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+  
+  @Override
+  public List<XAttr> listXAttrs(String src)
+      throws IOException {
+    ListXAttrsRequestProto.Builder builder = ListXAttrsRequestProto.newBuilder();
+    builder.setSrc(src);
+    ListXAttrsRequestProto req = builder.build();
+    try {
+      return PBHelperClient.convert(rpcProxy.listXAttrs(null, req));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void removeXAttr(String src, XAttr xAttr) throws IOException {
+    RemoveXAttrRequestProto req = RemoveXAttrRequestProto
+        .newBuilder().setSrc(src)
+        .setXAttr(PBHelperClient.convertXAttrProto(xAttr)).build();
+    try {
+      rpcProxy.removeXAttr(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void checkAccess(String path, FsAction mode) throws IOException {
+    CheckAccessRequestProto req = CheckAccessRequestProto.newBuilder()
+        .setPath(path).setMode(PBHelperClient.convert(mode)).build();
+    try {
+      rpcProxy.checkAccess(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public void setStoragePolicy(String src, String policyName)
+      throws IOException {
+    SetStoragePolicyRequestProto req = SetStoragePolicyRequestProto
+        .newBuilder().setSrc(src).setPolicyName(policyName).build();
+    try {
+      rpcProxy.setStoragePolicy(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
+    GetStoragePolicyRequestProto request = GetStoragePolicyRequestProto
+        .newBuilder().setPath(path).build();
+    try {
+      return PBHelperClient.convert(rpcProxy.getStoragePolicy(null, request)
+          .getStoragePolicy());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+    try {
+      GetStoragePoliciesResponseProto response = rpcProxy
+          .getStoragePolicies(null, VOID_GET_STORAGE_POLICIES_REQUEST);
+      return PBHelperClient.convertStoragePolicies(response.getPoliciesList());
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  public long getCurrentEditLogTxid() throws IOException {
+    GetCurrentEditLogTxidRequestProto req = GetCurrentEditLogTxidRequestProto
+        .getDefaultInstance();
+    try {
+      return rpcProxy.getCurrentEditLogTxid(null, req).getTxid();
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
+  public EventBatchList getEditsFromTxid(long txid) throws IOException {
+    GetEditsFromTxidRequestProto req = GetEditsFromTxidRequestProto.newBuilder()
+        .setTxid(txid).build();
+    try {
+      return PBHelperClient.convert(rpcProxy.getEditsFromTxid(null, req));
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
new file mode 100644
index 0000000..78cd160
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
@@ -0,0 +1,55 @@
+/**
+ * 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.hdfs.server.namenode.ha;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.retry.FailoverProxyProvider;
+
+public abstract class AbstractNNFailoverProxyProvider<T> implements
+   FailoverProxyProvider <T> {
+
+  private AtomicBoolean fallbackToSimpleAuth;
+
+  /**
+   * Inquire whether logical HA URI is used for the implementation. If it is
+   * used, a special token handling may be needed to make sure a token acquired 
+   * from a node in the HA pair can be used against the other node. 
+   *
+   * @return true if logical HA URI is used. false, if not used.
+   */
+  public abstract boolean useLogicalURI(); 
+
+  /**
+   * Set for tracking if a secure client falls back to simple auth.  This method
+   * is synchronized only to stifle a Findbugs warning.
+   *
+   * @param fallbackToSimpleAuth - set to true or false during this method to
+   *   indicate if a secure client falls back to simple auth
+   */
+  public synchronized void setFallbackToSimpleAuth(
+      AtomicBoolean fallbackToSimpleAuth) {
+    this.fallbackToSimpleAuth = fallbackToSimpleAuth;
+  }
+
+  public synchronized AtomicBoolean getFallbackToSimpleAuth() {
+    return fallbackToSimpleAuth;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.java
new file mode 100644
index 0000000..0b387b7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.java
@@ -0,0 +1,71 @@
+/**
+ * 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.hdfs.server.namenode.ha;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.retry.FailoverProxyProvider;
+
+/**
+ * A NNFailoverProxyProvider implementation which wrapps old implementations
+ * directly implementing the {@link FailoverProxyProvider} interface.
+ *
+ * It is assumed that the old impelmentation is using logical URI.
+ */
+public class WrappedFailoverProxyProvider<T> extends
+    AbstractNNFailoverProxyProvider<T> {
+  private final FailoverProxyProvider<T> proxyProvider;
+  
+  /**
+   * Wrap the given instance of an old FailoverProxyProvider.
+   */
+  public WrappedFailoverProxyProvider(FailoverProxyProvider<T> provider) {
+    proxyProvider = provider;
+  }
+    
+  @Override
+  public Class<T> getInterface() {
+    return proxyProvider.getInterface();
+  }
+
+  @Override
+  public synchronized ProxyInfo<T> getProxy() {
+    return proxyProvider.getProxy();
+  }
+
+  @Override
+  public void performFailover(T currentProxy) {
+    proxyProvider.performFailover(currentProxy);
+  }
+
+  /**
+   * Close the proxy,
+   */
+  @Override
+  public synchronized void close() throws IOException {
+    proxyProvider.close();
+  }
+
+  /**
+   * Assume logical URI is used for old proxy provider implementations.
+   */
+  @Override
+  public boolean useLogicalURI() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d9f159/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index cf54cd2..0718a3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -944,6 +944,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12428. Fix inconsistency between log-level guards and statements.
     (Jagadesh Kiran N and Jackie Chang via ozawa)
 
+    HDFS-9039. Separate client and server side methods of o.a.h.hdfs.
+    NameNodeProxies. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than


[06/21] hadoop git commit: YARN-4095. Avoid sharing AllocatorPerContext object in LocalDirAllocator between ShuffleHandler and LocalDirsHandlerService. Contributed by Zhihai Xu

Posted by wa...@apache.org.
YARN-4095. Avoid sharing AllocatorPerContext object in LocalDirAllocator between ShuffleHandler and LocalDirsHandlerService. Contributed by Zhihai Xu


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

Branch: refs/heads/YARN-1197
Commit: c890c51a916894a985439497b8a44e8eee82d762
Parents: a2c76e5
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Sep 23 15:42:01 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Sep 23 15:42:01 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../nodemanager/LocalDirsHandlerService.java    | 33 +++++++++++++++++---
 .../TestLocalDirsHandlerService.java            | 18 +++++++++++
 3 files changed, 50 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c890c51a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 329e743..3cd92f6 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -460,6 +460,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12428. Fix inconsistency between log-level guards and statements.
     (Jagadesh Kiran N and Jackie Chang via ozawa)
 
+    YARN-4095. Avoid sharing AllocatorPerContext object in LocalDirAllocator
+    between ShuffleHandler and LocalDirsHandlerService. (Zhihai Xu via jlowe)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c890c51a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
index 6709c90..769044a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
@@ -30,6 +30,7 @@ import java.util.TimerTask;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileSystem;
@@ -52,6 +53,22 @@ public class LocalDirsHandlerService extends AbstractService {
 
   private static Log LOG = LogFactory.getLog(LocalDirsHandlerService.class);
 
+  /**
+   * Good local directories, use internally,
+   * initial value is the same as NM_LOCAL_DIRS.
+   */
+  @Private
+  static final String NM_GOOD_LOCAL_DIRS =
+      YarnConfiguration.NM_PREFIX + "good-local-dirs";
+
+  /**
+   * Good log directories, use internally,
+   * initial value is the same as NM_LOG_DIRS.
+   */
+  @Private
+  static final String NM_GOOD_LOG_DIRS =
+      YarnConfiguration.NM_PREFIX + "good-log-dirs";
+
   /** Timer used to schedule disk health monitoring code execution */
   private Timer dirsHandlerScheduler;
   private long diskHealthCheckInterval;
@@ -113,9 +130,17 @@ public class LocalDirsHandlerService extends AbstractService {
           new DirectoryCollection(
             validatePaths(conf.getTrimmedStrings(YarnConfiguration.NM_LOG_DIRS)),
             maxUsableSpacePercentagePerDisk, minFreeSpacePerDiskMB);
+
+      String local = conf.get(YarnConfiguration.NM_LOCAL_DIRS);
+      conf.set(NM_GOOD_LOCAL_DIRS,
+          (local != null) ? local : "");
       localDirsAllocator = new LocalDirAllocator(
-          YarnConfiguration.NM_LOCAL_DIRS);
-      logDirsAllocator = new LocalDirAllocator(YarnConfiguration.NM_LOG_DIRS);
+          NM_GOOD_LOCAL_DIRS);
+      String log = conf.get(YarnConfiguration.NM_LOG_DIRS);
+      conf.set(NM_GOOD_LOG_DIRS,
+          (log != null) ? log : "");
+      logDirsAllocator = new LocalDirAllocator(
+          NM_GOOD_LOG_DIRS);
     }
 
     @Override
@@ -373,10 +398,10 @@ public class LocalDirsHandlerService extends AbstractService {
 
     Configuration conf = getConfig();
     List<String> localDirs = getLocalDirs();
-    conf.setStrings(YarnConfiguration.NM_LOCAL_DIRS,
+    conf.setStrings(NM_GOOD_LOCAL_DIRS,
                     localDirs.toArray(new String[localDirs.size()]));
     List<String> logDirs = getLogDirs();
-    conf.setStrings(YarnConfiguration.NM_LOG_DIRS,
+    conf.setStrings(NM_GOOD_LOG_DIRS,
                       logDirs.toArray(new String[logDirs.size()]));
     if (!areDisksHealthy()) {
       // Just log.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c890c51a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLocalDirsHandlerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLocalDirsHandlerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLocalDirsHandlerService.java
index c61d1f0..e704c8f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLocalDirsHandlerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLocalDirsHandlerService.java
@@ -120,6 +120,15 @@ public class TestLocalDirsHandlerService {
     Assert.assertEquals(0, nm.getGoodLocalDirsDiskUtilizationPerc());
     Assert.assertEquals(0, nm.getGoodLogDirsDiskUtilizationPerc());
 
+    Assert.assertEquals("",
+        dirSvc.getConfig().get(LocalDirsHandlerService.NM_GOOD_LOCAL_DIRS));
+    Assert.assertEquals("",
+        dirSvc.getConfig().get(LocalDirsHandlerService.NM_GOOD_LOG_DIRS));
+    Assert.assertEquals(localDir1 + "," + localDir2,
+        dirSvc.getConfig().get(YarnConfiguration.NM_LOCAL_DIRS));
+    Assert.assertEquals(logDir1 + "," + logDir2,
+        dirSvc.getConfig().get(YarnConfiguration.NM_LOG_DIRS));
+
     conf.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE,
       100.0f);
     nm = NodeManagerMetrics.create();
@@ -141,6 +150,15 @@ public class TestLocalDirsHandlerService {
     Assert
       .assertEquals(utilizationPerc, nm.getGoodLogDirsDiskUtilizationPerc());
 
+    Assert.assertEquals(localDir2,
+        dirSvc.getConfig().get(LocalDirsHandlerService.NM_GOOD_LOCAL_DIRS));
+    Assert.assertEquals(logDir2,
+        dirSvc.getConfig().get(LocalDirsHandlerService.NM_GOOD_LOG_DIRS));
+    Assert.assertEquals(localDir1 + "," + localDir2,
+        dirSvc.getConfig().get(YarnConfiguration.NM_LOCAL_DIRS));
+    Assert.assertEquals(logDir1 + "," + logDir2,
+        dirSvc.getConfig().get(YarnConfiguration.NM_LOG_DIRS));
+
     FileUtils.deleteDirectory(new File(localDir1));
     FileUtils.deleteDirectory(new File(localDir2));
     FileUtils.deleteDirectory(new File(logDir1));


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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index dc843b9..168280a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -64,6 +64,7 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -331,11 +332,15 @@ public class TestAMRestart {
     MockAM am2 = MockRM.launchAM(app1, rm1, nm1);
     RegisterApplicationMasterResponse registerResponse =
         am2.registerAppAttempt();
-    rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
+    rm1.waitForState(am2.getApplicationAttemptId(), RMAppAttemptState.RUNNING);
 
     // check am2 get the nm token from am1.
-    Assert.assertEquals(expectedNMTokens,
-      registerResponse.getNMTokensFromPreviousAttempts());
+    Assert.assertEquals(expectedNMTokens.size(),
+        registerResponse.getNMTokensFromPreviousAttempts().size());
+    for (int i = 0; i < expectedNMTokens.size(); i++) {
+      Assert.assertTrue(expectedNMTokens.get(i)
+          .equals(registerResponse.getNMTokensFromPreviousAttempts().get(i)));
+    }
 
     // am2 allocate 1 container on nm2
     containers = new ArrayList<Container>();
@@ -365,7 +370,7 @@ public class TestAMRestart {
     // restart am
     MockAM am3 = MockRM.launchAM(app1, rm1, nm1);
     registerResponse = am3.registerAppAttempt();
-    rm1.waitForState(app1.getApplicationId(), RMAppState.RUNNING);
+    rm1.waitForState(am3.getApplicationAttemptId(), RMAppAttemptState.RUNNING);
 
     // check am3 get the NM token from both am1 and am2;
     List<NMToken> transferredTokens = registerResponse.getNMTokensFromPreviousAttempts();
@@ -430,7 +435,7 @@ public class TestAMRestart {
 
     ContainerStatus containerStatus =
         BuilderUtils.newContainerStatus(amContainer, ContainerState.COMPLETE,
-            "", ContainerExitStatus.DISKS_FAILED);
+            "", ContainerExitStatus.DISKS_FAILED, Resources.createResource(200));
     currentNode.containerStatus(containerStatus);
     am1.waitForState(RMAppAttemptState.FAILED);
     rm1.waitForState(app1.getApplicationId(), RMAppState.ACCEPTED);

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
index 9dcab2e..88c7c13 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
@@ -132,11 +132,11 @@ public class TestChildQueueOrder {
         final Resource allocatedResource = Resources.createResource(allocation);
         if (queue instanceof ParentQueue) {
           ((ParentQueue)queue).allocateResource(clusterResource, 
-              allocatedResource, RMNodeLabelsManager.NO_LABEL);
+              allocatedResource, RMNodeLabelsManager.NO_LABEL, false);
         } else {
           FiCaSchedulerApp app1 = getMockApplication(0, "");
           ((LeafQueue)queue).allocateResource(clusterResource, app1, 
-              allocatedResource, null, null);
+              allocatedResource, null, null, false);
         }
 
         // Next call - nothing

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
index fe8be06..b85c697 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
@@ -770,9 +770,9 @@ public class TestLeafQueue {
     qb.finishApplication(app_0.getApplicationId(), user_0);
     qb.finishApplication(app_2.getApplicationId(), user_1);
     qb.releaseResource(clusterResource, app_0, app_0.getResource(u0Priority),
-        null, null);
+        null, null, false);
     qb.releaseResource(clusterResource, app_2, app_2.getResource(u1Priority),
-        null, null);
+        null, null, false);
 
     qb.setUserLimit(50);
     qb.setUserLimitFactor(1);


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

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
index ef35093..4a815f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestParentQueue.java
@@ -144,11 +144,11 @@ public class TestParentQueue {
         final Resource allocatedResource = Resources.createResource(allocation);
         if (queue instanceof ParentQueue) {
           ((ParentQueue)queue).allocateResource(clusterResource, 
-              allocatedResource, RMNodeLabelsManager.NO_LABEL);
+              allocatedResource, RMNodeLabelsManager.NO_LABEL, false);
         } else {
           FiCaSchedulerApp app1 = getMockApplication(0, "");
           ((LeafQueue)queue).allocateResource(clusterResource, app1, 
-              allocatedResource, null, null);
+              allocatedResource, null, null, false);
         }
         
         // Next call - nothing

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java
index dd7ed41..daccead 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerTestBase.java
@@ -169,7 +169,7 @@ public class FairSchedulerTestBase {
     resourceManager.getRMContext().getRMApps()
         .put(id.getApplicationId(), rmApp);
 
-    scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null, null, null);
     return id;
   }
   
@@ -195,7 +195,7 @@ public class FairSchedulerTestBase {
     resourceManager.getRMContext().getRMApps()
         .put(id.getApplicationId(), rmApp);
 
-    scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(id, ask, new ArrayList<ContainerId>(), null, null, null, null);
     return id;
   }
 
@@ -217,7 +217,7 @@ public class FairSchedulerTestBase {
       ResourceRequest request, ApplicationAttemptId attId) {
     List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
     ask.add(request);
-    scheduler.allocate(attId, ask,  new ArrayList<ContainerId>(), null, null);
+    scheduler.allocate(attId, ask,  new ArrayList<ContainerId>(), null, null, null, null);
   }
 
   protected void createApplicationWithAMResource(ApplicationAttemptId attId,

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

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

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


[04/21] hadoop git commit: HDFS-8733. Keep server related definition in hdfs.proto on server side. Contributed by Mingliang Liu.

Posted by wa...@apache.org.
HDFS-8733. Keep server related definition in hdfs.proto on server side. Contributed by Mingliang Liu.


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

Branch: refs/heads/YARN-1197
Commit: 7c5c099324d9168114be2f1233d49fdb65a8c1f2
Parents: 63d9f15
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Sep 22 20:57:05 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Sep 22 20:57:05 2015 -0700

----------------------------------------------------------------------
 .../src/main/proto/hdfs.proto                   | 166 ---------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   1 +
 .../hadoop-hdfs/src/contrib/bkjournal/pom.xml   |   1 +
 .../bkjournal/src/main/proto/bkjournal.proto    |   1 +
 .../DatanodeProtocolClientSideTranslatorPB.java |   2 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   4 +-
 .../NamenodeProtocolServerSideTranslatorPB.java |   4 +-
 .../NamenodeProtocolTranslatorPB.java           |   5 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  32 +--
 .../src/main/proto/DatanodeProtocol.proto       |   1 +
 .../hadoop-hdfs/src/main/proto/HdfsServer.proto | 201 +++++++++++++++++++
 .../src/main/proto/InterDatanodeProtocol.proto  |   1 +
 .../src/main/proto/JournalProtocol.proto        |   1 +
 .../src/main/proto/NamenodeProtocol.proto       |   1 +
 .../src/main/proto/QJournalProtocol.proto       |   1 +
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |  24 +--
 17 files changed, 247 insertions(+), 202 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index ee77dc0..0e2d541 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@ -199,12 +199,6 @@ message BlockStoragePolicyProto {
   optional StorageTypesProto replicationFallbackPolicy = 5;
 }
 
-/**
- * A list of storage IDs. 
- */
-message StorageUuidsProto {
-  repeated string storageUuids = 1;
-}
 
 /**
  * A LocatedBlock gives information about a block and its location.
@@ -415,68 +409,6 @@ message SnapshotDiffReportProto {
 }
 
 /**
- * Common node information shared by all the nodes in the cluster
- */
-message StorageInfoProto {
-  required uint32 layoutVersion = 1; // Layout version of the file system
-  required uint32 namespceID = 2;    // File system namespace ID
-  required string clusterID = 3;     // ID of the cluster
-  required uint64 cTime = 4;         // File system creation time
-}
-
-/**
- * Information sent by a namenode to identify itself to the primary namenode.
- */
-message NamenodeRegistrationProto {
-  required string rpcAddress = 1;    // host:port of the namenode RPC address
-  required string httpAddress = 2;   // host:port of the namenode http server
-  enum NamenodeRoleProto {
-    NAMENODE = 1;
-    BACKUP = 2;
-    CHECKPOINT = 3;
-  }
-  required StorageInfoProto storageInfo = 3;  // Node information
-  optional NamenodeRoleProto role = 4 [default = NAMENODE];        // Namenode role
-}
-
-/**
- * Unique signature to identify checkpoint transactions.
- */
-message CheckpointSignatureProto {
-  required string blockPoolId = 1;
-  required uint64 mostRecentCheckpointTxId = 2;
-  required uint64 curSegmentTxId = 3;
-  required StorageInfoProto storageInfo = 4;
-}
-
-/**
- * Command sent from one namenode to another namenode.
- */
-message NamenodeCommandProto {
-  enum Type {
-    NamenodeCommand = 0;      // Base command
-    CheckPointCommand = 1;    // Check point command
-  }
-  required uint32 action = 1;
-  required Type type = 2;
-  optional CheckpointCommandProto checkpointCmd = 3;
-}
-
-/**
- * Command returned from primary to checkpointing namenode.
- * This command has checkpoint signature that identifies
- * checkpoint transaction and is needed for further
- * communication related to checkpointing.
- */
-message CheckpointCommandProto {
-  // Unique signature to identify checkpoint transation
-  required CheckpointSignatureProto signature = 1; 
-
-  // If true, return transfer image to primary upon the completion of checkpoint
-  required bool needToReturnImage = 2;
-}
-
-/**
  * Block information
  *
  * Please be wary of adding additional fields here, since INodeFiles
@@ -492,104 +424,6 @@ message BlockProto {
 }
 
 /**
- * Block and datanodes where is it located
- */
-message BlockWithLocationsProto {
-  required BlockProto block = 1;   // Block
-  repeated string datanodeUuids = 2; // Datanodes with replicas of the block
-  repeated string storageUuids = 3;  // Storages with replicas of the block
-  repeated StorageTypeProto storageTypes = 4;
-}
-
-/**
- * List of block with locations
- */
-message BlocksWithLocationsProto {
-  repeated BlockWithLocationsProto blocks = 1;
-}
-
-/**
- * Editlog information with available transactions
- */
-message RemoteEditLogProto {
-  required uint64 startTxId = 1;  // Starting available edit log transaction
-  required uint64 endTxId = 2;    // Ending available edit log transaction
-  optional bool isInProgress = 3 [default = false];
-}
-
-/**
- * Enumeration of editlogs available on a remote namenode
- */
-message RemoteEditLogManifestProto {
-  repeated RemoteEditLogProto logs = 1;
-}
-
-/**
- * Namespace information that describes namespace on a namenode
- */
-message NamespaceInfoProto {
-  required string buildVersion = 1;         // Software revision version (e.g. an svn or git revision)
-  required uint32 unused = 2;               // Retained for backward compatibility
-  required string blockPoolID = 3;          // block pool used by the namespace
-  required StorageInfoProto storageInfo = 4;// Node information
-  required string softwareVersion = 5;      // Software version number (e.g. 2.0.0)
-  optional uint64 capabilities = 6 [default = 0]; // feature flags
-}
-
-/**
- * Block access token information
- */
-message BlockKeyProto {
-  required uint32 keyId = 1;      // Key identifier
-  required uint64 expiryDate = 2; // Expiry time in milliseconds
-  optional bytes keyBytes = 3;    // Key secret
-}
-
-/**
- * Current key and set of block keys at the namenode.
- */
-message ExportedBlockKeysProto {
-  required bool isBlockTokenEnabled = 1;
-  required uint64 keyUpdateInterval = 2;
-  required uint64 tokenLifeTime = 3;
-  required BlockKeyProto currentKey = 4;
-  repeated BlockKeyProto allKeys = 5;
-}
-
-/**
- * State of a block replica at a datanode
- */
-enum ReplicaStateProto {
-  FINALIZED = 0;  // State of a replica when it is not modified
-  RBW = 1;        // State of replica that is being written to
-  RWR = 2;        // State of replica that is waiting to be recovered
-  RUR = 3;        // State of replica that is under recovery
-  TEMPORARY = 4;  // State of replica that is created for replication
-}
-
-/**
- * Block that needs to be recovered with at a given location
- */
-message RecoveringBlockProto {
-  required uint64 newGenStamp = 1;        // New genstamp post recovery
-  required LocatedBlockProto block = 2;   // Block to be recovered
-  optional BlockProto truncateBlock = 3;  // New block for recovery (truncate)
-}
-
-/**
- * void request
- */
-message VersionRequestProto {
-}
-
-/**
- * Version response from namenode.
- */
-message VersionResponseProto {
-  required NamespaceInfoProto info = 1;
-}
-
-/**
  * Information related to a snapshot
  * TODO: add more information
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 0718a3a..b900d91 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -947,6 +947,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-9039. Separate client and server side methods of o.a.h.hdfs.
     NameNodeProxies. (Mingliang Liu via wheat9)
 
+    HDFS-8733. Keep server related definition in hdfs.proto on server side.
+    (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index d0c2dc7..6a93331 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -340,6 +340,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
               <source>
                 <directory>${basedir}/src/main/proto</directory>
                 <includes>
+                  <include>HdfsServer.proto</include>
                   <include>DatanodeProtocol.proto</include>
                   <include>HAZKInfo.proto</include>
                   <include>InterDatanodeProtocol.proto</include>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
index 7e58606..005ee4d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
@@ -113,6 +113,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
               <imports>
                 <param>${basedir}/../../../../../hadoop-common-project/hadoop-common/src/main/proto</param>
                 <param>${basedir}/../../../../../hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto</param>
+                <param>${basedir}/../../../../../hadoop-hdfs-project/hadoop-hdfs/src/main/proto</param>
                 <param>${basedir}/src/main/proto</param>
               </imports>
               <source>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/proto/bkjournal.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/proto/bkjournal.proto b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/proto/bkjournal.proto
index c809105..15fa479 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/proto/bkjournal.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/proto/bkjournal.proto
@@ -25,6 +25,7 @@ option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
 import "hdfs.proto";
+import "HdfsServer.proto";
 
 message VersionProto {
   required int32 layoutVersion = 1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index 18f89f8..705d573 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@ -46,7 +46,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterData
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlockReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index 94d1f0c..4b9f7c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@ -46,8 +46,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlock
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionResponseProto;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
index 91ffb1b..db7a8d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs.protocolPB;
 import java.io.IOException;
 
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
index bcb96ba..6fc5fc7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java
@@ -24,9 +24,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 7c08f71..75b3811 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -45,27 +45,27 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDele
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointCommandProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeCommandProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.NamenodeRoleProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamespaceInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlockKeyProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlockWithLocationsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlocksWithLocationsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.CheckpointCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.CheckpointSignatureProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.ExportedBlockKeysProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto.NamenodeRoleProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamespaceInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RecoveringBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogManifestProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.ReplicaStateProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.StorageInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index b87e753..727259f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -33,6 +33,7 @@ package hadoop.hdfs.datanode;
 
 import "HAServiceProtocol.proto";
 import "hdfs.proto";
+import "HdfsServer.proto";
 
 /**
  * Information to identify a datanode to a namenode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
new file mode 100644
index 0000000..3b60e51
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto
@@ -0,0 +1,201 @@
+/**
+ * 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.
+ */
+
+/**
+ * These .proto interfaces are private and stable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *stable* .proto interface.
+ */
+
+// This file contains protocol buffers that are used throughout HDFS -- i.e.
+// by the client, server, and data transfer protocols.
+
+
+option java_package = "org.apache.hadoop.hdfs.protocol.proto";
+option java_outer_classname = "HdfsServerProtos";
+option java_generate_equals_and_hash = true;
+package hadoop.hdfs;
+
+import "hdfs.proto";
+
+/**
+ * A list of storage IDs.
+ */
+message StorageUuidsProto {
+  repeated string storageUuids = 1;
+}
+
+/**
+ * Block access token information
+ */
+message BlockKeyProto {
+  required uint32 keyId = 1;      // Key identifier
+  required uint64 expiryDate = 2; // Expiry time in milliseconds
+  optional bytes keyBytes = 3;    // Key secret
+}
+
+/**
+ * Current key and set of block keys at the namenode.
+ */
+message ExportedBlockKeysProto {
+  required bool isBlockTokenEnabled = 1;
+  required uint64 keyUpdateInterval = 2;
+  required uint64 tokenLifeTime = 3;
+  required BlockKeyProto currentKey = 4;
+  repeated BlockKeyProto allKeys = 5;
+}
+
+/**
+ * Block and datanodes where is it located
+ */
+message BlockWithLocationsProto {
+  required BlockProto block = 1;   // Block
+  repeated string datanodeUuids = 2; // Datanodes with replicas of the block
+  repeated string storageUuids = 3;  // Storages with replicas of the block
+  repeated StorageTypeProto storageTypes = 4;
+}
+
+/**
+ * List of block with locations
+ */
+message BlocksWithLocationsProto {
+  repeated BlockWithLocationsProto blocks = 1;
+}
+
+/**
+ * Editlog information with available transactions
+ */
+message RemoteEditLogProto {
+  required uint64 startTxId = 1;  // Starting available edit log transaction
+  required uint64 endTxId = 2;    // Ending available edit log transaction
+  optional bool isInProgress = 3 [default = false];
+}
+
+/**
+ * Enumeration of editlogs available on a remote namenode
+ */
+message RemoteEditLogManifestProto {
+  repeated RemoteEditLogProto logs = 1;
+}
+
+/**
+ * Namespace information that describes namespace on a namenode
+ */
+message NamespaceInfoProto {
+  required string buildVersion = 1;         // Software revision version (e.g. an svn or git revision)
+  required uint32 unused = 2;               // Retained for backward compatibility
+  required string blockPoolID = 3;          // block pool used by the namespace
+  required StorageInfoProto storageInfo = 4;// Node information
+  required string softwareVersion = 5;      // Software version number (e.g. 2.0.0)
+  optional uint64 capabilities = 6 [default = 0]; // feature flags
+}
+
+/**
+ * State of a block replica at a datanode
+ */
+enum ReplicaStateProto {
+  FINALIZED = 0;  // State of a replica when it is not modified
+  RBW = 1;        // State of replica that is being written to
+  RWR = 2;        // State of replica that is waiting to be recovered
+  RUR = 3;        // State of replica that is under recovery
+  TEMPORARY = 4;  // State of replica that is created for replication
+}
+
+/**
+ * Block that needs to be recovered with at a given location
+ */
+message RecoveringBlockProto {
+  required uint64 newGenStamp = 1;        // New genstamp post recovery
+  required LocatedBlockProto block = 2;   // Block to be recovered
+  optional BlockProto truncateBlock = 3;  // New block for recovery (truncate)
+}
+
+/**
+ * Unique signature to identify checkpoint transactions.
+ */
+message CheckpointSignatureProto {
+  required string blockPoolId = 1;
+  required uint64 mostRecentCheckpointTxId = 2;
+  required uint64 curSegmentTxId = 3;
+  required StorageInfoProto storageInfo = 4;
+}
+
+/**
+ * Command returned from primary to checkpointing namenode.
+ * This command has checkpoint signature that identifies
+ * checkpoint transaction and is needed for further
+ * communication related to checkpointing.
+ */
+message CheckpointCommandProto {
+  // Unique signature to identify checkpoint transation
+  required CheckpointSignatureProto signature = 1;
+
+  // If true, return transfer image to primary upon the completion of checkpoint
+  required bool needToReturnImage = 2;
+}
+
+/**
+ * Command sent from one namenode to another namenode.
+ */
+message NamenodeCommandProto {
+  enum Type {
+    NamenodeCommand = 0;      // Base command
+    CheckPointCommand = 1;    // Check point command
+  }
+  required uint32 action = 1;
+  required Type type = 2;
+  optional CheckpointCommandProto checkpointCmd = 3;
+}
+
+/**
+ * void request
+ */
+message VersionRequestProto {
+}
+
+/**
+ * Version response from namenode.
+ */
+message VersionResponseProto {
+  required NamespaceInfoProto info = 1;
+}
+
+/**
+ * Common node information shared by all the nodes in the cluster
+ */
+message StorageInfoProto {
+  required uint32 layoutVersion = 1; // Layout version of the file system
+  required uint32 namespceID = 2;    // File system namespace ID
+  required string clusterID = 3;     // ID of the cluster
+  required uint64 cTime = 4;         // File system creation time
+}
+
+/**
+ * Information sent by a namenode to identify itself to the primary namenode.
+ */
+message NamenodeRegistrationProto {
+  required string rpcAddress = 1;    // host:port of the namenode RPC address
+  required string httpAddress = 2;   // host:port of the namenode http server
+  enum NamenodeRoleProto {
+    NAMENODE = 1;
+    BACKUP = 2;
+    CHECKPOINT = 3;
+  }
+  required StorageInfoProto storageInfo = 3;  // Node information
+  optional NamenodeRoleProto role = 4 [default = NAMENODE];        // Namenode role
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto
index 1a21777..580f8d3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto
@@ -32,6 +32,7 @@ option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
 import "hdfs.proto";
+import "HdfsServer.proto";
 
 /**
  * Block with location information and new generation stamp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto
index 0de717e..3fd029b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/JournalProtocol.proto
@@ -32,6 +32,7 @@ option java_generate_equals_and_hash = true;
 package hadoop.hdfs;
 
 import "hdfs.proto";
+import "HdfsServer.proto";
 
 /**
  * Journal information used by the journal receiver to identify a journal.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
index f7c1312..d8b1e44 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/NamenodeProtocol.proto
@@ -32,6 +32,7 @@ option java_generate_equals_and_hash = true;
 package hadoop.hdfs.namenode;
 
 import "hdfs.proto";
+import "HdfsServer.proto";
 
 /**
  * Get list of blocks for a given datanode with the total length 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
index 809ee35..960a21f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
@@ -29,6 +29,7 @@ option java_generate_equals_and_hash = true;
 package hadoop.hdfs.qjournal;
 
 import "hdfs.proto";
+import "HdfsServer.proto";
 
 message JournalIdProto {
   required string identifier = 1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c5c0993/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
index 851e5b9..2bfba98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
@@ -43,23 +43,23 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommand
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlocksWithLocationsProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CheckpointSignatureProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamenodeRegistrationProto.NamenodeRoleProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.NamespaceInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlockKeyProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlockWithLocationsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlocksWithLocationsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.CheckpointSignatureProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.ExportedBlockKeysProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto.NamenodeRoleProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamespaceInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RecoveringBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogManifestProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.StorageInfoProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;


[14/21] hadoop git commit: YARN-1643. Make ContainersMonitor support changing monitoring size of an allocated container. Contributed by Meng Ding and Wangda Tan

Posted by wa...@apache.org.
YARN-1643. Make ContainersMonitor support changing monitoring size of an allocated container. Contributed by Meng Ding and Wangda Tan


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

Branch: refs/heads/YARN-1197
Commit: c59ae4eeb17e52e8fc659b9962d20628719fc621
Parents: 5f5a968
Author: Jian He <ji...@apache.org>
Authored: Wed Aug 5 15:19:33 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Sep 23 13:29:37 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../monitor/ContainersMonitorImpl.java          | 207 ++++++++++------
 .../TestContainerManagerWithLCE.java            |  11 +
 .../containermanager/TestContainerManager.java  |  96 +++++++
 .../monitor/MockResourceCalculatorPlugin.java   |  69 ++++++
 .../MockResourceCalculatorProcessTree.java      |  57 +++++
 .../TestContainersMonitorResourceChange.java    | 248 +++++++++++++++++++
 7 files changed, 615 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59ae4ee/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 309059f..3734fa6 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -212,6 +212,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3867. ContainerImpl changes to support container resizing. (Meng Ding 
     via jianhe)
 
+    YARN-1643. Make ContainersMonitor support changing monitoring size of an
+    allocated container. (Meng Ding and Wangda Tan)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59ae4ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
index afb51ad..b3839d2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
@@ -18,13 +18,11 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -32,12 +30,14 @@ import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.server.api.records.ResourceUtilization;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent;
 import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
@@ -56,16 +56,16 @@ public class ContainersMonitorImpl extends AbstractService implements
   private boolean containerMetricsEnabled;
   private long containerMetricsPeriodMs;
 
-  final List<ContainerId> containersToBeRemoved;
-  final Map<ContainerId, ProcessTreeInfo> containersToBeAdded;
-  Map<ContainerId, ProcessTreeInfo> trackingContainers =
-      new HashMap<ContainerId, ProcessTreeInfo>();
+  @VisibleForTesting
+  final Map<ContainerId, ProcessTreeInfo> trackingContainers =
+      new ConcurrentHashMap<>();
 
-  final ContainerExecutor containerExecutor;
+  private final ContainerExecutor containerExecutor;
   private final Dispatcher eventDispatcher;
   private final Context context;
   private ResourceCalculatorPlugin resourceCalculatorPlugin;
   private Configuration conf;
+  private static float vmemRatio;
   private Class<? extends ResourceCalculatorProcessTree> processTreeClass;
 
   private long maxVmemAllottedForContainers = UNKNOWN_MEMORY_LIMIT;
@@ -82,6 +82,8 @@ public class ContainersMonitorImpl extends AbstractService implements
 
   private ResourceUtilization containersUtilization;
 
+  private volatile boolean stopped = false;
+
   public ContainersMonitorImpl(ContainerExecutor exec,
       AsyncDispatcher dispatcher, Context context) {
     super("containers-monitor");
@@ -90,8 +92,6 @@ public class ContainersMonitorImpl extends AbstractService implements
     this.eventDispatcher = dispatcher;
     this.context = context;
 
-    this.containersToBeAdded = new HashMap<ContainerId, ProcessTreeInfo>();
-    this.containersToBeRemoved = new ArrayList<ContainerId>();
     this.monitoringThread = new MonitoringThread();
 
     this.containersUtilization = ResourceUtilization.newInstance(0, 0, 0.0f);
@@ -140,7 +140,7 @@ public class ContainersMonitorImpl extends AbstractService implements
     this.maxVCoresAllottedForContainers = configuredVCoresForContainers;
 
     // ///////// Virtual memory configuration //////
-    float vmemRatio = conf.getFloat(YarnConfiguration.NM_VMEM_PMEM_RATIO,
+    vmemRatio = conf.getFloat(YarnConfiguration.NM_VMEM_PMEM_RATIO,
         YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO);
     Preconditions.checkArgument(vmemRatio > 0.99f,
         YarnConfiguration.NM_VMEM_PMEM_RATIO + " should be at least 1.0");
@@ -218,6 +218,7 @@ public class ContainersMonitorImpl extends AbstractService implements
   @Override
   protected void serviceStop() throws Exception {
     if (containersMonitorEnabled) {
+      stopped = true;
       this.monitoringThread.interrupt();
       try {
         this.monitoringThread.join();
@@ -228,7 +229,8 @@ public class ContainersMonitorImpl extends AbstractService implements
     super.serviceStop();
   }
 
-  private static class ProcessTreeInfo {
+  @VisibleForTesting
+  static class ProcessTreeInfo {
     private ContainerId containerId;
     private String pid;
     private ResourceCalculatorProcessTree pTree;
@@ -267,26 +269,43 @@ public class ContainersMonitorImpl extends AbstractService implements
       this.pTree = pTree;
     }
 
-    public long getVmemLimit() {
+    /**
+     * @return Virtual memory limit for the process tree in bytes
+     */
+    public synchronized long getVmemLimit() {
       return this.vmemLimit;
     }
 
     /**
      * @return Physical memory limit for the process tree in bytes
      */
-    public long getPmemLimit() {
+    public synchronized long getPmemLimit() {
       return this.pmemLimit;
     }
 
     /**
-     * Return the number of cpu vcores assigned
-     * @return
+     * @return Number of cpu vcores assigned
      */
-    public int getCpuVcores() {
+    public synchronized int getCpuVcores() {
       return this.cpuVcores;
     }
-  }
 
+    /**
+     * Set resource limit for enforcement
+     * @param pmemLimit
+     *          Physical memory limit for the process tree in bytes
+     * @param vmemLimit
+     *          Virtual memory limit for the process tree in bytes
+     * @param cpuVcores
+     *          Number of cpu vcores assigned
+     */
+    public synchronized void setResourceLimit(
+        long pmemLimit, long vmemLimit, int cpuVcores) {
+      this.pmemLimit = pmemLimit;
+      this.vmemLimit = vmemLimit;
+      this.cpuVcores = cpuVcores;
+    }
+  }
 
   /**
    * Check whether a container's process tree's current memory usage is over
@@ -359,8 +378,7 @@ public class ContainersMonitorImpl extends AbstractService implements
     @Override
     public void run() {
 
-      while (true) {
-
+      while (!stopped && !Thread.currentThread().isInterrupted()) {
         // Print the processTrees for debugging.
         if (LOG.isDebugEnabled()) {
           StringBuilder tmp = new StringBuilder("[ ");
@@ -372,31 +390,6 @@ public class ContainersMonitorImpl extends AbstractService implements
               + tmp.substring(0, tmp.length()) + "]");
         }
 
-        // Add new containers
-        synchronized (containersToBeAdded) {
-          for (Entry<ContainerId, ProcessTreeInfo> entry : containersToBeAdded
-              .entrySet()) {
-            ContainerId containerId = entry.getKey();
-            ProcessTreeInfo processTreeInfo = entry.getValue();
-            LOG.info("Starting resource-monitoring for " + containerId);
-            trackingContainers.put(containerId, processTreeInfo);
-          }
-          containersToBeAdded.clear();
-        }
-
-        // Remove finished containers
-        synchronized (containersToBeRemoved) {
-          for (ContainerId containerId : containersToBeRemoved) {
-            if (containerMetricsEnabled) {
-              ContainerMetrics.forContainer(
-                  containerId, containerMetricsPeriodMs).finished();
-            }
-            trackingContainers.remove(containerId);
-            LOG.info("Stopping resource-monitoring for " + containerId);
-          }
-          containersToBeRemoved.clear();
-        }
-
         // Temporary structure to calculate the total resource utilization of
         // the containers
         ResourceUtilization trackedContainersUtilization  =
@@ -408,10 +401,8 @@ public class ContainersMonitorImpl extends AbstractService implements
         long pmemByAllContainers = 0;
         long cpuUsagePercentPerCoreByAllContainers = 0;
         long cpuUsageTotalCoresByAllContainers = 0;
-        for (Iterator<Map.Entry<ContainerId, ProcessTreeInfo>> it =
-            trackingContainers.entrySet().iterator(); it.hasNext();) {
-
-          Map.Entry<ContainerId, ProcessTreeInfo> entry = it.next();
+        for (Entry<ContainerId, ProcessTreeInfo> entry : trackingContainers
+            .entrySet()) {
           ContainerId containerId = entry.getKey();
           ProcessTreeInfo ptInfo = entry.getValue();
           try {
@@ -435,11 +426,6 @@ public class ContainersMonitorImpl extends AbstractService implements
                 if (containerMetricsEnabled) {
                   ContainerMetrics usageMetrics = ContainerMetrics
                       .forContainer(containerId, containerMetricsPeriodMs);
-                  int cpuVcores = ptInfo.getCpuVcores();
-                  final int vmemLimit = (int) (ptInfo.getVmemLimit() >> 20);
-                  final int pmemLimit = (int) (ptInfo.getPmemLimit() >> 20);
-                  usageMetrics.recordResourceLimit(
-                      vmemLimit, pmemLimit, cpuVcores);
                   usageMetrics.recordProcessId(pId);
                 }
               }
@@ -548,7 +534,7 @@ public class ContainersMonitorImpl extends AbstractService implements
               eventDispatcher.getEventHandler().handle(
                   new ContainerKillEvent(containerId,
                       containerExitStatus, msg));
-              it.remove();
+              trackingContainers.remove(containerId);
               LOG.info("Removed ProcessTree with root " + pId);
             }
           } catch (Exception e) {
@@ -605,6 +591,60 @@ public class ContainersMonitorImpl extends AbstractService implements
     }
   }
 
+  private void changeContainerResource(
+      ContainerId containerId, Resource resource) {
+    Container container = context.getContainers().get(containerId);
+    // Check container existence
+    if (container == null) {
+      LOG.warn("Container " + containerId.toString() + "does not exist");
+      return;
+    }
+    container.setResource(resource);
+  }
+
+  private void updateContainerMetrics(ContainersMonitorEvent monitoringEvent) {
+    if (!containerMetricsEnabled || monitoringEvent == null) {
+      return;
+    }
+
+    ContainerId containerId = monitoringEvent.getContainerId();
+    ContainerMetrics usageMetrics = ContainerMetrics
+        .forContainer(containerId, containerMetricsPeriodMs);
+
+    int vmemLimitMBs;
+    int pmemLimitMBs;
+    int cpuVcores;
+    switch (monitoringEvent.getType()) {
+    case START_MONITORING_CONTAINER:
+      ContainerStartMonitoringEvent startEvent =
+          (ContainerStartMonitoringEvent) monitoringEvent;
+      usageMetrics.recordStateChangeDurations(
+          startEvent.getLaunchDuration(),
+          startEvent.getLocalizationDuration());
+      cpuVcores = startEvent.getCpuVcores();
+      vmemLimitMBs = (int) (startEvent.getVmemLimit() >> 20);
+      pmemLimitMBs = (int) (startEvent.getPmemLimit() >> 20);
+      usageMetrics.recordResourceLimit(
+          vmemLimitMBs, pmemLimitMBs, cpuVcores);
+      break;
+    case STOP_MONITORING_CONTAINER:
+      usageMetrics.finished();
+      break;
+    case CHANGE_MONITORING_CONTAINER_RESOURCE:
+      ChangeMonitoringContainerResourceEvent changeEvent =
+          (ChangeMonitoringContainerResourceEvent) monitoringEvent;
+      Resource resource = changeEvent.getResource();
+      pmemLimitMBs = resource.getMemory();
+      vmemLimitMBs = (int) (pmemLimitMBs * vmemRatio);
+      cpuVcores = resource.getVirtualCores();
+      usageMetrics.recordResourceLimit(
+          vmemLimitMBs, pmemLimitMBs, cpuVcores);
+      break;
+    default:
+      break;
+    }
+  }
+
   @Override
   public long getVmemAllocatedForContainers() {
     return this.maxVmemAllottedForContainers;
@@ -650,38 +690,53 @@ public class ContainersMonitorImpl extends AbstractService implements
   }
 
   @Override
+  @SuppressWarnings("unchecked")
   public void handle(ContainersMonitorEvent monitoringEvent) {
-
+    ContainerId containerId = monitoringEvent.getContainerId();
     if (!containersMonitorEnabled) {
+      if (monitoringEvent.getType() == ContainersMonitorEventType
+          .CHANGE_MONITORING_CONTAINER_RESOURCE) {
+        // Nothing to enforce. Update container resource immediately.
+        ChangeMonitoringContainerResourceEvent changeEvent =
+            (ChangeMonitoringContainerResourceEvent) monitoringEvent;
+        changeContainerResource(containerId, changeEvent.getResource());
+      }
       return;
     }
 
-    ContainerId containerId = monitoringEvent.getContainerId();
     switch (monitoringEvent.getType()) {
     case START_MONITORING_CONTAINER:
       ContainerStartMonitoringEvent startEvent =
           (ContainerStartMonitoringEvent) monitoringEvent;
-
-      if (containerMetricsEnabled) {
-        ContainerMetrics usageMetrics = ContainerMetrics
-            .forContainer(containerId, containerMetricsPeriodMs);
-        usageMetrics.recordStateChangeDurations(
-            startEvent.getLaunchDuration(),
-            startEvent.getLocalizationDuration());
-      }
-
-      synchronized (this.containersToBeAdded) {
-        ProcessTreeInfo processTreeInfo =
-            new ProcessTreeInfo(containerId, null, null,
-                startEvent.getVmemLimit(), startEvent.getPmemLimit(),
-                startEvent.getCpuVcores());
-        this.containersToBeAdded.put(containerId, processTreeInfo);
-      }
+      LOG.info("Starting resource-monitoring for " + containerId);
+      updateContainerMetrics(monitoringEvent);
+      trackingContainers.put(containerId,
+          new ProcessTreeInfo(containerId, null, null,
+              startEvent.getVmemLimit(), startEvent.getPmemLimit(),
+              startEvent.getCpuVcores()));
       break;
     case STOP_MONITORING_CONTAINER:
-      synchronized (this.containersToBeRemoved) {
-        this.containersToBeRemoved.add(containerId);
+      LOG.info("Stopping resource-monitoring for " + containerId);
+      updateContainerMetrics(monitoringEvent);
+      trackingContainers.remove(containerId);
+      break;
+    case CHANGE_MONITORING_CONTAINER_RESOURCE:
+      ChangeMonitoringContainerResourceEvent changeEvent =
+          (ChangeMonitoringContainerResourceEvent) monitoringEvent;
+      ProcessTreeInfo processTreeInfo = trackingContainers.get(containerId);
+      if (processTreeInfo == null) {
+        LOG.warn("Failed to track container "
+            + containerId.toString()
+            + ". It may have already completed.");
+        break;
       }
+      LOG.info("Changing resource-monitoring for " + containerId);
+      updateContainerMetrics(monitoringEvent);
+      long pmemLimit = changeEvent.getResource().getMemory() * 1024L * 1024L;
+      long vmemLimit = (long) (pmemLimit * vmemRatio);
+      int cpuVcores = changeEvent.getResource().getVirtualCores();
+      processTreeInfo.setResourceLimit(pmemLimit, vmemLimit, cpuVcores);
+      changeContainerResource(containerId, changeEvent.getResource());
       break;
     default:
       // TODO: Wrong event.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59ae4ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
index 9a05278..75bcdae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
@@ -211,6 +211,17 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
     super.testIncreaseContainerResourceWithInvalidResource();
   }
 
+  @Override
+  public void testChangeContainerResource() throws Exception {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testChangeContainerResource");
+    super.testChangeContainerResource();
+  }
+
   private boolean shouldRunTest() {
     return System
         .getProperty(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH) != null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59ae4ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index e2f12ba..2ea9146 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -1046,6 +1046,102 @@ public class TestContainerManager extends BaseContainerManagerTest {
     }
   }
 
+  @Test
+  public void testChangeContainerResource() throws Exception {
+    containerManager.start();
+    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
+    PrintWriter fileWriter = new PrintWriter(scriptFile);
+    // Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    if (Shell.WINDOWS) {
+      fileWriter.println("@ping -n 100 127.0.0.1 >nul");
+    } else {
+      fileWriter.write("\numask 0");
+      fileWriter.write("\nexec sleep 100");
+    }
+    fileWriter.close();
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+    URL resource_alpha =
+        ConverterUtils.getYarnUrlFromPath(localFS
+            .makeQualified(new Path(scriptFile.getAbsolutePath())));
+    LocalResource rsrc_alpha =
+        recordFactory.newRecordInstance(LocalResource.class);
+    rsrc_alpha.setResource(resource_alpha);
+    rsrc_alpha.setSize(-1);
+    rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
+    rsrc_alpha.setType(LocalResourceType.FILE);
+    rsrc_alpha.setTimestamp(scriptFile.lastModified());
+    String destinationFile = "dest_file";
+    Map<String, LocalResource> localResources =
+        new HashMap<String, LocalResource>();
+    localResources.put(destinationFile, rsrc_alpha);
+    containerLaunchContext.setLocalResources(localResources);
+    List<String> commands =
+        Arrays.asList(Shell.getRunScriptCommand(scriptFile));
+    containerLaunchContext.setCommands(commands);
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+            containerLaunchContext,
+                createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+                    context.getNodeId(), user,
+                        context.getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+    // Make sure the container reaches RUNNING state
+    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
+        org.apache.hadoop.yarn.server.nodemanager.
+            containermanager.container.ContainerState.RUNNING);
+    // Construct container resource increase request,
+    List<Token> increaseTokens = new ArrayList<Token>();
+    // Add increase request.
+    Resource targetResource = Resource.newInstance(4096, 2);
+    Token containerToken = createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+        context.getNodeId(), user, targetResource,
+            context.getContainerTokenSecretManager(), null);
+    increaseTokens.add(containerToken);
+    IncreaseContainersResourceRequest increaseRequest =
+        IncreaseContainersResourceRequest.newInstance(increaseTokens);
+    IncreaseContainersResourceResponse increaseResponse =
+        containerManager.increaseContainersResource(increaseRequest);
+    Assert.assertEquals(
+        1, increaseResponse.getSuccessfullyIncreasedContainers().size());
+    Assert.assertTrue(increaseResponse.getFailedRequests().isEmpty());
+    // Check status
+    List<ContainerId> containerIds = new ArrayList<>();
+    containerIds.add(cId);
+    GetContainerStatusesRequest gcsRequest =
+        GetContainerStatusesRequest.newInstance(containerIds);
+    ContainerStatus containerStatus = containerManager
+        .getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
+    // Check status immediately as resource increase is blocking
+    assertEquals(targetResource, containerStatus.getCapability());
+    // Simulate a decrease request
+    List<org.apache.hadoop.yarn.api.records.Container> containersToDecrease
+        = new ArrayList<>();
+    targetResource = Resource.newInstance(2048, 2);
+    org.apache.hadoop.yarn.api.records.Container decreasedContainer =
+        org.apache.hadoop.yarn.api.records.Container
+            .newInstance(cId, null, null, targetResource, null, null);
+    containersToDecrease.add(decreasedContainer);
+    containerManager.handle(
+        new CMgrDecreaseContainersResourceEvent(containersToDecrease));
+    // Check status with retry
+    containerStatus = containerManager
+        .getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
+    int retry = 0;
+    while (!targetResource.equals(containerStatus.getCapability()) &&
+        (retry++ < 5)) {
+      Thread.sleep(200);
+      containerStatus = containerManager.getContainerStatuses(gcsRequest)
+          .getContainerStatuses().get(0);
+    }
+    assertEquals(targetResource, containerStatus.getCapability());
+  }
+
   public static Token createContainerToken(ContainerId cId, long rmIdentifier,
       NodeId nodeId, String user,
       NMContainerTokenSecretManager containerTokenSecretManager)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59ae4ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorPlugin.java
new file mode 100644
index 0000000..4a18a8c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorPlugin.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
+
+import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
+
+public class MockResourceCalculatorPlugin extends ResourceCalculatorPlugin {
+
+  @Override
+  public long getVirtualMemorySize() {
+    return 0;
+  }
+
+  @Override
+  public long getPhysicalMemorySize() {
+    return 0;
+  }
+
+  @Override
+  public long getAvailableVirtualMemorySize() {
+    return 0;
+  }
+
+  @Override
+  public long getAvailablePhysicalMemorySize() {
+    return 0;
+  }
+
+  @Override
+  public int getNumProcessors() {
+    return 0;
+  }
+
+  @Override
+  public int getNumCores() {
+    return  0;
+  }
+
+  @Override
+  public long getCpuFrequency() {
+    return 0;
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    return 0;
+  }
+
+  @Override
+  public float getCpuUsage() {
+    return 0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59ae4ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.java
new file mode 100644
index 0000000..c5aaa77
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/MockResourceCalculatorProcessTree.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
+
+import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
+
+public class MockResourceCalculatorProcessTree extends ResourceCalculatorProcessTree {
+
+  private long rssMemorySize = 0;
+
+  public MockResourceCalculatorProcessTree(String root) {
+    super(root);
+  }
+
+  @Override
+  public void updateProcessTree() {
+  }
+
+  @Override
+  public String getProcessTreeDump() {
+    return "";
+  }
+
+  @Override
+  public long getCumulativeCpuTime() {
+    return 0;
+  }
+
+  @Override
+  public boolean checkPidPgrpidForMatch() {
+    return true;
+  }
+
+  public void setRssMemorySize(long rssMemorySize) {
+    this.rssMemorySize = rssMemorySize;
+  }
+
+  public long getRssMemorySize() {
+    return this.rssMemorySize;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59ae4ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java
new file mode 100644
index 0000000..d7f89fc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitorResourceChange.java
@@ -0,0 +1,248 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl.ProcessTreeInfo;
+import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerLivenessContext;
+import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext;
+import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
+import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext;
+import org.apache.hadoop.yarn.server.nodemanager.executor.LocalizerStartContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+
+public class TestContainersMonitorResourceChange {
+
+  private ContainersMonitorImpl containersMonitor;
+  private MockExecutor executor;
+  private Configuration conf;
+  private AsyncDispatcher dispatcher;
+  private Context context;
+  private MockContainerEventHandler containerEventHandler;
+
+  private static class MockExecutor extends ContainerExecutor {
+    @Override
+    public void init() throws IOException {
+    }
+    @Override
+    public void startLocalizer(LocalizerStartContext ctx)
+        throws IOException, InterruptedException {
+    }
+    @Override
+    public int launchContainer(ContainerStartContext ctx) throws
+        IOException {
+      return 0;
+    }
+    @Override
+    public boolean signalContainer(ContainerSignalContext ctx)
+        throws IOException {
+      return true;
+    }
+    @Override
+    public void deleteAsUser(DeletionAsUserContext ctx)
+        throws IOException, InterruptedException {
+    }
+    @Override
+    public String getProcessId(ContainerId containerId) {
+      return String.valueOf(containerId.getContainerId());
+    }
+    @Override
+    public boolean isContainerAlive(ContainerLivenessContext ctx)
+        throws IOException {
+      return true;
+    }
+  }
+
+  private static class MockContainerEventHandler implements
+      EventHandler<ContainerEvent> {
+    final private Set<ContainerId> killedContainer
+        = new HashSet<>();
+    @Override
+    public void handle(ContainerEvent event) {
+      if (event.getType() == ContainerEventType.KILL_CONTAINER) {
+        synchronized (killedContainer) {
+          killedContainer.add(event.getContainerID());
+        }
+      }
+    }
+    public boolean isContainerKilled(ContainerId containerId) {
+      synchronized (killedContainer) {
+        return killedContainer.contains(containerId);
+      }
+    }
+  }
+
+  @Before
+  public void setup() {
+    executor = new MockExecutor();
+    dispatcher = new AsyncDispatcher();
+    context = Mockito.mock(Context.class);
+    Mockito.doReturn(new ConcurrentSkipListMap<ContainerId, Container>())
+        .when(context).getContainers();
+    conf = new Configuration();
+    conf.set(
+        YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR,
+        MockResourceCalculatorPlugin.class.getCanonicalName());
+    conf.set(
+        YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE,
+        MockResourceCalculatorProcessTree.class.getCanonicalName());
+    dispatcher.init(conf);
+    dispatcher.start();
+    containerEventHandler = new MockContainerEventHandler();
+    dispatcher.register(ContainerEventType.class, containerEventHandler);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (containersMonitor != null) {
+      containersMonitor.stop();
+    }
+    if (dispatcher != null) {
+      dispatcher.stop();
+    }
+  }
+
+  @Test
+  public void testContainersResourceChange() throws Exception {
+    // set container monitor interval to be 20ms
+    conf.setLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS, 20L);
+    containersMonitor = createContainersMonitor(executor, dispatcher, context);
+    containersMonitor.init(conf);
+    containersMonitor.start();
+    // create container 1
+    containersMonitor.handle(new ContainerStartMonitoringEvent(
+        getContainerId(1), 2100L, 1000L, 1, 0, 0));
+    // verify that this container is properly tracked
+    assertNotNull(getProcessTreeInfo(getContainerId(1)));
+    assertEquals(1000L, getProcessTreeInfo(getContainerId(1))
+        .getPmemLimit());
+    assertEquals(2100L, getProcessTreeInfo(getContainerId(1))
+        .getVmemLimit());
+    // sleep longer than the monitor interval to make sure resource
+    // enforcement has started
+    Thread.sleep(200);
+    // increase pmem usage, the container should be killed
+    MockResourceCalculatorProcessTree mockTree =
+        (MockResourceCalculatorProcessTree) getProcessTreeInfo(
+            getContainerId(1)).getProcessTree();
+    mockTree.setRssMemorySize(2500L);
+    // verify that this container is killed
+    Thread.sleep(200);
+    assertTrue(containerEventHandler
+        .isContainerKilled(getContainerId(1)));
+    // create container 2
+    containersMonitor.handle(new ContainerStartMonitoringEvent(
+        getContainerId(2), 2202009L, 1048576L, 1, 0, 0));
+    // verify that this container is properly tracked
+    assertNotNull(getProcessTreeInfo(getContainerId(2)));
+    assertEquals(1048576L, getProcessTreeInfo(getContainerId(2))
+        .getPmemLimit());
+    assertEquals(2202009L, getProcessTreeInfo(getContainerId(2))
+        .getVmemLimit());
+    // trigger a change resource event, check limit after change
+    containersMonitor.handle(new ChangeMonitoringContainerResourceEvent(
+        getContainerId(2), Resource.newInstance(2, 1)));
+    assertEquals(2097152L, getProcessTreeInfo(getContainerId(2))
+        .getPmemLimit());
+    assertEquals(4404019L, getProcessTreeInfo(getContainerId(2))
+        .getVmemLimit());
+    // sleep longer than the monitor interval to make sure resource
+    // enforcement has started
+    Thread.sleep(200);
+    // increase pmem usage, the container should NOT be killed
+    mockTree =
+        (MockResourceCalculatorProcessTree) getProcessTreeInfo(
+            getContainerId(2)).getProcessTree();
+    mockTree.setRssMemorySize(2000000L);
+    // verify that this container is not killed
+    Thread.sleep(200);
+    assertFalse(containerEventHandler
+        .isContainerKilled(getContainerId(2)));
+    containersMonitor.stop();
+  }
+
+  @Test
+  public void testContainersResourceChangeIsTriggeredImmediately()
+      throws Exception {
+    // set container monitor interval to be 20s
+    conf.setLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS, 20000L);
+    containersMonitor = createContainersMonitor(executor, dispatcher, context);
+    containersMonitor.init(conf);
+    containersMonitor.start();
+    // sleep 1 second to make sure the container monitor thread is
+    // now waiting for the next monitor cycle
+    Thread.sleep(1000);
+    // create a container with id 3
+    containersMonitor.handle(new ContainerStartMonitoringEvent(
+        getContainerId(3), 2202009L, 1048576L, 1, 0, 0));
+    // Verify that this container has been tracked
+    assertNotNull(getProcessTreeInfo(getContainerId(3)));
+    // trigger a change resource event, check limit after change
+    containersMonitor.handle(new ChangeMonitoringContainerResourceEvent(
+        getContainerId(3), Resource.newInstance(2, 1)));
+    // verify that this container has been properly tracked with the
+    // correct size
+    assertEquals(2097152L, getProcessTreeInfo(getContainerId(3))
+        .getPmemLimit());
+    assertEquals(4404019L, getProcessTreeInfo(getContainerId(3))
+        .getVmemLimit());
+    containersMonitor.stop();
+  }
+
+  private ContainersMonitorImpl createContainersMonitor(
+      ContainerExecutor containerExecutor, AsyncDispatcher dispatcher,
+      Context context) {
+    return new ContainersMonitorImpl(containerExecutor, dispatcher, context);
+  }
+
+  private ContainerId getContainerId(int id) {
+    return ContainerId.newContainerId(ApplicationAttemptId.newInstance(
+        ApplicationId.newInstance(123456L, 1), 1), id);
+  }
+
+  private ProcessTreeInfo getProcessTreeInfo(ContainerId id) {
+    return containersMonitor.trackingContainers.get(id);
+  }
+}


[10/21] hadoop git commit: YARN-3866. AM-RM protocol changes to support container resizing. Contributed by Meng Ding

Posted by wa...@apache.org.
YARN-3866. AM-RM protocol changes to support container resizing. Contributed by Meng Ding


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

Branch: refs/heads/YARN-1197
Commit: dfe2cb849fcb273d32e840d8562eaaee904af054
Parents: 692d51c
Author: Jian He <ji...@apache.org>
Authored: Mon Jul 13 17:34:26 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Sep 23 13:29:36 2015 -0700

----------------------------------------------------------------------
 .../app/local/TestLocalContainerAllocator.java  |   6 +-
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../api/protocolrecords/AllocateRequest.java    |  57 ++++--
 .../api/protocolrecords/AllocateResponse.java   |  42 +++--
 .../records/ContainerResourceChangeRequest.java | 117 +++++++++++++
 .../api/records/ContainerResourceDecrease.java  |  78 ---------
 .../api/records/ContainerResourceIncrease.java  |  84 ---------
 .../ContainerResourceIncreaseRequest.java       |  80 ---------
 .../yarn/api/records/ContainerStatus.java       |  13 ++
 .../src/main/proto/yarn_protos.proto            |  14 +-
 .../src/main/proto/yarn_service_protos.proto    |  16 +-
 .../impl/pb/AllocateRequestPBImpl.java          | 119 ++++++++++---
 .../impl/pb/AllocateResponsePBImpl.java         | 175 +++++--------------
 .../ContainerResourceChangeRequestPBImpl.java   | 141 +++++++++++++++
 .../pb/ContainerResourceDecreasePBImpl.java     | 136 --------------
 .../pb/ContainerResourceIncreasePBImpl.java     | 171 ------------------
 .../ContainerResourceIncreaseRequestPBImpl.java | 141 ---------------
 .../records/impl/pb/ContainerStatusPBImpl.java  |  31 +++-
 .../hadoop/yarn/api/TestAllocateRequest.java    |  73 --------
 .../hadoop/yarn/api/TestAllocateResponse.java   | 114 ------------
 .../yarn/api/TestContainerResourceDecrease.java |  66 -------
 .../yarn/api/TestContainerResourceIncrease.java |  74 --------
 .../TestContainerResourceIncreaseRequest.java   |  68 -------
 .../hadoop/yarn/api/TestPBImplRecords.java      |  34 +---
 24 files changed, 534 insertions(+), 1319 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
index f901ed8..167d804 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
@@ -46,8 +46,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 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.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;
@@ -254,8 +252,8 @@ public class TestLocalContainerAllocator {
           Resources.none(), null, 1, null,
           Collections.<NMToken>emptyList(),
           yarnToken,
-          Collections.<ContainerResourceIncrease>emptyList(),
-          Collections.<ContainerResourceDecrease>emptyList());
+          Collections.<Container>emptyList(),
+          Collections.<Container>emptyList());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e461fe3..7e4edce 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -200,6 +200,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3212. RMNode State Transition Update with DECOMMISSIONING state.
     (Junping Du via wangda)
 
+    YARN-3866. AM-RM protocol changes to support container resizing. (Meng Ding
+    via jianhe)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/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 2458d9b..0b65e5c 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
@@ -22,11 +22,12 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
+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.ResourceBlacklistRequest;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
+import org.apache.hadoop.yarn.api.records.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.util.Records;
 
@@ -46,6 +47,16 @@ import org.apache.hadoop.yarn.util.Records;
  *   <li>
  *     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
+ *     requirements of running containers.
+ *   </li>
  * </ul>
  * 
  * @see ApplicationMasterProtocol#allocate(AllocateRequest)
@@ -61,7 +72,7 @@ public abstract class AllocateRequest {
       List<ContainerId> containersToBeReleased,
       ResourceBlacklistRequest resourceBlacklistRequest) {
     return newInstance(responseID, appProgress, resourceAsk,
-        containersToBeReleased, resourceBlacklistRequest, null);
+        containersToBeReleased, resourceBlacklistRequest, null, null);
   }
   
   @Public
@@ -70,7 +81,8 @@ public abstract class AllocateRequest {
       List<ResourceRequest> resourceAsk,
       List<ContainerId> containersToBeReleased,
       ResourceBlacklistRequest resourceBlacklistRequest,
-      List<ContainerResourceIncreaseRequest> increaseRequests) {
+      List<ContainerResourceChangeRequest> increaseRequests,
+      List<ContainerResourceChangeRequest> decreaseRequests) {
     AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
     allocateRequest.setResponseId(responseID);
     allocateRequest.setProgress(appProgress);
@@ -78,6 +90,7 @@ public abstract class AllocateRequest {
     allocateRequest.setReleaseList(containersToBeReleased);
     allocateRequest.setResourceBlacklistRequest(resourceBlacklistRequest);
     allocateRequest.setIncreaseRequests(increaseRequests);
+    allocateRequest.setDecreaseRequests(decreaseRequests);
     return allocateRequest;
   }
   
@@ -184,20 +197,38 @@ public abstract class AllocateRequest {
       ResourceBlacklistRequest resourceBlacklistRequest);
   
   /**
-   * Get the <code>ContainerResourceIncreaseRequest</code> being sent by the
-   * <code>ApplicationMaster</code>
+   * Get the list of container resource increase requests being sent by the
+   * <code>ApplicationMaster</code>.
    */
   @Public
-  @Stable
-  public abstract List<ContainerResourceIncreaseRequest> getIncreaseRequests();
-  
+  @Unstable
+  public abstract List<ContainerResourceChangeRequest> getIncreaseRequests();
+
   /**
-   * Set the <code>ContainerResourceIncreaseRequest</code> to inform the
-   * <code>ResourceManager</code> about some container's resources need to be
-   * increased
+   * Set the list of container resource increase requests to inform the
+   * <code>ResourceManager</code> about the containers whose resources need
+   * to be increased.
    */
   @Public
-  @Stable
+  @Unstable
   public abstract void setIncreaseRequests(
-      List<ContainerResourceIncreaseRequest> increaseRequests);
+      List<ContainerResourceChangeRequest> increaseRequests);
+
+  /**
+   * Get the list of container resource decrease requests being sent by the
+   * <code>ApplicationMaster</code>.
+   */
+  @Public
+  @Unstable
+  public abstract List<ContainerResourceChangeRequest> getDecreaseRequests();
+
+  /**
+   * Set the list of container resource decrease requests to inform the
+   * <code>ResourceManager</code> about the containers whose resources need
+   * to be decreased.
+   */
+  @Public
+  @Unstable
+  public abstract void setDecreaseRequests(
+      List<ContainerResourceChangeRequest> decreaseRequests);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/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 c4fdb79..c363070 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
@@ -28,8 +28,6 @@ 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;
@@ -59,6 +57,14 @@ import org.apache.hadoop.yarn.util.Records;
  *   <li>The number of available nodes in a cluster.</li>
  *   <li>A description of resources requested back by the cluster</li>
  *   <li>AMRMToken, if AMRMToken has been rolled over</li>
+ *   <li>
+ *     A list of {@link Container} representing the containers
+ *     whose resource has been increased.
+ *   </li>
+ *   <li>
+ *     A list of {@link Container} representing the containers
+ *     whose resource has been decreased.
+ *   </li>
  * </ul>
  * 
  * @see ApplicationMasterProtocol#allocate(AllocateRequest)
@@ -94,8 +100,8 @@ public abstract class AllocateResponse {
       List<Container> allocatedContainers, List<NodeReport> updatedNodes,
       Resource availResources, AMCommand command, int numClusterNodes,
       PreemptionMessage preempt, List<NMToken> nmTokens,
-      List<ContainerResourceIncrease> increasedContainers,
-      List<ContainerResourceDecrease> decreasedContainers) {
+      List<Container> increasedContainers,
+      List<Container> decreasedContainers) {
     AllocateResponse response = newInstance(responseId, completedContainers,
         allocatedContainers, updatedNodes, availResources, command,
         numClusterNodes, preempt, nmTokens);
@@ -111,8 +117,8 @@ public abstract class AllocateResponse {
       List<Container> allocatedContainers, List<NodeReport> updatedNodes,
       Resource availResources, AMCommand command, int numClusterNodes,
       PreemptionMessage preempt, List<NMToken> nmTokens, Token amRMToken,
-      List<ContainerResourceIncrease> increasedContainers,
-      List<ContainerResourceDecrease> decreasedContainers) {
+      List<Container> increasedContainers,
+      List<Container> decreasedContainers) {
     AllocateResponse response =
         newInstance(responseId, completedContainers, allocatedContainers,
           updatedNodes, availResources, command, numClusterNodes, preempt,
@@ -263,34 +269,38 @@ public abstract class AllocateResponse {
   public abstract void setNMTokens(List<NMToken> nmTokens);
   
   /**
-   * Get the list of newly increased containers by <code>ResourceManager</code>
+   * Get the list of newly increased containers by
+   * <code>ResourceManager</code>.
    */
   @Public
-  @Stable
-  public abstract List<ContainerResourceIncrease> getIncreasedContainers();
+  @Unstable
+  public abstract List<Container> getIncreasedContainers();
 
   /**
-   * Set the list of newly increased containers by <code>ResourceManager</code>
+   * Set the list of newly increased containers by
+   * <code>ResourceManager</code>.
    */
   @Private
   @Unstable
   public abstract void setIncreasedContainers(
-      List<ContainerResourceIncrease> increasedContainers);
+      List<Container> increasedContainers);
 
   /**
-   * Get the list of newly decreased containers by <code>NodeManager</code>
+   * Get the list of newly decreased containers by
+   * <code>ResourceManager</code>.
    */
   @Public
-  @Stable
-  public abstract List<ContainerResourceDecrease> getDecreasedContainers();
+  @Unstable
+  public abstract List<Container> getDecreasedContainers();
 
   /**
-   * Set the list of newly decreased containers by <code>NodeManager</code>
+   * Set the list of newly decreased containers by
+   * <code>ResourceManager</code>.
    */
   @Private
   @Unstable
   public abstract void setDecreasedContainers(
-      List<ContainerResourceDecrease> decreasedContainers);
+      List<Container> decreasedContainers);
 
   /**
    * The AMRMToken that belong to this attempt

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceChangeRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceChangeRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceChangeRequest.java
new file mode 100644
index 0000000..117015b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceChangeRequest.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;
+
+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;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/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
deleted file mode 100644
index d766d92..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceDecrease.java
+++ /dev/null
@@ -1,78 +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.yarn.util.Records;
-
-/**
- * Used by Application Master to ask Node Manager reduce size of a specified
- * container
- */
-public abstract class ContainerResourceDecrease {
-  @Public
-  public static ContainerResourceDecrease newInstance(
-      ContainerId existingContainerId, Resource targetCapability) {
-    ContainerResourceDecrease context = Records
-        .newRecord(ContainerResourceDecrease.class);
-    context.setContainerId(existingContainerId);
-    context.setCapability(targetCapability);
-    return context;
-  }
-
-  @Public
-  public abstract ContainerId getContainerId();
-
-  @Public
-  public abstract void setContainerId(ContainerId containerId);
-
-  @Public
-  public abstract Resource getCapability();
-
-  @Public
-  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/dfe2cb84/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
deleted file mode 100644
index f4c1560..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncrease.java
+++ /dev/null
@@ -1,84 +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.yarn.util.Records;
-
-/**
- * Represent a new increased container accepted by Resource Manager
- */
-public abstract class ContainerResourceIncrease {
-  @Public
-  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
-  public abstract ContainerId getContainerId();
-
-  @Public
-  public abstract void setContainerId(ContainerId containerId);
-
-  @Public
-  public abstract Resource getCapability();
-
-  @Public
-  public abstract void setCapability(Resource capability);
-  
-  @Public
-  public abstract Token getContainerToken();
-
-  @Public
-  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/dfe2cb84/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
deleted file mode 100644
index 9e3b640..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerResourceIncreaseRequest.java
+++ /dev/null
@@ -1,80 +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.yarn.util.Records;
-
-/**
- * Used by Application Master, send a container resource increase request to
- * Resource Manager
- */
-@Public
-public abstract class ContainerResourceIncreaseRequest {
-  @Public
-  public static ContainerResourceIncreaseRequest newInstance(
-      ContainerId existingContainerId, Resource targetCapability) {
-    ContainerResourceIncreaseRequest context = Records
-        .newRecord(ContainerResourceIncreaseRequest.class);
-    context.setContainerId(existingContainerId);
-    context.setCapability(targetCapability);
-    return context;
-  }
-
-  @Public
-  public abstract ContainerId getContainerId();
-
-  @Public
-  public abstract void setContainerId(ContainerId containerId);
-
-  @Public
-  public abstract Resource getCapability();
-
-  @Public
-  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/dfe2cb84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
index 5ccf6dc..2c2238f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerStatus.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.util.Records;
  *   <li>{@code ContainerState} of the container.</li>
  *   <li><em>Exit status</em> of a completed container.</li>
  *   <li><em>Diagnostic</em> message for a failed container.</li>
+ *   <li>{@link Resource} allocated to the container.</li>
  * </ul>
  */
 @Public
@@ -114,4 +115,16 @@ public abstract class ContainerStatus {
   @Private
   @Unstable
   public abstract void setDiagnostics(String diagnostics);
+
+  /**
+   * Get the <code>Resource</code> allocated to the container.
+   * @return <code>Resource</code> allocated to the container
+   */
+  @Public
+  @Unstable
+  public abstract Resource getCapability();
+
+  @Private
+  @Unstable
+  public abstract void setCapability(Resource capability);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/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 0bccfc4..057aeee 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
@@ -470,6 +470,7 @@ message ContainerStatusProto {
   optional ContainerStateProto state = 2;
   optional string diagnostics = 3 [default = "N/A"];
   optional int32 exit_status = 4 [default = -1000];
+  optional ResourceProto capability = 5;
 }
 
 enum ContainerExitStatusProto {
@@ -479,22 +480,11 @@ enum ContainerExitStatusProto {
   DISKS_FAILED = -101;
 }
 
-message ContainerResourceIncreaseRequestProto {
+message ContainerResourceChangeRequestProto {
   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;
-}
-
 ////////////////////////////////////////////////////////////////////////
 ////// From common//////////////////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/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 b0b12d1..ff5a127 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
@@ -66,7 +66,8 @@ message AllocateRequestProto {
   optional ResourceBlacklistRequestProto blacklist_request = 3;
   optional int32 response_id = 4;
   optional float progress = 5;
-  repeated ContainerResourceIncreaseRequestProto increase_request = 6;
+  repeated ContainerResourceChangeRequestProto increase_request = 6;
+  repeated ContainerResourceChangeRequestProto decrease_request = 7;
 }
 
 message NMTokenProto {
@@ -84,8 +85,8 @@ message AllocateResponseProto {
   optional int32 num_cluster_nodes = 7;
   optional PreemptionMessageProto preempt = 8;
   repeated NMTokenProto nm_tokens = 9;
-  repeated ContainerResourceIncreaseProto increased_containers = 10;
-  repeated ContainerResourceDecreaseProto decreased_containers = 11;
+  repeated ContainerProto increased_containers = 10;
+  repeated ContainerProto decreased_containers = 11;
   optional hadoop.common.TokenProto am_rm_token = 12;
 }
 
@@ -286,6 +287,15 @@ message GetContainerStatusesResponseProto {
   repeated ContainerExceptionMapProto failed_requests = 2;
 }
 
+message IncreaseContainersResourceRequestProto {
+  repeated hadoop.common.TokenProto increase_containers = 1;
+}
+
+message IncreaseContainersResourceResponseProto {
+  repeated ContainerIdProto succeeded_requests = 1;
+  repeated ContainerExceptionMapProto failed_requests = 2;
+}
+
 //////////////////////////////////////////////////////
 /////// Application_History_Protocol /////////////////
 //////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/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 dc11165..d6db32c 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,15 +27,15 @@ 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.ContainerResourceChangeRequest;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 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.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.proto.YarnProtos.ContainerIdProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto;
+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.AllocateRequestProto;
@@ -52,7 +52,8 @@ public class AllocateRequestPBImpl extends AllocateRequest {
 
   private List<ResourceRequest> ask = null;
   private List<ContainerId> release = null;
-  private List<ContainerResourceIncreaseRequest> increaseRequests = null;
+  private List<ContainerResourceChangeRequest> increaseRequests = null;
+  private List<ContainerResourceChangeRequest> decreaseRequests = null;
   private ResourceBlacklistRequest blacklistRequest = null;
   
   public AllocateRequestPBImpl() {
@@ -101,6 +102,9 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     if (this.increaseRequests != null) {
       addIncreaseRequestsToProto();
     }
+    if (this.decreaseRequests != null) {
+      addDecreaseRequestsToProto();
+    }
     if (this.blacklistRequest != null) {
       builder.setBlacklistRequest(convertToProtoFormat(this.blacklistRequest));
     }
@@ -162,14 +166,14 @@ public class AllocateRequestPBImpl extends AllocateRequest {
   }
   
   @Override
-  public List<ContainerResourceIncreaseRequest> getIncreaseRequests() {
+  public List<ContainerResourceChangeRequest> getIncreaseRequests() {
     initIncreaseRequests();
     return this.increaseRequests;
   }
 
   @Override
   public void setIncreaseRequests(
-      List<ContainerResourceIncreaseRequest> increaseRequests) {
+      List<ContainerResourceChangeRequest> increaseRequests) {
     if (increaseRequests == null) {
       return;
     }
@@ -177,7 +181,24 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     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);
+  }
+
   @Override
   public ResourceBlacklistRequest getResourceBlacklistRequest() {
     AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
@@ -252,28 +273,42 @@ public class AllocateRequestPBImpl extends AllocateRequest {
       return;
     }
     AllocateRequestProtoOrBuilder p = viaProto ? proto : builder;
-    List<ContainerResourceIncreaseRequestProto> list =
+    List<ContainerResourceChangeRequestProto> list =
         p.getIncreaseRequestList();
-    this.increaseRequests = new ArrayList<ContainerResourceIncreaseRequest>();
+    this.increaseRequests = new ArrayList<ContainerResourceChangeRequest>();
 
-    for (ContainerResourceIncreaseRequestProto c : list) {
+    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;
     }
-    Iterable<ContainerResourceIncreaseRequestProto> iterable =
-        new Iterable<ContainerResourceIncreaseRequestProto>() {
+    Iterable<ContainerResourceChangeRequestProto> iterable =
+        new Iterable<ContainerResourceChangeRequestProto>() {
           @Override
-          public Iterator<ContainerResourceIncreaseRequestProto> iterator() {
-            return new Iterator<ContainerResourceIncreaseRequestProto>() {
+          public Iterator<ContainerResourceChangeRequestProto> iterator() {
+            return new Iterator<ContainerResourceChangeRequestProto>() {
 
-              Iterator<ContainerResourceIncreaseRequest> iter =
+              Iterator<ContainerResourceChangeRequest> iter =
                   increaseRequests.iterator();
 
               @Override
@@ -282,7 +317,7 @@ public class AllocateRequestPBImpl extends AllocateRequest {
               }
 
               @Override
-              public ContainerResourceIncreaseRequestProto next() {
+              public ContainerResourceChangeRequestProto next() {
                 return convertToProtoFormat(iter.next());
               }
 
@@ -296,7 +331,43 @@ public class AllocateRequestPBImpl extends AllocateRequest {
         };
     builder.addAllIncreaseRequest(iterable);
   }
-  
+
+  private void addDecreaseRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearDecreaseRequest();
+    if (decreaseRequests == null) {
+      return;
+    }
+    Iterable<ContainerResourceChangeRequestProto> iterable =
+        new Iterable<ContainerResourceChangeRequestProto>() {
+          @Override
+          public Iterator<ContainerResourceChangeRequestProto> iterator() {
+            return new Iterator<ContainerResourceChangeRequestProto>() {
+
+              Iterator<ContainerResourceChangeRequest> iter =
+                      decreaseRequests.iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iter.hasNext();
+              }
+
+              @Override
+              public ContainerResourceChangeRequestProto next() {
+                return convertToProtoFormat(iter.next());
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+            };
+
+          }
+        };
+    builder.addAllDecreaseRequest(iterable);
+  }
+
   @Override
   public List<ContainerId> getReleaseList() {
     initReleases();
@@ -367,14 +438,14 @@ public class AllocateRequestPBImpl extends AllocateRequest {
     return ((ResourceRequestPBImpl)t).getProto();
   }
   
-  private ContainerResourceIncreaseRequestPBImpl convertFromProtoFormat(
-      ContainerResourceIncreaseRequestProto p) {
-    return new ContainerResourceIncreaseRequestPBImpl(p);
+  private ContainerResourceChangeRequestPBImpl convertFromProtoFormat(
+      ContainerResourceChangeRequestProto p) {
+    return new ContainerResourceChangeRequestPBImpl(p);
   }
 
-  private ContainerResourceIncreaseRequestProto convertToProtoFormat(
-      ContainerResourceIncreaseRequest t) {
-    return ((ContainerResourceIncreaseRequestPBImpl) t).getProto();
+  private ContainerResourceChangeRequestProto convertToProtoFormat(
+      ContainerResourceChangeRequest t) {
+    return ((ContainerResourceChangeRequestPBImpl) t).getProto();
   }
 
   private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/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 f2796fd..dd7d1a9 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,8 +29,6 @@ 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;
@@ -38,8 +36,6 @@ import org.apache.hadoop.yarn.api.records.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceDecreasePBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreasePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeReportPBImpl;
@@ -48,8 +44,6 @@ 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.proto.YarnProtos.ContainerProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionMessageProto;
@@ -72,8 +66,8 @@ public class AllocateResponsePBImpl extends AllocateResponse {
   private List<Container> allocatedContainers = null;
   private List<NMToken> nmTokens = null;
   private List<ContainerStatus> completedContainersStatuses = null;
-  private List<ContainerResourceIncrease> increasedContainers = null;
-  private List<ContainerResourceDecrease> decreasedContainers = null;
+  private List<Container> increasedContainers = null;
+  private List<Container> decreasedContainers = null;
 
   private List<NodeReport> updatedNodes = null;
   private PreemptionMessage preempt;
@@ -147,14 +141,14 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     }
     if (this.increasedContainers != null) {
       builder.clearIncreasedContainers();
-      Iterable<ContainerResourceIncreaseProto> iterable =
-          getIncreaseProtoIterable(this.increasedContainers);
+      Iterable<ContainerProto> iterable =
+          getContainerProtoIterable(this.increasedContainers);
       builder.addAllIncreasedContainers(iterable);
     }
     if (this.decreasedContainers != null) {
       builder.clearDecreasedContainers();
-      Iterable<ContainerResourceDecreaseProto> iterable =
-          getChangeProtoIterable(this.decreasedContainers);
+      Iterable<ContainerProto> iterable =
+          getContainerProtoIterable(this.decreasedContainers);
       builder.addAllDecreasedContainers(iterable);
     }
     if (this.amrmToken != null) {
@@ -262,6 +256,36 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     allocatedContainers.addAll(containers);
   }
 
+  @Override
+  public synchronized List<Container> getIncreasedContainers() {
+    initLocalIncreasedContainerList();
+    return this.increasedContainers;
+  }
+
+  @Override
+  public synchronized void setIncreasedContainers(
+      final List<Container> containers) {
+    if (containers == null)
+      return;
+    initLocalIncreasedContainerList();
+    increasedContainers.addAll(containers);
+  }
+
+  @Override
+  public synchronized List<Container> getDecreasedContainers() {
+    initLocalDecreasedContainerList();
+    return this.decreasedContainers;
+  }
+
+  @Override
+  public synchronized void setDecreasedContainers(
+      final List<Container> containers) {
+    if (containers == null)
+      return;
+    initLocalDecreasedContainerList();
+    decreasedContainers.addAll(containers);
+  }
+
   //// Finished containers
   @Override
   public synchronized List<ContainerStatus> getCompletedContainersStatuses() {
@@ -333,37 +357,6 @@ public class AllocateResponsePBImpl extends AllocateResponse {
   }
 
   @Override
-  public synchronized List<ContainerResourceIncrease> getIncreasedContainers() {
-    initLocalIncreasedContainerList();
-    return increasedContainers;
-  }
-
-  @Override
-  public synchronized void setIncreasedContainers(
-      List<ContainerResourceIncrease> increasedContainers) {
-    if (increasedContainers == null)
-      return;
-    initLocalIncreasedContainerList();
-    this.increasedContainers.addAll(increasedContainers);
-  }
-
-  @Override
-  public synchronized List<ContainerResourceDecrease> getDecreasedContainers() {
-    initLocalDecreasedContainerList();
-    return decreasedContainers;
-  }
-
-  @Override
-  public synchronized void setDecreasedContainers(
-      List<ContainerResourceDecrease> decreasedContainers) {
-    if (decreasedContainers == null) {
-      return;
-    }
-    initLocalDecreasedContainerList();
-    this.decreasedContainers.addAll(decreasedContainers);
-  }
-
-  @Override
   public synchronized Token getAMRMToken() {
     AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
     if (amrmToken != null) {
@@ -390,10 +383,10 @@ public class AllocateResponsePBImpl extends AllocateResponse {
       return;
     }
     AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
-    List<ContainerResourceIncreaseProto> list = p.getIncreasedContainersList();
-    increasedContainers = new ArrayList<ContainerResourceIncrease>();
+    List<ContainerProto> list = p.getIncreasedContainersList();
+    increasedContainers = new ArrayList<>();
 
-    for (ContainerResourceIncreaseProto c : list) {
+    for (ContainerProto c : list) {
       increasedContainers.add(convertFromProtoFormat(c));
     }
   }
@@ -403,10 +396,10 @@ public class AllocateResponsePBImpl extends AllocateResponse {
       return;
     }
     AllocateResponseProtoOrBuilder p = viaProto ? proto : builder;
-    List<ContainerResourceDecreaseProto> list = p.getDecreasedContainersList();
-    decreasedContainers = new ArrayList<ContainerResourceDecrease>();
+    List<ContainerProto> list = p.getDecreasedContainersList();
+    decreasedContainers = new ArrayList<>();
 
-    for (ContainerResourceDecreaseProto c : list) {
+    for (ContainerProto c : list) {
       decreasedContainers.add(convertFromProtoFormat(c));
     }
   }
@@ -453,70 +446,6 @@ public class AllocateResponsePBImpl extends AllocateResponse {
     }
   }
 
-  private synchronized Iterable<ContainerResourceIncreaseProto>
-      getIncreaseProtoIterable(
-          final List<ContainerResourceIncrease> newContainersList) {
-    maybeInitBuilder();
-    return new Iterable<ContainerResourceIncreaseProto>() {
-      @Override
-      public synchronized Iterator<ContainerResourceIncreaseProto> iterator() {
-        return new Iterator<ContainerResourceIncreaseProto>() {
-
-          Iterator<ContainerResourceIncrease> iter = newContainersList
-              .iterator();
-
-          @Override
-          public synchronized boolean hasNext() {
-            return iter.hasNext();
-          }
-
-          @Override
-          public synchronized ContainerResourceIncreaseProto next() {
-            return convertToProtoFormat(iter.next());
-          }
-
-          @Override
-          public synchronized void remove() {
-            throw new UnsupportedOperationException();
-          }
-        };
-
-      }
-    };
-  }
-
-  private synchronized Iterable<ContainerResourceDecreaseProto>
-      getChangeProtoIterable(
-          final List<ContainerResourceDecrease> newContainersList) {
-    maybeInitBuilder();
-    return new Iterable<ContainerResourceDecreaseProto>() {
-      @Override
-      public synchronized Iterator<ContainerResourceDecreaseProto> iterator() {
-        return new Iterator<ContainerResourceDecreaseProto>() {
-
-          Iterator<ContainerResourceDecrease> iter = newContainersList
-              .iterator();
-
-          @Override
-          public synchronized boolean hasNext() {
-            return iter.hasNext();
-          }
-
-          @Override
-          public synchronized ContainerResourceDecreaseProto next() {
-            return convertToProtoFormat(iter.next());
-          }
-
-          @Override
-          public synchronized void remove() {
-            throw new UnsupportedOperationException();
-          }
-        };
-
-      }
-    };
-  }
-  
   private synchronized Iterable<ContainerProto> getContainerProtoIterable(
       final List<Container> newContainersList) {
     maybeInitBuilder();
@@ -654,26 +583,6 @@ public class AllocateResponsePBImpl extends AllocateResponse {
       completedContainersStatuses.add(convertFromProtoFormat(c));
     }
   }
-  
-  private synchronized ContainerResourceIncrease convertFromProtoFormat(
-      ContainerResourceIncreaseProto p) {
-    return new ContainerResourceIncreasePBImpl(p);
-  }
-
-  private synchronized ContainerResourceIncreaseProto convertToProtoFormat(
-      ContainerResourceIncrease t) {
-    return ((ContainerResourceIncreasePBImpl) t).getProto();
-  }
-
-  private synchronized ContainerResourceDecrease convertFromProtoFormat(
-      ContainerResourceDecreaseProto p) {
-    return new ContainerResourceDecreasePBImpl(p);
-  }
-
-  private synchronized ContainerResourceDecreaseProto convertToProtoFormat(
-      ContainerResourceDecrease t) {
-    return ((ContainerResourceDecreasePBImpl) t).getProto();
-  }
 
   private synchronized NodeReportPBImpl convertFromProtoFormat(
       NodeReportProto p) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceChangeRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceChangeRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceChangeRequestPBImpl.java
new file mode 100644
index 0000000..f382b8c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceChangeRequestPBImpl.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.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));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/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
deleted file mode 100644
index 1834132..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceDecreasePBImpl.java
+++ /dev/null
@@ -1,136 +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.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;
-
-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/dfe2cb84/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
deleted file mode 100644
index 4e4f3a7..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreasePBImpl.java
+++ /dev/null
@@ -1,171 +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.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;
-
-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/dfe2cb84/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
deleted file mode 100644
index f5ebf6c..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerResourceIncreaseRequestPBImpl.java
+++ /dev/null
@@ -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.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;
-
-
-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/dfe2cb84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java
index 86f2af9..d33d06d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerStatusPBImpl.java
@@ -24,6 +24,8 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 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.Resource;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
@@ -78,6 +80,7 @@ public class ContainerStatusPBImpl extends ContainerStatus {
     sb.append("ContainerStatus: [");
     sb.append("ContainerId: ").append(getContainerId()).append(", ");
     sb.append("State: ").append(getState()).append(", ");
+    sb.append("Capability: ").append(getCapability()).append(", ");
     sb.append("Diagnostics: ").append(getDiagnostics()).append(", ");
     sb.append("ExitStatus: ").append(getExitStatus()).append(", ");
     sb.append("]");
@@ -168,6 +171,25 @@ public class ContainerStatusPBImpl extends ContainerStatus {
     builder.setDiagnostics(diagnostics);
   }
 
+  @Override
+  public synchronized Resource getCapability() {
+    ContainerStatusProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasCapability()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getCapability());
+  }
+
+  @Override
+  public synchronized void setCapability(Resource capability) {
+    maybeInitBuilder();
+    if (capability == null) {
+      builder.clearCapability();
+      return;
+    }
+    builder.setCapability(convertToProtoFormat(capability));
+  }
+
   private ContainerStateProto convertToProtoFormat(ContainerState e) {
     return ProtoUtils.convertToProtoFormat(e);
   }
@@ -184,6 +206,11 @@ public class ContainerStatusPBImpl extends ContainerStatus {
     return ((ContainerIdPBImpl)t).getProto();
   }
 
+  private ResourceProto convertToProtoFormat(Resource e) {
+    return ((ResourcePBImpl)e).getProto();
+  }
 
-
-}  
+  private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
+    return new ResourcePBImpl(p);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateRequest.java
deleted file mode 100644
index 5ea29f8..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateRequest.java
+++ /dev/null
@@ -1,73 +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;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.Assert;
-
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
-import org.junit.Test;
-
-public class TestAllocateRequest {
-  @Test
-  public void testAllcoateRequestWithIncrease() {
-    List<ContainerResourceIncreaseRequest> incRequests =
-        new ArrayList<ContainerResourceIncreaseRequest>();
-    for (int i = 0; i < 3; i++) {
-      incRequests.add(ContainerResourceIncreaseRequest.newInstance(null,
-          Resource.newInstance(0, i)));
-    }
-    AllocateRequest r =
-        AllocateRequest.newInstance(123, 0f, null, null, null, incRequests);
-
-    // serde
-    AllocateRequestProto p = ((AllocateRequestPBImpl) r).getProto();
-    r = new AllocateRequestPBImpl(p);
-
-    // check value
-    Assert.assertEquals(123, r.getResponseId());
-    Assert.assertEquals(incRequests.size(), r.getIncreaseRequests().size());
-
-    for (int i = 0; i < incRequests.size(); i++) {
-      Assert.assertEquals(r.getIncreaseRequests().get(i).getCapability()
-          .getVirtualCores(), incRequests.get(i).getCapability()
-          .getVirtualCores());
-    }
-  }
-
-  @Test
-  public void testAllcoateRequestWithoutIncrease() {
-    AllocateRequest r =
-        AllocateRequest.newInstance(123, 0f, null, null, null, null);
-
-    // serde
-    AllocateRequestProto p = ((AllocateRequestPBImpl) r).getProto();
-    r = new AllocateRequestPBImpl(p);
-
-    // check value
-    Assert.assertEquals(123, r.getResponseId());
-    Assert.assertEquals(0, r.getIncreaseRequests().size());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java
deleted file mode 100644
index fbe9af9..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestAllocateResponse.java
+++ /dev/null
@@ -1,114 +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;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
-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;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * 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.
- */
-public class TestAllocateResponse {
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testAllocateResponseWithIncDecContainers() {
-    List<ContainerResourceIncrease> incContainers =
-        new ArrayList<ContainerResourceIncrease>();
-    List<ContainerResourceDecrease> decContainers =
-        new ArrayList<ContainerResourceDecrease>();
-    for (int i = 0; i < 3; i++) {
-      incContainers.add(ContainerResourceIncrease.newInstance(null,
-          Resource.newInstance(1024, i), null));
-    }
-    for (int i = 0; i < 5; i++) {
-      decContainers.add(ContainerResourceDecrease.newInstance(null,
-          Resource.newInstance(1024, i)));
-    }
-
-    AllocateResponse r =
-        AllocateResponse.newInstance(3, new ArrayList<ContainerStatus>(),
-            new ArrayList<Container>(), new ArrayList<NodeReport>(), null,
-            AMCommand.AM_RESYNC, 3, null, new ArrayList<NMToken>(),
-            incContainers, decContainers);
-
-    // serde
-    AllocateResponseProto p = ((AllocateResponsePBImpl) r).getProto();
-    r = new AllocateResponsePBImpl(p);
-
-    // check value
-    Assert
-        .assertEquals(incContainers.size(), r.getIncreasedContainers().size());
-    Assert
-        .assertEquals(decContainers.size(), r.getDecreasedContainers().size());
-
-    for (int i = 0; i < incContainers.size(); i++) {
-      Assert.assertEquals(i, r.getIncreasedContainers().get(i).getCapability()
-          .getVirtualCores());
-    }
-
-    for (int i = 0; i < decContainers.size(); i++) {
-      Assert.assertEquals(i, r.getDecreasedContainers().get(i).getCapability()
-          .getVirtualCores());
-    }
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testAllocateResponseWithoutIncDecContainers() {
-    AllocateResponse r =
-        AllocateResponse.newInstance(3, new ArrayList<ContainerStatus>(),
-            new ArrayList<Container>(), new ArrayList<NodeReport>(), null,
-            AMCommand.AM_RESYNC, 3, null, new ArrayList<NMToken>(), null, null);
-
-    // serde
-    AllocateResponseProto p = ((AllocateResponsePBImpl) r).getProto();
-    r = new AllocateResponsePBImpl(p);
-
-    // check value
-    Assert.assertEquals(0, r.getIncreasedContainers().size());
-    Assert.assertEquals(0, r.getDecreasedContainers().size());
-  }
-}


[05/21] hadoop git commit: HDFS-9013. Deprecate NameNodeMXBean#getNNStarted in branch2 and remove from trunk (Contributed by Surendra Singh Lilhore)

Posted by wa...@apache.org.
HDFS-9013. Deprecate NameNodeMXBean#getNNStarted in branch2 and remove from trunk (Contributed by Surendra Singh Lilhore)


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

Branch: refs/heads/YARN-1197
Commit: a2c76e5f26301d4b01e56b347442f3dec171591d
Parents: 7c5c099
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Sep 23 15:28:41 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Wed Sep 23 15:28:41 2015 +0530

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Metrics.md              | 1 -
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                 | 5 ++++-
 .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java    | 5 -----
 .../apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java  | 7 -------
 .../server/blockmanagement/TestPendingInvalidateBlock.java  | 9 +++------
 .../hadoop/hdfs/server/namenode/TestNameNodeMXBean.java     | 3 ---
 6 files changed, 7 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2c76e5f/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
index efe1f2f..3e01fe3 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
@@ -192,7 +192,6 @@ Each metrics record contains tags such as ProcessName, SessionId, and Hostname a
 | `PutImageNumOps` | Total number of fsimage uploads to SecondaryNameNode |
 | `PutImageAvgTime` | Average fsimage upload time in milliseconds |
 | `TotalFileOps`| Total number of file operations performed |
-| `NNStarted`| NameNode start time |
 | `NNStartedTimeInMillis`| NameNode start time in milliseconds |
 
 FSNamesystem

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2c76e5f/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index b900d91..b610bde 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1385,7 +1385,10 @@ Release 2.8.0 - UNRELEASED
     HDFS-9063. Correctly handle snapshot path for getContentSummary. (jing9)
 
     HDFS-8780. Fetching live/dead datanode list with arg true for remove-
-    DecommissionNode,returns list with decom node. (J.Andreina via vinayakumab)
+    DecommissionNode,returns list with decom node. (J.Andreina via vinayakumarb)
+
+    HDFS-9013. Deprecate NameNodeMXBean#getNNStarted in branch2 and remove from
+    trunk (Surendra Singh Lilhore via vinayakumarb)
 
 Release 2.7.2 - UNRELEASED
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2c76e5f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 4a9d13b..0b638bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -6110,11 +6110,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return JSON.toString(txnIdMap);
   }
   
-  @Override  // NameNodeMXBean
-  public String getNNStarted() {
-    return getStartTime().toString();
-  }
-
   @Override // NameNodeMXBean
   public long getNNStartedTimeInMillis() {
     return startTime;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2c76e5f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
index 00c1abe..3628092 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
@@ -232,13 +232,6 @@ public interface NameNodeMXBean {
   public String getJournalTransactionInfo();
 
   /**
-   * Gets the NN start time
-   *
-   * @return the NN start time
-   */
-  public String getNNStarted();
-
-  /**
    * Gets the NN start time in milliseconds.
    * @return the NN start time in msec
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2c76e5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
index 84ae771..60e0f7c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
@@ -90,15 +90,12 @@ public class TestPendingInvalidateBlock {
     Thread.sleep(6000);
     Assert.assertEquals(0, cluster.getNamesystem().getBlocksTotal());
     Assert.assertEquals(0, cluster.getNamesystem().getPendingDeletionBlocks());
-    String nnStartedStr = cluster.getNamesystem().getNNStarted();
-    long nnStarted = new SimpleDateFormat("EEE MMM dd HH:mm:ss zzz yyyy")
-        .parse(nnStartedStr).getTime();
+    long nnStarted = cluster.getNamesystem().getNNStartedTimeInMillis();
     long blockDeletionStartTime = cluster.getNamesystem()
         .getBlockDeletionStartTime();
     Assert.assertTrue(String.format(
-        "Expect blockDeletionStartTime = %d > nnStarted = %d/nnStartedStr = %s.",
-        blockDeletionStartTime, nnStarted, nnStartedStr),
-        blockDeletionStartTime > nnStarted);
+        "Expect blockDeletionStartTime = %d > nnStarted = %d.",
+        blockDeletionStartTime, nnStarted), blockDeletionStartTime > nnStarted);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2c76e5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
index f10d57e..559aae6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
@@ -164,9 +164,6 @@ public class TestNameNodeMXBean {
           "JournalTransactionInfo");
       assertEquals("Bad value for NameTxnIds", fsn.getJournalTransactionInfo(),
           journalTxnInfo);
-      // get attribute "NNStarted"
-      String nnStarted = (String) mbs.getAttribute(mxbeanName, "NNStarted");
-      assertEquals("Bad value for NNStarted", fsn.getNNStarted(), nnStarted);
       // get attribute "CompileInfo"
       String compileInfo = (String) mbs.getAttribute(mxbeanName, "CompileInfo");
       assertEquals("Bad value for CompileInfo", fsn.getCompileInfo(), compileInfo);


[20/21] hadoop git commit: YARN-3868. Recovery support for container resizing. Contributed by Meng Ding

Posted by wa...@apache.org.
YARN-3868. Recovery support for container resizing. Contributed by Meng Ding


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

Branch: refs/heads/YARN-1197
Commit: c57eac5dfe277845ab4522a1188023a73ee41539
Parents: c3dc1af
Author: Jian He <ji...@apache.org>
Authored: Thu Aug 20 21:18:23 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Sep 23 13:29:38 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../containermanager/ContainerManagerImpl.java  |   5 +-
 .../container/ContainerImpl.java                |   8 +-
 .../recovery/NMLeveldbStateStoreService.java    |  22 ++
 .../recovery/NMNullStateStoreService.java       |   6 +
 .../recovery/NMStateStoreService.java           |  15 ++
 .../TestContainerManagerRecovery.java           | 233 ++++++++++++++++++-
 .../recovery/NMMemoryStateStoreService.java     |  11 +-
 .../TestNMLeveldbStateStoreService.java         |  11 +
 9 files changed, 301 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c57eac5d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1872b1a..d2aafa0 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -218,6 +218,8 @@ Release 2.8.0 - UNRELEASED
     YARN-1644. RM-NM protocol changes and NodeStatusUpdater implementation to
     support container resizing. (Meng Ding via jianhe)
 
+    YARN-3868. Recovery support for container resizing. (Meng Ding via jianhe)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c57eac5d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 868d8d3..39d2983 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -346,7 +346,7 @@ public class ContainerManagerImpl extends CompositeService implements
       Container container = new ContainerImpl(getConfig(), dispatcher,
           context.getNMStateStore(), req.getContainerLaunchContext(),
           credentials, metrics, token, rcs.getStatus(), rcs.getExitCode(),
-          rcs.getDiagnostics(), rcs.getKilled());
+          rcs.getDiagnostics(), rcs.getKilled(), rcs.getCapability());
       context.getContainers().put(containerId, container);
       dispatcher.getEventHandler().handle(
           new ApplicationContainerInitEvent(container));
@@ -1101,6 +1101,9 @@ public class ContainerManagerImpl extends CompositeService implements
     this.readLock.lock();
     try {
       if (!serviceStopped) {
+        // Persist container resource change for recovery
+        this.context.getNMStateStore().storeContainerResourceChanged(
+            containerId, targetResource);
         getContainersMonitor().handle(
             new ChangeMonitoringContainerResourceEvent(
                 containerId, targetResource));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c57eac5d/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 5c61a92..eff2188 100644
--- a/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
+++ b/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
@@ -154,13 +154,19 @@ public class ContainerImpl implements Container {
       Credentials creds, NodeManagerMetrics metrics,
       ContainerTokenIdentifier containerTokenIdentifier,
       RecoveredContainerStatus recoveredStatus, int exitCode,
-      String diagnostics, boolean wasKilled) {
+      String diagnostics, boolean wasKilled, Resource recoveredCapability) {
     this(conf, dispatcher, stateStore, launchContext, creds, metrics,
         containerTokenIdentifier);
     this.recoveredStatus = recoveredStatus;
     this.exitCode = exitCode;
     this.recoveredAsKilled = wasKilled;
     this.diagnostics.append(diagnostics);
+    if (recoveredCapability != null
+        && !this.resource.equals(recoveredCapability)) {
+      // resource capability had been updated before NM was down
+      this.resource = Resource.newInstance(recoveredCapability.getMemory(),
+          recoveredCapability.getVirtualCores());
+    }
   }
 
   private static final ContainerDiagnosticsUpdateTransition UPDATE_DIAGNOSTICS_TRANSITION =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c57eac5d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
index df58182..89c71bb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
@@ -40,7 +40,10 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerRequestP
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.VersionProto;
@@ -99,6 +102,8 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
   private static final String CONTAINER_REQUEST_KEY_SUFFIX = "/request";
   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 =
+      "/resourceChanged";
   private static final String CONTAINER_KILLED_KEY_SUFFIX = "/killed";
   private static final String CONTAINER_EXIT_CODE_KEY_SUFFIX = "/exitcode";
 
@@ -230,6 +235,9 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
       } else if (suffix.equals(CONTAINER_EXIT_CODE_KEY_SUFFIX)) {
         rcs.status = RecoveredContainerStatus.COMPLETED;
         rcs.exitCode = Integer.parseInt(asString(entry.getValue()));
+      } else if (suffix.equals(CONTAINER_RESOURCE_CHANGED_KEY_SUFFIX)) {
+        rcs.capability = new ResourcePBImpl(
+            ResourceProto.parseFrom(entry.getValue()));
       } else {
         throw new IOException("Unexpected container state key: " + key);
       }
@@ -275,6 +283,20 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
   }
 
   @Override
+  public void storeContainerResourceChanged(ContainerId containerId,
+      Resource capability) throws IOException {
+    String key = CONTAINERS_KEY_PREFIX + containerId.toString()
+        + CONTAINER_RESOURCE_CHANGED_KEY_SUFFIX;
+    try {
+      // New value will overwrite old values for the same key
+      db.put(bytes(key),
+          ((ResourcePBImpl) capability).getProto().toByteArray());
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
   public void storeContainerKilled(ContainerId containerId)
       throws IOException {
     String key = CONTAINERS_KEY_PREFIX + containerId.toString()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c57eac5d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
index ab49543..d5dce9b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMNullStateStoreService.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto;
@@ -88,6 +89,11 @@ public class NMNullStateStoreService extends NMStateStoreService {
   }
 
   @Override
+  public void storeContainerResourceChanged(ContainerId containerId,
+      Resource capability) throws IOException {
+  }
+
+  @Override
   public void storeContainerKilled(ContainerId containerId)
       throws IOException {
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c57eac5d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
index fa66349..e8ccf54 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto;
@@ -74,6 +75,7 @@ public abstract class NMStateStoreService extends AbstractService {
     boolean killed = false;
     String diagnostics = "";
     StartContainerRequest startRequest;
+    Resource capability;
 
     public RecoveredContainerStatus getStatus() {
       return status;
@@ -94,6 +96,10 @@ public abstract class NMStateStoreService extends AbstractService {
     public StartContainerRequest getStartRequest() {
       return startRequest;
     }
+
+    public Resource getCapability() {
+      return capability;
+    }
   }
 
   public static class LocalResourceTrackerState {
@@ -284,6 +290,15 @@ public abstract class NMStateStoreService extends AbstractService {
       throws IOException;
 
   /**
+   * Record that a container resource has been changed
+   * @param containerId the container ID
+   * @param capability the container resource capability
+   * @throws IOException
+   */
+  public abstract void storeContainerResourceChanged(ContainerId containerId,
+      Resource capability) throws IOException;
+
+  /**
    * Record that a container has completed
    * @param containerId the container ID
    * @param exitCode the exit code from the container

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c57eac5d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
index 4d0aacd..43f1b29 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
@@ -28,18 +28,30 @@ import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
@@ -48,9 +60,17 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
@@ -58,6 +78,9 @@ import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
+import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
+import org.apache.hadoop.yarn.server.nodemanager.NodeHealthCheckerService;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
 import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
@@ -65,6 +88,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncher;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
@@ -77,18 +101,50 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.junit.Before;
 import org.junit.Test;
 
-public class TestContainerManagerRecovery {
+public class TestContainerManagerRecovery extends BaseContainerManagerTest {
 
-  private NodeManagerMetrics metrics = NodeManagerMetrics.create();
+  public TestContainerManagerRecovery() throws UnsupportedFileSystemException {
+    super();
+  }
+
+  @Override
+  @Before
+  public void setup() throws IOException {
+    localFS.delete(new Path(localDir.getAbsolutePath()), true);
+    localFS.delete(new Path(tmpDir.getAbsolutePath()), true);
+    localFS.delete(new Path(localLogDir.getAbsolutePath()), true);
+    localFS.delete(new Path(remoteLogDir.getAbsolutePath()), true);
+    localDir.mkdir();
+    tmpDir.mkdir();
+    localLogDir.mkdir();
+    remoteLogDir.mkdir();
+    LOG.info("Created localDir in " + localDir.getAbsolutePath());
+    LOG.info("Created tmpDir in " + tmpDir.getAbsolutePath());
+
+    String bindAddress = "0.0.0.0:12345";
+    conf.set(YarnConfiguration.NM_ADDRESS, bindAddress);
+    conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath());
+    conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
+    conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogDir.getAbsolutePath());
+    conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1);
+    // Default delSrvc
+    delSrvc = createDeletionService();
+    delSrvc.init(conf);
+    exec = createContainerExecutor();
+    dirsHandler = new LocalDirsHandlerService();
+    nodeHealthChecker = new NodeHealthCheckerService(
+        NodeManager.getNodeHealthScriptRunner(conf), dirsHandler);
+    nodeHealthChecker.init(conf);
+  }
 
   @Test
   public void testApplicationRecovery() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.NM_RECOVERY_ENABLED, true);
     conf.setBoolean(YarnConfiguration.NM_RECOVERY_SUPERVISED, true);
-    conf.set(YarnConfiguration.NM_ADDRESS, "localhost:1234");
     conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, "yarn_admin_user");
     NMStateStoreService stateStore = new NMMemoryStateStoreService();
@@ -234,6 +290,91 @@ public class TestContainerManagerRecovery {
   }
 
   @Test
+  public void testContainerResizeRecovery() throws Exception {
+    conf.setBoolean(YarnConfiguration.NM_RECOVERY_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.NM_RECOVERY_SUPERVISED, true);
+    NMStateStoreService stateStore = new NMMemoryStateStoreService();
+    stateStore.init(conf);
+    stateStore.start();
+    Context context = createContext(conf, stateStore);
+    ContainerManagerImpl cm = createContainerManager(context, delSrvc);
+    cm.init(conf);
+    cm.start();
+    // add an application by starting a container
+    ApplicationId appId = ApplicationId.newInstance(0, 1);
+    ApplicationAttemptId attemptId =
+        ApplicationAttemptId.newInstance(appId, 1);
+    ContainerId cid = ContainerId.newContainerId(attemptId, 1);
+    Map<String, String> containerEnv = Collections.emptyMap();
+    Map<String, ByteBuffer> serviceData = Collections.emptyMap();
+    Credentials containerCreds = new Credentials();
+    DataOutputBuffer dob = new DataOutputBuffer();
+    containerCreds.writeTokenStorageToStream(dob);
+    ByteBuffer containerTokens = ByteBuffer.wrap(dob.getData(), 0,
+        dob.getLength());
+    Map<ApplicationAccessType, String> acls = Collections.emptyMap();
+    File tmpDir = new File("target",
+        this.getClass().getSimpleName() + "-tmpDir");
+    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
+    PrintWriter fileWriter = new PrintWriter(scriptFile);
+    if (Shell.WINDOWS) {
+      fileWriter.println("@ping -n 100 127.0.0.1 >nul");
+    } else {
+      fileWriter.write("\numask 0");
+      fileWriter.write("\nexec sleep 100");
+    }
+    fileWriter.close();
+    FileContext localFS = FileContext.getLocalFSFileContext();
+    URL resource_alpha =
+        ConverterUtils.getYarnUrlFromPath(localFS
+            .makeQualified(new Path(scriptFile.getAbsolutePath())));
+    LocalResource rsrc_alpha = RecordFactoryProvider
+        .getRecordFactory(null).newRecordInstance(LocalResource.class);
+    rsrc_alpha.setResource(resource_alpha);
+    rsrc_alpha.setSize(-1);
+    rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
+    rsrc_alpha.setType(LocalResourceType.FILE);
+    rsrc_alpha.setTimestamp(scriptFile.lastModified());
+    String destinationFile = "dest_file";
+    Map<String, LocalResource> localResources = new HashMap<>();
+    localResources.put(destinationFile, rsrc_alpha);
+    List<String> commands =
+        Arrays.asList(Shell.getRunScriptCommand(scriptFile));
+    ContainerLaunchContext clc = ContainerLaunchContext.newInstance(
+        localResources, containerEnv, commands, serviceData,
+        containerTokens, acls);
+    StartContainersResponse startResponse = startContainer(
+        context, cm, cid, clc, null);
+    assertTrue(startResponse.getFailedRequests().isEmpty());
+    assertEquals(1, context.getApplications().size());
+    Application app = context.getApplications().get(appId);
+    assertNotNull(app);
+    // make sure the container reaches RUNNING state
+    waitForNMContainerState(cm, cid,
+        org.apache.hadoop.yarn.server.nodemanager
+            .containermanager.container.ContainerState.RUNNING);
+    Resource targetResource = Resource.newInstance(2048, 2);
+    IncreaseContainersResourceResponse increaseResponse =
+        increaseContainersResource(context, cm, cid, targetResource);
+    assertTrue(increaseResponse.getFailedRequests().isEmpty());
+    // check status
+    ContainerStatus containerStatus = getContainerStatus(context, cm, cid);
+    assertEquals(targetResource, containerStatus.getCapability());
+    // restart and verify container is running and recovered
+    // to the correct size
+    cm.stop();
+    context = createContext(conf, stateStore);
+    cm = createContainerManager(context);
+    cm.init(conf);
+    cm.start();
+    assertEquals(1, context.getApplications().size());
+    app = context.getApplications().get(appId);
+    assertNotNull(app);
+    containerStatus = getContainerStatus(context, cm, cid);
+    assertEquals(targetResource, containerStatus.getCapability());
+  }
+
+  @Test
   public void testContainerCleanupOnShutdown() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     ApplicationAttemptId attemptId =
@@ -257,10 +398,8 @@ public class TestContainerManagerRecovery {
         LogAggregationContext.newInstance("includePattern", "excludePattern");
 
     // verify containers are stopped on shutdown without recovery
-    YarnConfiguration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.NM_RECOVERY_ENABLED, false);
     conf.setBoolean(YarnConfiguration.NM_RECOVERY_SUPERVISED, false);
-    conf.set(YarnConfiguration.NM_ADDRESS, "localhost:1234");
     Context context = createContext(conf, new NMNullStateStoreService());
     ContainerManagerImpl cm = spy(createContainerManager(context));
     cm.init(conf);
@@ -306,12 +445,36 @@ public class TestContainerManagerRecovery {
     verify(cm, never()).handle(isA(CMgrCompletedAppsEvent.class));
   }
 
-  private NMContext createContext(YarnConfiguration conf,
+  private ContainerManagerImpl createContainerManager(Context context,
+      DeletionService delSrvc) {
+    return new ContainerManagerImpl(context, exec, delSrvc,
+        mock(NodeStatusUpdater.class), metrics, dirsHandler) {
+      @Override
+      public void
+      setBlockNewContainerRequests(boolean blockNewContainerRequests) {
+        // do nothing
+      }
+      @Override
+      protected void authorizeGetAndStopContainerRequest(
+          ContainerId containerId, Container container,
+          boolean stopRequest, NMTokenIdentifier identifier)
+          throws YarnException {
+        if(container == null || container.getUser().equals("Fail")){
+          throw new YarnException("Reject this container");
+        }
+      }
+    };
+  }
+
+  private NMContext createContext(Configuration conf,
       NMStateStoreService stateStore) {
     NMContext context = new NMContext(new NMContainerTokenSecretManager(
         conf), new NMTokenSecretManagerInNM(), null,
-        new ApplicationACLsManager(conf), stateStore);
-
+        new ApplicationACLsManager(conf), stateStore){
+      public int getHttpPort() {
+        return HTTP_PORT;
+      }
+    };
     // simulate registration with RM
     MasterKey masterKey = new MasterKeyPBImpl();
     masterKey.setKeyId(123);
@@ -349,6 +512,58 @@ public class TestContainerManagerRecovery {
     });
   }
 
+  private IncreaseContainersResourceResponse increaseContainersResource(
+      Context context, final ContainerManagerImpl cm, ContainerId cid,
+      Resource capability) throws Exception {
+    UserGroupInformation user = UserGroupInformation.createRemoteUser(
+        cid.getApplicationAttemptId().toString());
+    // construct container resource increase request
+    final List<Token> increaseTokens = new ArrayList<Token>();
+    // add increase request
+    Token containerToken = TestContainerManager.createContainerToken(
+        cid, 0, context.getNodeId(), user.getShortUserName(),
+        capability, context.getContainerTokenSecretManager(), null);
+    increaseTokens.add(containerToken);
+    final IncreaseContainersResourceRequest increaseRequest =
+        IncreaseContainersResourceRequest.newInstance(increaseTokens);
+    NMTokenIdentifier nmToken = new NMTokenIdentifier(
+        cid.getApplicationAttemptId(), context.getNodeId(),
+        user.getShortUserName(),
+        context.getNMTokenSecretManager().getCurrentKey().getKeyId());
+    user.addTokenIdentifier(nmToken);
+    return user.doAs(
+        new PrivilegedExceptionAction<IncreaseContainersResourceResponse>() {
+          @Override
+          public IncreaseContainersResourceResponse run() throws Exception {
+            return cm.increaseContainersResource(increaseRequest);
+          }
+        });
+  }
+
+  private ContainerStatus getContainerStatus(
+      Context context, final ContainerManagerImpl cm, ContainerId cid)
+      throws  Exception {
+    UserGroupInformation user = UserGroupInformation.createRemoteUser(
+        cid.getApplicationAttemptId().toString());
+    NMTokenIdentifier nmToken = new NMTokenIdentifier(
+        cid.getApplicationAttemptId(), context.getNodeId(),
+        user.getShortUserName(),
+        context.getNMTokenSecretManager().getCurrentKey().getKeyId());
+    user.addTokenIdentifier(nmToken);
+    List<ContainerId> containerIds = new ArrayList<>();
+    containerIds.add(cid);
+    final GetContainerStatusesRequest gcsRequest =
+        GetContainerStatusesRequest.newInstance(containerIds);
+    return user.doAs(
+        new PrivilegedExceptionAction<ContainerStatus>() {
+          @Override
+          public ContainerStatus run() throws Exception {
+            return cm.getContainerStatuses(gcsRequest)
+                .getContainerStatuses().get(0);
+          }
+        });
+  }
+
   private void waitForAppState(Application app, ApplicationState state)
       throws Exception {
     final int msecPerSleep = 10;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c57eac5d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
index e0487e7..a1c95ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMMemoryStateStoreService.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.DeletionServiceDeleteTaskProto;
@@ -122,9 +123,10 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
       rcsCopy.killed = rcs.killed;
       rcsCopy.diagnostics = rcs.diagnostics;
       rcsCopy.startRequest = rcs.startRequest;
+      rcsCopy.capability = rcs.capability;
       result.add(rcsCopy);
     }
-    return new ArrayList<RecoveredContainerState>();
+    return result;
   }
 
   @Override
@@ -153,6 +155,13 @@ public class NMMemoryStateStoreService extends NMStateStoreService {
   }
 
   @Override
+  public synchronized void storeContainerResourceChanged(
+      ContainerId containerId, Resource capability) throws IOException {
+    RecoveredContainerState rcs = getRecoveredContainerState(containerId);
+    rcs.capability = capability;
+  }
+
+  @Override
   public synchronized void storeContainerKilled(ContainerId containerId)
       throws IOException {
     RecoveredContainerState rcs = getRecoveredContainerState(containerId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c57eac5d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
index 1804424..08b49e7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
@@ -298,6 +298,17 @@ public class TestNMLeveldbStateStoreService {
     assertEquals(containerReq, rcs.getStartRequest());
     assertEquals(diags.toString(), rcs.getDiagnostics());
 
+    // increase the container size, and verify recovered
+    stateStore.storeContainerResourceChanged(containerId, Resource.newInstance(2468, 4));
+    restartStateStore();
+    recoveredContainers = stateStore.loadContainersState();
+    assertEquals(1, recoveredContainers.size());
+    rcs = recoveredContainers.get(0);
+    assertEquals(RecoveredContainerStatus.LAUNCHED, rcs.getStatus());
+    assertEquals(ContainerExitStatus.INVALID, rcs.getExitCode());
+    assertEquals(false, rcs.getKilled());
+    assertEquals(Resource.newInstance(2468, 4), rcs.getCapability());
+
     // mark the container killed, add some more diags, and verify recovered
     diags.append("some more diags for container");
     stateStore.storeContainerDiagnostics(containerId, diags);


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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89cab1ba/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java
new file mode 100644
index 0000000..b986b1f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java
@@ -0,0 +1,131 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
+import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+/**
+ * For an application, resource limits and resource requests, decide how to
+ * allocate container. This is to make application resource allocation logic
+ * extensible.
+ */
+public abstract class AbstractContainerAllocator {
+  private static final Log LOG = LogFactory.getLog(AbstractContainerAllocator.class);
+
+  FiCaSchedulerApp application;
+  final ResourceCalculator rc;
+  final RMContext rmContext;
+  
+  public AbstractContainerAllocator(FiCaSchedulerApp application,
+      ResourceCalculator rc, RMContext rmContext) {
+    this.application = application;
+    this.rc = rc;
+    this.rmContext = rmContext;
+  }
+
+  protected CSAssignment getCSAssignmentFromAllocateResult(
+      Resource clusterResource, ContainerAllocation result,
+      RMContainer rmContainer) {
+    // Handle skipped
+    boolean skipped =
+        (result.getAllocationState() == AllocationState.APP_SKIPPED);
+    CSAssignment assignment = new CSAssignment(skipped);
+    assignment.setApplication(application);
+    
+    // Handle excess reservation
+    assignment.setExcessReservation(result.getContainerToBeUnreserved());
+
+    // If we allocated something
+    if (Resources.greaterThan(rc, clusterResource,
+        result.getResourceToBeAllocated(), Resources.none())) {
+      Resource allocatedResource = result.getResourceToBeAllocated();
+      Container updatedContainer = result.getUpdatedContainer();
+
+      assignment.setResource(allocatedResource);
+      assignment.setType(result.getContainerNodeType());
+
+      if (result.getAllocationState() == AllocationState.RESERVED) {
+        // This is a reserved container
+        LOG.info("Reserved container " + " application="
+            + application.getApplicationId() + " resource=" + allocatedResource
+            + " queue=" + this.toString() + " cluster=" + clusterResource);
+        assignment.getAssignmentInformation().addReservationDetails(
+            updatedContainer.getId(),
+            application.getCSLeafQueue().getQueuePath());
+        assignment.getAssignmentInformation().incrReservations();
+        Resources.addTo(assignment.getAssignmentInformation().getReserved(),
+            allocatedResource);
+      } else if (result.getAllocationState() == AllocationState.ALLOCATED){
+        // This is a new container
+        // Inform the ordering policy
+        LOG.info("assignedContainer" + " application attempt="
+            + application.getApplicationAttemptId() + " container="
+            + updatedContainer.getId() + " queue=" + this + " clusterResource="
+            + clusterResource);
+
+        application
+            .getCSLeafQueue()
+            .getOrderingPolicy()
+            .containerAllocated(application,
+                application.getRMContainer(updatedContainer.getId()));
+
+        assignment.getAssignmentInformation().addAllocationDetails(
+            updatedContainer.getId(),
+            application.getCSLeafQueue().getQueuePath());
+        assignment.getAssignmentInformation().incrAllocations();
+        Resources.addTo(assignment.getAssignmentInformation().getAllocated(),
+            allocatedResource);
+        
+        if (rmContainer != null) {
+          assignment.setFulfilledReservation(true);
+        }
+      }
+    }
+    
+    return assignment;
+  }
+  
+  /**
+   * allocate needs to handle following stuffs:
+   * 
+   * <ul>
+   * <li>Select request: Select a request to allocate. E.g. select a resource
+   * request based on requirement/priority/locality.</li>
+   * <li>Check if a given resource can be allocated based on resource
+   * availability</li>
+   * <li>Do allocation: this will decide/create allocated/reserved
+   * container, this will also update metrics</li>
+   * </ul>
+   */
+  public abstract CSAssignment assignContainers(Resource clusterResource,
+      FiCaSchedulerNode node, SchedulingMode schedulingMode,
+      ResourceLimits resourceLimits, RMContainer reservedContainer);
+}
\ No newline at end of file

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

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

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

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

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

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

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

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

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

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


[07/21] hadoop git commit: YARN-3975. WebAppProxyServlet should not redirect to RM page if AHS is enabled. Contributed by Mit Desai

Posted by wa...@apache.org.
YARN-3975. WebAppProxyServlet should not redirect to RM page if AHS is enabled. Contributed by Mit Desai


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

Branch: refs/heads/YARN-1197
Commit: 692d51c09d3668cde47cc297296d095ddfa933a3
Parents: c890c51
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Sep 23 16:38:55 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Sep 23 16:38:55 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../yarn/client/api/impl/YarnClientImpl.java    |   7 +-
 .../yarn/server/webproxy/AppReportFetcher.java  |  49 ++++++--
 .../server/webproxy/WebAppProxyServlet.java     |  46 ++++++--
 .../server/webproxy/TestWebAppProxyServlet.java | 113 ++++++++++++++++---
 5 files changed, 174 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/692d51c0/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 3cd92f6..e461fe3 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -963,6 +963,9 @@ Release 2.7.2 - UNRELEASED
     YARN-3433. Jersey tests failing with Port in Use -again.
     (Brahma Reddy Battula)
 
+    YARN-3975. WebAppProxyServlet should not redirect to RM page if AHS is
+    enabled (Mit Desai via jlowe)
+
 Release 2.7.1 - 2015-07-06
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692d51c0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
index 1713f9e..bc97a12 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
@@ -432,16 +432,11 @@ public class YarnClientImpl extends YarnClient {
           .newRecord(GetApplicationReportRequest.class);
       request.setApplicationId(appId);
       response = rmClient.getApplicationReport(request);
-    } catch (YarnException e) {
+    } catch (ApplicationNotFoundException e) {
       if (!historyServiceEnabled) {
         // Just throw it as usual if historyService is not enabled.
         throw e;
       }
-      // Even if history-service is enabled, treat all exceptions still the same
-      // except the following
-      if (!(e.getClass() == ApplicationNotFoundException.class)) {
-        throw e;
-      }
       return historyClient.getApplicationReport(appId);
     }
     return response.getApplicationReport();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692d51c0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/AppReportFetcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/AppReportFetcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/AppReportFetcher.java
index 6aa43eb..11ec2e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/AppReportFetcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/AppReportFetcher.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.client.AHSProxy;
@@ -42,6 +41,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
  * This class abstracts away how ApplicationReports are fetched.
  */
 public class AppReportFetcher {
+  enum AppReportSource { RM, AHS }
   private static final Log LOG = LogFactory.getLog(AppReportFetcher.class);
   private final Configuration conf;
   private final ApplicationClientProtocol applicationsManager;
@@ -115,28 +115,29 @@ public class AppReportFetcher {
    * @throws YarnException on any error.
    * @throws IOException
    */
-  public ApplicationReport getApplicationReport(ApplicationId appId)
+  public FetchedAppReport getApplicationReport(ApplicationId appId)
   throws YarnException, IOException {
     GetApplicationReportRequest request = recordFactory
         .newRecordInstance(GetApplicationReportRequest.class);
     request.setApplicationId(appId);
 
-    GetApplicationReportResponse response;
+    ApplicationReport appReport;
+    FetchedAppReport fetchedAppReport;
     try {
-      response = applicationsManager.getApplicationReport(request);
-    } catch (YarnException e) {
+      appReport = applicationsManager.
+          getApplicationReport(request).getApplicationReport();
+      fetchedAppReport = new FetchedAppReport(appReport, AppReportSource.RM);
+    } catch (ApplicationNotFoundException e) {
       if (!isAHSEnabled) {
         // Just throw it as usual if historyService is not enabled.
         throw e;
       }
-      // Even if history-service is enabled, treat all exceptions still the same
-      // except the following
-      if (!(e.getClass() == ApplicationNotFoundException.class)) {
-        throw e;
-      }
-      response = historyManager.getApplicationReport(request);
+      //Fetch the application report from AHS
+      appReport = historyManager.
+          getApplicationReport(request).getApplicationReport();
+      fetchedAppReport = new FetchedAppReport(appReport, AppReportSource.AHS);
     }
-    return response.getApplicationReport();
+    return fetchedAppReport;
   }
 
   public void stop() {
@@ -147,4 +148,28 @@ public class AppReportFetcher {
       RPC.stopProxy(this.historyManager);
     }
   }
+
+  /*
+   * This class creates a bundle of the application report and the source from
+   * where the the report was fetched. This allows the WebAppProxyServlet
+   * to make decisions for the application report based on the source.
+   */
+  static class FetchedAppReport {
+    private ApplicationReport appReport;
+    private AppReportSource appReportSource;
+
+    public FetchedAppReport(ApplicationReport appReport,
+        AppReportSource appReportSource) {
+      this.appReport = appReport;
+      this.appReportSource = appReportSource;
+    }
+
+    public AppReportSource getAppReportSource() {
+      return this.appReportSource;
+    }
+
+    public ApplicationReport getApplicationReport() {
+      return this.appReport;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692d51c0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
index 33f36f0..0e988b8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
@@ -49,6 +49,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.webproxy.AppReportFetcher.AppReportSource;
+import org.apache.hadoop.yarn.server.webproxy.AppReportFetcher.FetchedAppReport;
 import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.StringHelper;
 import org.apache.hadoop.yarn.util.TrackingUriPlugin;
@@ -90,6 +92,7 @@ public class WebAppProxyServlet extends HttpServlet {
 
   private transient List<TrackingUriPlugin> trackingUriPlugins;
   private final String rmAppPageUrlBase;
+  private final String ahsAppPageUrlBase;
   private transient YarnConfiguration conf;
 
   /**
@@ -125,6 +128,9 @@ public class WebAppProxyServlet extends HttpServlet {
             TrackingUriPlugin.class);
     this.rmAppPageUrlBase = StringHelper.pjoin(
         WebAppUtils.getResolvedRMWebAppURLWithScheme(conf), "cluster", "app");
+    this.ahsAppPageUrlBase = StringHelper.pjoin(
+        WebAppUtils.getHttpSchemePrefix(conf) + WebAppUtils
+        .getAHSWebAppURLWithoutScheme(conf), "applicationhistory", "apps");
   }
 
   /**
@@ -266,7 +272,7 @@ public class WebAppProxyServlet extends HttpServlet {
     return b != null ? b : false;
   }
   
-  private ApplicationReport getApplicationReport(ApplicationId id)
+  private FetchedAppReport getApplicationReport(ApplicationId id)
       throws IOException, YarnException {
     return ((AppReportFetcher) getServletContext()
         .getAttribute(WebAppProxy.FETCHER_ATTRIBUTE)).getApplicationReport(id);
@@ -345,9 +351,18 @@ public class WebAppProxyServlet extends HttpServlet {
       
       boolean checkUser = securityEnabled && (!userWasWarned || !userApproved);
 
-      ApplicationReport applicationReport;
+      FetchedAppReport fetchedAppReport = null;
+      ApplicationReport applicationReport = null;
       try {
-        applicationReport = getApplicationReport(id);
+        fetchedAppReport = getApplicationReport(id);
+        if (fetchedAppReport != null) {
+          if (fetchedAppReport.getAppReportSource() != AppReportSource.RM &&
+              fetchedAppReport.getAppReportSource() != AppReportSource.AHS) {
+            throw new UnsupportedOperationException("Application report not "
+                + "fetched from RM or history server.");
+          }
+          applicationReport = fetchedAppReport.getApplicationReport();
+        }
       } catch (ApplicationNotFoundException e) {
         applicationReport = null;
       }
@@ -363,16 +378,29 @@ public class WebAppProxyServlet extends HttpServlet {
           return;
         }
 
-        notFound(resp, "Application " + appId + " could not be found, " +
-                       "please try the history server");
+        notFound(resp, "Application " + appId + " could not be found " +
+            "in RM or history server");
         return;
       }
       String original = applicationReport.getOriginalTrackingUrl();
       URI trackingUri;
-      // fallback to ResourceManager's app page if no tracking URI provided
-      if(original == null || original.equals("N/A")) {
-        ProxyUtils.sendRedirect(req, resp, 
-            StringHelper.pjoin(rmAppPageUrlBase, id.toString()));
+      if (original == null || original.equals("N/A") || original.equals("")) {
+        if (fetchedAppReport.getAppReportSource() == AppReportSource.RM) {
+          // fallback to ResourceManager's app page if no tracking URI provided
+          // and Application Report was fetched from RM
+          LOG.debug("Original tracking url is '{}'. Redirecting to RM app page",
+              original == null? "NULL" : original);
+          ProxyUtils.sendRedirect(req, resp,
+              StringHelper.pjoin(rmAppPageUrlBase, id.toString()));
+        } else if (fetchedAppReport.getAppReportSource()
+              == AppReportSource.AHS) {
+          // fallback to Application History Server app page if the application
+          // report was fetched from AHS
+          LOG.debug("Original tracking url is '{}'. Redirecting to AHS app page"
+              , original == null? "NULL" : original);
+          ProxyUtils.sendRedirect(req, resp,
+              StringHelper.pjoin(ahsAppPageUrlBase, id.toString()));
+        }
         return;
       } else {
         if (ProxyUriUtils.getSchemeFromUrl(original).isEmpty()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692d51c0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
index 8e68c38..68d1258 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
@@ -27,6 +27,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.net.ConnectException;
 import java.net.HttpCookie;
 import java.net.HttpURLConnection;
 import java.net.URI;
@@ -76,6 +77,7 @@ public class TestWebAppProxyServlet {
   private static int numberOfHeaders = 0;
   private static final String UNKNOWN_HEADER = "Unknown-Header";
   private static boolean hasUnknownHeader = false;
+  Configuration configuration = new Configuration();
 
 
   /**
@@ -137,8 +139,6 @@ public class TestWebAppProxyServlet {
 
   @Test(timeout=5000)
   public void testWebAppProxyServlet() throws Exception {
-
-    Configuration configuration = new Configuration();
     configuration.set(YarnConfiguration.PROXY_ADDRESS, "localhost:9090");
     // overriding num of web server threads, see HttpServer.HTTP_MAXTHREADS 
     configuration.setInt("hadoop.http.max.threads", 5);
@@ -166,6 +166,7 @@ public class TestWebAppProxyServlet {
       proxyConn.connect();
       assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR,
           proxyConn.getResponseCode());
+
       // set true Application ID in url
       URL url = new URL("http://localhost:" + proxyPort + "/proxy/application_00_0");
       proxyConn = (HttpURLConnection) url.openConnection();
@@ -220,13 +221,70 @@ public class TestWebAppProxyServlet {
       LOG.info("ProxyConn.getHeaderField(): " +  proxyConn.getHeaderField(ProxyUtils.LOCATION));
       assertEquals("http://localhost:" + originalPort
           + "/foo/bar/test/tez?a=b&x=y&h=p#main", proxyConn.getURL().toString());
-
     } finally {
       proxy.close();
     }
   }
 
   @Test(timeout=5000)
+  public void testAppReportForEmptyTrackingUrl() throws Exception {
+    configuration.set(YarnConfiguration.PROXY_ADDRESS, "localhost:9090");
+    // overriding num of web server threads, see HttpServer.HTTP_MAXTHREADS
+    configuration.setInt("hadoop.http.max.threads", 5);
+    WebAppProxyServerForTest proxy = new WebAppProxyServerForTest();
+    proxy.init(configuration);
+    proxy.start();
+
+    int proxyPort = proxy.proxy.proxyServer.getConnectorAddress(0).getPort();
+    AppReportFetcherForTest appReportFetcher = proxy.proxy.appReportFetcher;
+
+    try {
+    //set AHS_ENBALED = false to simulate getting the app report from RM
+    configuration.setBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
+        false);
+    ApplicationId app = ApplicationId.newInstance(0, 0);
+    appReportFetcher.answer = 6;
+    URL url = new URL("http://localhost:" + proxyPort +
+        "/proxy/" + app.toString());
+    HttpURLConnection proxyConn = (HttpURLConnection) url.openConnection();
+    proxyConn.connect();
+    try {
+      proxyConn.getResponseCode();
+    } catch (ConnectException e) {
+      // Connection Exception is expected as we have set
+      // appReportFetcher.answer = 6, which does not set anything for
+      // original tracking url field in the app report.
+    }
+    String appAddressInRm =
+        WebAppUtils.getResolvedRMWebAppURLWithScheme(configuration) +
+        "/cluster" + "/app/" + app.toString();
+    assertTrue("Webapp proxy servlet should have redirected to RM",
+        proxyConn.getURL().toString().equals(appAddressInRm));
+
+    //set AHS_ENBALED = true to simulate getting the app report from AHS
+    configuration.setBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
+        true);
+    proxyConn = (HttpURLConnection) url.openConnection();
+    proxyConn.connect();
+    try {
+      proxyConn.getResponseCode();
+    } catch (ConnectException e) {
+      // Connection Exception is expected as we have set
+      // appReportFetcher.answer = 6, which does not set anything for
+      // original tracking url field in the app report.
+    }
+    String appAddressInAhs = WebAppUtils.getHttpSchemePrefix(configuration) +
+        WebAppUtils.getAHSWebAppURLWithoutScheme(configuration) +
+        "/applicationhistory" + "/apps/" + app.toString();
+    assertTrue("Webapp proxy servlet should have redirected to AHS",
+        proxyConn.getURL().toString().equals(appAddressInAhs));
+    }
+    finally {
+      proxy.close();
+    }
+  }
+
+  @Test(timeout=5000)
   public void testWebAppProxyPassThroughHeaders() throws Exception {
     Configuration configuration = new Configuration();
     configuration.set(YarnConfiguration.PROXY_ADDRESS, "localhost:9091");
@@ -398,49 +456,70 @@ public class TestWebAppProxyServlet {
   }
 
   private class AppReportFetcherForTest extends AppReportFetcher {
-    
     int answer = 0;
     
     public AppReportFetcherForTest(Configuration conf) {
       super(conf);
     }
 
-    public ApplicationReport getApplicationReport(ApplicationId appId)
+    public FetchedAppReport getApplicationReport(ApplicationId appId)
         throws YarnException {
       if (answer == 0) {
         return getDefaultApplicationReport(appId);
       } else if (answer == 1) {
         return null;
       } else if (answer == 2) {
-        ApplicationReport result = getDefaultApplicationReport(appId);
-        result.setUser("user");
+        FetchedAppReport result = getDefaultApplicationReport(appId);
+        result.getApplicationReport().setUser("user");
         return result;
       } else if (answer == 3) {
-        ApplicationReport result =  getDefaultApplicationReport(appId);
-        result.setYarnApplicationState(YarnApplicationState.KILLED);
+        FetchedAppReport result =  getDefaultApplicationReport(appId);
+        result.getApplicationReport().
+            setYarnApplicationState(YarnApplicationState.KILLED);
         return result;
       } else if (answer == 4) {
         throw new ApplicationNotFoundException("Application is not found");
       } else if (answer == 5) {
         // test user-provided path and query parameter can be appended to the
         // original tracking url
-        ApplicationReport result = getDefaultApplicationReport(appId);
-        result.setOriginalTrackingUrl("localhost:" + originalPort
-            + "/foo/bar?a=b#main");
-        result.setYarnApplicationState(YarnApplicationState.FINISHED);
+        FetchedAppReport result = getDefaultApplicationReport(appId);
+        result.getApplicationReport().setOriginalTrackingUrl("localhost:"
+            + originalPort + "/foo/bar?a=b#main");
+        result.getApplicationReport().
+            setYarnApplicationState(YarnApplicationState.FINISHED);
         return result;
+      } else if (answer == 6) {
+        return getDefaultApplicationReport(appId, false);
       }
       return null;
     }
 
-    private ApplicationReport getDefaultApplicationReport(ApplicationId appId) {
+    /*
+     * If this method is called with isTrackingUrl=false, no tracking url
+     * will set in the app report. Hence, there will be a connection exception
+     * when the prxyCon tries to connect.
+     */
+    private FetchedAppReport getDefaultApplicationReport(ApplicationId appId,
+        boolean isTrackingUrl) {
+      FetchedAppReport fetchedReport;
       ApplicationReport result = new ApplicationReportPBImpl();
       result.setApplicationId(appId);
-      result.setOriginalTrackingUrl("localhost:" + originalPort + "/foo/bar");
       result.setYarnApplicationState(YarnApplicationState.RUNNING);
       result.setUser(CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER);
-      return result;
+      if (isTrackingUrl) {
+        result.setOriginalTrackingUrl("localhost:" + originalPort + "/foo/bar");
+      }
+      if(configuration.getBoolean(YarnConfiguration.
+          APPLICATION_HISTORY_ENABLED, false)) {
+        fetchedReport = new FetchedAppReport(result, AppReportSource.AHS);
+      } else {
+        fetchedReport = new FetchedAppReport(result, AppReportSource.RM);
+      }
+      return fetchedReport;
+    }
+
+    private FetchedAppReport getDefaultApplicationReport(ApplicationId appId) {
+      return getDefaultApplicationReport(appId, true);
     }
-    
   }
 }


[09/21] hadoop git commit: YARN-3866. AM-RM protocol changes to support container resizing. Contributed by Meng Ding

Posted by wa...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java
deleted file mode 100644
index 29b0ffe..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java
+++ /dev/null
@@ -1,66 +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;
-
-import org.junit.Assert;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-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.api.records.impl.pb.ContainerResourceDecreasePBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto;
-import org.junit.Test;
-
-public class TestContainerResourceDecrease {
-  @Test
-  public void testResourceDecreaseContext() {
-    ContainerId containerId = ContainerId
-        .newContainerId(ApplicationAttemptId.newInstance(
-            ApplicationId.newInstance(1234, 3), 3), 7);
-    Resource resource = Resource.newInstance(1023, 3);
-    ContainerResourceDecrease ctx = ContainerResourceDecrease.newInstance(
-        containerId, resource);
-
-    // get proto and recover to ctx
-    ContainerResourceDecreaseProto proto = 
-        ((ContainerResourceDecreasePBImpl) ctx).getProto();
-    ctx = new ContainerResourceDecreasePBImpl(proto);
-
-    // check values
-    Assert.assertEquals(ctx.getCapability(), resource);
-    Assert.assertEquals(ctx.getContainerId(), containerId);
-  }
-  
-  @Test
-  public void testResourceDecreaseContextWithNull() {
-    ContainerResourceDecrease ctx = ContainerResourceDecrease.newInstance(null,
-        null);
-    
-    // get proto and recover to ctx;
-    ContainerResourceDecreaseProto proto = 
-        ((ContainerResourceDecreasePBImpl) ctx).getProto();
-    ctx = new ContainerResourceDecreasePBImpl(proto);
-
-    // check values
-    Assert.assertNull(ctx.getCapability());
-    Assert.assertNull(ctx.getContainerId());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java
deleted file mode 100644
index 932d5a7..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java
+++ /dev/null
@@ -1,74 +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;
-
-import java.util.Arrays;
-
-import org.junit.Assert;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-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.api.records.impl.pb.ContainerResourceIncreasePBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto;
-import org.junit.Test;
-
-public class TestContainerResourceIncrease {
-  @Test
-  public void testResourceIncreaseContext() {
-    byte[] identifier = new byte[] { 1, 2, 3, 4 };
-    Token token = Token.newInstance(identifier, "", "".getBytes(), "");
-    ContainerId containerId = ContainerId
-        .newContainerId(ApplicationAttemptId.newInstance(
-            ApplicationId.newInstance(1234, 3), 3), 7);
-    Resource resource = Resource.newInstance(1023, 3);
-    ContainerResourceIncrease ctx = ContainerResourceIncrease.newInstance(
-        containerId, resource, token);
-
-    // get proto and recover to ctx
-    ContainerResourceIncreaseProto proto = 
-        ((ContainerResourceIncreasePBImpl) ctx).getProto();
-    ctx = new ContainerResourceIncreasePBImpl(proto);
-
-    // check values
-    Assert.assertEquals(ctx.getCapability(), resource);
-    Assert.assertEquals(ctx.getContainerId(), containerId);
-    Assert.assertTrue(Arrays.equals(ctx.getContainerToken().getIdentifier()
-        .array(), identifier));
-  }
-  
-  @Test
-  public void testResourceIncreaseContextWithNull() {
-    ContainerResourceIncrease ctx = ContainerResourceIncrease.newInstance(null,
-        null, null);
-    
-    // get proto and recover to ctx;
-    ContainerResourceIncreaseProto proto = 
-        ((ContainerResourceIncreasePBImpl) ctx).getProto();
-    ctx = new ContainerResourceIncreasePBImpl(proto);
-
-    // check values
-    Assert.assertNull(ctx.getContainerToken());
-    Assert.assertNull(ctx.getCapability());
-    Assert.assertNull(ctx.getContainerId());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java
deleted file mode 100644
index cf4dabf..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java
+++ /dev/null
@@ -1,68 +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;
-
-import org.junit.Assert;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-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.api.records.impl.pb.ContainerResourceIncreaseRequestPBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto;
-import org.junit.Test;
-
-public class TestContainerResourceIncreaseRequest {
-  @Test
-  public void ContainerResourceIncreaseRequest() {
-    ContainerId containerId = ContainerId
-        .newContainerId(ApplicationAttemptId.newInstance(
-            ApplicationId.newInstance(1234, 3), 3), 7);
-    Resource resource = Resource.newInstance(1023, 3);
-    ContainerResourceIncreaseRequest context = ContainerResourceIncreaseRequest
-        .newInstance(containerId, resource);
-
-    // to proto and get it back
-    ContainerResourceIncreaseRequestProto proto = 
-        ((ContainerResourceIncreaseRequestPBImpl) context).getProto();
-    ContainerResourceIncreaseRequest contextRecover = 
-        new ContainerResourceIncreaseRequestPBImpl(proto);
-
-    // check value
-    Assert.assertEquals(contextRecover.getContainerId(), containerId);
-    Assert.assertEquals(contextRecover.getCapability(), resource);
-  }
-
-  @Test
-  public void testResourceChangeContextWithNullField() {
-    ContainerResourceIncreaseRequest context = ContainerResourceIncreaseRequest
-        .newInstance(null, null);
-
-    // to proto and get it back
-    ContainerResourceIncreaseRequestProto proto = 
-        ((ContainerResourceIncreaseRequestPBImpl) context).getProto();
-    ContainerResourceIncreaseRequest contextRecover = 
-        new ContainerResourceIncreaseRequestPBImpl(proto);
-
-    // check value
-    Assert.assertNull(contextRecover.getContainerId());
-    Assert.assertNull(contextRecover.getCapability());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfe2cb84/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 6357c36..0979c75 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
@@ -113,9 +113,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.ContainerResourceDecrease;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
-import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
+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;
@@ -155,9 +153,7 @@ 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.ContainerResourceDecreasePBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreasePBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreaseRequestPBImpl;
+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;
@@ -190,9 +186,7 @@ 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.ContainerResourceDecreaseProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto;
+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;
@@ -471,9 +465,7 @@ public class TestPBImplRecords {
     generateByNewInstance(ContainerLaunchContext.class);
     generateByNewInstance(ApplicationSubmissionContext.class);
     generateByNewInstance(ContainerReport.class);
-    generateByNewInstance(ContainerResourceDecrease.class);
-    generateByNewInstance(ContainerResourceIncrease.class);
-    generateByNewInstance(ContainerResourceIncreaseRequest.class);
+    generateByNewInstance(ContainerResourceChangeRequest.class);
     generateByNewInstance(ContainerStatus.class);
     generateByNewInstance(PreemptionContainer.class);
     generateByNewInstance(PreemptionResourceRequest.class);
@@ -959,21 +951,9 @@ public class TestPBImplRecords {
   }
 
   @Test
-  public void testContainerResourceDecreasePBImpl() throws Exception {
-    validatePBImplRecord(ContainerResourceDecreasePBImpl.class,
-        ContainerResourceDecreaseProto.class);
-  }
-
-  @Test
-  public void testContainerResourceIncreasePBImpl() throws Exception {
-    validatePBImplRecord(ContainerResourceIncreasePBImpl.class,
-        ContainerResourceIncreaseProto.class);
-  }
-
-  @Test
-  public void testContainerResourceIncreaseRequestPBImpl() throws Exception {
-    validatePBImplRecord(ContainerResourceIncreaseRequestPBImpl.class,
-        ContainerResourceIncreaseRequestProto.class);
+  public void testContainerResourceChangeRequestPBImpl() throws Exception {
+    validatePBImplRecord(ContainerResourceChangeRequestPBImpl.class,
+        ContainerResourceChangeRequestProto.class);
   }
 
   @Test


[13/21] hadoop git commit: YARN-1644. RM-NM protocol changes and NodeStatusUpdater implementation to support container resizing. Contributed by Meng Ding

Posted by wa...@apache.org.
YARN-1644. RM-NM protocol changes and NodeStatusUpdater implementation to support container resizing. Contributed by Meng Ding


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

Branch: refs/heads/YARN-1197
Commit: c3dc1af072574f5890a8d43e4d60526951b4b8bc
Parents: c59ae4e
Author: Jian He <ji...@apache.org>
Authored: Thu Aug 20 21:04:14 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Sep 23 13:29:37 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../yarn/client/TestResourceTrackerOnHA.java    |   2 +-
 .../protocolrecords/NodeHeartbeatResponse.java  |   4 +
 .../impl/pb/NodeHeartbeatResponsePBImpl.java    |  76 +++++-
 .../yarn/server/api/records/NodeStatus.java     |  15 +-
 .../api/records/impl/pb/NodeStatusPBImpl.java   |  75 +++++-
 .../main/proto/yarn_server_common_protos.proto  |   3 +-
 .../yarn_server_common_service_protos.proto     |   1 +
 .../hadoop/yarn/TestYarnServerApiClasses.java   |  39 ++-
 .../hadoop/yarn/server/nodemanager/Context.java |   3 +
 .../yarn/server/nodemanager/NodeManager.java    |  10 +
 .../nodemanager/NodeStatusUpdaterImpl.java      |  57 +++-
 .../containermanager/ContainerManagerImpl.java  | 159 +++++++-----
 .../nodemanager/TestNodeManagerResync.java      | 258 +++++++++++++++++++
 .../amrmproxy/BaseAMRMProxyTest.java            |   5 +
 .../amrmproxy/MockResourceManagerFacade.java    |   6 +-
 .../containermanager/TestContainerManager.java  |   2 +-
 17 files changed, 628 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 3734fa6..1872b1a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -215,6 +215,9 @@ Release 2.8.0 - UNRELEASED
     YARN-1643. Make ContainersMonitor support changing monitoring size of an
     allocated container. (Meng Ding and Wangda Tan)
 
+    YARN-1644. RM-NM protocol changes and NodeStatusUpdater implementation to
+    support container resizing. (Meng Ding via jianhe)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java
index 6cdf87f..338198b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestResourceTrackerOnHA.java
@@ -68,7 +68,7 @@ public class TestResourceTrackerOnHA extends ProtocolHATestBase{
     failoverThread = createAndStartFailoverThread();
     NodeStatus status =
         NodeStatus.newInstance(NodeId.newInstance("localhost", 0), 0, null,
-            null, null, null, null);
+            null, null, null, null, null);
     NodeHeartbeatRequest request2 =
         NodeHeartbeatRequest.newInstance(status, null, null,null);
     resourceTracker.nodeHeartbeat(request2);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
index 1498a0c..38fbc82 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeHeartbeatResponse.java
@@ -24,6 +24,7 @@ import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 
@@ -70,4 +71,7 @@ public interface NodeHeartbeatResponse {
   
   boolean getAreNodeLabelsAcceptedByRM();
   void setAreNodeLabelsAcceptedByRM(boolean areNodeLabelsAcceptedByRM);
+
+  List<Container> getContainersToDecrease();
+  void addAllContainersToDecrease(List<Container> containersToDecrease);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
index e27d8ca..12c5230 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
@@ -27,12 +27,15 @@ import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.MasterKeyProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeActionProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto;
@@ -58,7 +61,9 @@ public class NodeHeartbeatResponsePBImpl extends
 
   private MasterKey containerTokenMasterKey = null;
   private MasterKey nmTokenMasterKey = null;
-  
+
+  private List<Container> containersToDecrease = null;
+
   public NodeHeartbeatResponsePBImpl() {
     builder = NodeHeartbeatResponseProto.newBuilder();
   }
@@ -96,6 +101,9 @@ public class NodeHeartbeatResponsePBImpl extends
     if (this.systemCredentials != null) {
       addSystemCredentialsToProto();
     }
+    if (this.containersToDecrease != null) {
+      addContainersToDecreaseToProto();
+    }
   }
 
   private void addSystemCredentialsToProto() {
@@ -408,6 +416,64 @@ public class NodeHeartbeatResponsePBImpl extends
     builder.addAllApplicationsToCleanup(iterable);
   }
 
+  private void initContainersToDecrease() {
+    if (this.containersToDecrease != null) {
+      return;
+    }
+    NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerProto> list = p.getContainersToDecreaseList();
+    this.containersToDecrease = new ArrayList<>();
+
+    for (ContainerProto c : list) {
+      this.containersToDecrease.add(convertFromProtoFormat(c));
+    }
+  }
+
+  @Override
+  public List<Container> getContainersToDecrease() {
+    initContainersToDecrease();
+    return this.containersToDecrease;
+  }
+
+  @Override
+  public void addAllContainersToDecrease(
+      final List<Container> containersToDecrease) {
+    if (containersToDecrease == null) {
+      return;
+    }
+    initContainersToDecrease();
+    this.containersToDecrease.addAll(containersToDecrease);
+  }
+
+  private void addContainersToDecreaseToProto() {
+    maybeInitBuilder();
+    builder.clearContainersToDecrease();
+    if (this.containersToDecrease == null) {
+      return;
+    }
+    Iterable<ContainerProto> iterable = new
+        Iterable<ContainerProto>() {
+      @Override
+      public Iterator<ContainerProto> iterator() {
+        return new Iterator<ContainerProto>() {
+          private Iterator<Container> iter = containersToDecrease.iterator();
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+          @Override
+          public ContainerProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+    builder.addAllContainersToDecrease(iterable);
+  }
 
   @Override
   public Map<ApplicationId, ByteBuffer> getSystemCredentialsForApps() {
@@ -484,6 +550,14 @@ public class NodeHeartbeatResponsePBImpl extends
     return ((MasterKeyPBImpl) t).getProto();
   }
 
+  private ContainerPBImpl convertFromProtoFormat(ContainerProto p) {
+    return new ContainerPBImpl(p);
+  }
+
+  private ContainerProto convertToProtoFormat(Container t) {
+    return ((ContainerPBImpl) t).getProto();
+  }
+
   @Override
   public boolean getAreNodeLabelsAcceptedByRM() {
     NodeHeartbeatResponseProtoOrBuilder p =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java
index 7b8262f..2d62db5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java
@@ -24,6 +24,7 @@ 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.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.util.Records;
@@ -48,6 +49,7 @@ public abstract class NodeStatus {
    * @param nodeHealthStatus Health status of the node.
    * @param containersUtilization Utilization of the containers in this node.
    * @param nodeUtilization Utilization of the node.
+   * @param increasedContainers Containers whose resource has been increased.
    * @return New {@code NodeStatus} with the provided information.
    */
   public static NodeStatus newInstance(NodeId nodeId, int responseId,
@@ -55,7 +57,8 @@ public abstract class NodeStatus {
       List<ApplicationId> keepAliveApplications,
       NodeHealthStatus nodeHealthStatus,
       ResourceUtilization containersUtilization,
-      ResourceUtilization nodeUtilization) {
+      ResourceUtilization nodeUtilization,
+      List<Container> increasedContainers) {
     NodeStatus nodeStatus = Records.newRecord(NodeStatus.class);
     nodeStatus.setResponseId(responseId);
     nodeStatus.setNodeId(nodeId);
@@ -64,6 +67,7 @@ public abstract class NodeStatus {
     nodeStatus.setNodeHealthStatus(nodeHealthStatus);
     nodeStatus.setContainersUtilization(containersUtilization);
     nodeStatus.setNodeUtilization(nodeUtilization);
+    nodeStatus.setIncreasedContainers(increasedContainers);
     return nodeStatus;
   }
 
@@ -108,4 +112,13 @@ public abstract class NodeStatus {
   @Unstable
   public abstract void setNodeUtilization(
       ResourceUtilization nodeUtilization);
+
+  @Public
+  @Unstable
+  public abstract List<Container> getIncreasedContainers();
+
+  @Private
+  @Unstable
+  public abstract void setIncreasedContainers(
+      List<Container> increasedContainers);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java
index 7d4e83f..e34451d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java
@@ -24,13 +24,16 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
+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.NodeIdPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeHealthStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
@@ -49,7 +52,8 @@ public class NodeStatusPBImpl extends NodeStatus {
   private List<ContainerStatus> containers = null;
   private NodeHealthStatus nodeHealthStatus = null;
   private List<ApplicationId> keepAliveApplications = null;
-  
+  private List<Container> increasedContainers = null;
+
   public NodeStatusPBImpl() {
     builder = NodeStatusProto.newBuilder();
   }
@@ -79,6 +83,9 @@ public class NodeStatusPBImpl extends NodeStatus {
     if (this.keepAliveApplications != null) {
       addKeepAliveApplicationsToProto();
     }
+    if (this.increasedContainers != null) {
+      addIncreasedContainersToProto();
+    }
   }
 
   private synchronized void mergeLocalToProto() {
@@ -165,6 +172,37 @@ public class NodeStatusPBImpl extends NodeStatus {
     builder.addAllKeepAliveApplications(iterable);
   }
 
+  private synchronized void addIncreasedContainersToProto() {
+    maybeInitBuilder();
+    builder.clearIncreasedContainers();
+    if (increasedContainers == null) {
+      return;
+    }
+    Iterable<ContainerProto> iterable = new
+        Iterable<ContainerProto>() {
+      @Override
+      public Iterator<ContainerProto> iterator() {
+        return new Iterator<ContainerProto>() {
+          private Iterator<Container> iter =
+                  increasedContainers.iterator();
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+          @Override
+          public ContainerProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+    builder.addAllIncreasedContainers(iterable);
+  }
+
   @Override
   public int hashCode() {
     return getProto().hashCode();
@@ -336,6 +374,31 @@ public class NodeStatusPBImpl extends NodeStatus {
         .setNodeUtilization(convertToProtoFormat(nodeUtilization));
   }
 
+  @Override
+  public synchronized List<Container> getIncreasedContainers() {
+    if (increasedContainers != null) {
+      return increasedContainers;
+    }
+    NodeStatusProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerProto> list = p.getIncreasedContainersList();
+    this.increasedContainers = new ArrayList<>();
+    for (ContainerProto c : list) {
+      this.increasedContainers.add(convertFromProtoFormat(c));
+    }
+    return this.increasedContainers;
+  }
+
+  @Override
+  public synchronized void setIncreasedContainers(
+      List<Container> increasedContainers) {
+    maybeInitBuilder();
+    if (increasedContainers == null) {
+      builder.clearIncreasedContainers();
+      return;
+    }
+    this.increasedContainers = increasedContainers;
+  }
+
   private NodeIdProto convertToProtoFormat(NodeId nodeId) {
     return ((NodeIdPBImpl)nodeId).getProto();
   }
@@ -377,4 +440,14 @@ public class NodeStatusPBImpl extends NodeStatus {
       ResourceUtilizationProto p) {
     return new ResourceUtilizationPBImpl(p);
   }
+
+  private ContainerPBImpl convertFromProtoFormat(
+      ContainerProto c) {
+    return new ContainerPBImpl(c);
+  }
+
+  private ContainerProto convertToProtoFormat(
+      Container c) {
+    return ((ContainerPBImpl)c).getProto();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
index 901051f..b161f5b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto
@@ -38,6 +38,7 @@ message NodeStatusProto {
   repeated ApplicationIdProto keep_alive_applications = 5;
   optional ResourceUtilizationProto containers_utilization = 6;
   optional ResourceUtilizationProto node_utilization = 7;
+  repeated ContainerProto increased_containers = 8;
 }
 
 message MasterKeyProto {
@@ -60,4 +61,4 @@ message ResourceUtilizationProto {
   optional int32 pmem = 1;
   optional int32 vmem = 2;
   optional float cpu = 3;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
index c122b2a..2db8919 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_service_protos.proto
@@ -82,6 +82,7 @@ message NodeHeartbeatResponseProto {
   repeated ContainerIdProto containers_to_be_removed_from_nm = 9;
   repeated SystemCredentialsForAppsProto system_credentials_for_apps = 10;
   optional bool areNodeLabelsAcceptedByRM = 11 [default = false];
+  repeated ContainerProto containers_to_decrease = 12;
 }
 
 message SystemCredentialsForAppsProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
index d9eeb9d..c9427dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
@@ -29,6 +29,7 @@ import java.util.HashSet;
 
 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.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -168,6 +169,20 @@ public class TestYarnServerApiClasses {
     assertTrue(copy.getAreNodeLabelsAcceptedByRM());
   }
 
+  @Test
+  public void testNodeHeartbeatResponsePBImplWithDecreasedContainers() {
+    NodeHeartbeatResponsePBImpl original = new NodeHeartbeatResponsePBImpl();
+    original.addAllContainersToDecrease(
+        Arrays.asList(getDecreasedContainer(1, 2, 2048, 2),
+            getDecreasedContainer(2, 3, 1024, 1)));
+    NodeHeartbeatResponsePBImpl copy =
+        new NodeHeartbeatResponsePBImpl(original.getProto());
+    assertEquals(1, copy.getContainersToDecrease().get(0)
+        .getId().getContainerId());
+    assertEquals(1024, copy.getContainersToDecrease().get(1)
+        .getResource().getMemory());
+  }
+
   /**
    * Test RegisterNodeManagerRequestPBImpl.
    */
@@ -244,6 +259,9 @@ public class TestYarnServerApiClasses {
     original.setNodeHealthStatus(getNodeHealthStatus());
     original.setNodeId(getNodeId());
     original.setResponseId(1);
+    original.setIncreasedContainers(
+        Arrays.asList(getIncreasedContainer(1, 2, 2048, 2),
+            getIncreasedContainer(2, 3, 4096, 3)));
 
     NodeStatusPBImpl copy = new NodeStatusPBImpl(original.getProto());
     assertEquals(3L, copy.getContainersStatuses().get(1).getContainerId()
@@ -252,7 +270,10 @@ public class TestYarnServerApiClasses {
     assertEquals(1000, copy.getNodeHealthStatus().getLastHealthReportTime());
     assertEquals(9090, copy.getNodeId().getPort());
     assertEquals(1, copy.getResponseId());
-
+    assertEquals(1, copy.getIncreasedContainers().get(0)
+        .getId().getContainerId());
+    assertEquals(4096, copy.getIncreasedContainers().get(1)
+        .getResource().getMemory());
   }
 
   @Test
@@ -347,6 +368,22 @@ public class TestYarnServerApiClasses {
     return new ApplicationIdPBImpl(appId.getProto());
   }
 
+  private Container getDecreasedContainer(int containerID,
+      int appAttemptId, int memory, int vCores) {
+    ContainerId containerId = getContainerId(containerID, appAttemptId);
+    Resource capability = Resource.newInstance(memory, vCores);
+    return Container.newInstance(
+        containerId, null, null, capability, null, null);
+  }
+
+  private Container getIncreasedContainer(int containerID,
+      int appAttemptId, int memory, int vCores) {
+    ContainerId containerId = getContainerId(containerID, appAttemptId);
+    Resource capability = Resource.newInstance(memory, vCores);
+    return Container.newInstance(
+        containerId, null, null, capability, null, null);
+  }
+
   private NodeStatus getNodeStatus() {
     NodeStatus status = recordFactory.newRecordInstance(NodeStatus.class);
     status.setContainersStatuses(new ArrayList<ContainerStatus>());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
index 52d937b..9c2d1fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
@@ -62,6 +62,9 @@ public interface Context {
 
   ConcurrentMap<ContainerId, Container> getContainers();
 
+  ConcurrentMap<ContainerId, org.apache.hadoop.yarn.api.records.Container>
+      getIncreasedContainers();
+
   NMContainerTokenSecretManager getContainerTokenSecretManager();
   
   NMTokenSecretManagerInNM getNMTokenSecretManager();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 3cf9f1a..184f489 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -439,6 +439,10 @@ public class NodeManager extends CompositeService
     protected final ConcurrentMap<ContainerId, Container> containers =
         new ConcurrentSkipListMap<ContainerId, Container>();
 
+    protected final ConcurrentMap<ContainerId,
+        org.apache.hadoop.yarn.api.records.Container> increasedContainers =
+            new ConcurrentHashMap<>();
+
     private final NMContainerTokenSecretManager containerTokenSecretManager;
     private final NMTokenSecretManagerInNM nmTokenSecretManager;
     private ContainerManagementProtocol containerManager;
@@ -493,6 +497,12 @@ public class NodeManager extends CompositeService
     }
 
     @Override
+    public ConcurrentMap<ContainerId, org.apache.hadoop.yarn.api.records.Container>
+        getIncreasedContainers() {
+      return this.increasedContainers;
+    }
+
+    @Override
     public NMContainerTokenSecretManager getContainerTokenSecretManager() {
       return this.containerTokenSecretManager;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index aa51e5c..f8ce90f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -310,18 +310,28 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
   @VisibleForTesting
   protected void registerWithRM()
       throws YarnException, IOException {
-    List<NMContainerStatus> containerReports = getNMContainerStatuses();
+    RegisterNodeManagerResponse regNMResponse;
     Set<NodeLabel> nodeLabels = nodeLabelsHandler.getNodeLabelsForRegistration();
-    RegisterNodeManagerRequest request =
-        RegisterNodeManagerRequest.newInstance(nodeId, httpPort, totalResource,
-            nodeManagerVersionId, containerReports, getRunningApplications(),
-            nodeLabels);
-    if (containerReports != null) {
-      LOG.info("Registering with RM using containers :" + containerReports);
+ 
+    // Synchronize NM-RM registration with
+    // ContainerManagerImpl#increaseContainersResource and
+    // ContainerManagerImpl#startContainers to avoid race condition
+    // during RM recovery
+    synchronized (this.context) {
+      List<NMContainerStatus> containerReports = getNMContainerStatuses();
+      RegisterNodeManagerRequest request =
+          RegisterNodeManagerRequest.newInstance(nodeId, httpPort, totalResource,
+              nodeManagerVersionId, containerReports, getRunningApplications(),
+              nodeLabels);
+      if (containerReports != null) {
+        LOG.info("Registering with RM using containers :" + containerReports);
+      }
+      regNMResponse =
+          resourceTracker.registerNodeManager(request);
+      // Make sure rmIdentifier is set before we release the lock
+      this.rmIdentifier = regNMResponse.getRMIdentifier();
     }
-    RegisterNodeManagerResponse regNMResponse =
-        resourceTracker.registerNodeManager(request);
-    this.rmIdentifier = regNMResponse.getRMIdentifier();
+
     // if the Resource Manager instructs NM to shutdown.
     if (NodeAction.SHUTDOWN.equals(regNMResponse.getNodeAction())) {
       String message =
@@ -418,10 +428,12 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     List<ContainerStatus> containersStatuses = getContainerStatuses();
     ResourceUtilization containersUtilization = getContainersUtilization();
     ResourceUtilization nodeUtilization = getNodeUtilization();
+    List<org.apache.hadoop.yarn.api.records.Container> increasedContainers
+        = getIncreasedContainers();
     NodeStatus nodeStatus =
         NodeStatus.newInstance(nodeId, responseId, containersStatuses,
           createKeepAliveApplicationList(), nodeHealthStatus,
-          containersUtilization, nodeUtilization);
+          containersUtilization, nodeUtilization, increasedContainers);
 
     return nodeStatus;
   }
@@ -448,6 +460,21 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
     return nodeResourceMonitor.getUtilization();
   }
 
+  /* Get the containers whose resource has been increased since last
+   * NM-RM heartbeat.
+   */
+  private List<org.apache.hadoop.yarn.api.records.Container>
+      getIncreasedContainers() {
+    List<org.apache.hadoop.yarn.api.records.Container>
+        increasedContainers = new ArrayList<>(
+            this.context.getIncreasedContainers().values());
+    for (org.apache.hadoop.yarn.api.records.Container
+        container : increasedContainers) {
+      this.context.getIncreasedContainers().remove(container.getId());
+    }
+    return increasedContainers;
+  }
+
   // Iterate through the NMContext and clone and get all the containers'
   // statuses. If it's a completed container, add into the
   // recentlyStoppedContainers collections.
@@ -765,6 +792,14 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
               ((NMContext) context)
                 .setSystemCrendentialsForApps(parseCredentials(systemCredentials));
             }
+
+            List<org.apache.hadoop.yarn.api.records.Container>
+                containersToDecrease = response.getContainersToDecrease();
+            if (!containersToDecrease.isEmpty()) {
+              dispatcher.getEventHandler().handle(
+                  new CMgrDecreaseContainersResourceEvent(containersToDecrease)
+              );
+            }
           } catch (ConnectException e) {
             //catch and throw the exception if tried MAX wait time to connect RM
             dispatcher.getEventHandler().handle(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 4f2ccbe..868d8d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -563,8 +563,7 @@ public class ContainerManagerImpl extends CompositeService implements
 
     List<ApplicationId> appIds =
         new ArrayList<ApplicationId>(applications.keySet());
-    this.handle(
-        new CMgrCompletedAppsEvent(appIds,
+    this.handle(new CMgrCompletedAppsEvent(appIds,
             CMgrCompletedAppsEvent.Reason.ON_SHUTDOWN));
 
     LOG.info("Waiting for Applications to be Finished");
@@ -584,8 +583,8 @@ public class ContainerManagerImpl extends CompositeService implements
     if (applications.isEmpty()) {
       LOG.info("All applications in FINISHED state");
     } else {
-      LOG.info("Done waiting for Applications to be Finished. Still alive: " +
-          applications.keySet());
+      LOG.info("Done waiting for Applications to be Finished. Still alive: "
+          + applications.keySet());
     }
   }
 
@@ -759,13 +758,12 @@ public class ContainerManagerImpl extends CompositeService implements
    * Start a list of containers on this NodeManager.
    */
   @Override
-  public StartContainersResponse
-      startContainers(StartContainersRequest requests) throws YarnException,
-          IOException {
+  public StartContainersResponse startContainers(
+      StartContainersRequest requests) throws YarnException, IOException {
     if (blockNewContainerRequests.get()) {
       throw new NMNotYetReadyException(
-        "Rejecting new containers as NodeManager has not"
-            + " yet connected with ResourceManager");
+          "Rejecting new containers as NodeManager has not"
+              + " yet connected with ResourceManager");
     }
     UserGroupInformation remoteUgi = getRemoteUgi();
     NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
@@ -773,42 +771,50 @@ public class ContainerManagerImpl extends CompositeService implements
     List<ContainerId> succeededContainers = new ArrayList<ContainerId>();
     Map<ContainerId, SerializedException> failedContainers =
         new HashMap<ContainerId, SerializedException>();
-    for (StartContainerRequest request : requests.getStartContainerRequests()) {
-      ContainerId containerId = null;
-      try {
-        if (request.getContainerToken() == null ||
-            request.getContainerToken().getIdentifier() == null) {
-          throw new IOException(INVALID_CONTAINERTOKEN_MSG);
-        }
-        ContainerTokenIdentifier containerTokenIdentifier =
-            BuilderUtils.newContainerTokenIdentifier(request.getContainerToken());
-        verifyAndGetContainerTokenIdentifier(request.getContainerToken(),
-          containerTokenIdentifier);
-        containerId = containerTokenIdentifier.getContainerID();
-
-        // Initialize the AMRMProxy service instance only if the container is of
-        // type AM and if the AMRMProxy service is enabled
-        if (isARMRMProxyEnabled()
-            && containerTokenIdentifier.getContainerType().equals(
-                ContainerType.APPLICATION_MASTER)) {
-          this.amrmProxyService.processApplicationStartRequest(request);
-        }
+    // Synchronize with NodeStatusUpdaterImpl#registerWithRM
+    // to avoid race condition during NM-RM resync (due to RM restart) while a
+    // container is being started, in particular when the container has not yet
+    // been added to the containers map in NMContext.
+    synchronized (this.context) {
+      for (StartContainerRequest request : requests
+          .getStartContainerRequests()) {
+        ContainerId containerId = null;
+        try {
+          if (request.getContainerToken() == null
+              || request.getContainerToken().getIdentifier() == null) {
+            throw new IOException(INVALID_CONTAINERTOKEN_MSG);
+          }
 
-        startContainerInternal(nmTokenIdentifier,
-            containerTokenIdentifier, request);
-        succeededContainers.add(containerId);
-      } catch (YarnException e) {
-        failedContainers.put(containerId, SerializedException.newInstance(e));
-      } catch (InvalidToken ie) {
-        failedContainers.put(containerId, SerializedException.newInstance(ie));
-        throw ie;
-      } catch (IOException e) {
-        throw RPCUtil.getRemoteException(e);
+          ContainerTokenIdentifier containerTokenIdentifier = BuilderUtils
+              .newContainerTokenIdentifier(request.getContainerToken());
+          verifyAndGetContainerTokenIdentifier(request.getContainerToken(),
+              containerTokenIdentifier);
+          containerId = containerTokenIdentifier.getContainerID();
+
+          // Initialize the AMRMProxy service instance only if the container is of
+          // type AM and if the AMRMProxy service is enabled
+          if (isARMRMProxyEnabled() && containerTokenIdentifier
+              .getContainerType().equals(ContainerType.APPLICATION_MASTER)) {
+            this.amrmProxyService.processApplicationStartRequest(request);
+          }
+
+          startContainerInternal(nmTokenIdentifier, containerTokenIdentifier,
+              request);
+          succeededContainers.add(containerId);
+        } catch (YarnException e) {
+          failedContainers.put(containerId, SerializedException.newInstance(e));
+        } catch (InvalidToken ie) {
+          failedContainers
+              .put(containerId, SerializedException.newInstance(ie));
+          throw ie;
+        } catch (IOException e) {
+          throw RPCUtil.getRemoteException(e);
+        }
       }
+      return StartContainersResponse
+          .newInstance(getAuxServiceMetaData(), succeededContainers,
+              failedContainers);
     }
-
-    return StartContainersResponse.newInstance(getAuxServiceMetaData(),
-        succeededContainers, failedContainers);
   }
 
   private ContainerManagerApplicationProto buildAppProto(ApplicationId appId,
@@ -959,7 +965,7 @@ public class ContainerManagerImpl extends CompositeService implements
       InvalidToken {
     byte[] password =
         context.getContainerTokenSecretManager().retrievePassword(
-          containerTokenIdentifier);
+            containerTokenIdentifier);
     byte[] tokenPass = token.getPassword().array();
     if (password == null || tokenPass == null
         || !Arrays.equals(password, tokenPass)) {
@@ -989,32 +995,39 @@ public class ContainerManagerImpl extends CompositeService implements
         = new ArrayList<ContainerId>();
     Map<ContainerId, SerializedException> failedContainers =
         new HashMap<ContainerId, SerializedException>();
-    // Process container resource increase requests
-    for (org.apache.hadoop.yarn.api.records.Token token :
-        requests.getContainersToIncrease()) {
-      ContainerId containerId = null;
-      try {
-        if (token.getIdentifier() == null) {
-          throw new IOException(INVALID_CONTAINERTOKEN_MSG);
+    // Synchronize with NodeStatusUpdaterImpl#registerWithRM
+    // to avoid race condition during NM-RM resync (due to RM restart) while a
+    // container resource is being increased in NM, in particular when the
+    // increased container has not yet been added to the increasedContainers
+    // map in NMContext.
+    synchronized (this.context) {
+      // Process container resource increase requests
+      for (org.apache.hadoop.yarn.api.records.Token token :
+          requests.getContainersToIncrease()) {
+        ContainerId containerId = null;
+        try {
+          if (token.getIdentifier() == null) {
+            throw new IOException(INVALID_CONTAINERTOKEN_MSG);
+          }
+          ContainerTokenIdentifier containerTokenIdentifier =
+              BuilderUtils.newContainerTokenIdentifier(token);
+          verifyAndGetContainerTokenIdentifier(token,
+              containerTokenIdentifier);
+          authorizeStartAndResourceIncreaseRequest(
+              nmTokenIdentifier, containerTokenIdentifier, false);
+          containerId = containerTokenIdentifier.getContainerID();
+          // Reuse the startContainer logic to update NMToken,
+          // as container resource increase request will have come with
+          // an updated NMToken.
+          updateNMTokenIdentifier(nmTokenIdentifier);
+          Resource resource = containerTokenIdentifier.getResource();
+          changeContainerResourceInternal(containerId, resource, true);
+          successfullyIncreasedContainers.add(containerId);
+        } catch (YarnException | InvalidToken e) {
+          failedContainers.put(containerId, SerializedException.newInstance(e));
+        } catch (IOException e) {
+          throw RPCUtil.getRemoteException(e);
         }
-        ContainerTokenIdentifier containerTokenIdentifier =
-            BuilderUtils.newContainerTokenIdentifier(token);
-        verifyAndGetContainerTokenIdentifier(token,
-            containerTokenIdentifier);
-        authorizeStartAndResourceIncreaseRequest(
-            nmTokenIdentifier, containerTokenIdentifier, false);
-        containerId = containerTokenIdentifier.getContainerID();
-        // Reuse the startContainer logic to update NMToken,
-        // as container resource increase request will have come with
-        // an updated NMToken.
-        updateNMTokenIdentifier(nmTokenIdentifier);
-        Resource resource = containerTokenIdentifier.getResource();
-        changeContainerResourceInternal(containerId, resource, true);
-        successfullyIncreasedContainers.add(containerId);
-      } catch (YarnException | InvalidToken e) {
-        failedContainers.put(containerId, SerializedException.newInstance(e));
-      } catch (IOException e) {
-        throw RPCUtil.getRemoteException(e);
       }
     }
     return IncreaseContainersResourceResponse.newInstance(
@@ -1075,6 +1088,16 @@ public class ContainerManagerImpl extends CompositeService implements
           + " is not smaller than the current resource "
           + currentResource.toString());
     }
+    if (increase) {
+      org.apache.hadoop.yarn.api.records.Container increasedContainer =
+          org.apache.hadoop.yarn.api.records.Container.newInstance(
+              containerId, null, null, targetResource, null, null);
+      if (context.getIncreasedContainers().putIfAbsent(containerId,
+          increasedContainer) != null){
+        throw RPCUtil.getRemoteException("Container " + containerId.toString()
+            + " resource is being increased.");
+      }
+    }
     this.readLock.lock();
     try {
       if (!serviceStopped) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
index c22d475..4250ac3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
@@ -18,21 +18,35 @@
 
 package org.apache.hadoop.yarn.server.nodemanager;
 
+import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -41,8 +55,13 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 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.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
@@ -50,6 +69,8 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 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.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
@@ -57,12 +78,15 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -87,7 +111,10 @@ public class TestNodeManagerResync {
   private AtomicBoolean isNMShutdownCalled = new AtomicBoolean(false);
   private final NodeManagerEvent resyncEvent =
       new NodeManagerEvent(NodeManagerEventType.RESYNC);
+  private final long DUMMY_RM_IDENTIFIER = 1234;
 
+  protected static Log LOG = LogFactory
+      .getLog(TestNodeManagerResync.class);
 
   @Before
   public void setup() throws UnsupportedFileSystemException {
@@ -209,6 +236,32 @@ public class TestNodeManagerResync {
     nm.stop();
   }
 
+  @SuppressWarnings("unchecked")
+  @Test(timeout=60000)
+  public void testContainerResourceIncreaseIsSynchronizedWithRMResync()
+      throws IOException, InterruptedException, YarnException {
+    NodeManager nm = new TestNodeManager4();
+    YarnConfiguration conf = createNMConfig();
+    conf.setBoolean(
+        YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
+    nm.init(conf);
+    nm.start();
+    // Start a container and make sure it is in RUNNING state
+    ((TestNodeManager4)nm).startContainer();
+    // Simulate a container resource increase in a separate thread
+    ((TestNodeManager4)nm).increaseContainersResource();
+    // Simulate RM restart by sending a RESYNC event
+    LOG.info("Sending out RESYNC event");
+    nm.getNMDispatcher().getEventHandler().handle(
+        new NodeManagerEvent(NodeManagerEventType.RESYNC));
+    try {
+      syncBarrier.await();
+    } catch (BrokenBarrierException e) {
+      e.printStackTrace();
+    }
+    Assert.assertFalse(assertionFailedInThread.get());
+    nm.stop();
+  }
 
   // This is to test when NM gets the resync response from last heart beat, it
   // should be able to send the already-sent-via-last-heart-beat container
@@ -588,6 +641,211 @@ public class TestNodeManagerResync {
       }
     }}
 
+  class TestNodeManager4 extends NodeManager {
+
+    private Thread increaseContainerResourceThread = null;
+
+    @Override
+    protected NodeStatusUpdater createNodeStatusUpdater(Context context,
+        Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
+      return new TestNodeStatusUpdaterImpl4(context, dispatcher,
+          healthChecker, metrics);
+    }
+
+    @Override
+    protected ContainerManagerImpl createContainerManager(Context context,
+        ContainerExecutor exec, DeletionService del,
+        NodeStatusUpdater nodeStatusUpdater,
+        ApplicationACLsManager aclsManager,
+        LocalDirsHandlerService dirsHandler) {
+      return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
+          metrics, dirsHandler){
+        @Override
+        public void
+        setBlockNewContainerRequests(boolean blockNewContainerRequests) {
+          // do nothing
+        }
+
+        @Override
+        protected void authorizeGetAndStopContainerRequest(
+            ContainerId containerId, Container container,
+            boolean stopRequest, NMTokenIdentifier identifier)
+            throws YarnException {
+          // do nothing
+        }
+        @Override
+        protected void authorizeUser(UserGroupInformation remoteUgi,
+            NMTokenIdentifier nmTokenIdentifier) {
+          // do nothing
+        }
+        @Override
+        protected void authorizeStartAndResourceIncreaseRequest(
+            NMTokenIdentifier nmTokenIdentifier,
+            ContainerTokenIdentifier containerTokenIdentifier,
+            boolean startRequest) throws YarnException {
+          try {
+            // Sleep 2 seconds to simulate a pro-longed increase action.
+            // If during this time a RESYNC event is sent by RM, the
+            // resync action should block until the increase action is
+            // completed.
+            // See testContainerResourceIncreaseIsSynchronizedWithRMResync()
+            Thread.sleep(2000);
+          } catch (InterruptedException e) {
+            e.printStackTrace();
+          }
+        }
+        @Override
+        protected void updateNMTokenIdentifier(
+            NMTokenIdentifier nmTokenIdentifier)
+                throws SecretManager.InvalidToken {
+          // Do nothing
+        }
+        @Override
+        public Map<String, ByteBuffer> getAuxServiceMetaData() {
+          return new HashMap<>();
+        }
+        @Override
+        protected NMTokenIdentifier selectNMTokenIdentifier(
+            UserGroupInformation remoteUgi) {
+          return new NMTokenIdentifier();
+        }
+      };
+    }
+
+    // Start a container in NM
+    public void startContainer()
+        throws IOException, InterruptedException, YarnException {
+      LOG.info("Start a container and wait until it is in RUNNING state");
+      File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
+      PrintWriter fileWriter = new PrintWriter(scriptFile);
+      if (Shell.WINDOWS) {
+        fileWriter.println("@ping -n 100 127.0.0.1 >nul");
+      } else {
+        fileWriter.write("\numask 0");
+        fileWriter.write("\nexec sleep 100");
+      }
+      fileWriter.close();
+      ContainerLaunchContext containerLaunchContext =
+          recordFactory.newRecordInstance(ContainerLaunchContext.class);
+      URL resource_alpha =
+          ConverterUtils.getYarnUrlFromPath(localFS
+              .makeQualified(new Path(scriptFile.getAbsolutePath())));
+      LocalResource rsrc_alpha =
+          recordFactory.newRecordInstance(LocalResource.class);
+      rsrc_alpha.setResource(resource_alpha);
+      rsrc_alpha.setSize(-1);
+      rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
+      rsrc_alpha.setType(LocalResourceType.FILE);
+      rsrc_alpha.setTimestamp(scriptFile.lastModified());
+      String destinationFile = "dest_file";
+      Map<String, LocalResource> localResources =
+          new HashMap<String, LocalResource>();
+      localResources.put(destinationFile, rsrc_alpha);
+      containerLaunchContext.setLocalResources(localResources);
+      List<String> commands =
+          Arrays.asList(Shell.getRunScriptCommand(scriptFile));
+      containerLaunchContext.setCommands(commands);
+      Resource resource = Resource.newInstance(1024, 1);
+      StartContainerRequest scRequest =
+          StartContainerRequest.newInstance(
+              containerLaunchContext,
+              getContainerToken(resource));
+      List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+      list.add(scRequest);
+      StartContainersRequest allRequests =
+          StartContainersRequest.newInstance(list);
+      getContainerManager().startContainers(allRequests);
+      // Make sure the container reaches RUNNING state
+      ContainerId cId = TestContainerManager.createContainerId(0);
+      BaseContainerManagerTest.waitForNMContainerState(
+          getContainerManager(), cId,
+          org.apache.hadoop.yarn.server.nodemanager.
+              containermanager.container.ContainerState.RUNNING);
+    }
+
+    // Increase container resource in a thread
+    public void increaseContainersResource()
+        throws InterruptedException {
+      LOG.info("Increase a container resource in a separate thread");
+      increaseContainerResourceThread = new IncreaseContainersResourceThread();
+      increaseContainerResourceThread.start();
+    }
+
+    class TestNodeStatusUpdaterImpl4 extends MockNodeStatusUpdater {
+
+      public TestNodeStatusUpdaterImpl4(Context context, Dispatcher dispatcher,
+          NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
+        super(context, dispatcher, healthChecker, metrics);
+      }
+
+      @Override
+      protected void rebootNodeStatusUpdaterAndRegisterWithRM() {
+        try {
+          try {
+            // Check status before registerWithRM
+            List<ContainerId> containerIds = new ArrayList<>();
+            ContainerId cId = TestContainerManager.createContainerId(0);
+            containerIds.add(cId);
+            GetContainerStatusesRequest gcsRequest =
+                GetContainerStatusesRequest.newInstance(containerIds);
+            ContainerStatus containerStatus = getContainerManager()
+                .getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
+            assertEquals(Resource.newInstance(1024, 1),
+                containerStatus.getCapability());
+            // Call the actual rebootNodeStatusUpdaterAndRegisterWithRM().
+            // This function should be synchronized with
+            // increaseContainersResource().
+            super.rebootNodeStatusUpdaterAndRegisterWithRM();
+            // Check status after registerWithRM
+            containerStatus = getContainerManager()
+                .getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
+            assertEquals(Resource.newInstance(4096, 2),
+                containerStatus.getCapability());
+          } catch (AssertionError ae) {
+            ae.printStackTrace();
+            assertionFailedInThread.set(true);
+          }   finally {
+            syncBarrier.await();
+          }
+        } catch (Exception e) {
+          e.printStackTrace();
+        }
+      }
+    }
+
+    class IncreaseContainersResourceThread extends Thread {
+      @Override
+      public void run() {
+        // Construct container resource increase request
+        List<Token> increaseTokens = new ArrayList<Token>();
+        // Add increase request.
+        Resource targetResource = Resource.newInstance(4096, 2);
+        try {
+          increaseTokens.add(getContainerToken(targetResource));
+          IncreaseContainersResourceRequest increaseRequest =
+              IncreaseContainersResourceRequest.newInstance(increaseTokens);
+          IncreaseContainersResourceResponse increaseResponse =
+              getContainerManager()
+                  .increaseContainersResource(increaseRequest);
+          Assert.assertEquals(
+              1, increaseResponse.getSuccessfullyIncreasedContainers()
+                  .size());
+          Assert.assertTrue(increaseResponse.getFailedRequests().isEmpty());
+        } catch (Exception e) {
+          e.printStackTrace();
+        }
+      }
+    }
+
+    private Token getContainerToken(Resource resource) throws IOException {
+      ContainerId cId = TestContainerManager.createContainerId(0);
+      return TestContainerManager.createContainerToken(
+          cId, DUMMY_RM_IDENTIFIER,
+          getNMContext().getNodeId(), user, resource,
+          getNMContext().getContainerTokenSecretManager(), null);
+    }
+  }
+
   public static NMContainerStatus createNMContainerStatus(int id,
       ContainerState containerState) {
     ApplicationId applicationId = ApplicationId.newInstance(0, 1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
index 964379a..9bc23f6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
@@ -620,6 +620,11 @@ public abstract class BaseAMRMProxyTest {
     }
 
     @Override
+    public ConcurrentMap<ContainerId, org.apache.hadoop.yarn.api.records.Container> getIncreasedContainers() {
+      return null;
+    }
+
+    @Override
     public NMContainerTokenSecretManager getContainerTokenSecretManager() {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
index 7573a7a..f482784 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
@@ -93,8 +93,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-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.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
@@ -292,8 +290,8 @@ public class MockResourceManagerFacade implements
         new ArrayList<ContainerStatus>(), containerList,
         new ArrayList<NodeReport>(), null, AMCommand.AM_RESYNC, 1, null,
         new ArrayList<NMToken>(),
-        new ArrayList<ContainerResourceIncrease>(),
-        new ArrayList<ContainerResourceDecrease>());
+        new ArrayList<Container>(),
+        new ArrayList<Container>());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3dc1af0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index 2ea9146..3fb4112 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -108,7 +108,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     super.setup();
   }
 
-  private ContainerId createContainerId(int id) {
+  public static ContainerId createContainerId(int id) {
     ApplicationId appId = ApplicationId.newInstance(0, 0);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);


[11/21] hadoop git commit: YARN-1645. ContainerManager implementation to support container resizing. Contributed by Meng Ding & Wangda Tan

Posted by wa...@apache.org.
YARN-1645. ContainerManager implementation to support container resizing. Contributed by Meng Ding & Wangda Tan


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

Branch: refs/heads/YARN-1197
Commit: ffd820c27a4f8cf4676ad8758696ed89fde80218
Parents: 83a18ad
Author: Jian He <ji...@apache.org>
Authored: Tue Jul 21 16:10:40 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Wed Sep 23 13:29:37 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../CMgrDecreaseContainersResourceEvent.java    |  37 ++++
 .../nodemanager/ContainerManagerEventType.java  |   1 +
 .../containermanager/ContainerManagerImpl.java  | 180 ++++++++++++++++--
 .../container/ChangeContainerResourceEvent.java |  36 ++++
 .../container/ContainerEventType.java           |   4 +
 .../nodemanager/DummyContainerManager.java      |   6 +-
 .../TestContainerManagerWithLCE.java            |  22 +++
 .../BaseContainerManagerTest.java               |  43 ++++-
 .../containermanager/TestContainerManager.java  | 190 ++++++++++++++++++-
 10 files changed, 486 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd820c2/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index bf6d9c4..346fe85 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -206,6 +206,9 @@ Release 2.8.0 - UNRELEASED
     YARN-1449. AM-NM protocol changes to support container resizing.
     (Meng Ding & Wangda Tan via jianhe)
 
+    YARN-1645. ContainerManager implementation to support container resizing.
+    (Meng Ding & Wangda Tan via jianhe)
+
   IMPROVEMENTS
 
     YARN-644. Basic null check is not performed on passed in arguments before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd820c2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrDecreaseContainersResourceEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrDecreaseContainersResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrDecreaseContainersResourceEvent.java
new file mode 100644
index 0000000..9479d0b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/CMgrDecreaseContainersResourceEvent.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager;
+
+import org.apache.hadoop.yarn.api.records.Container;
+import java.util.List;
+
+public class CMgrDecreaseContainersResourceEvent extends ContainerManagerEvent {
+
+  private final List<Container> containersToDecrease;
+
+  public CMgrDecreaseContainersResourceEvent(List<Container>
+      containersToDecrease) {
+    super(ContainerManagerEventType.DECREASE_CONTAINERS_RESOURCE);
+    this.containersToDecrease = containersToDecrease;
+  }
+
+  public List<Container> getContainersToDecrease() {
+    return this.containersToDecrease;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd820c2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java
index 4278ce0..fcb0252 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerManagerEventType.java
@@ -21,4 +21,5 @@ package org.apache.hadoop.yarn.server.nodemanager;
 public enum ContainerManagerEventType {
   FINISH_APPS,
   FINISH_CONTAINERS,
+  DECREASE_CONTAINERS_RESOURCE
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd820c2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index ba1aec2..890a4e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -74,6 +74,7 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.LogAggregationContextPBImpl;
@@ -95,6 +96,7 @@ import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ContainerType;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedContainersEvent;
+import org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerManagerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
@@ -113,6 +115,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationInitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ChangeContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
@@ -141,6 +144,7 @@ import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
+import org.apache.hadoop.yarn.util.resource.Resources;
 
 public class ContainerManagerImpl extends CompositeService implements
     ServiceStateChangeListener, ContainerManagementProtocol,
@@ -681,33 +685,45 @@ public class ContainerManagerImpl extends CompositeService implements
 
   /**
    * @param containerTokenIdentifier
-   *          of the container to be started
+   *          of the container whose resource is to be started or increased
    * @throws YarnException
    */
   @Private
   @VisibleForTesting
-  protected void authorizeStartRequest(NMTokenIdentifier nmTokenIdentifier,
-      ContainerTokenIdentifier containerTokenIdentifier) throws YarnException {
+  protected void authorizeStartAndResourceIncreaseRequest(
+      NMTokenIdentifier nmTokenIdentifier,
+      ContainerTokenIdentifier containerTokenIdentifier,
+      boolean startRequest)
+      throws YarnException {
     if (nmTokenIdentifier == null) {
       throw RPCUtil.getRemoteException(INVALID_NMTOKEN_MSG);
     }
     if (containerTokenIdentifier == null) {
       throw RPCUtil.getRemoteException(INVALID_CONTAINERTOKEN_MSG);
     }
+    /*
+     * Check the following:
+     * 1. The request comes from the same application attempt
+     * 2. The request possess a container token that has not expired
+     * 3. The request possess a container token that is granted by a known RM
+     */
     ContainerId containerId = containerTokenIdentifier.getContainerID();
     String containerIDStr = containerId.toString();
     boolean unauthorized = false;
     StringBuilder messageBuilder =
-        new StringBuilder("Unauthorized request to start container. ");
+        new StringBuilder("Unauthorized request to " + (startRequest ?
+            "start container." : "increase container resource."));
     if (!nmTokenIdentifier.getApplicationAttemptId().getApplicationId().
         equals(containerId.getApplicationAttemptId().getApplicationId())) {
       unauthorized = true;
       messageBuilder.append("\nNMToken for application attempt : ")
         .append(nmTokenIdentifier.getApplicationAttemptId())
-        .append(" was used for starting container with container token")
+        .append(" was used for "
+            + (startRequest ? "starting " : "increasing resource of ")
+            + "container with container token")
         .append(" issued for application attempt : ")
         .append(containerId.getApplicationAttemptId());
-    } else if (!this.context.getContainerTokenSecretManager()
+    } else if (startRequest && !this.context.getContainerTokenSecretManager()
         .isValidStartContainerRequest(containerTokenIdentifier)) {
       // Is the container being relaunched? Or RPC layer let startCall with
       // tokens generated off old-secret through?
@@ -729,6 +745,14 @@ public class ContainerManagerImpl extends CompositeService implements
       LOG.error(msg);
       throw RPCUtil.getRemoteException(msg);
     }
+    if (containerTokenIdentifier.getRMIdentifier() != nodeStatusUpdater
+        .getRMIdentifier()) {
+      // Is the container coming from unknown RM
+      StringBuilder sb = new StringBuilder("\nContainer ");
+      sb.append(containerTokenIdentifier.getContainerID().toString())
+        .append(" rejected as it is allocated by a previous RM");
+      throw new InvalidContainerException(sb.toString());
+    }
   }
 
   /**
@@ -745,7 +769,7 @@ public class ContainerManagerImpl extends CompositeService implements
     }
     UserGroupInformation remoteUgi = getRemoteUgi();
     NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
-    authorizeUser(remoteUgi,nmTokenIdentifier);
+    authorizeUser(remoteUgi, nmTokenIdentifier);
     List<ContainerId> succeededContainers = new ArrayList<ContainerId>();
     Map<ContainerId, SerializedException> failedContainers =
         new HashMap<ContainerId, SerializedException>();
@@ -844,16 +868,8 @@ public class ContainerManagerImpl extends CompositeService implements
      * belongs to correct Node Manager (part of retrieve password). c) It has
      * correct RMIdentifier. d) It is not expired.
      */
-    authorizeStartRequest(nmTokenIdentifier, containerTokenIdentifier);
- 
-    if (containerTokenIdentifier.getRMIdentifier() != nodeStatusUpdater
-        .getRMIdentifier()) {
-        // Is the container coming from unknown RM
-        StringBuilder sb = new StringBuilder("\nContainer ");
-        sb.append(containerTokenIdentifier.getContainerID().toString())
-          .append(" rejected as it is allocated by a previous RM");
-        throw new InvalidContainerException(sb.toString());
-    }
+    authorizeStartAndResourceIncreaseRequest(
+        nmTokenIdentifier, containerTokenIdentifier, true);
     // update NMToken
     updateNMTokenIdentifier(nmTokenIdentifier);
 
@@ -960,9 +976,118 @@ public class ContainerManagerImpl extends CompositeService implements
   @Override
   public IncreaseContainersResourceResponse increaseContainersResource(
       IncreaseContainersResourceRequest requests)
-      throws YarnException, IOException {
-    // To be implemented in YARN-1645
-    return null;
+          throws YarnException, IOException {
+    if (blockNewContainerRequests.get()) {
+      throw new NMNotYetReadyException(
+          "Rejecting container resource increase as NodeManager has not"
+              + " yet connected with ResourceManager");
+    }
+    UserGroupInformation remoteUgi = getRemoteUgi();
+    NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
+    authorizeUser(remoteUgi, nmTokenIdentifier);
+    List<ContainerId> successfullyIncreasedContainers
+        = new ArrayList<ContainerId>();
+    Map<ContainerId, SerializedException> failedContainers =
+        new HashMap<ContainerId, SerializedException>();
+    // Process container resource increase requests
+    for (org.apache.hadoop.yarn.api.records.Token token :
+        requests.getContainersToIncrease()) {
+      ContainerId containerId = null;
+      try {
+        if (token.getIdentifier() == null) {
+          throw new IOException(INVALID_CONTAINERTOKEN_MSG);
+        }
+        ContainerTokenIdentifier containerTokenIdentifier =
+            BuilderUtils.newContainerTokenIdentifier(token);
+        verifyAndGetContainerTokenIdentifier(token,
+            containerTokenIdentifier);
+        authorizeStartAndResourceIncreaseRequest(
+            nmTokenIdentifier, containerTokenIdentifier, false);
+        containerId = containerTokenIdentifier.getContainerID();
+        // Reuse the startContainer logic to update NMToken,
+        // as container resource increase request will have come with
+        // an updated NMToken.
+        updateNMTokenIdentifier(nmTokenIdentifier);
+        Resource resource = containerTokenIdentifier.getResource();
+        changeContainerResourceInternal(containerId, resource, true);
+        successfullyIncreasedContainers.add(containerId);
+      } catch (YarnException | InvalidToken e) {
+        failedContainers.put(containerId, SerializedException.newInstance(e));
+      } catch (IOException e) {
+        throw RPCUtil.getRemoteException(e);
+      }
+    }
+    return IncreaseContainersResourceResponse.newInstance(
+        successfullyIncreasedContainers, failedContainers);
+  }
+
+  @SuppressWarnings("unchecked")
+  private void changeContainerResourceInternal(
+      ContainerId containerId, Resource targetResource, boolean increase)
+          throws YarnException, IOException {
+    Container container = context.getContainers().get(containerId);
+    // Check container existence
+    if (container == null) {
+      if (nodeStatusUpdater.isContainerRecentlyStopped(containerId)) {
+        throw RPCUtil.getRemoteException("Container " + containerId.toString()
+            + " was recently stopped on node manager.");
+      } else {
+        throw RPCUtil.getRemoteException("Container " + containerId.toString()
+            + " is not handled by this NodeManager");
+      }
+    }
+    // Check container state
+    org.apache.hadoop.yarn.server.nodemanager.
+        containermanager.container.ContainerState currentState =
+        container.getContainerState();
+    if (currentState != org.apache.hadoop.yarn.server.
+        nodemanager.containermanager.container.ContainerState.RUNNING) {
+      throw RPCUtil.getRemoteException("Container " + containerId.toString()
+          + " is in " + currentState.name() + " state."
+          + " Resource can only be changed when a container is in"
+          + " RUNNING state");
+    }
+    // Check validity of the target resource.
+    Resource currentResource = container.getResource();
+    if (currentResource.equals(targetResource)) {
+      LOG.warn("Unable to change resource for container "
+          + containerId.toString()
+          + ". The target resource "
+          + targetResource.toString()
+          + " is the same as the current resource");
+      return;
+    }
+    if (increase && !Resources.fitsIn(currentResource, targetResource)) {
+      throw RPCUtil.getRemoteException("Unable to increase resource for "
+          + "container " + containerId.toString()
+          + ". The target resource "
+          + targetResource.toString()
+          + " is smaller than the current resource "
+          + currentResource.toString());
+    }
+    if (!increase &&
+        (!Resources.fitsIn(Resources.none(), targetResource)
+            || !Resources.fitsIn(targetResource, currentResource))) {
+      throw RPCUtil.getRemoteException("Unable to decrease resource for "
+          + "container " + containerId.toString()
+          + ". The target resource "
+          + targetResource.toString()
+          + " is not smaller than the current resource "
+          + currentResource.toString());
+    }
+    this.readLock.lock();
+    try {
+      if (!serviceStopped) {
+        dispatcher.getEventHandler().handle(new ChangeContainerResourceEvent(
+            containerId, targetResource));
+      } else {
+        throw new YarnException(
+            "Unable to change container resource as the NodeManager is "
+                + "in the process of shutting down");
+      }
+    } finally {
+      this.readLock.unlock();
+    }
   }
 
   @Private
@@ -1182,6 +1307,21 @@ public class ContainerManagerImpl extends CompositeService implements
                   "Container Killed by ResourceManager"));
       }
       break;
+    case DECREASE_CONTAINERS_RESOURCE:
+      CMgrDecreaseContainersResourceEvent containersDecreasedEvent =
+          (CMgrDecreaseContainersResourceEvent) event;
+      for (org.apache.hadoop.yarn.api.records.Container container
+          : containersDecreasedEvent.getContainersToDecrease()) {
+        try {
+          changeContainerResourceInternal(container.getId(),
+              container.getResource(), false);
+        } catch (YarnException e) {
+          LOG.error("Unable to decrease container resource", e);
+        } catch (IOException e) {
+          LOG.error("Unable to update container resource in store", e);
+        }
+      }
+      break;
     default:
         throw new YarnRuntimeException(
             "Got an unknown ContainerManagerEvent type: " + event.getType());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd820c2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.java
new file mode 100644
index 0000000..3944a3d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ChangeContainerResourceEvent.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Resource;
+
+public class ChangeContainerResourceEvent extends ContainerEvent {
+
+  private Resource resource;
+
+  public ChangeContainerResourceEvent(ContainerId c, Resource resource) {
+    super(c, ContainerEventType.CHANGE_CONTAINER_RESOURCE);
+    this.resource = resource;
+  }
+
+  public Resource getResource() {
+    return this.resource;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd820c2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
index 5622f8c..dc712bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
@@ -25,6 +25,10 @@ public enum ContainerEventType {
   KILL_CONTAINER,
   UPDATE_DIAGNOSTICS_MSG,
   CONTAINER_DONE,
+  CHANGE_CONTAINER_RESOURCE,
+
+  // Producer: ContainerMonitor
+  CONTAINER_RESOURCE_CHANGED,
 
   // DownloadManager
   CONTAINER_INITED,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd820c2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java
index 349340b..3ff04d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java
@@ -191,8 +191,10 @@ public class DummyContainerManager extends ContainerManagerImpl {
   }
   
   @Override
-  protected void authorizeStartRequest(NMTokenIdentifier nmTokenIdentifier,
-      ContainerTokenIdentifier containerTokenIdentifier) throws YarnException {
+  protected void authorizeStartAndResourceIncreaseRequest(
+      NMTokenIdentifier nmTokenIdentifier,
+      ContainerTokenIdentifier containerTokenIdentifier,
+      boolean startRequest) throws YarnException {
     // do nothing
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd820c2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
index a47e7f7..9a05278 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
@@ -189,6 +189,28 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
     super.testStartContainerFailureWithUnknownAuxService();
   }
 
+  @Override
+  public void testIncreaseContainerResourceWithInvalidRequests() throws Exception {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testIncreaseContainerResourceWithInvalidRequests");
+    super.testIncreaseContainerResourceWithInvalidRequests();
+  }
+
+  @Override
+  public void testIncreaseContainerResourceWithInvalidResource() throws Exception {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testIncreaseContainerResourceWithInvalidResource");
+    super.testIncreaseContainerResourceWithInvalidResource();
+  }
+
   private boolean shouldRunTest() {
     return System
         .getProperty(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH) != null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd820c2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
index 2810662..3938342 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
@@ -209,12 +209,13 @@ public abstract class BaseContainerManagerTest {
         // do nothing
       }
       @Override
-        protected void authorizeStartRequest(
-            NMTokenIdentifier nmTokenIdentifier,
-            ContainerTokenIdentifier containerTokenIdentifier) throws YarnException {
-          // do nothing
-        }
-      
+      protected void authorizeStartAndResourceIncreaseRequest(
+          NMTokenIdentifier nmTokenIdentifier,
+          ContainerTokenIdentifier containerTokenIdentifier,
+          boolean startRequest) throws YarnException {
+        // do nothing
+      }
+
       @Override
         protected void updateNMTokenIdentifier(
             NMTokenIdentifier nmTokenIdentifier) throws InvalidToken {
@@ -310,4 +311,34 @@ public abstract class BaseContainerManagerTest {
         app.getApplicationState().equals(finalState));
   }
 
+  public static void waitForNMContainerState(ContainerManagerImpl
+      containerManager, ContainerId containerID,
+          org.apache.hadoop.yarn.server.nodemanager.containermanager
+              .container.ContainerState finalState)
+                  throws InterruptedException, YarnException, IOException {
+    waitForNMContainerState(containerManager, containerID, finalState, 20);
+  }
+
+  public static void waitForNMContainerState(ContainerManagerImpl
+      containerManager, ContainerId containerID,
+          org.apache.hadoop.yarn.server.nodemanager.containermanager
+          .container.ContainerState finalState, int timeOutMax)
+              throws InterruptedException, YarnException, IOException {
+    Container container =
+        containerManager.getContext().getContainers().get(containerID);
+    org.apache.hadoop.yarn.server.nodemanager
+        .containermanager.container.ContainerState currentState =
+            container.getContainerState();
+    int timeoutSecs = 0;
+    while (!currentState.equals(finalState)
+        && timeoutSecs++ < timeOutMax) {
+      Thread.sleep(1000);
+      LOG.info("Waiting for NM container to get into state " + finalState
+          + ". Current state is " + currentState);
+      currentState = container.getContainerState();
+    }
+    LOG.info("Container state is " + currentState);
+    Assert.assertEquals("ContainerState is not correct (timedout)",
+        finalState, currentState);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd820c2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index e508424..e2f12ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -38,6 +38,8 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@@ -72,6 +74,7 @@ import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
+import org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestAuxServices.ServiceA;
@@ -87,6 +90,8 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import static org.junit.Assert.assertEquals;
+
 public class TestContainerManager extends BaseContainerManagerTest {
 
   public TestContainerManager() throws UnsupportedFileSystemException {
@@ -803,7 +808,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
         metrics, dirsHandler);
     String strExceptionMsg = "";
     try {
-      cMgrImpl.authorizeStartRequest(null, new ContainerTokenIdentifier());
+      cMgrImpl.authorizeStartAndResourceIncreaseRequest(
+          null, new ContainerTokenIdentifier(), true);
     } catch(YarnException ye) {
       strExceptionMsg = ye.getMessage();
     }
@@ -812,7 +818,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
     strExceptionMsg = "";
     try {
-      cMgrImpl.authorizeStartRequest(new NMTokenIdentifier(), null);
+      cMgrImpl.authorizeStartAndResourceIncreaseRequest(
+          new NMTokenIdentifier(), null, true);
     } catch(YarnException ye) {
       strExceptionMsg = ye.getMessage();
     }
@@ -878,6 +885,167 @@ public class TestContainerManager extends BaseContainerManagerTest {
         ContainerManagerImpl.INVALID_CONTAINERTOKEN_MSG);
   }
 
+  @Test
+  public void testIncreaseContainerResourceWithInvalidRequests() throws Exception {
+    containerManager.start();
+    // Start 4 containers 0..4 with default resource (1024, 1)
+    List<StartContainerRequest> list = new ArrayList<>();
+    ContainerLaunchContext containerLaunchContext = recordFactory
+        .newRecordInstance(ContainerLaunchContext.class);
+    for (int i = 0; i < 4; i++) {
+      ContainerId cId = createContainerId(i);
+      long identifier = DUMMY_RM_IDENTIFIER;
+      Token containerToken = createContainerToken(cId, identifier,
+          context.getNodeId(), user, context.getContainerTokenSecretManager());
+      StartContainerRequest request = StartContainerRequest.newInstance(
+          containerLaunchContext, containerToken);
+      list.add(request);
+    }
+    StartContainersRequest requestList = StartContainersRequest
+        .newInstance(list);
+    StartContainersResponse response = containerManager
+        .startContainers(requestList);
+
+    Assert.assertEquals(4, response.getSuccessfullyStartedContainers().size());
+    int i = 0;
+    for (ContainerId id : response.getSuccessfullyStartedContainers()) {
+      Assert.assertEquals(i, id.getContainerId());
+      i++;
+    }
+
+    Thread.sleep(2000);
+    // Construct container resource increase request,
+    List<Token> increaseTokens = new ArrayList<Token>();
+    // Add increase request for container-0, the request will fail as the
+    // container will have exited, and won't be in RUNNING state
+    ContainerId cId0 = createContainerId(0);
+    Token containerToken =
+        createContainerToken(cId0, DUMMY_RM_IDENTIFIER,
+            context.getNodeId(), user,
+                Resource.newInstance(1234, 3),
+                    context.getContainerTokenSecretManager(), null);
+    increaseTokens.add(containerToken);
+    // Add increase request for container-7, the request will fail as the
+    // container does not exist
+    ContainerId cId7 = createContainerId(7);
+    containerToken =
+        createContainerToken(cId7, DUMMY_RM_IDENTIFIER,
+            context.getNodeId(), user,
+            Resource.newInstance(1234, 3),
+            context.getContainerTokenSecretManager(), null);
+    increaseTokens.add(containerToken);
+
+    IncreaseContainersResourceRequest increaseRequest =
+        IncreaseContainersResourceRequest
+          .newInstance(increaseTokens);
+    IncreaseContainersResourceResponse increaseResponse =
+        containerManager.increaseContainersResource(increaseRequest);
+    // Check response
+    Assert.assertEquals(
+        0, increaseResponse.getSuccessfullyIncreasedContainers().size());
+    Assert.assertEquals(2, increaseResponse.getFailedRequests().size());
+    for (Map.Entry<ContainerId, SerializedException> entry : increaseResponse
+        .getFailedRequests().entrySet()) {
+      Assert.assertNotNull("Failed message", entry.getValue().getMessage());
+      if (cId0.equals(entry.getKey())) {
+        Assert.assertTrue(entry.getValue().getMessage()
+          .contains("Resource can only be changed when a "
+              + "container is in RUNNING state"));
+      } else if (cId7.equals(entry.getKey())) {
+        Assert.assertTrue(entry.getValue().getMessage()
+            .contains("Container " + cId7.toString()
+                + " is not handled by this NodeManager"));
+      } else {
+        throw new YarnException("Received failed request from wrong"
+            + " container: " + entry.getKey().toString());
+      }
+    }
+  }
+
+  @Test
+  public void testIncreaseContainerResourceWithInvalidResource() throws Exception {
+    containerManager.start();
+    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
+    PrintWriter fileWriter = new PrintWriter(scriptFile);
+    // Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    if (Shell.WINDOWS) {
+      fileWriter.println("@ping -n 100 127.0.0.1 >nul");
+    } else {
+      fileWriter.write("\numask 0");
+      fileWriter.write("\nexec sleep 100");
+    }
+    fileWriter.close();
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+    URL resource_alpha =
+        ConverterUtils.getYarnUrlFromPath(localFS
+            .makeQualified(new Path(scriptFile.getAbsolutePath())));
+    LocalResource rsrc_alpha =
+        recordFactory.newRecordInstance(LocalResource.class);
+    rsrc_alpha.setResource(resource_alpha);
+    rsrc_alpha.setSize(-1);
+    rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
+    rsrc_alpha.setType(LocalResourceType.FILE);
+    rsrc_alpha.setTimestamp(scriptFile.lastModified());
+    String destinationFile = "dest_file";
+    Map<String, LocalResource> localResources =
+        new HashMap<String, LocalResource>();
+    localResources.put(destinationFile, rsrc_alpha);
+    containerLaunchContext.setLocalResources(localResources);
+    List<String> commands =
+        Arrays.asList(Shell.getRunScriptCommand(scriptFile));
+    containerLaunchContext.setCommands(commands);
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+            containerLaunchContext,
+            createContainerToken(cId, DUMMY_RM_IDENTIFIER, context.getNodeId(),
+            user, context.getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+    // Make sure the container reaches RUNNING state
+    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
+        org.apache.hadoop.yarn.server.nodemanager.
+            containermanager.container.ContainerState.RUNNING);
+    // Construct container resource increase request,
+    List<Token> increaseTokens = new ArrayList<Token>();
+    // Add increase request. The increase request should fail
+    // as the current resource does not fit in the target resource
+    Token containerToken =
+        createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+            context.getNodeId(), user,
+            Resource.newInstance(512, 1),
+            context.getContainerTokenSecretManager(), null);
+    increaseTokens.add(containerToken);
+    IncreaseContainersResourceRequest increaseRequest =
+        IncreaseContainersResourceRequest
+            .newInstance(increaseTokens);
+    IncreaseContainersResourceResponse increaseResponse =
+        containerManager.increaseContainersResource(increaseRequest);
+    // Check response
+    Assert.assertEquals(
+        0, increaseResponse.getSuccessfullyIncreasedContainers().size());
+    Assert.assertEquals(1, increaseResponse.getFailedRequests().size());
+    for (Map.Entry<ContainerId, SerializedException> entry : increaseResponse
+        .getFailedRequests().entrySet()) {
+      if (cId.equals(entry.getKey())) {
+        Assert.assertNotNull("Failed message", entry.getValue().getMessage());
+        Assert.assertTrue(entry.getValue().getMessage()
+            .contains("The target resource "
+                + Resource.newInstance(512, 1).toString()
+                + " is smaller than the current resource "
+                + Resource.newInstance(1024, 1)));
+      } else {
+        throw new YarnException("Received failed request from wrong"
+            + " container: " + entry.getKey().toString());
+      }
+    }
+  }
+
   public static Token createContainerToken(ContainerId cId, long rmIdentifier,
       NodeId nodeId, String user,
       NMContainerTokenSecretManager containerTokenSecretManager)
@@ -892,15 +1060,21 @@ public class TestContainerManager extends BaseContainerManagerTest {
       LogAggregationContext logAggregationContext)
       throws IOException {
     Resource r = BuilderUtils.newResource(1024, 1);
+    return createContainerToken(cId, rmIdentifier, nodeId, user, r,
+        containerTokenSecretManager, logAggregationContext);
+  }
+
+  public static Token createContainerToken(ContainerId cId, long rmIdentifier,
+      NodeId nodeId, String user, Resource resource,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext)
+      throws IOException {
     ContainerTokenIdentifier containerTokenIdentifier =
-        new ContainerTokenIdentifier(cId, nodeId.toString(), user, r,
+        new ContainerTokenIdentifier(cId, nodeId.toString(), user, resource,
           System.currentTimeMillis() + 100000L, 123, rmIdentifier,
           Priority.newInstance(0), 0, logAggregationContext, null);
-    Token containerToken =
-        BuilderUtils
-          .newContainerToken(nodeId, containerTokenSecretManager
-            .retrievePassword(containerTokenIdentifier),
+    return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager
+        .retrievePassword(containerTokenIdentifier),
             containerTokenIdentifier);
-    return containerToken;
   }
 }