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 sn...@apache.org on 2021/07/28 14:40:33 UTC

[hadoop] branch trunk updated: YARN-10790. CS Flexible AQC: Add separate parent and leaf template property. Contributed by Andras Gyori

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

snemeth pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new baea191  YARN-10790. CS Flexible AQC: Add separate parent and leaf template property. Contributed by Andras Gyori
baea191 is described below

commit baea191a91cd881e2df987aa7de8553627e8e109
Author: Szilard Nemeth <sn...@apache.org>
AuthorDate: Wed Jul 28 16:40:23 2021 +0200

    YARN-10790. CS Flexible AQC: Add separate parent and leaf template property. Contributed by Andras Gyori
---
 .../capacity/AutoCreatedQueueTemplate.java         | 102 ++++++++++++++++++---
 .../webapp/dao/CapacitySchedulerInfo.java          |  14 ++-
 .../webapp/dao/CapacitySchedulerQueueInfo.java     |  16 +++-
 .../dao/helper/CapacitySchedulerInfoHelper.java    |   5 +-
 .../capacity/TestAutoCreatedQueueTemplate.java     |  40 ++++++++
 .../webapp/TestRMWebServicesCapacitySched.java     |   6 +-
 .../TestRMWebServicesForCSWithPartitions.java      |   2 +-
 7 files changed, 162 insertions(+), 23 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/capacity/AutoCreatedQueueTemplate.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/AutoCreatedQueueTemplate.java
