Преглед изворни кода

Merge -c 1227788 from trunk to branch-0.23 to fix MAPREDUCE-3326. Added detailed information about queue's to the CapacityScheduler web-ui.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1227794 13f79535-47bb-0310-9956-ffa450edef68
Arun Murthy пре 13 година
родитељ
комит
1828be291a

+ 3 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -360,6 +360,9 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-1744. DistributedCache creates its own FileSytem instance when 
     adding a file/archive to the path. (Dick King via tucu)
 
+    MAPREDUCE-3326. Added detailed information about queue's to the
+    CapacityScheduler web-ui. (Jason Lowe via acmurthy) 
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 4 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/ResponseInfo.java

@@ -80,6 +80,10 @@ public class ResponseInfo implements Iterable<ResponseInfo.Item> {
     return this;
   }
 
+  public void clear() {
+    items.clear();
+  }
+
   @Override
   public Iterator<Item> iterator() {
     return items.iterator();

+ 72 - 23
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java

@@ -20,18 +20,22 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
 import static org.apache.hadoop.yarn.util.StringHelper.join;
 
+import java.util.ArrayList;
+
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerLeafQueueInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfo;
+import org.apache.hadoop.yarn.webapp.ResponseInfo;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.LI;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.UL;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
 import com.google.inject.Inject;
 import com.google.inject.servlet.RequestScoped;
@@ -45,23 +49,61 @@ class CapacitySchedulerPage extends RmView {
   static final float EPSILON = 1e-8f;
 
   @RequestScoped
-  static class Parent {
-    CSQueue queue;
+  static class CSQInfo {
+    CapacitySchedulerInfo csinfo;
+    CapacitySchedulerQueueInfo qinfo;
+  }
+
+  static class LeafQueueInfoBlock extends HtmlBlock {
+    final CapacitySchedulerLeafQueueInfo lqinfo;
+
+    @Inject LeafQueueInfoBlock(ViewContext ctx, CSQInfo info) {
+      super(ctx);
+      lqinfo = (CapacitySchedulerLeafQueueInfo) info.qinfo;
+    }
+
+    @Override
+    protected void render(Block html) {
+      ResponseInfo ri = info("\'" + lqinfo.getQueuePath().substring(5) + "\' Queue Status").
+          _("Queue State:", lqinfo.getQueueState()).
+          _("Capacity:", percent(lqinfo.getCapacity() / 100)).
+          _("Max Capacity:", percent(lqinfo.getMaxCapacity() / 100)).
+          _("Used Capacity:", percent(lqinfo.getUsedCapacity() / 100)).
+          _("Absolute Capacity:", percent(lqinfo.getAbsoluteCapacity() / 100)).
+          _("Absolute Max Capacity:", percent(lqinfo.getAbsoluteMaxCapacity() / 100)).
+          _("Utilization:", percent(lqinfo.getUtilization() / 100)).
+          _("Used Resources:", lqinfo.getUsedResources().toString()).
+          _("Num Active Applications:", Integer.toString(lqinfo.getNumActiveApplications())).
+          _("Num Pending Applications:", Integer.toString(lqinfo.getNumPendingApplications())).
+          _("Num Containers:", Integer.toString(lqinfo.getNumContainers())).
+          _("Max Applications:", Integer.toString(lqinfo.getMaxApplications())).
+          _("Max Applications Per User:", Integer.toString(lqinfo.getMaxApplicationsPerUser())).
+          _("Max Active Applications:", Integer.toString(lqinfo.getMaxActiveApplications())).
+          _("Max Active Applications Per User:", Integer.toString(lqinfo.getMaxActiveApplicationsPerUser())).
+          _("User Limit:", Integer.toString(lqinfo.getUserLimit()) + "%").
+          _("User Limit Factor:", String.format("%.1f", lqinfo.getUserLimitFactor()));
+
+      html._(InfoBlock.class);
+
+      // clear the info contents so this queue's info doesn't accumulate into another queue's info
+      ri.clear();
+    }
   }
 
   public static class QueueBlock extends HtmlBlock {
-    final Parent parent;
-    final CapacitySchedulerInfo sinfo;
+    final CSQInfo csqinfo;
 
-    @Inject QueueBlock(Parent parent) {
-      this.parent = parent;
-      sinfo = new CapacitySchedulerInfo(parent.queue);
+    @Inject QueueBlock(CSQInfo info) {
+      csqinfo = info;
     }
 
     @Override
     public void render(Block html) {
+      ArrayList<CapacitySchedulerQueueInfo> subQueues =
+          (csqinfo.qinfo == null) ? csqinfo.csinfo.getSubQueues()
+              : csqinfo.qinfo.getSubQueues();
       UL<Hamlet> ul = html.ul();
-      for (CapacitySchedulerQueueInfo info : sinfo.getSubQueues()) {
+      for (CapacitySchedulerQueueInfo info : subQueues) {
         float used = info.getUsedCapacity() / 100;
         float set = info.getCapacity() / 100;
         float delta = Math.abs(set - used) + 0.001f;
@@ -76,11 +118,12 @@ class CapacitySchedulerPage extends RmView {
                 used > set ? OVER : UNDER, ';',
                 used > set ? left(set/max) : left(used/max)))._('.')._().
               span(".q", info.getQueuePath().substring(5))._();
-        if (info.getQueue() instanceof ParentQueue) {
-          // this could be optimized better
-          parent.queue = info.getQueue();
-          li.
-            _(QueueBlock.class);
+
+        csqinfo.qinfo = info;
+        if (info.getSubQueues() == null) {
+          li.ul("#lq").li()._(LeafQueueInfoBlock.class)._()._();
+        } else {
+          li._(QueueBlock.class);
         }
         li._();
       }
@@ -91,11 +134,11 @@ class CapacitySchedulerPage extends RmView {
 
   static class QueuesBlock extends HtmlBlock {
     final CapacityScheduler cs;
-    final Parent parent;
+    final CSQInfo csqinfo;
 
-    @Inject QueuesBlock(ResourceManager rm, Parent parent) {
+    @Inject QueuesBlock(ResourceManager rm, CSQInfo info) {
       cs = (CapacityScheduler) rm.getResourceScheduler();
-      this.parent = parent;
+      csqinfo = info;
     }
 
     @Override
@@ -115,8 +158,10 @@ class CapacitySchedulerPage extends RmView {
               span(".q", "default")._()._();
       } else {
         CSQueue root = cs.getRootQueue();
-        parent.queue = root;
-        CapacitySchedulerInfo sinfo = new CapacitySchedulerInfo(parent.queue);
+        CapacitySchedulerInfo sinfo = new CapacitySchedulerInfo(root);
+        csqinfo.csinfo = sinfo;
+        csqinfo.qinfo = null;
+
         float used = sinfo.getUsedCapacity() / 100;
         float set = sinfo.getCapacity() / 100;
         float delta = Math.abs(set - used) + 0.001f;
@@ -144,13 +189,16 @@ class CapacitySchedulerPage extends RmView {
           "#cs ul { list-style: none }",
           "#cs a { font-weight: normal; margin: 2px; position: relative }",
           "#cs a span { font-weight: normal; font-size: 80% }",
-          "#cs-wrapper .ui-widget-header { padding: 0.2em 0.5em }")._().
+          "#cs-wrapper .ui-widget-header { padding: 0.2em 0.5em }",
+          "table.info tr th {width: 50%}")._(). // to center info table
       script("/static/jt/jquery.jstree.js").
       script().$type("text/javascript").
         _("$(function() {",
           "  $('#cs a span').addClass('ui-corner-all').css('position', 'absolute');",
           "  $('#cs').bind('loaded.jstree', function (e, data) {",
-          "    data.inst.open_all(); }).",
+          "    data.inst.open_all();",
+          "    data.inst.close_node('#lq', true);",
+          "   }).",
           "    jstree({",
           "    core: { animation: 188, html_titles: true },",
           "    plugins: ['themeroller', 'html_data', 'ui'],",
@@ -160,8 +208,9 @@ class CapacitySchedulerPage extends RmView {
           "  });",
           "  $('#cs').bind('select_node.jstree', function(e, data) {",
           "    var q = $('.q', data.rslt.obj).first().text();",
-            "    if (q == 'root') q = '';",
-          "    $('#apps').dataTable().fnFilter(q, 3);",
+          "    if (q == 'root') q = '';",
+          "    else q = '^' + q.substr(q.lastIndexOf('.') + 1) + '$';",
+          "    $('#apps').dataTable().fnFilter(q, 3, true);",
           "  });",
           "  $('#cs').show();",
           "});")._();

+ 6 - 17
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java

@@ -26,9 +26,8 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlTransient;
 import javax.xml.bind.annotation.XmlType;
 
-import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 
 @XmlRootElement(name = "capacityScheduler")
 @XmlType(name = "capacityScheduler")
@@ -83,21 +82,11 @@ public class CapacitySchedulerInfo extends SchedulerInfo {
     CSQueue parentQueue = parent;
     ArrayList<CapacitySchedulerQueueInfo> queuesInfo = new ArrayList<CapacitySchedulerQueueInfo>();
     for (CSQueue queue : parentQueue.getChildQueues()) {
-      float usedCapacity = queue.getUsedCapacity() * 100;
-      float capacity = queue.getCapacity() * 100;
-      String queueName = queue.getQueueName();
-      String queuePath = queue.getQueuePath();
-      float max = queue.getMaximumCapacity();
-      if (max < EPSILON || max > 1f)
-        max = 1f;
-      float maxCapacity = max * 100;
-      QueueState state = queue.getState();
-      CapacitySchedulerQueueInfo info = new CapacitySchedulerQueueInfo(
-          capacity, usedCapacity, maxCapacity, queueName, state, queuePath);
-
-      if (queue instanceof ParentQueue) {
-        info.isParent = true;
-        info.queue = queue;
+      CapacitySchedulerQueueInfo info;
+      if (queue instanceof LeafQueue) {
+        info = new CapacitySchedulerLeafQueueInfo((LeafQueue)queue);
+      } else {
+        info = new CapacitySchedulerQueueInfo(queue);
         info.subQueues = getQueues(queue);
       }
       queuesInfo.add(info);

+ 91 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java

@@ -0,0 +1,91 @@
+/**
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
+
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
+
+  protected int numActiveApplications;
+  protected int numPendingApplications;
+  protected int numContainers;
+  protected int maxApplications;
+  protected int maxApplicationsPerUser;
+  protected int maxActiveApplications;
+  protected int maxActiveApplicationsPerUser;
+  protected int userLimit;
+  protected float userLimitFactor;
+
+  CapacitySchedulerLeafQueueInfo() {
+  };
+
+  CapacitySchedulerLeafQueueInfo(LeafQueue q) {
+    super(q);
+    numActiveApplications = q.getNumActiveApplications();
+    numPendingApplications = q.getNumPendingApplications();
+    numContainers = q.getNumContainers();
+    maxApplications = q.getMaxApplications();
+    maxApplicationsPerUser = q.getMaxApplicationsPerUser();
+    maxActiveApplications = q.getMaximumActiveApplications();
+    maxActiveApplicationsPerUser = q.getMaximumActiveApplicationsPerUser();
+    userLimit = q.getUserLimit();
+    userLimitFactor = q.getUserLimitFactor();
+  }
+
+  public int getNumActiveApplications() {
+    return numActiveApplications;
+  }
+
+  public int getNumPendingApplications() {
+    return numPendingApplications;
+  }
+
+  public int getNumContainers() {
+    return numContainers;
+  }
+
+  public int getMaxApplications() {
+    return maxApplications;
+  }
+
+  public int getMaxApplicationsPerUser() {
+    return maxApplicationsPerUser;
+  }
+
+  public int getMaxActiveApplications() {
+    return maxActiveApplications;
+  }
+
+  public int getMaxActiveApplicationsPerUser() {
+    return maxActiveApplicationsPerUser;
+  }
+
+  public int getUserLimit() {
+    return userLimit;
+  }
+
+  public float getUserLimitFactor() {
+    return userLimitFactor;
+  }
+}

+ 58 - 24
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java

@@ -22,50 +22,54 @@ import java.util.ArrayList;
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlSeeAlso;
 import javax.xml.bind.annotation.XmlTransient;
 
-import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 
 @XmlRootElement
 @XmlAccessorType(XmlAccessType.FIELD)
+@XmlSeeAlso({CapacitySchedulerLeafQueueInfo.class})
 public class CapacitySchedulerQueueInfo {
 
   @XmlTransient
-  protected String queuePath;
-  @XmlTransient
-  protected Boolean isParent = false;
+  static final float EPSILON = 1e-8f;
 
-  // bit odd to store this but makes html easier for now
   @XmlTransient
-  protected CSQueue queue;
+  protected String queuePath;
 
   protected float capacity;
   protected float usedCapacity;
   protected float maxCapacity;
+  protected float absoluteCapacity;
+  protected float absoluteMaxCapacity;
+  protected float utilization;
+  protected int numApplications;
+  protected String usedResources;
   protected String queueName;
-  protected QueueState state;
+  protected String state;
   protected ArrayList<CapacitySchedulerQueueInfo> subQueues;
 
   CapacitySchedulerQueueInfo() {
   };
 
-  CapacitySchedulerQueueInfo(float cap, float used, float max, String name,
-      QueueState state, String path) {
-    this.capacity = cap;
-    this.usedCapacity = used;
-    this.maxCapacity = max;
-    this.queueName = name;
-    this.state = state;
-    this.queuePath = path;
-  }
-
-  public Boolean isParent() {
-    return this.isParent;
-  }
-
-  public CSQueue getQueue() {
-    return this.queue;
+  CapacitySchedulerQueueInfo(CSQueue q) {
+    queuePath = q.getQueuePath();
+    capacity = q.getCapacity() * 100;
+    usedCapacity = q.getUsedCapacity() * 100;
+
+    maxCapacity = q.getMaximumCapacity();
+    if (maxCapacity < EPSILON || maxCapacity > 1f)
+      maxCapacity = 1f;
+    maxCapacity *= 100;
+
+    absoluteCapacity = cap(q.getAbsoluteCapacity(), 0f, 1f) * 100;
+    absoluteMaxCapacity = cap(q.getAbsoluteMaximumCapacity(), 0f, 1f) * 100;
+    utilization = q.getUtilization() * 100;
+    numApplications = q.getNumApplications();
+    usedResources = q.getUsedResources().toString();
+    queueName = q.getQueueName();
+    state = q.getState().toString();
   }
 
   public float getCapacity() {
@@ -80,12 +84,32 @@ public class CapacitySchedulerQueueInfo {
     return this.maxCapacity;
   }
 
+  public float getAbsoluteCapacity() {
+    return absoluteCapacity;
+  }
+
+  public float getAbsoluteMaxCapacity() {
+    return absoluteMaxCapacity;
+  }
+
+  public float getUtilization() {
+    return utilization;
+  }
+
+  public int getNumApplications() {
+    return numApplications;
+  }
+
+  public String getUsedResources() {
+    return usedResources;
+  }
+
   public String getQueueName() {
     return this.queueName;
   }
 
   public String getQueueState() {
-    return this.state.toString();
+    return this.state;
   }
 
   public String getQueuePath() {
@@ -96,4 +120,14 @@ public class CapacitySchedulerQueueInfo {
     return this.subQueues;
   }
 
+  /**
+   * Limit a value to a specified range.
+   * @param val the value to be capped
+   * @param low the lower bound of the range (inclusive)
+   * @param hi the upper bound of the range (inclusive)
+   * @return the capped value
+   */
+  static float cap(float val, float low, float hi) {
+    return Math.min(Math.max(val, low), hi);
+  }
 }

+ 51 - 15
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java

@@ -210,17 +210,21 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
         Element qElem = (Element) queues.item(j);
         String qName = WebServicesTestUtils.getXmlString(qElem, "queueName");
         String q = CapacitySchedulerConfiguration.ROOT + "." + qName;
-        verifySubQueueXML(qElem, q);
+        verifySubQueueXML(qElem, q, 100);
       }
     }
   }
 
-  public void verifySubQueueXML(Element qElem, String q) throws Exception {
-
+  public void verifySubQueueXML(Element qElem, String q, float parentAbsCapacity)
+      throws Exception {
+    float absCapacity = WebServicesTestUtils.getXmlFloat(qElem, "absoluteCapacity");
     verifySubQueueGeneric(q,
         WebServicesTestUtils.getXmlFloat(qElem, "usedCapacity"),
         WebServicesTestUtils.getXmlFloat(qElem, "capacity"),
         WebServicesTestUtils.getXmlFloat(qElem, "maxCapacity"),
+        absCapacity,
+        WebServicesTestUtils.getXmlFloat(qElem, "absoluteMaxCapacity"),
+        parentAbsCapacity,
         WebServicesTestUtils.getXmlString(qElem, "queueName"),
         WebServicesTestUtils.getXmlString(qElem, "state"));
 
@@ -230,8 +234,12 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
         Element subqElem = (Element) queues.item(j);
         String qName = WebServicesTestUtils.getXmlString(subqElem, "queueName");
         String q2 = q + "." + qName;
-        verifySubQueueXML(subqElem, q2);
+        verifySubQueueXML(subqElem, q2, absCapacity);
       }
+    } else {
+      verifyLeafQueueGeneric(q,
+          WebServicesTestUtils.getXmlInt(qElem, "userLimit"),
+          WebServicesTestUtils.getXmlFloat(qElem, "userLimitFactor"));
     }
   }
 
@@ -254,7 +262,7 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
     for (int i = 0; i < arr.length(); i++) {
       JSONObject obj = arr.getJSONObject(i);
       String q = CapacitySchedulerConfiguration.ROOT + "." + obj.getString("queueName");
-      verifySubQueue(obj, q);
+      verifySubQueue(obj, q, 100);
     }
   }
 
@@ -268,31 +276,46 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
     assertTrue("queueName doesn't match", "root".matches(queueName));
   }
 
-  private void verifySubQueue(JSONObject info, String q) throws JSONException,
-      Exception {
-    if (info.has("subQueues")) {
-      assertEquals("incorrect number of elements", 6, info.length());
-    } else {
-      assertEquals("incorrect number of elements", 5, info.length());
+  private void verifySubQueue(JSONObject info, String q, float parentAbsCapacity)
+      throws JSONException, Exception {
+    int numExpectedElements = 11;
+    boolean isParentQueue = true;
+    if (!info.has("subQueues")) {
+      numExpectedElements = 20;
+      isParentQueue = false;
     }
+    assertEquals("incorrect number of elements", numExpectedElements, info.length());
+
+    float absCapacity = (float) info.getDouble("absoluteCapacity");
+
     verifySubQueueGeneric(q, (float) info.getDouble("usedCapacity"),
         (float) info.getDouble("capacity"),
-        (float) info.getDouble("maxCapacity"), info.getString("queueName"),
+        (float) info.getDouble("maxCapacity"),
+        absCapacity,
+        (float) info.getDouble("absoluteMaxCapacity"),
+        parentAbsCapacity,
+        info.getString("queueName"),
         info.getString("state"));
 
-    if (info.has("subQueues")) {
+    if (isParentQueue) {
       JSONArray arr = info.getJSONArray("subQueues");
       // test subqueues
       for (int i = 0; i < arr.length(); i++) {
         JSONObject obj = arr.getJSONObject(i);
         String q2 = q + "." + obj.getString("queueName");
-        verifySubQueue(obj, q2);
+        verifySubQueue(obj, q2, absCapacity);
       }
+    } else {
+      verifyLeafQueueGeneric(q, info.getInt("userLimit"),
+          (float) info.getDouble("userLimitFactor"));
     }
   }
 
   private void verifySubQueueGeneric(String q, float usedCapacity,
-      float capacity, float maxCapacity, String qname, String state)
+      float capacity, float maxCapacity,
+      float absCapacity, float absMaxCapacity,
+      float parentAbsCapacity,
+      String qname, String state)
       throws Exception {
     String[] qArr = q.split("\\.");
     assertTrue("q name invalid: " + q, qArr.length > 1);
@@ -302,15 +325,28 @@ public class TestRMWebServicesCapacitySched extends JerseyTest {
     assertEquals("capacity doesn't match", csConf.getCapacity(q), capacity,
         1e-3f);
     float expectCapacity = csConf.getMaximumCapacity(q);
+    float expectAbsMaxCapacity = parentAbsCapacity * (maxCapacity/100);
     if (CapacitySchedulerConfiguration.UNDEFINED == expectCapacity) {
       expectCapacity = 100;
+      expectAbsMaxCapacity = 100;
     }
     assertEquals("maxCapacity doesn't match", expectCapacity, maxCapacity,
         1e-3f);
+    assertEquals("absoluteCapacity doesn't match",
+        parentAbsCapacity * (capacity/100), absCapacity, 1e-3f);
+    assertEquals("absoluteMaxCapacity doesn't match",
+        expectAbsMaxCapacity, absMaxCapacity, 1e-3f);
     assertTrue("queueName doesn't match, got: " + qname + " expected: " + q,
         qshortName.matches(qname));
     assertTrue("state doesn't match",
         (csConf.getState(q).toString()).matches(state));
 
   }
+
+  private void verifyLeafQueueGeneric(String q, int userLimit,
+      float userLimitFactor) throws Exception {
+    assertEquals("userLimit doesn't match", csConf.getUserLimit(q), userLimit);
+    assertEquals("userLimitFactor doesn't match",
+        csConf.getUserLimitFactor(q), userLimitFactor, 1e-3f);
+  }
 }