浏览代码

YARN-10154. Addendum Patch which fixes below bugs

1. RM fails to start when LeafQueueTemplate max capacity is not specified.
2. Job stuck in ACCEPTED state with DominantResourceCalculator as Queue
   Capacity is set to NaN during RM startup with clusterResource is zero.

Reviewed by Sunil G and Manikandan R.
Prabhu Joseph 5 年之前
父节点
当前提交
450e5aa9dd

+ 54 - 29
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ManagedParentQueue.java

@@ -192,40 +192,55 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
      *
      *
      */
      */
     if (this.capacityConfigType.equals(CapacityConfigType.ABSOLUTE_RESOURCE)) {
     if (this.capacityConfigType.equals(CapacityConfigType.ABSOLUTE_RESOURCE)) {
-      for (String label : queueCapacities.getExistingNodeLabels()) {
-        queueCapacities.setCapacity(label,
-            this.csContext.getResourceCalculator().divide(
-                this.csContext.getClusterResource(),
-                this.csContext.getConfiguration().getMinimumResourceRequirement(
-                    label,
-                    this.csContext.getConfiguration()
-                        .getAutoCreatedQueueTemplateConfPrefix(getQueuePath()),
-                    resourceTypes),
-                getQueueResourceQuotas().getConfiguredMinResource(label)));
-
-        queueCapacities.setMaximumCapacity(label,
-            this.csContext.getResourceCalculator().divide(
-                this.csContext.getClusterResource(),
-                this.csContext.getConfiguration().getMaximumResourceRequirement(
-                    label,
-                    this.csContext.getConfiguration()
-                        .getAutoCreatedQueueTemplateConfPrefix(getQueuePath()),
-                    resourceTypes),
-                getQueueResourceQuotas().getConfiguredMaxResource(label)));
-
-        queueCapacities.setAbsoluteCapacity(label,
-            queueCapacities.getCapacity(label)
-                * getQueueCapacities().getAbsoluteCapacity(label));
-
-        queueCapacities.setAbsoluteMaximumCapacity(label,
-            queueCapacities.getMaximumCapacity(label)
-                * getQueueCapacities().getAbsoluteMaximumCapacity(label));
-      }
+      updateQueueCapacities(queueCapacities);
     }
     }
     builder.capacities(queueCapacities);
     builder.capacities(queueCapacities);
     return builder;
     return builder;
   }
   }
 
 