index 203ec4d..cf78fed 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/AutoCreatedQueueTemplate.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/AutoCreatedQueueTemplate.java
@@ -30,6 +30,7 @@ import java.util.Set;
 
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.AUTO_QUEUE_CREATION_V2_PREFIX;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getQueuePrefix;
 
 /**
  * A handler for storing and setting auto created queue template settings.
@@ -37,10 +38,17 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.C
 public class AutoCreatedQueueTemplate {
   public static final String AUTO_QUEUE_TEMPLATE_PREFIX =
       AUTO_QUEUE_CREATION_V2_PREFIX + "template.";
+  public static final String AUTO_QUEUE_LEAF_TEMPLATE_PREFIX =
+      AUTO_QUEUE_CREATION_V2_PREFIX + "leaf-template.";
+  public static final String AUTO_QUEUE_PARENT_TEMPLATE_PREFIX =
+      AUTO_QUEUE_CREATION_V2_PREFIX + "parent-template.";
+
   private static final String WILDCARD_QUEUE = "*";
   private static final int MAX_WILDCARD_LEVEL = 1;
 
   private final Map<String, String> templateProperties = new HashMap<>();
+  private final Map<String, String> leafOnlyProperties = new HashMap<>();
+  private final Map<String, String> parentOnlyProperties = new HashMap<>();
 
   public AutoCreatedQueueTemplate(Configuration configuration,
                                   String queuePath) {
@@ -54,7 +62,7 @@ public class AutoCreatedQueueTemplate {
   }
 
   /**
-   * Get the template properties attached to a parent queue.
+   * Get the common template properties specified for a parent queue.
    * @return template property names and values
    */
   public Map<String, String> getTemplateProperties() {
@@ -62,13 +70,43 @@ public class AutoCreatedQueueTemplate {
   }
 
   /**
-   * Sets the configuration properties of a child queue based on its parent
-   * template settings.
-   * @param conf configuration to set
+   * Get the leaf specific template properties specified for a parent queue.
+   * @return template property names and values
+   */
+  public Map<String, String> getLeafOnlyProperties() {
+    return leafOnlyProperties;
+  }
+
+  /**
+   * Get the parent specific template properties specified for a parent queue.
+   * @return template property names and values
+   */
+  public Map<String, String> getParentOnlyProperties() {
+    return parentOnlyProperties;
+  }
+
+  /**
+   * Sets the common template properties and parent specific template
+   * properties of a child queue based on its parent template settings.
+ * @param conf configuration to set
    * @param childQueuePath child queue path used for prefixing the properties
    */
   public void setTemplateEntriesForChild(Configuration conf,
                                          String childQueuePath) {
+    setTemplateEntriesForChild(conf, childQueuePath, false);
+  }
+
+  /**
+   * Sets the common template properties and leaf or parent specific template
+   * properties of a child queue based on its parent template settings.
+   * @param conf configuration to set
+   * @param isLeaf whether to include leaf specific template properties, or
+   *               parent specific template properties
+   * @param childQueuePath child queue path used for prefixing the properties
+   */
+  public void setTemplateEntriesForChild(Configuration conf,
+                                         String childQueuePath,
+                                         boolean isLeaf) {
     if (childQueuePath.equals(ROOT)) {
       return;
     }
@@ -77,7 +115,14 @@ public class AutoCreatedQueueTemplate {
     Set<String> alreadySetProps = conf.getPropsWithPrefix(
         CapacitySchedulerConfiguration.getQueuePrefix(childQueuePath)).keySet();
 
-    for (Map.Entry<String, String> entry : templateProperties.entrySet()) {
+    // Check template properties only set for leaf or parent queues
+    Map<String, String> queueTypeSpecificTemplates = parentOnlyProperties;
+    if (isLeaf) {
+      queueTypeSpecificTemplates = leafOnlyProperties;
+    }
+
+    for (Map.Entry<String, String> entry :
+        queueTypeSpecificTemplates.entrySet()) {
       // Do not overwrite explicitly configured properties
       if (alreadySetProps.contains(entry.getKey())) {
         continue;
@@ -85,6 +130,17 @@ public class AutoCreatedQueueTemplate {
       conf.set(CapacitySchedulerConfiguration.getQueuePrefix(
           childQueuePath) + entry.getKey(), entry.getValue());
     }
+
+    for (Map.Entry<String, String> entry : templateProperties.entrySet()) {
+      // Do not overwrite explicitly configured properties or properties set
+      // by queue type specific templates (parent-template and leaf-template)
+      if (alreadySetProps.contains(entry.getKey())
+          || queueTypeSpecificTemplates.containsKey(entry.getKey())) {
+        continue;
+      }
+      conf.set(CapacitySchedulerConfiguration.getQueuePrefix(
+          childQueuePath) + entry.getKey(), entry.getValue());
+    }
   }
 
   /**
@@ -118,16 +174,15 @@ public class AutoCreatedQueueTemplate {
 
     // Collect all template entries
     while (wildcardLevel <= supportedWildcardLevel) {
-      // Get all config entries with the specified prefix
       String templateQueuePath = String.join(".", queuePathParts);
       // Get all configuration entries with
-      // <queuePath>.auto-queue-creation-v2.template prefix
-      Map<String, String> props = configuration.getPropsWithPrefix(
-          getAutoQueueTemplatePrefix(templateQueuePath));
+      // yarn.scheduler.capacity.<queuePath> prefix
+      Map<String, String> queueProps = configuration.getPropsWithPrefix(
+          getQueuePrefix(templateQueuePath));
 
-      for (Map.Entry<String, String> entry : props.entrySet()) {
-        // If an entry is already present, it had a higher precedence
-        templateProperties.putIfAbsent(entry.getKey(), entry.getValue());
+      // Store template, parent-template and leaf-template properties
+      for (Map.Entry<String, String> entry : queueProps.entrySet()) {
+        storeConfiguredTemplates(entry.getKey(), entry.getValue());
       }
 
       // Replace a queue part with a wildcard based on the wildcard level
@@ -138,4 +193,27 @@ public class AutoCreatedQueueTemplate {
       ++wildcardLevel;
     }
   }
+
+  private void storeConfiguredTemplates(
+      String templateKey, String templateValue) {
+    String prefix = "";
+    Map<String, String> properties = templateProperties;
+
+    if (templateKey.startsWith(AUTO_QUEUE_TEMPLATE_PREFIX)) {
+      prefix = AUTO_QUEUE_TEMPLATE_PREFIX;
+    } else if (templateKey.startsWith(AUTO_QUEUE_LEAF_TEMPLATE_PREFIX)) {
+      prefix = AUTO_QUEUE_LEAF_TEMPLATE_PREFIX;
+      properties = leafOnlyProperties;
+    } else if (templateKey.startsWith(AUTO_QUEUE_PARENT_TEMPLATE_PREFIX)) {
+      prefix = AUTO_QUEUE_PARENT_TEMPLATE_PREFIX;
+      properties = parentOnlyProperties;
+    }
+
+    if (!prefix.isEmpty()) {
+      // Trim template prefix from key
+      String key = templateKey.substring(prefix.length());
+      // If an entry is already present, it had a higher precedence
+      properties.putIfAbsent(key, templateValue);
+    }
+  }
 }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
index cc02c23..79d11e2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java
@@ -63,6 +63,8 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
   protected String autoCreationEligibility;
   protected String defaultNodeLabelExpression;
   protected AutoQueueTemplatePropertiesInfo autoQueueTemplateProperties;
+  protected AutoQueueTemplatePropertiesInfo autoQueueParentTemplateProperties;
+  protected AutoQueueTemplatePropertiesInfo autoQueueLeafTemplateProperties;
 
   @XmlTransient
   static final float EPSILON = 1e-8f;
@@ -108,10 +110,18 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
 
     queuePriority = parent.getPriority().getPriority();
     if (parent instanceof ParentQueue) {
-      orderingPolicyInfo = ((ParentQueue) parent).getQueueOrderingPolicy()
+      ParentQueue queue = (ParentQueue) parent;
+      orderingPolicyInfo = queue.getQueueOrderingPolicy()
           .getConfigName();
       autoQueueTemplateProperties = CapacitySchedulerInfoHelper
-            .getAutoCreatedTemplate((ParentQueue) parent);
+          .getAutoCreatedTemplate(queue.getAutoCreatedQueueTemplate()
+              .getTemplateProperties());
+      autoQueueParentTemplateProperties = CapacitySchedulerInfoHelper
+          .getAutoCreatedTemplate(queue.getAutoCreatedQueueTemplate()
+              .getParentOnlyProperties());
+      autoQueueLeafTemplateProperties = CapacitySchedulerInfoHelper
+          .getAutoCreatedTemplate(queue.getAutoCreatedQueueTemplate()
+              .getLeafOnlyProperties());
     }
     mode = CapacitySchedulerInfoHelper.getMode(parent);
     queueType = CapacitySchedulerInfoHelper.getQueueType(parent);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
index 152620d..452fa50 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java
@@ -92,6 +92,10 @@ public class CapacitySchedulerQueueInfo {
   protected String defaultNodeLabelExpression;
   protected AutoQueueTemplatePropertiesInfo autoQueueTemplateProperties =
       new AutoQueueTemplatePropertiesInfo();
+  protected AutoQueueTemplatePropertiesInfo autoQueueParentTemplateProperties =
+      new AutoQueueTemplatePropertiesInfo();
+  protected AutoQueueTemplatePropertiesInfo autoQueueLeafTemplateProperties =
+      new AutoQueueTemplatePropertiesInfo();
 
   CapacitySchedulerQueueInfo() {
   };
@@ -172,10 +176,18 @@ public class CapacitySchedulerQueueInfo {
 
     queuePriority = q.getPriority().getPriority();
     if (q instanceof ParentQueue) {
-      orderingPolicyInfo = ((ParentQueue) q).getQueueOrderingPolicy()
+      ParentQueue queue = (ParentQueue) q;
+      orderingPolicyInfo = queue.getQueueOrderingPolicy()
           .getConfigName();
       autoQueueTemplateProperties = CapacitySchedulerInfoHelper
-            .getAutoCreatedTemplate((ParentQueue) q);
+            .getAutoCreatedTemplate(queue.getAutoCreatedQueueTemplate()
+                .getTemplateProperties());
+      autoQueueParentTemplateProperties = CapacitySchedulerInfoHelper
+          .getAutoCreatedTemplate(queue.getAutoCreatedQueueTemplate()
+              .getParentOnlyProperties());
+      autoQueueLeafTemplateProperties = CapacitySchedulerInfoHelper
+          .getAutoCreatedTemplate(queue.getAutoCreatedQueueTemplate()
+              .getLeafOnlyProperties());
     }
 
     String configuredCapacity = conf.get(
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/helper/CapacitySchedulerInfoHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/helper/CapacitySchedulerInfoHelper.java
index f71928b..8b3602d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/helper/CapacitySchedulerInfoHelper.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/helper/CapacitySchedulerInfoHelper.java
@@ -112,12 +112,11 @@ public class CapacitySchedulerInfoHelper {
   }
 
   public static AutoQueueTemplatePropertiesInfo getAutoCreatedTemplate(
-      ParentQueue parent) {
+      Map<String, String> templateProperties) {
     AutoQueueTemplatePropertiesInfo propertiesInfo =
         new AutoQueueTemplatePropertiesInfo();
     for (Map.Entry<String, String> e :
-        parent.getAutoCreatedQueueTemplate().getTemplateProperties()
-            .entrySet()) {
+        templateProperties.entrySet()) {
       propertiesInfo.add(new ConfItem(e.getKey(), e.getValue()));
     }
 
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/TestAutoCreatedQueueTemplate.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/TestAutoCreatedQueueTemplate.java
index 2763af0..f65c788 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/TestAutoCreatedQueueTemplate.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/TestAutoCreatedQueueTemplate.java
@@ -22,10 +22,13 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.AUTO_CREATE_CHILD_QUEUE_AUTO_REMOVAL_ENABLE;
+
 public class TestAutoCreatedQueueTemplate {
   private static final String TEST_QUEUE_ABC = "root.a.b.c";
   private static final String TEST_QUEUE_AB = "root.a.b";
   private static final String TEST_QUEUE_A = "root.a";
+  private static final String TEST_QUEUE_B = "root.b";
   private static final String ROOT = "root";
   private CapacitySchedulerConfiguration conf;
 
@@ -120,8 +123,45 @@ public class TestAutoCreatedQueueTemplate {
         conf.getNonLabeledQueueWeight(TEST_QUEUE_A), 10e-6);
   }
 
+  @Test
+  public void testQueueSpecificTemplates() {
+    conf.set(getTemplateKey("root", "capacity"), "2w");
+    conf.set(getLeafTemplateKey("root",
+        "default-node-label-expression"), "test");
+    conf.set(getLeafTemplateKey("root", "capacity"), "10w");
+    conf.setBoolean(getParentTemplateKey(
+        "root", AUTO_CREATE_CHILD_QUEUE_AUTO_REMOVAL_ENABLE), false);
+
+    AutoCreatedQueueTemplate template =
+        new AutoCreatedQueueTemplate(conf, ROOT);
+    template.setTemplateEntriesForChild(conf, TEST_QUEUE_A);
+    template.setTemplateEntriesForChild(conf, TEST_QUEUE_B, true);
+
+    Assert.assertNull("default-node-label-expression is set for parent",
+        conf.getDefaultNodeLabelExpression(TEST_QUEUE_A));
+    Assert.assertEquals("default-node-label-expression is not set for leaf",
+        "test", conf.getDefaultNodeLabelExpression(TEST_QUEUE_B));
+    Assert.assertFalse("auto queue removal is not disabled for parent",
+        conf.isAutoExpiredDeletionEnabled(TEST_QUEUE_A));
+    Assert.assertEquals("weight should not be overridden when set by " +
+            "queue type specific template",
+        10f, conf.getNonLabeledQueueWeight(TEST_QUEUE_B), 10e-6);
+    Assert.assertEquals("weight should be set by common template",
+        2f, conf.getNonLabeledQueueWeight(TEST_QUEUE_A), 10e-6);
+  }
+
   private String getTemplateKey(String queuePath, String entryKey) {
     return CapacitySchedulerConfiguration.getQueuePrefix(queuePath)
         + AutoCreatedQueueTemplate.AUTO_QUEUE_TEMPLATE_PREFIX + entryKey;
   }
+
+  private String getParentTemplateKey(String queuePath, String entryKey) {
+    return CapacitySchedulerConfiguration.getQueuePrefix(queuePath)
+        + AutoCreatedQueueTemplate.AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + entryKey;
+  }
+
+  private String getLeafTemplateKey(String queuePath, String entryKey) {
+    return CapacitySchedulerConfiguration.getQueuePrefix(queuePath)
+        + AutoCreatedQueueTemplate.AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + entryKey;
+  }
 }
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
index 459645f..b472e09 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
@@ -369,7 +369,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
     JSONObject info = json.getJSONObject("scheduler");
     assertEquals("incorrect number of elements in: " + info, 1, info.length());
     info = info.getJSONObject("schedulerInfo");
-    assertEquals("incorrect number of elements in: " + info, 20, info.length());
+    assertEquals("incorrect number of elements in: " + info, 22, info.length());
     verifyClusterSchedulerGeneric(info.getString("type"),
         (float) info.getDouble("usedCapacity"),
         (float) info.getDouble("capacity"),
@@ -424,10 +424,10 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
   private void verifySubQueue(JSONObject info, String q,
       float parentAbsCapacity, float parentAbsMaxCapacity)
       throws JSONException, Exception {
-    int numExpectedElements = 35;
+    int numExpectedElements = 37;
     boolean isParentQueue = true;
     if (!info.has("queues")) {
-      numExpectedElements = 53;
+      numExpectedElements = 55;
       isParentQueue = false;
     }
     assertEquals("incorrect number of elements", numExpectedElements, info.length());
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.java
index 2e67d69..36b27a2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesForCSWithPartitions.java
@@ -574,7 +574,7 @@ public class TestRMWebServicesForCSWithPartitions extends JerseyTestBase {
     JSONObject info = json.getJSONObject("scheduler");
     assertEquals("incorrect number of elements", 1, info.length());
     info = info.getJSONObject("schedulerInfo");
-    assertEquals("incorrect number of elements", 20, info.length());
+    assertEquals("incorrect number of elements", 22, info.length());
     JSONObject capacitiesJsonObject = info.getJSONObject(CAPACITIES);
     JSONArray partitionsCapsArray =
         capacitiesJsonObject.getJSONArray(QUEUE_CAPACITIES_BY_PARTITION);

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