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/08/21 22:52:41 UTC

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

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

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

commit 0cfa058adf48fcf73ff5987a2b5dfffed60fb755
Author: Jonathan Hung <jh...@linkedin.com>
AuthorDate: Wed Mar 27 09:55:05 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 67592cc..a4c1f6c 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
@@ -225,6 +225,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 b945183..6e8eb81 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
@@ -201,9 +201,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);
 
@@ -249,6 +263,13 @@ public class ResourceUtils {
 
     setAllocationForMandatoryResources(resourceInformationMap, conf);
 
+    return resourceInformationMap;
+  }
+
+  @VisibleForTesting
+  static void initializeResourcesMap(Configuration conf) {
+    Map<String, ResourceInformation> resourceInformationMap =
+        getResourceInformationMapFromConfig(conf);
     initializeResourcesFromResourceInformationMap(resourceInformationMap);
   }
 
@@ -546,19 +567,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 38ee606..b9b57e4 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
@@ -407,14 +407,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 28b8c6b..27859b7 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
@@ -143,6 +143,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;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -442,12 +443,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 1754a97..7cdf8b7 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;
@@ -790,16 +791,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);
@@ -823,6 +814,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);
@@ -834,7 +827,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();
@@ -843,8 +836,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 4ef26e6..d4fe1c9 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
@@ -527,8 +527,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..2b36ed0 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(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 b7f69fc..caf16fb 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
@@ -156,6 +156,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;
@@ -2943,7 +2944,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);
@@ -3044,10 +3045,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);
@@ -3066,10 +3067,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