Prechádzať zdrojové kódy

YARN-10604. Support auto queue creation without mapping rules. Contributed by Andras Gyori

Szilard Nemeth 4 rokov pred
rodič
commit
7d88953768

+ 16 - 2
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

@@ -932,12 +932,26 @@ public class CapacityScheduler extends
       boolean isRecovery) {
 
     CSQueue queue = getQueue(queueName);
+    ApplicationPlacementContext fallbackContext = placementContext;
 
     if (queue == null) {
-      if (placementContext != null && placementContext.hasParentQueue()) {
+      // Even if placement rules are turned off, we still have the opportunity
+      // to auto create a queue.
+      if (placementContext == null) {
+        fallbackContext = CSQueueUtils.extractQueuePath(queueName);
+      }
+      if (fallbackContext.hasParentQueue()) {
         try {
-          return autoCreateLeafQueue(placementContext);
+          return autoCreateLeafQueue(fallbackContext);
         } catch (YarnException | IOException e) {
+          // A null queue is expected if the placementContext is null. In order
+          // not to disrupt the control flow, if we fail to auto create a queue,
+          // we fall back to the original logic.
+          if (placementContext == null) {
+            LOG.error("Could not auto-create leaf queue " + queueName +
+                " due to : ", e);
+            return null;
+          }
           if (isRecovery) {
             if (!getConfiguration().shouldAppFailFast(getConfig())) {
               LOG.error("Could not auto-create leaf queue " + queueName +

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

@@ -18,14 +18,21 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 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.placement.ApplicationPlacementContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -461,7 +468,28 @@ public class TestCapacitySchedulerNewQueueAutoCreation
             "for auto queue creation",
         ((ParentQueue)empty).isEligibleForAutoQueueCreation());
   }
-  
+
+  @Test
+  public void testAutoQueueCreationWithDisabledMappingRules() throws Exception {
+    startScheduler();
+
+    ApplicationId appId = BuilderUtils.newApplicationId(1, 1);
+    // Set ApplicationPlacementContext to null in the submitted application
+    // in order to imitate a submission with mapping rules turned off
+    SchedulerEvent addAppEvent = new AppAddedSchedulerEvent(appId,
+        "root.a.a1-auto.a2-auto", USER0, null);
+    ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
+        appId, 1);
+    SchedulerEvent addAttemptEvent = new AppAttemptAddedSchedulerEvent(
+        appAttemptId, false);
+    cs.handle(addAppEvent);
+    cs.handle(addAttemptEvent);
+
+    CSQueue a2Auto = cs.getQueue("root.a.a1-auto.a2-auto");
+    Assert.assertNotNull(a2Auto);
+  }
+
+  @Test
   public void testAutoCreateQueueUserLimitDisabled() throws Exception {
     startScheduler();
     createBasicQueueStructureAndValidate();