|
@@ -22,14 +22,18 @@ import static org.apache.hadoop.yarn.util.StringHelper.join;
|
|
|
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.HashMap;
|
|
|
+import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
+import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth;
|
|
|
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.QueueCapacities;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.UserInfo;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerInfo;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerLeafQueueInfo;
|
|
@@ -63,48 +67,92 @@ class CapacitySchedulerPage extends RmView {
|
|
|
static class CSQInfo {
|
|
|
CapacitySchedulerInfo csinfo;
|
|
|
CapacitySchedulerQueueInfo qinfo;
|
|
|
+ String label;
|
|
|
}
|
|
|
|
|
|
static class LeafQueueInfoBlock extends HtmlBlock {
|
|
|
final CapacitySchedulerLeafQueueInfo lqinfo;
|
|
|
+ private String nodeLabel;
|
|
|
|
|
|
@Inject LeafQueueInfoBlock(ViewContext ctx, CSQInfo info) {
|
|
|
super(ctx);
|
|
|
lqinfo = (CapacitySchedulerLeafQueueInfo) info.qinfo;
|
|
|
+ nodeLabel = info.label;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
protected void render(Block html) {
|
|
|
+ if (nodeLabel == null) {
|
|
|
+ renderLeafQueueInfoWithoutParition(html);
|
|
|
+ } else {
|
|
|
+ renderLeafQueueInfoWithPartition(html);
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- ResponseInfo ri = info("\'" + lqinfo.getQueuePath().substring(5) + "\' Queue Status").
|
|
|
- _("Queue State:", lqinfo.getQueueState()).
|
|
|
- _("Used Capacity:", percent(lqinfo.getUsedCapacity() / 100)).
|
|
|
- _("Absolute Used Capacity:", percent(lqinfo.getAbsoluteUsedCapacity() / 100)).
|
|
|
- _("Absolute Capacity:", percent(lqinfo.getAbsoluteCapacity() / 100)).
|
|
|
- _("Absolute Max Capacity:", percent(lqinfo.getAbsoluteMaxCapacity() / 100)).
|
|
|
- _("Used Resources:", lqinfo.getResourcesUsed().toString()).
|
|
|
- _("Num Schedulable Applications:", Integer.toString(lqinfo.getNumActiveApplications())).
|
|
|
- _("Num Non-Schedulable 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 Application Master Resources:", lqinfo.getAMResourceLimit().toString()).
|
|
|
- _("Used Application Master Resources:", lqinfo.getUsedAMResource().toString()).
|
|
|
- _("Max Application Master Resources Per User:", lqinfo.getUserAMResourceLimit().toString()).
|
|
|
- _("Configured Capacity:", percent(lqinfo.getCapacity() / 100)).
|
|
|
- _("Configured Max Capacity:", percent(lqinfo.getMaxCapacity() / 100)).
|
|
|
- _("Configured Minimum User Limit Percent:", Integer.toString(lqinfo.getUserLimit()) + "%").
|
|
|
- _("Configured User Limit Factor:", StringUtils.format(
|
|
|
- "%.1f", lqinfo.getUserLimitFactor())).
|
|
|
- _("Accessible Node Labels:", StringUtils.join(",", lqinfo.getNodeLabels())).
|
|
|
- _("Ordering Policy: ", lqinfo.getOrderingPolicyInfo()).
|
|
|
- _("Preemption:", lqinfo.getPreemptionDisabled() ? "disabled" : "enabled");
|
|
|
+ private void renderLeafQueueInfoWithPartition(Block html) {
|
|
|
+ nodeLabel = nodeLabel.length() == 0 ? "<DEFAULT_PARTITION>" : nodeLabel;
|
|
|
+ // first display the queue's label specific details :
|
|
|
+ ResponseInfo ri =
|
|
|
+ info("\'" + lqinfo.getQueuePath().substring(5)
|
|
|
+ + "\' Queue Status for Partition \'" + nodeLabel + "\'");
|
|
|
+ renderQueueCapacityInfo(ri);
|
|
|
+ html._(InfoBlock.class);
|
|
|
+ // clear the info contents so this queue's info doesn't accumulate into
|
|
|
+ // another queue's info
|
|
|
+ ri.clear();
|
|
|
+
|
|
|
+ // second display the queue specific details :
|
|
|
+ ri =
|
|
|
+ info("\'" + lqinfo.getQueuePath().substring(5) + "\' Queue Status")
|
|
|
+ ._("Queue State:", lqinfo.getQueueState());
|
|
|
+ renderCommonLeafQueueInfo(ri);
|
|
|
|
|
|
html._(InfoBlock.class);
|
|
|
+ // clear the info contents so this queue's info doesn't accumulate into
|
|
|
+ // another queue's info
|
|
|
+ ri.clear();
|
|
|
+ }
|
|
|
|
|
|
- // clear the info contents so this queue's info doesn't accumulate into another queue's info
|
|
|
+ private void renderLeafQueueInfoWithoutParition(Block html) {
|
|
|
+ ResponseInfo ri =
|
|
|
+ info("\'" + lqinfo.getQueuePath().substring(5) + "\' Queue Status")
|
|
|
+ ._("Queue State:", lqinfo.getQueueState());
|
|
|
+ renderQueueCapacityInfo(ri);
|
|
|
+ renderCommonLeafQueueInfo(ri);
|
|
|
+ html._(InfoBlock.class);
|
|
|
+ // clear the info contents so this queue's info doesn't accumulate into
|
|
|
+ // another queue's info
|
|
|
ri.clear();
|
|
|
}
|
|
|
+
|
|
|
+ private void renderQueueCapacityInfo(ResponseInfo ri) {
|
|
|
+ ri.
|
|
|
+ _("Used Capacity:", percent(lqinfo.getUsedCapacity() / 100)).
|
|
|
+ _("Configured Capacity:", percent(lqinfo.getCapacity() / 100)).
|
|
|
+ _("Configured Max Capacity:", percent(lqinfo.getMaxCapacity() / 100)).
|
|
|
+ _("Absolute Used Capacity:", percent(lqinfo.getAbsoluteUsedCapacity() / 100)).
|
|
|
+ _("Absolute Configured Capacity:", percent(lqinfo.getAbsoluteCapacity() / 100)).
|
|
|
+ _("Absolute Configured Max Capacity:", percent(lqinfo.getAbsoluteMaxCapacity() / 100)).
|
|
|
+ _("Used Resources:", lqinfo.getResourcesUsed().toString());
|
|
|
+ }
|
|
|
+
|
|
|
+ private void renderCommonLeafQueueInfo(ResponseInfo ri) {
|
|
|
+ ri.
|
|
|
+ _("Num Schedulable Applications:", Integer.toString(lqinfo.getNumActiveApplications())).
|
|
|
+ _("Num Non-Schedulable 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 Application Master Resources:", lqinfo.getAMResourceLimit().toString()).
|
|
|
+ _("Used Application Master Resources:", lqinfo.getUsedAMResource().toString()).
|
|
|
+ _("Max Application Master Resources Per User:", lqinfo.getUserAMResourceLimit().toString()).
|
|
|
+ _("Configured Minimum User Limit Percent:", Integer.toString(lqinfo.getUserLimit()) + "%").
|
|
|
+ _("Configured User Limit Factor:", StringUtils.format(
|
|
|
+ "%.1f", lqinfo.getUserLimitFactor())).
|
|
|
+ _("Accessible Node Labels:", StringUtils.join(",", lqinfo.getNodeLabels())).
|
|
|
+ _("Ordering Policy: ", lqinfo.getOrderingPolicyInfo()).
|
|
|
+ _("Preemption:", lqinfo.getPreemptionDisabled() ? "disabled" : "enabled");
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
static class QueueUsersInfoBlock extends HtmlBlock {
|
|
@@ -172,7 +220,7 @@ class CapacitySchedulerPage extends RmView {
|
|
|
span().$style(join(width(absUsedCap/absMaxCap),
|
|
|
";font-size:1px;left:0%;", absUsedCap > absCap ? Q_OVER : Q_UNDER)).
|
|
|
_('.')._().
|
|
|
- span(".q", info.getQueuePath().substring(5))._().
|
|
|
+ span(".q", "Queue: "+info.getQueuePath().substring(5))._().
|
|
|
span().$class("qstats").$style(left(Q_STATS_POS)).
|
|
|
_(join(percent(used), " used"))._();
|
|
|
|
|
@@ -194,11 +242,15 @@ class CapacitySchedulerPage extends RmView {
|
|
|
final CapacityScheduler cs;
|
|
|
final CSQInfo csqinfo;
|
|
|
private final ResourceManager rm;
|
|
|
+ private List<RMNodeLabel> nodeLabelsInfo;
|
|
|
|
|
|
@Inject QueuesBlock(ResourceManager rm, CSQInfo info) {
|
|
|
cs = (CapacityScheduler) rm.getResourceScheduler();
|
|
|
csqinfo = info;
|
|
|
this.rm = rm;
|
|
|
+ RMNodeLabelsManager nodeLabelManager =
|
|
|
+ rm.getRMContext().getNodeLabelManager();
|
|
|
+ nodeLabelsInfo = nodeLabelManager.pullRMNodeLabelsInfo();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -268,12 +320,6 @@ class CapacitySchedulerPage extends RmView {
|
|
|
span().$style(Q_END)._("100% ")._().
|
|
|
span(".q", "default")._()._();
|
|
|
} else {
|
|
|
- CSQueue root = cs.getRootQueue();
|
|
|
- CapacitySchedulerInfo sinfo = new CapacitySchedulerInfo(root, cs);
|
|
|
- csqinfo.csinfo = sinfo;
|
|
|
- csqinfo.qinfo = null;
|
|
|
-
|
|
|
- float used = sinfo.getUsedCapacity() / 100;
|
|
|
ul.
|
|
|
li().$style("margin-bottom: 1em").
|
|
|
span().$style("font-weight: bold")._("Legend:")._().
|
|
@@ -285,8 +331,22 @@ class CapacitySchedulerPage extends RmView {
|
|
|
_("Used (over capacity)")._().
|
|
|
span().$class("qlegend ui-corner-all ui-state-default").
|
|
|
_("Max Capacity")._().
|
|
|
- _().
|
|
|
- li().
|
|
|
+ _();
|
|
|
+
|
|
|
+ float used = 0;
|
|
|
+ if (null == nodeLabelsInfo
|
|
|
+ || (nodeLabelsInfo.size() == 1 && nodeLabelsInfo.get(0)
|
|
|
+ .getLabelName().isEmpty())) {
|
|
|
+ CSQueue root = cs.getRootQueue();
|
|
|
+ CapacitySchedulerInfo sinfo =
|
|
|
+ new CapacitySchedulerInfo(root, cs, new RMNodeLabel(
|
|
|
+ RMNodeLabelsManager.NO_LABEL));
|
|
|
+ csqinfo.csinfo = sinfo;
|
|
|
+ csqinfo.qinfo = null;
|
|
|
+
|
|
|
+ used = sinfo.getUsedCapacity() / 100;
|
|
|
+ //label is not enabled in the cluster or there's only "default" label,
|
|
|
+ ul.li().
|
|
|
a(_Q).$style(width(Q_MAX_WIDTH)).
|
|
|
span().$style(join(width(used), ";left:0%;",
|
|
|
used > 1 ? Q_OVER : Q_UNDER))._(".")._().
|
|
@@ -294,6 +354,41 @@ class CapacitySchedulerPage extends RmView {
|
|
|
span().$class("qstats").$style(left(Q_STATS_POS)).
|
|
|
_(join(percent(used), " used"))._().
|
|
|
_(QueueBlock.class)._();
|
|
|
+ } else {
|
|
|
+ for (RMNodeLabel label : nodeLabelsInfo) {
|
|
|
+ CSQueue root = cs.getRootQueue();
|
|
|
+ CapacitySchedulerInfo sinfo =
|
|
|
+ new CapacitySchedulerInfo(root, cs, label);
|
|
|
+ csqinfo.csinfo = sinfo;
|
|
|
+ csqinfo.qinfo = null;
|
|
|
+ csqinfo.label = label.getLabelName();
|
|
|
+ String nodeLabel =
|
|
|
+ csqinfo.label.length() == 0 ? "<DEFAULT_PARTITION>"
|
|
|
+ : csqinfo.label;
|
|
|
+ QueueCapacities queueCapacities = root.getQueueCapacities();
|
|
|
+ used = queueCapacities.getUsedCapacity(label.getLabelName());
|
|
|
+ String partitionUiTag =
|
|
|
+ "Partition: " + nodeLabel + " " + label.getResource();
|
|
|
+ ul.li().
|
|
|
+ a(_Q).$style(width(Q_MAX_WIDTH)).
|
|
|
+ span().$style(join(width(used), ";left:0%;",
|
|
|
+ used > 1 ? Q_OVER : Q_UNDER))._(".")._().
|
|
|
+ span(".q", partitionUiTag)._().
|
|
|
+ span().$class("qstats").$style(left(Q_STATS_POS)).
|
|
|
+ _(join(percent(used), " used"))._();
|
|
|
+
|
|
|
+ //for the queue hierarchy under label
|
|
|
+ UL<Hamlet> underLabel = html.ul("#pq");
|
|
|
+ underLabel.li().
|
|
|
+ a(_Q).$style(width(Q_MAX_WIDTH)).
|
|
|
+ span().$style(join(width(used), ";left:0%;",
|
|
|
+ used > 1 ? Q_OVER : Q_UNDER))._(".")._().
|
|
|
+ span(".q", "Queue: root")._().
|
|
|
+ span().$class("qstats").$style(left(Q_STATS_POS)).
|
|
|
+ _(join(percent(used), " used"))._().
|
|
|
+ _(QueueBlock.class)._()._();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
ul._()._().
|
|
|
script().$type("text/javascript").
|