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 jh...@apache.org on 2019/03/27 18:11:11 UTC

[hadoop] branch YARN-8200 updated (9c6dbd8 -> df87983)

This is an automated email from the ASF dual-hosted git repository.

jhung pushed a change to branch YARN-8200
in repository https://gitbox.apache.org/repos/asf/hadoop.git.


    from 9c6dbd8  YARN-9271. Backport YARN-6927 for resource type support in MapReduce
     new bfc74ae  YARN-9272. Backport YARN-7738 for refreshing max allocation for multiple resource types
     new df87983  YARN-9409. Port resource type changes from YARN-7237 to branch-2

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../yarn/api/records/ResourceInformation.java      |   6 +
 .../hadoop/yarn/util/resource/ResourceUtils.java   |  40 +++--
 .../yarn/server/resourcemanager/AdminService.java  |  22 ++-
 .../scheduler/AbstractYarnScheduler.java           |  15 +-
 .../scheduler/capacity/CapacityScheduler.java      |  10 +-
 .../capacity/CapacitySchedulerConfiguration.java   |  22 +--
 .../scheduler/capacity/LeafQueue.java              |   4 +-
 .../hadoop/yarn/server/resourcemanager/MockNM.java |  41 +++--
 .../hadoop/yarn/server/resourcemanager/MockRM.java |   9 +
 .../yarn/server/resourcemanager/RMHATestBase.java  |  29 +++-
 .../scheduler/capacity/TestCapacityScheduler.java  |  11 +-
 ...estCapacitySchedulerWithMultiResourceTypes.java | 190 +++++++++++++++++++++
 .../scheduler/capacity/TestUtils.java              |  41 +++--
 13 files changed, 362 insertions(+), 78 deletions(-)
 create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerWithMultiResourceTypes.java


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


[hadoop] 02/02: YARN-9409. Port resource type changes from YARN-7237 to branch-2

Posted by jh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jhung pushed a commit to branch YARN-8200
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit df87983bb947eed840f3f08cdcba025eed1ecd8f
Author: Jonathan Hung <jh...@linkedin.com>
AuthorDate: Wed Mar 27 09:55:51 2019 -0700

    YARN-9409. Port resource type changes from YARN-7237 to branch-2
---
 .../scheduler/AbstractYarnScheduler.java           | 15 +++-----
 .../hadoop/yarn/server/resourcemanager/MockNM.java | 41 ++++++++++++++--------
 .../hadoop/yarn/server/resourcemanager/MockRM.java |  9 +++++
 .../scheduler/capacity/TestUtils.java              | 41 +++++++++++++++-------
 4 files changed, 70 insertions(+), 36 deletions(-)

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 425a0b1..fd50f20 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
@@ -58,7 +58,6 @@ import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
@@ -797,7 +796,7 @@ public abstract class AbstractYarnScheduler
       writeLock.unlock();
     }
   }
-  
+
   /**
    * Process resource update on a node.
    */
@@ -900,12 +899,12 @@ public abstract class AbstractYarnScheduler
     LOG.info("Updated the cluste max priority to maxClusterLevelAppPriority = "
         + maxClusterLevelAppPriority);
   }
-  
+
   /**
    * Sanity check increase/decrease request, and return
    * SchedulerContainerResourceChangeRequest according to given
    * UpdateContainerRequest.
-   * 
+   *
    * <pre>
    * - Returns non-null value means validation succeeded
    * - Throw exception when any other error happens
@@ -1334,9 +1333,7 @@ public abstract class AbstractYarnScheduler
   }
 
   /*
-   * Get a Resource object with for the minimum allocation possible. If resource
-   * profiles are enabled then the 'minimum' resource profile will be used. If
-   * they are not enabled, use the minimums specified in the config files.
+   * Get a Resource object with for the minimum allocation possible.
    *
    * @return a Resource object with the minimum allocation for the scheduler
    */
