Prechádzať zdrojové kódy

YARN-10869. CS considers only the default maximum-allocation-mb/vcore property as a maximum when it creates dynamic queues (#3225)

Co-authored-by: Benjamin Teke <bteke@cloudera.com>
Benjamin Teke 3 rokov pred
rodič
commit
ac0a4e7f58

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

@@ -518,8 +518,14 @@ public abstract class AbstractCSQueue implements CSQueue {
 
   private void setupMaximumAllocation(CapacitySchedulerConfiguration csConf) {
     String myQueuePath = getQueuePath();
+    /* YARN-10869: When using AutoCreatedLeafQueues, the passed configuration
+    * object is a cloned one containing only the template configs
+    * (see ManagedParentQueue#getLeafQueueConfigs). To ensure that the actual
+    * cluster maximum allocation is fetched the original config object should
+    * be used.
+    */
     Resource clusterMax = ResourceUtils
-        .fetchMaximumAllocationFromConfig(csConf);
+        .fetchMaximumAllocationFromConfig(this.csContext.getConfiguration());
     Resource queueMax = csConf.getQueueMaximumAllocation(myQueuePath);
 
     maximumAllocation = Resources.clone(

+ 14 - 0
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

@@ -1043,6 +1043,11 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     }
   }
 
+  public void setQueueMaximumAllocation(String queue, String maximumAllocation) {
+    String queuePrefix = getQueuePrefix(queue);
+    set(queuePrefix + MAXIMUM_ALLOCATION, maximumAllocation);
+  }
+
   public long getQueueMaximumAllocationMb(String queue) {
     String queuePrefix = getQueuePrefix(queue);
     return getInt(queuePrefix + MAXIMUM_ALLOCATION_MB, (int)UNDEFINED);
@@ -2465,6 +2470,15 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     setDefaultNodeLabelExpression(leafQueueConfPrefix, expression);
   }
 
+  @Private
+  @VisibleForTesting
+  public void setAutoCreatedLeafQueueConfigMaximumAllocation(String
+         queuePath, String expression) {
+    String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
+        queuePath);
+    setQueueMaximumAllocation(leafQueueConfPrefix, expression);
+  }
+
   public static String getUnits(String resourceValue) {
     String units;
     for (int i = 0; i < resourceValue.length(); i++) {

+ 29 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoCreatedQueueBase.java

@@ -408,6 +408,9 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
     conf.setAutoCreatedLeafQueueConfigMaxCapacity(C, 100.0f);
     conf.setAutoCreatedLeafQueueConfigUserLimit(C, 100);
     conf.setAutoCreatedLeafQueueConfigUserLimitFactor(C, 3.0f);
+    conf.setAutoCreatedLeafQueueConfigUserLimitFactor(C, 3.0f);
+    conf.setAutoCreatedLeafQueueConfigMaximumAllocation(C,
+        "memory-mb=10240,vcores=6");
 
     conf.setAutoCreatedLeafQueueTemplateCapacityByLabel(C, NODEL_LABEL_GPU,
         NODE_LABEL_GPU_TEMPLATE_CAPACITY);
@@ -563,9 +566,31 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
     schedConf.setInt(YarnConfiguration.RESOURCE_TYPES
         + ".memory-mb.maximum-allocation", 16384);
 
+
     return new CapacitySchedulerConfiguration(schedConf);
   }
 
+  protected void setSchedulerMinMaxAllocation(CapacitySchedulerConfiguration conf) {
+    unsetMinMaxAllocation(conf);
+
+    conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, 1);
+    conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, 8);
+    conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 1024);
+    conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 18384);
+
+  }
+
+  private void unsetMinMaxAllocation(CapacitySchedulerConfiguration conf) {
+    conf.unset(YarnConfiguration.RESOURCE_TYPES
+        + ".vcores.minimum-allocation");
+    conf.unset(YarnConfiguration.RESOURCE_TYPES
+        + ".vcores.maximum-allocation");
+    conf.unset(YarnConfiguration.RESOURCE_TYPES
+        + ".memory-mb.minimum-allocation");
+    conf.unset(YarnConfiguration.RESOURCE_TYPES
+        + ".memory-mb.maximum-allocation");
+  }
+
   protected MockRM setupSchedulerInstance() throws Exception {
 
     if (mockRM != null) {
@@ -663,10 +688,11 @@ public class TestCapacitySchedulerAutoCreatedQueueBase {
   }
 
   protected void validateContainerLimits(
-      AutoCreatedLeafQueue autoCreatedLeafQueue) {
-    assertEquals(8,
+      AutoCreatedLeafQueue autoCreatedLeafQueue, int vCoreLimit,
+      long memorySize) {
+    assertEquals(vCoreLimit,
         autoCreatedLeafQueue.getMaximumAllocation().getVirtualCores());
-    assertEquals(16384,
+    assertEquals(memorySize,
         autoCreatedLeafQueue.getMaximumAllocation().getMemorySize());
   }
 

+ 31 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerAutoQueueCreation.java

@@ -142,7 +142,7 @@ public class TestCapacitySchedulerAutoQueueCreation
           expectedChildQueueAbsCapacity, accessibleNodeLabelsOnC);
 
       validateUserAndAppLimits(autoCreatedLeafQueue, 4000, 4000);
-      validateContainerLimits(autoCreatedLeafQueue);
+      validateContainerLimits(autoCreatedLeafQueue, 6, 10240);
 
       assertTrue(autoCreatedLeafQueue
           .getOrderingPolicy() instanceof FairOrderingPolicy);
@@ -171,6 +171,35 @@ public class TestCapacitySchedulerAutoQueueCreation
     }
   }
 
+  @Test(timeout = 20000)
+  public void testAutoCreateLeafQueueCreationSchedulerMaximumAllocation()
+      throws Exception {
+    try {
+      // Check the minimum/maximum allocation settings via the
+      // yarn.scheduler.minimum/maximum-allocation-mb/vcore property
+      setSchedulerMinMaxAllocation(cs.getConfiguration());
+      cs.getConfiguration().setAutoCreatedLeafQueueConfigMaximumAllocation(C,
+          "memory-mb=18384,vcores=8");
+      cs.reinitialize(cs.getConfiguration(), mockRM.getRMContext());
+
+      // submit an app
+      submitApp(mockRM, cs.getQueue(PARENT_QUEUE), USER0, USER0, 1, 1);
+
+      // check preconditions
+      List<ApplicationAttemptId> appsInC = cs.getAppsInQueue(PARENT_QUEUE);
+      assertEquals(1, appsInC.size());
+      assertNotNull(cs.getQueue(USER0));
+
+      AutoCreatedLeafQueue autoCreatedLeafQueue =
+          (AutoCreatedLeafQueue) cs.getQueue(USER0);
+
+      validateContainerLimits(autoCreatedLeafQueue, 8, 18384);
+    } finally {
+      cleanupQueue(USER0);
+      cleanupQueue(TEST_GROUPUSER);
+    }
+  }
+
   @Test(timeout = 20000)
   public void testAutoCreateLeafQueueCreationUsingFullParentPath()
       throws Exception {
@@ -936,7 +965,7 @@ public class TestCapacitySchedulerAutoQueueCreation
       validateCapacities(user3Queue, 0.3f, 0.09f, 0.4f,0.2f);
 
       validateUserAndAppLimits(user3Queue, 4000, 4000);
-      validateContainerLimits(user3Queue);
+      validateContainerLimits(user3Queue, 6, 10240);
 
       GuaranteedOrZeroCapacityOverTimePolicy autoCreatedQueueManagementPolicy =
           (GuaranteedOrZeroCapacityOverTimePolicy) ((ManagedParentQueue)