+  private void updateQueueCapacities(QueueCapacities queueCapacities) {
+    for (String label : queueCapacities.getExistingNodeLabels()) {
+      queueCapacities.setCapacity(label,
+          this.csContext.getResourceCalculator().divide(
+              this.csContext.getClusterResource(),
+              this.csContext.getConfiguration().getMinimumResourceRequirement(
+                  label,
+                  this.csContext.getConfiguration()
+                      .getAutoCreatedQueueTemplateConfPrefix(getQueuePath()),
+                  resourceTypes),
+              getQueueResourceQuotas().getConfiguredMinResource(label)));
+
+      Resource childMaxResource = this.csContext.getConfiguration()
+          .getMaximumResourceRequirement(label,
+              this.csContext.getConfiguration()
+                  .getAutoCreatedQueueTemplateConfPrefix(getQueuePath()),
+              resourceTypes);
+      Resource parentMaxRes = getQueueResourceQuotas()
+          .getConfiguredMaxResource(label);
+
+      Resource effMaxResource = Resources.min(
+          this.csContext.getResourceCalculator(),
+          this.csContext.getClusterResource(),
+          childMaxResource.equals(Resources.none()) ? parentMaxRes
+              : childMaxResource,
+          parentMaxRes);
+
+      queueCapacities.setMaximumCapacity(
+          label, this.csContext.getResourceCalculator().divide(
+               this.csContext.getClusterResource(),
+               effMaxResource,
+               getQueueResourceQuotas().getConfiguredMaxResource(label)));
+
+      queueCapacities.setAbsoluteCapacity(
+          label, queueCapacities.getCapacity(label)
+          * getQueueCapacities().getAbsoluteCapacity(label));
+
+      queueCapacities.setAbsoluteMaximumCapacity(label,
+          queueCapacities.getMaximumCapacity(label)
+          * getQueueCapacities().getAbsoluteMaximumCapacity(label));
+    }
+  }
+
   protected void validate(final CSQueue newlyParsedQueue) throws IOException {
   protected void validate(final CSQueue newlyParsedQueue) throws IOException {
     // Sanity check
     // Sanity check
     if (!(newlyParsedQueue instanceof ManagedParentQueue) || !newlyParsedQueue
     if (!(newlyParsedQueue instanceof ManagedParentQueue) || !newlyParsedQueue
@@ -276,6 +291,16 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
 
 
       AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) childQueue;
       AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) childQueue;
       super.addChildQueue(leafQueue);
       super.addChildQueue(leafQueue);
+
+      /* Below is to avoid Setting Queue Capacity to NaN when ClusterResource
+         is zero during RM Startup with DominantResourceCalculator */
+      if (this.capacityConfigType.equals(
+          CapacityConfigType.ABSOLUTE_RESOURCE)) {
+        QueueCapacities queueCapacities =
+            getLeafQueueTemplate().getQueueCapacities();
+        updateQueueCapacities(queueCapacities);
+      }
+
       final AutoCreatedLeafQueueConfig initialLeafQueueTemplate =
       final AutoCreatedLeafQueueConfig initialLeafQueueTemplate =
           queueManagementPolicy.getInitialLeafQueueConfiguration(leafQueue);
           queueManagementPolicy.getInitialLeafQueueConfiguration(leafQueue);
 
 

+ 41 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAbsoluteResourceWithAutoQueue.java

@@ -22,6 +22,7 @@ import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueueUtils.EPSILON;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
 
 
 import java.util.HashMap;
 import java.util.HashMap;
@@ -33,11 +34,17 @@ import java.util.Set;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 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.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmissionData;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmitter;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.queuemanagement.GuaranteedOrZeroCapacityOverTimePolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.queuemanagement.GuaranteedOrZeroCapacityOverTimePolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderingPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderingPolicy;
+import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
@@ -274,4 +281,38 @@ public class TestAbsoluteResourceWithAutoQueue
     fail("Exception should be thrown as leaf queue template configuration is "
     fail("Exception should be thrown as leaf queue template configuration is "
         + "not same as Parent configuration");
         + "not same as Parent configuration");
   }
   }
+
+  @Test(timeout = 20000)
+  public void testApplicationRunningWithDRF() throws Exception {
+    CapacitySchedulerConfiguration csConf =
+        setupSimpleQueueConfiguration(false);
+    setupMinMaxResourceConfiguration(csConf);
+    csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        ResourceScheduler.class);
+
+    // Validate Leaf Queue Template in Absolute Resource with DRF
+    csConf.setResourceComparator(DominantResourceCalculator.class);
+    setupGroupQueueMappings(QUEUED, csConf, "%user");
+
+    mockRM = new MockRM(csConf);
+    mockRM.start();
+
+    MockNM nm1 = mockRM.registerNode("127.0.0.1:1234", 250 * GB, 40);
+
+    // Submit a Application and validate if it is moving to RUNNING state
+    RMApp app1 = MockRMAppSubmitter.submit(mockRM,
+        MockRMAppSubmissionData.Builder.createWithMemory(1024, mockRM)
+            .withAppName("app1")
+            .withUser(TEST_GROUPUSER)
+            .withAcls(null)
+            .build());
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, mockRM, nm1);
+
+    cs = (CapacityScheduler) mockRM.getResourceScheduler();
+    AutoCreatedLeafQueue autoCreatedLeafQueue =
+        (AutoCreatedLeafQueue) cs.getQueue(TEST_GROUPUSER);
+    assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue);
+    ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue(QUEUED);
+    assertEquals(parentQueue, autoCreatedLeafQueue.getParent());
+  }
 }
 }