@@ -1347,9 +1344,7 @@ public abstract class AbstractYarnScheduler
   }
 
   /**
-   * Get a Resource object with for the maximum allocation possible. If resource
-   * profiles are enabled then the 'maximum' resource profile will be used. If
-   * they are not enabled, use the maximums specified in the config files.
+   * Get a Resource object with for the maximum allocation possible.
    *
    * @return a Resource object with the maximum allocation for the scheduler
    */
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 db92d7c..a02ac84 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
@@ -47,14 +47,14 @@ import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
+import org.apache.hadoop.yarn.util.resource.Resources;
 import org.mortbay.log.Log;
 
 public class MockNM {
 
   private int responseId;
   private NodeId nodeId;
-  private long memory;
-  private int vCores;
+  private Resource capatibility;
   private ResourceTrackerService resourceTracker;
   private int httpPort = 2;
   private MasterKey currentContainerTokenMasterKey;
@@ -75,13 +75,25 @@ public class MockNM {
 
   public MockNM(String nodeIdStr, int memory, int vcores,
       ResourceTrackerService resourceTracker) {
-    this(nodeIdStr, memory, vcores, resourceTracker, YarnVersionInfo.getVersion());
+    this(nodeIdStr, memory, vcores, resourceTracker,
+        YarnVersionInfo.getVersion());
   }
 
   public MockNM(String nodeIdStr, int memory, int vcores,
       ResourceTrackerService resourceTracker, String version) {
-    this.memory = memory;
-    this.vCores = vcores;
+    this(nodeIdStr, Resource.newInstance(memory, vcores), resourceTracker,
+        version);
+  }
+
+  public MockNM(String nodeIdStr, Resource capatibility,
+      ResourceTrackerService resourceTracker) {
+    this(nodeIdStr, capatibility, resourceTracker,
+        YarnVersionInfo.getVersion());
+  }
+
+  public MockNM(String nodeIdStr, Resource capatibility,
+      ResourceTrackerService resourceTracker, String version) {
+    this.capatibility = capatibility;
     this.resourceTracker = resourceTracker;
     this.version = version;
     String[] splits = nodeIdStr.split(":");
@@ -146,8 +158,7 @@ public class MockNM {
         RegisterNodeManagerRequest.class);
     req.setNodeId(nodeId);
     req.setHttpPort(httpPort);
-    Resource resource = BuilderUtils.newResource(memory, vCores);
-    req.setResource(resource);
+    req.setResource(capatibility);
     req.setContainerStatuses(containerReports);
     req.setNMVersion(version);
     req.setRunningApplications(runningApplications);
@@ -158,8 +169,7 @@ public class MockNM {
     this.currentNMTokenMasterKey = registrationResponse.getNMTokenMasterKey();
     Resource newResource = registrationResponse.getResource();
     if (newResource != null) {
-      memory = (int) newResource.getMemorySize();
-      vCores = newResource.getVirtualCores();
+      capatibility = Resources.clone(newResource);
     }
     containerStats.clear();
     if (containerReports != null) {
@@ -185,7 +195,7 @@ public class MockNM {
       long containerId, ContainerState containerState) throws Exception {
     ContainerStatus containerStatus = BuilderUtils.newContainerStatus(
         BuilderUtils.newContainerId(attemptId, containerId), containerState,
-        "Success", 0, BuilderUtils.newResource(memory, vCores));
+        "Success", 0, capatibility);
     ArrayList<ContainerStatus> containerStatusList =
         new ArrayList<ContainerStatus>(1);
     containerStatusList.add(containerStatus);
@@ -266,19 +276,22 @@ public class MockNM {
 
     Resource newResource = heartbeatResponse.getResource();
     if (newResource != null) {
-      memory = newResource.getMemorySize();
-      vCores = newResource.getVirtualCores();
+      capatibility = Resources.clone(newResource);
     }
 
     return heartbeatResponse;
   }
 
   public long getMemory() {
-    return memory;
+    return capatibility.getMemorySize();
   }
 
   public int getvCores() {
-    return vCores;
+    return capatibility.getVirtualCores();
+  }
+
+  public Resource getCapatibility() {
+    return capatibility;
   }
 
   public String getVersion() {
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 0754584..fd327f1 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
@@ -848,6 +848,15 @@ public class MockRM extends ResourceManager {
     return nm;
   }
 
+  public MockNM registerNode(String nodeIdStr, Resource nodeCapatibility)
+      throws Exception {
+    MockNM nm = new MockNM(nodeIdStr, nodeCapatibility,
+        getResourceTrackerService());
+    nm.registerNode();
+    drainEventsImplicitly();
+    return nm;
+  }
+
   public void sendNodeStarted(MockNM nm) throws Exception {
     RMNodeImpl node = (RMNodeImpl) getRMContext().getRMNodes().get(
         nm.getNodeId());
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/TestUtils.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/TestUtils.java
index e81ffbd..cb150e0 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/TestUtils.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/TestUtils.java
@@ -18,16 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.Set;
-
+import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -38,6 +29,7 @@ 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.ResourceInformation;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.Event;
@@ -53,20 +45,28 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsMana
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 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.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
 public class TestUtils {
   private static final Log LOG = LogFactory.getLog(TestUtils.class);
@@ -456,4 +456,21 @@ public class TestUtils {
     cs.submitResourceCommitRequest(clusterResource,
         csAssignment);
   }
+
+  /**
+   * An easy way to create resources other than memory and vcores for tests.
+   * @param memory memory
+   * @param vcores vcores
+   * @param nameToValues resource types other than memory and vcores.
+   * @return created resource
+   */
+  public static Resource createResource(long memory, int vcores,
+      Map<String, Integer> nameToValues) {
+    Resource res = Resource.newInstance(memory, vcores);
+    for (Map.Entry<String, Integer> entry : nameToValues.entrySet()) {
+      res.setResourceInformation(entry.getKey(), ResourceInformation
+          .newInstance(entry.getKey(), "", entry.getValue()));
+    }
+    return res;
+  }
 }


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


[hadoop] 01/02: YARN-9272. Backport YARN-7738 for refreshing max allocation for multiple resource types

Posted by jh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jhung pushed a commit to branch YARN-8200
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit bfc74aecbaee241f3cf54c79778c912a8e783004
Author: Jonathan Hung <jh...@linkedin.com>
AuthorDate: Wed Mar 27 09:57:25 2019 -0700

    YARN-9272. Backport YARN-7738 for refreshing max allocation for multiple resource types
---
 .../yarn/api/records/ResourceInformation.java      |   6 +
 .../hadoop/yarn/util/resource/ResourceUtils.java   |  40 +++--
 .../yarn/server/resourcemanager/AdminService.java  |  22 ++-
 .../scheduler/capacity/CapacityScheduler.java      |  10 +-
 .../capacity/CapacitySchedulerConfiguration.java   |  22 +--
 .../scheduler/capacity/LeafQueue.java              |   4 +-
 .../yarn/server/resourcemanager/RMHATestBase.java  |  29 +++-
 .../scheduler/capacity/TestCapacityScheduler.java  |  11 +-
 ...estCapacitySchedulerWithMultiResourceTypes.java | 190 +++++++++++++++++++++
 9 files changed, 292 insertions(+), 42 deletions(-)

diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
index 8917a84..08746a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceInformation.java
@@ -214,6 +214,12 @@ public class ResourceInformation implements Comparable<ResourceInformation> {
             Long.MAX_VALUE);
   }
 
+  public static ResourceInformation newInstance(String name, String units,
+      long minRes, long maxRes) {
+    return ResourceInformation.newInstance(name, units, 0L,
+        ResourceTypes.COUNTABLE, minRes, maxRes);
+  }
+
   public static ResourceInformation newInstance(String name, long value) {
     return ResourceInformation
         .newInstance(name, "", value, ResourceTypes.COUNTABLE, 0L,
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
index 3806771..a359ad1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
@@ -210,9 +210,23 @@ public class ResourceUtils {
     }
   }
 
-  @VisibleForTesting
-  static void initializeResourcesMap(Configuration conf) {
+  /**
+   * Get maximum allocation from config, *THIS WILL NOT UPDATE INTERNAL DATA*
+   * @param conf config
+   * @return maximum allocation
+   */
+  public static Resource fetchMaximumAllocationFromConfig(Configuration conf) {
+    Map<String, ResourceInformation> resourceInformationMap =
+        getResourceInformationMapFromConfig(conf);
+    Resource ret = Resource.newInstance(0, 0);
+    for (ResourceInformation entry : resourceInformationMap.values()) {
+      ret.setResourceValue(entry.getName(), entry.getMaximumAllocation());
+    }
+    return ret;
+  }
 
+  private static Map<String, ResourceInformation> getResourceInformationMapFromConfig(
+      Configuration conf) {
     Map<String, ResourceInformation> resourceInformationMap = new HashMap<>();
     String[] resourceNames = conf.getStrings(YarnConfiguration.RESOURCE_TYPES);
 
@@ -254,6 +268,13 @@ public class ResourceUtils {
     setMinimumAllocationForMandatoryResources(resourceInformationMap, conf);
     setMaximumAllocationForMandatoryResources(resourceInformationMap, conf);
 
+    return resourceInformationMap;
+  }
+
+  @VisibleForTesting
+  static void initializeResourcesMap(Configuration conf) {
+    Map<String, ResourceInformation> resourceInformationMap =
+        getResourceInformationMapFromConfig(conf);
     initializeResourcesFromResourceInformationMap(resourceInformationMap);
   }
 
@@ -523,19 +544,8 @@ public class ResourceUtils {
   public static Resource getResourceTypesMaximumAllocation() {
     Resource ret = Resource.newInstance(0, 0);
     for (ResourceInformation entry : resourceTypesArray) {
-      String name = entry.getName();
-      if (name.equals(ResourceInformation.MEMORY_MB.getName())) {
-        ret.setMemorySize(entry.getMaximumAllocation());
-      } else if (name.equals(ResourceInformation.VCORES.getName())) {
-        Long tmp = entry.getMaximumAllocation();
-        if (tmp > Integer.MAX_VALUE) {
-          tmp = (long) Integer.MAX_VALUE;
-        }
-        ret.setVirtualCores(tmp.intValue());
-        continue;
-      } else {
-        ret.setResourceValue(name, entry.getMaximumAllocation());
-      }
+      ret.setResourceValue(entry.getName(),
+          entry.getMaximumAllocation());
     }
     return ret;
   }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index 6c0a854..3c117bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -400,14 +400,32 @@ public class AdminService extends CompositeService implements
     }
   }
 
+  protected Configuration loadNewConfiguration()
+      throws IOException, YarnException {
+    // Retrieve yarn-site.xml in order to refresh scheduling monitor properties.
+    Configuration conf = getConfiguration(new Configuration(false),
+        YarnConfiguration.YARN_SITE_CONFIGURATION_FILE,
+        YarnConfiguration.RESOURCE_TYPES_CONFIGURATION_FILE);
+    // The reason we call Configuration#size() is because when getConfiguration
+    // been called, it invokes Configuration#addResouce, which invokes
+    // Configuration#reloadConfiguration which triggers the reload process in a
+    // lazy way, the properties will only be reload when it's needed rather than
+    // reload it right after getConfiguration been called. So here we call
+    // Configuration#size() to force the Configuration#getProps been called to
+    // reload all the properties.
+    conf.size();
+    return conf;
+  }
+
   @Private
   public void refreshQueues() throws IOException, YarnException {
-    rm.getRMContext().getScheduler().reinitialize(getConfig(),
+    Configuration conf = loadNewConfiguration();
+    rm.getRMContext().getScheduler().reinitialize(conf,
         this.rm.getRMContext());
     // refresh the reservation system
     ReservationSystem rSystem = rm.getRMContext().getReservationSystem();
     if (rSystem != null) {
-      rSystem.reinitialize(getConfig(), rm.getRMContext());
+      rSystem.reinitialize(conf, rm.getRMContext());
     }
   }
 
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 ce95cff..86b9591 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
@@ -147,6 +147,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSe
 import org.apache.hadoop.yarn.server.utils.Lock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 @LimitedPrivate("yarn")
@@ -433,12 +434,15 @@ public class CapacityScheduler extends
       validateConf(this.conf);
       try {
         LOG.info("Re-initializing queues...");
-        refreshMaximumAllocation(this.conf.getMaximumAllocation());
+        refreshMaximumAllocation(
+            ResourceUtils.fetchMaximumAllocationFromConfig(this.conf));
         reinitializeQueues(this.conf);
       } catch (Throwable t) {
         this.conf = oldConf;
-        refreshMaximumAllocation(this.conf.getMaximumAllocation());
-        throw new IOException("Failed to re-init queues : "+ t.getMessage(), t);
+        refreshMaximumAllocation(
+            ResourceUtils.fetchMaximumAllocationFromConfig(this.conf));
+        throw new IOException("Failed to re-init queues : " + t.getMessage(),
+            t);
       }
 
       // update lazy preemption
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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
index 1e22e0b..8f3d59d 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPo
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import java.util.ArrayList;
@@ -773,16 +774,6 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     return Resources.createResource(minimumMemory, minimumCores);
   }
 
-  public Resource getMaximumAllocation() {
-    int maximumMemory = getInt(
-        YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
-    int maximumCores = getInt(
-        YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
-    return Resources.createResource(maximumMemory, maximumCores);
-  }
-
   @Private
   public Priority getQueuePriority(String queue) {
     String queuePolicyPrefix = getQueuePrefix(queue);
@@ -806,6 +797,8 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
    * @return setting specified per queue else falls back to the cluster setting
    */
   public Resource getMaximumAllocationPerQueue(String queue) {
+    // Only support to specify memory and vcores maximum allocation per queue
+    // for now.
     String queuePrefix = getQueuePrefix(queue);
     long maxAllocationMbPerQueue = getInt(queuePrefix + MAXIMUM_ALLOCATION_MB,
         (int)UNDEFINED);
@@ -817,7 +810,7 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
       LOG.debug("max alloc vcores per queue for " + queue + " is "
           + maxAllocationVcoresPerQueue);
     }
-    Resource clusterMax = getMaximumAllocation();
+    Resource clusterMax = ResourceUtils.fetchMaximumAllocationFromConfig(this);
     if (maxAllocationMbPerQueue == (int)UNDEFINED) {
       LOG.info("max alloc mb per queue for " + queue + " is undefined");
       maxAllocationMbPerQueue = clusterMax.getMemorySize();
@@ -826,8 +819,11 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
        LOG.info("max alloc vcore per queue for " + queue + " is undefined");
       maxAllocationVcoresPerQueue = clusterMax.getVirtualCores();
     }
-    Resource result = Resources.createResource(maxAllocationMbPerQueue,
-        maxAllocationVcoresPerQueue);
+    // Copy from clusterMax and overwrite per-queue's maximum memory/vcore
+    // allocation.
+    Resource result = Resources.clone(clusterMax);
+    result.setMemorySize(maxAllocationMbPerQueue);
+    result.setVirtualCores(maxAllocationVcoresPerQueue);
     if (maxAllocationMbPerQueue > clusterMax.getMemorySize()
         || maxAllocationVcoresPerQueue > clusterMax.getVirtualCores()) {
       throw new IllegalArgumentException(
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 c352ba4..993f089 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
@@ -540,8 +540,8 @@ public class LeafQueue extends AbstractCSQueue {
       // since we have already told running AM's the size
       Resource oldMax = getMaximumAllocation();
       Resource newMax = newlyParsedLeafQueue.getMaximumAllocation();
-      if (newMax.getMemorySize() < oldMax.getMemorySize()
-          || newMax.getVirtualCores() < oldMax.getVirtualCores()) {
+
+      if (!Resources.fitsIn(oldMax, newMax)) {
         throw new IOException("Trying to reinitialize " + getQueuePath()
             + " the maximum allocation size can not be decreased!"
             + " Current setting: " + oldMax + ", trying to set it to: "
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java
index 4ac4fc3..e756c49 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java
@@ -105,9 +105,34 @@ public abstract class RMHATestBase extends ClientBaseWithFixes{
     return am;
   }
 
+  private MockRM initMockRMWithOldConf(final Configuration confForRM1) {
+    return new MockRM(confForRM1, null, false, false) {
+      @Override
+      protected AdminService createAdminService() {
+        return new AdminService(this) {
+          @Override
+          protected void startServer() {
+            // override to not start rpc handler
+          }
+
+          @Override
+          protected void stopServer() {
+            // don't do anything
+          }
+
+          @Override
+          protected Configuration loadNewConfiguration() throws IOException, YarnException {
+            return confForRM1;
+          }
+        };
+      }
+    };
+  }
+
   protected void startRMs() throws IOException {
-    rm1 = new MockRM(confForRM1, null, false, false);
-    rm2 = new MockRM(confForRM2, null, false, false);
+    rm1 = initMockRMWithOldConf(confForRM1);
+    rm2 = initMockRMWithOldConf(confForRM2);
+
     startRMs(rm1, confForRM1, rm2, confForRM2);
   }
 
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 bb14e1e..a1ef3a0 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
@@ -135,6 +135,7 @@ import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
@@ -2941,7 +2942,7 @@ public class TestCapacityScheduler {
         conf.getMaximumAllocationPerQueue(A1).getMemorySize());
     assertEquals("max allocation",
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
-        conf.getMaximumAllocation().getMemorySize());
+        ResourceUtils.fetchMaximumAllocationFromConfig(conf).getMemorySize());
 
     CSQueue rootQueue = cs.getRootQueue();
     CSQueue queueA = findQueue(rootQueue, A);
@@ -3042,10 +3043,10 @@ public class TestCapacityScheduler {
         conf.getMaximumAllocationPerQueue(A1).getVirtualCores());
     assertEquals("cluster max allocation MB",
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
-        conf.getMaximumAllocation().getMemorySize());
+        ResourceUtils.fetchMaximumAllocationFromConfig(conf).getMemorySize());
     assertEquals("cluster max allocation vcores",
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
-        conf.getMaximumAllocation().getVirtualCores());
+        ResourceUtils.fetchMaximumAllocationFromConfig(conf).getVirtualCores());
 
     CSQueue rootQueue = cs.getRootQueue();
     CSQueue queueA = findQueue(rootQueue, A);
@@ -3064,10 +3065,10 @@ public class TestCapacityScheduler {
         conf.getMaximumAllocationPerQueue(A1).getVirtualCores());
     assertEquals("max allocation MB cluster",
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
-        conf.getMaximumAllocation().getMemorySize());
+        ResourceUtils.fetchMaximumAllocationFromConfig(conf).getMemorySize());
     assertEquals("max allocation vcores cluster",
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
-        conf.getMaximumAllocation().getVirtualCores());
+        ResourceUtils.fetchMaximumAllocationFromConfig(conf).getVirtualCores());
     assertEquals("queue max allocation MB", 6144,
         ((LeafQueue) queueA1).getMaximumAllocation().getMemorySize());
     assertEquals("queue max allocation vcores", 3,
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/TestCapacitySchedulerWithMultiResourceTypes.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/TestCapacitySchedulerWithMultiResourceTypes.java
new file mode 100644
index 0000000..1a30e1d
--- /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/TestCapacitySchedulerWithMultiResourceTypes.java
@@ -0,0 +1,190 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+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.rmapp.RMApp;
+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.event.NodeUpdateSchedulerEvent;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Test Capacity Scheduler with multiple resource types.
+ */
+public class TestCapacitySchedulerWithMultiResourceTypes {
+  private static String RESOURCE_1 = "res1";
+  private final int GB = 1024;
+
+  @Test
+  public void testMaximumAllocationRefreshWithMultipleResourceTypes() throws Exception {
+
+    // Initialize resource map
+    Map<String, ResourceInformation> riMap = new HashMap<>();
+
+    // Initialize mandatory resources
+    ResourceInformation memory = ResourceInformation.newInstance(
+        ResourceInformation.MEMORY_MB.getName(),
+        ResourceInformation.MEMORY_MB.getUnits(),
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
+    ResourceInformation vcores = ResourceInformation.newInstance(
+        ResourceInformation.VCORES.getName(),
+        ResourceInformation.VCORES.getUnits(),
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
+    riMap.put(ResourceInformation.MEMORY_URI, memory);
+    riMap.put(ResourceInformation.VCORES_URI, vcores);
+    riMap.put(RESOURCE_1, ResourceInformation.newInstance(RESOURCE_1, "", 0,
+        ResourceTypes.COUNTABLE, 0, 3333L));
+
+    ResourceUtils.initializeResourcesFromResourceInformationMap(riMap);
+
+    CapacitySchedulerConfiguration csconf =
+        new CapacitySchedulerConfiguration();
+    csconf.setMaximumApplicationMasterResourcePerQueuePercent("root", 100.0f);
+    csconf.setMaximumAMResourcePercentPerPartition("root", "", 100.0f);
+    csconf.setMaximumApplicationMasterResourcePerQueuePercent("root.default",
+        100.0f);
+    csconf.setMaximumAMResourcePercentPerPartition("root.default", "", 100.0f);
+    csconf.setResourceComparator(DominantResourceCalculator.class);
+    csconf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_1);
+    csconf.setInt(YarnConfiguration.RESOURCE_TYPES + "." + RESOURCE_1
+        + ".maximum-allocation", 3333);
+
+    YarnConfiguration conf = new YarnConfiguration(csconf);
+    // Don't reset resource types since we have already configured resource
+    // types
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+
+    MockRM rm = new MockRM(conf);
+    rm.start();
+
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    Assert.assertEquals(3333L,
+        cs.getMaximumResourceCapability().getResourceValue(RESOURCE_1));
+    Assert.assertEquals(3333L,
+        cs.getMaximumAllocation().getResourceValue(RESOURCE_1));
+    Assert.assertEquals(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+        cs.getMaximumResourceCapability()
+            .getResourceValue(ResourceInformation.MEMORY_URI));
+    Assert.assertEquals(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+        cs.getMaximumAllocation()
+            .getResourceValue(ResourceInformation.MEMORY_URI));
+    Assert.assertEquals(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
+        cs.getMaximumResourceCapability()
+            .getResourceValue(ResourceInformation.VCORES_URI));
+    Assert.assertEquals(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
+        cs.getMaximumAllocation()
+            .getResourceValue(ResourceInformation.VCORES_URI));
+
+    // Set RES_1 to 3332 (less than 3333) and refresh CS, failures expected.
+    csconf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_1);
+    csconf.setInt(YarnConfiguration.RESOURCE_TYPES + "." + RESOURCE_1
+        + ".maximum-allocation", 3332);
+
+    boolean exception = false;
+    try {
+      cs.reinitialize(csconf, rm.getRMContext());
+    } catch (IOException e) {
+      exception = true;
+    }
+
+    Assert.assertTrue("Should have exception in CS", exception);
+
+    // Maximum allocation won't be updated
+    Assert.assertEquals(3333L,
+        cs.getMaximumResourceCapability().getResourceValue(RESOURCE_1));
+    Assert.assertEquals(3333L,
+        cs.getMaximumAllocation().getResourceValue(RESOURCE_1));
+    Assert.assertEquals(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+        cs.getMaximumResourceCapability()
+            .getResourceValue(ResourceInformation.MEMORY_URI));
+    Assert.assertEquals(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+        cs.getMaximumAllocation()
+            .getResourceValue(ResourceInformation.MEMORY_URI));
+    Assert.assertEquals(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
+        cs.getMaximumResourceCapability()
+            .getResourceValue(ResourceInformation.VCORES_URI));
+    Assert.assertEquals(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
+        cs.getMaximumAllocation()
+            .getResourceValue(ResourceInformation.VCORES_URI));
+
+    // Set RES_1 to 3334 and refresh CS, should success
+    csconf.set(YarnConfiguration.RESOURCE_TYPES, RESOURCE_1);
+    csconf.setInt(YarnConfiguration.RESOURCE_TYPES + "." + RESOURCE_1
+        + ".maximum-allocation", 3334);
+    cs.reinitialize(csconf, rm.getRMContext());
+
+    // Maximum allocation will be updated
+    Assert.assertEquals(3334,
+        cs.getMaximumResourceCapability().getResourceValue(RESOURCE_1));
+
+    // Since we haven't updated the real configuration of ResourceUtils,
+    // cs.getMaximumAllocation won't be updated.
+    Assert.assertEquals(3333,
+        cs.getMaximumAllocation().getResourceValue(RESOURCE_1));
+    Assert.assertEquals(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+        cs.getMaximumResourceCapability()
+            .getResourceValue(ResourceInformation.MEMORY_URI));
+    Assert.assertEquals(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+        cs.getMaximumAllocation()
+            .getResourceValue(ResourceInformation.MEMORY_URI));
+    Assert.assertEquals(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
+        cs.getMaximumResourceCapability()
+            .getResourceValue(ResourceInformation.VCORES_URI));
+    Assert.assertEquals(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
+        cs.getMaximumAllocation()
+            .getResourceValue(ResourceInformation.VCORES_URI));
+
+    rm.close();
+  }
+}


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