Browse Source

YARN-11354. [Federation] Add Yarn Router's NodeLabel Web Page. (#5073)

slfan1989 2 years ago
parent
commit
5d6ab15860
22 changed files with 544 additions and 29 deletions
  1. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
  2. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWSConsts.java
  3. 2 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServiceProtocol.java
  4. 28 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
  5. 21 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeLabelInfo.java
  6. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceInfo.java
  7. 31 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java
  8. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.java
  9. 30 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java
  10. 5 27
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/NavBlock.java
  11. 143 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/NodeLabelsBlock.java
  12. 49 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/NodeLabelsPage.java
  13. 83 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterBlock.java
  14. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterController.java
  15. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebApp.java
  16. 38 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServiceUtil.java
  17. 11 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java
  18. 34 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java
  19. 16 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java
  20. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockRESTRequestInterceptor.java
  21. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/PassThroughRESTRequestInterceptor.java
  22. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationWebApp.java

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java

@@ -467,6 +467,10 @@ public abstract class Resource implements Comparable<Resource> {
     return getFormattedString(String.valueOf(getMemorySize()));
   }
 
+  public String toFormattedString() {
+    return getFormattedString();
+  }
+
   private String getFormattedString(String memory) {
     StringBuilder sb = new StringBuilder();
 

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWSConsts.java

@@ -126,6 +126,9 @@ public final class RMWSConsts {
   /** Path for {@code RMWebServiceProtocol#getClusterNodeLabels}. */
   public static final String GET_NODE_LABELS = "/get-node-labels";
 
+  /** Path for {@code RMWebServiceProtocol#getRMNodeLabels}. */
+  public static final String GET_RM_NODE_LABELS = "/get-rm-node-labels";
+
   /** Path for {@code RMWebServiceProtocol#addToClusterNodeLabels}. */
   public static final String ADD_NODE_LABELS = "/add-node-labels";
 

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServiceProtocol.java

@@ -325,6 +325,8 @@ public interface RMWebServiceProtocol {
    */
   NodeToLabelsInfo getNodeToLabels(HttpServletRequest hsr) throws IOException;
 
+  NodeLabelsInfo getRMNodeLabels(HttpServletRequest hsr) throws IOException;
+
   /**
    * This method retrieves all the node within multiple node labels in the
    * cluster, and it is reachable by using {@link RMWSConsts#LABEL_MAPPINGS}.

+ 28 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java

@@ -129,6 +129,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.AdminService;
@@ -138,6 +139,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
@@ -1404,6 +1406,32 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
     return new NodeLabelsInfo(nodeLabelsInfo);
   }
 
+  @GET
+  @Path(RMWSConsts.GET_RM_NODE_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  public NodeLabelsInfo getRMNodeLabels(@Context HttpServletRequest hsr)
+      throws IOException {
+
+    initForReadableEndpoints();
+    RMNodeLabelsManager nlm = rm.getRMContext().getNodeLabelManager();
+
+    ArrayList<NodeLabelInfo> nodeLabelsInfo = new ArrayList<>();
+    for (RMNodeLabel info : nlm.pullRMNodeLabelsInfo()) {
+      String labelName = info.getLabelName().isEmpty() ?
+          NodeLabel.DEFAULT_NODE_LABEL_PARTITION : info.getLabelName();
+      int activeNMs = info.getNumActiveNMs();
+      PartitionInfo partitionInfo =
+          new PartitionInfo(new ResourceInfo(info.getResource()));
+      NodeLabel nodeLabel = NodeLabel.newInstance(labelName, info.getIsExclusive());
+      NodeLabelInfo nodeLabelInfo = new NodeLabelInfo(nodeLabel, partitionInfo);
+      nodeLabelInfo.setActiveNMs(activeNMs);
+      nodeLabelsInfo.add(nodeLabelInfo);
+    }
+
+    return new NodeLabelsInfo(nodeLabelsInfo);
+  }
+
   @POST
   @Path(RMWSConsts.ADD_NODE_LABELS)
   @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,

+ 21 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodeLabelInfo.java

@@ -31,6 +31,7 @@ public class NodeLabelInfo {
   private String name;
   private boolean exclusivity;
   private PartitionInfo partitionInfo;
+  private Integer activeNMs;
 
   public NodeLabelInfo() {
     // JAXB needs this
@@ -68,6 +69,26 @@ public class NodeLabelInfo {
     return partitionInfo;
   }
 
+  public Integer getActiveNMs() {
+    return activeNMs;
+  }
+
+  public void setActiveNMs(Integer activeNMs) {
+    this.activeNMs = activeNMs;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public void setExclusivity(boolean exclusivity) {
+    this.exclusivity = exclusivity;
+  }
+
+  public void setPartitionInfo(PartitionInfo partitionInfo) {
+    this.partitionInfo = partitionInfo;
+  }
+
   @Override
   public boolean equals(Object obj) {
     if (this == obj) {

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourceInfo.java

@@ -72,6 +72,10 @@ public class ResourceInfo {
     return getResource().toString();
   }
 
+  public String toFormattedString() {
+    return getResource().toFormattedString();
+  }
+
   public void setMemory(int memory) {
     if (resources == null) {
       resources = Resource.newInstance(memory, vCores);

+ 31 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java

@@ -121,6 +121,8 @@ public final class RouterMetrics {
   private MutableGaugeInt numGetAppTimeoutFailedRetrieved;
   @Metric("# of getAppTimeouts failed to be retrieved")
   private MutableGaugeInt numGetAppTimeoutsFailedRetrieved;
+  @Metric("# of getRMNodeLabels failed to be retrieved")
+  private MutableGaugeInt numGetRMNodeLabelsFailedRetrieved;
   @Metric("# of checkUserAccessToQueue failed to be retrieved")
   private MutableGaugeInt numCheckUserAccessToQueueFailedRetrieved;
 
@@ -205,6 +207,8 @@ public final class RouterMetrics {
   private MutableRate totalSucceededGetAppTimeoutRetrieved;
   @Metric("Total number of successful Retrieved GetAppTimeouts and latency(ms)")
   private MutableRate totalSucceededGetAppTimeoutsRetrieved;
+  @Metric("Total number of successful Retrieved GetRMNodeLabels and latency(ms)")
+  private MutableRate totalSucceededGetRMNodeLabelsRetrieved;
   @Metric("Total number of successful Retrieved CheckUserAccessToQueue and latency(ms)")
   private MutableRate totalSucceededCheckUserAccessToQueueRetrieved;
 
@@ -251,6 +255,7 @@ public final class RouterMetrics {
   private MutableQuantiles getUpdateQueueLatency;
   private MutableQuantiles getAppTimeoutLatency;
   private MutableQuantiles getAppTimeoutsLatency;
+  private MutableQuantiles getRMNodeLabelsLatency;
   private MutableQuantiles checkUserAccessToQueueLatency;
 
   private static volatile RouterMetrics instance = null;
@@ -405,6 +410,9 @@ public final class RouterMetrics {
     getAppTimeoutsLatency = registry.newQuantiles("getAppTimeoutsLatency",
          "latency of get apptimeouts timeouts", "ops", "latency", 10);
 
+    getRMNodeLabelsLatency = registry.newQuantiles("getRMNodeLabelsLatency",
+        "latency of get rmnodelabels timeouts", "ops", "latency", 10);
+
     checkUserAccessToQueueLatency = registry.newQuantiles("checkUserAccessToQueueLatency",
         "latency of get apptimeouts timeouts", "ops", "latency", 10);
   }
@@ -628,6 +636,11 @@ public final class RouterMetrics {
     return totalSucceededGetAppTimeoutsRetrieved.lastStat().numSamples();
   }
 
+  @VisibleForTesting
+  public long getNumSucceededGetRMNodeLabelsRetrieved() {
+    return totalSucceededGetRMNodeLabelsRetrieved.lastStat().numSamples();
+  }
+
   @VisibleForTesting
   public long getNumSucceededCheckUserAccessToQueueRetrievedRetrieved() {
     return totalSucceededCheckUserAccessToQueueRetrieved.lastStat().numSamples();
@@ -833,6 +846,11 @@ public final class RouterMetrics {
     return totalSucceededGetAppTimeoutsRetrieved.lastStat().mean();
   }
 
+  @VisibleForTesting
+  public double getLatencySucceededGetRMNodeLabelsRetrieved() {
+    return totalSucceededGetRMNodeLabelsRetrieved.lastStat().mean();
+  }
+
   @VisibleForTesting
   public double getLatencySucceededCheckUserAccessToQueueRetrieved() {
     return totalSucceededCheckUserAccessToQueueRetrieved.lastStat().mean();
@@ -1019,6 +1037,10 @@ public final class RouterMetrics {
     return numGetAppTimeoutsFailedRetrieved.value();
   }
 
+  public int getRMNodeLabelsFailedRetrieved() {
+    return numGetRMNodeLabelsFailedRetrieved.value();
+  }
+
   public int getCheckUserAccessToQueueFailedRetrieved() {
     return numCheckUserAccessToQueueFailedRetrieved.value();
   }
@@ -1223,6 +1245,11 @@ public final class RouterMetrics {
     getAppTimeoutsLatency.add(duration);
   }
 
+  public void succeededGetRMNodeLabelsRetrieved(long duration) {
+    totalSucceededGetRMNodeLabelsRetrieved.add(duration);
+    getRMNodeLabelsLatency.add(duration);
+  }
+
   public void succeededCheckUserAccessToQueueRetrieved(long duration) {
     totalSucceededCheckUserAccessToQueueRetrieved.add(duration);
     checkUserAccessToQueueLatency.add(duration);
@@ -1388,6 +1415,10 @@ public final class RouterMetrics {
     numGetAppTimeoutsFailedRetrieved.incr();
   }
 
+  public void incrGetRMNodeLabelsFailedRetrieved() {
+    numGetRMNodeLabelsFailedRetrieved.incr();
+  }
+
   public void incrCheckUserAccessToQueueFailedRetrieved() {
     numCheckUserAccessToQueueFailedRetrieved.incr();
   }

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.java

@@ -602,4 +602,12 @@ public class DefaultRequestInterceptorREST
                 + containerId + "/" + RMWSConsts.SIGNAL + "/" + command, null,
             null, getConf(), client);
   }
+
+  @Override
+  public NodeLabelsInfo getRMNodeLabels(HttpServletRequest hsr) {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        NodeLabelsInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.GET_RM_NODE_LABELS,
+        null, null, getConf(), client);
+  }
 }

+ 30 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java

@@ -1273,10 +1273,39 @@ public class FederationInterceptorREST extends AbstractRESTRequestInterceptor {
       routerMetrics.incrNodeToLabelsFailedRetrieved();
       RouterServerUtil.logAndThrowIOException("getNodeToLabels error.", e);
     }
-    routerMetrics.incrGetAppStatisticsFailedRetrieved();
+    routerMetrics.incrNodeToLabelsFailedRetrieved();
     throw new RuntimeException("getNodeToLabels Failed.");
   }
 
+  @Override
+  public NodeLabelsInfo getRMNodeLabels(HttpServletRequest hsr) throws IOException {
+    try {
+      long startTime = clock.getTime();
+      Map<SubClusterId, SubClusterInfo> subClustersActive = getActiveSubclusters();
+      final HttpServletRequest hsrCopy = clone(hsr);
+      Class[] argsClasses = new Class[]{HttpServletRequest.class};
+      Object[] args = new Object[]{hsrCopy};
+      ClientMethod remoteMethod = new ClientMethod("getRMNodeLabels", argsClasses, args);
+      Map<SubClusterInfo, NodeLabelsInfo> nodeToLabelsInfoMap =
+          invokeConcurrent(subClustersActive.values(), remoteMethod, NodeLabelsInfo.class);
+      NodeLabelsInfo nodeToLabelsInfo =
+          RouterWebServiceUtil.mergeNodeLabelsInfo(nodeToLabelsInfoMap);
+      if (nodeToLabelsInfo != null) {
+        long stopTime = clock.getTime();
+        routerMetrics.succeededGetRMNodeLabelsRetrieved(stopTime - startTime);
+        return nodeToLabelsInfo;
+      }
+    } catch (NotFoundException e) {
+      routerMetrics.incrGetRMNodeLabelsFailedRetrieved();
+      RouterServerUtil.logAndThrowIOException("get all active sub cluster(s) error.", e);
+    } catch (YarnException e) {
+      routerMetrics.incrGetRMNodeLabelsFailedRetrieved();
+      RouterServerUtil.logAndThrowIOException("getRMNodeLabels error.", e);
+    }
+    routerMetrics.incrGetRMNodeLabelsFailedRetrieved();
+    throw new RuntimeException("getRMNodeLabels Failed.");
+  }
+
   @Override
   public LabelsToNodesInfo getLabelsToNodes(Set<String> labels)
       throws IOException {

+ 5 - 27
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/NavBlock.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.yarn.server.router.webapp;
 
 import com.google.inject.Inject;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.server.router.Router;
 import org.apache.hadoop.yarn.server.webapp.WebPageUtils;
 import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
@@ -49,35 +48,14 @@ public class NavBlock extends RouterBlock {
 
     List<String> subClusterIds = getActiveSubClusterIds();
 
-    Hamlet.UL<Hamlet.LI<Hamlet.UL<Hamlet.DIV<Hamlet>>>> subAppsList1 =
-        mainList.li().a(url("nodes"), "Nodes").ul().$style("padding:0.3em 1em 0.1em 2em");
-
     // ### nodes info
-    subAppsList1.li().__();
-    for (String subClusterId : subClusterIds) {
-      subAppsList1.li().a(url("nodes", subClusterId), subClusterId).__();
-    }
-    subAppsList1.__().__();
+    initNodesMenu(mainList, subClusterIds);
 
-    // ### applications info
-    Hamlet.UL<Hamlet.LI<Hamlet.UL<Hamlet.DIV<Hamlet>>>> subAppsList2 =
-            mainList.li().a(url("apps"), "Applications").ul();
+    // ### nodelabels info
+    initNodeLabelsMenu(mainList, subClusterIds);
 
-    subAppsList2.li().__();
-    for (String subClusterId : subClusterIds) {
-      Hamlet.LI<Hamlet.UL<Hamlet.LI<Hamlet.UL<Hamlet.DIV<Hamlet>>>>> subAppsList3 = subAppsList2.
-          li().a(url("apps", subClusterId), subClusterId);
-      Hamlet.UL<Hamlet.LI<Hamlet.UL<Hamlet.LI<Hamlet.UL<Hamlet.DIV<Hamlet>>>>>> subAppsList4 =
-          subAppsList3.ul().$style("padding:0.3em 1em 0.1em 2em");
-      subAppsList4.li().__();
-      for (YarnApplicationState state : YarnApplicationState.values()) {
-        subAppsList4.
-            li().a(url("apps", subClusterId, state.toString()), state.toString()).__();
-      }
-      subAppsList4.li().__().__();
-      subAppsList3.__();
-    }
-    subAppsList2.__().__();
+    // ### applications info
+    initApplicationsMenu(mainList, subClusterIds);
 
     // ### tools
     Hamlet.DIV<Hamlet> sectionBefore = mainList.__();

+ 143 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/NodeLabelsBlock.java

@@ -0,0 +1,143 @@
+/**
+ * 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.router.webapp;
+
+import com.google.inject.Inject;
+import com.sun.jersey.api.client.Client;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.PartitionInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo;
+import org.apache.hadoop.yarn.server.router.Router;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.NODE_SC;
+
+/**
+ * Navigation block for the Router Web UI.
+ */
+public class NodeLabelsBlock extends RouterBlock {
+
+  private Router router;
+
+  @Inject
+  public NodeLabelsBlock(Router router, ViewContext ctx) {
+    super(router, ctx);
+    this.router = router;
+  }
+
+  @Override
+  protected void render(Block html) {
+    boolean isEnabled = isYarnFederationEnabled();
+
+    // Get subClusterName
+    String subClusterName = $(NODE_SC);
+
+    NodeLabelsInfo nodeLabelsInfo = null;
+    if (StringUtils.isNotEmpty(subClusterName)) {
+      nodeLabelsInfo = getSubClusterNodeLabelsInfo(subClusterName);
+    } else {
+      nodeLabelsInfo = getYarnFederationNodeLabelsInfo(isEnabled);
+    }
+
+    initYarnFederationNodeLabelsOfCluster(nodeLabelsInfo, html);
+  }
+
+  /**
+   * Get NodeLabels Info based on SubCluster.
+   * @return NodeLabelsInfo.
+   */
+  private NodeLabelsInfo getSubClusterNodeLabelsInfo(String subCluster) {
+    try {
+      SubClusterId subClusterId = SubClusterId.newInstance(subCluster);
+      FederationStateStoreFacade facade = FederationStateStoreFacade.getInstance();
+      SubClusterInfo subClusterInfo = facade.getSubCluster(subClusterId);
+
+      if (subClusterInfo != null) {
+        // Prepare webAddress
+        String webAddress = subClusterInfo.getRMWebServiceAddress();
+        String herfWebAppAddress = "";
+        if (webAddress != null && !webAddress.isEmpty()) {
+          herfWebAppAddress =
+              WebAppUtils.getHttpSchemePrefix(this.router.getConfig()) + webAddress;
+          return getSubClusterNodeLabelsByWebAddress(herfWebAppAddress);
+        }
+      }
+    } catch (Exception e) {
+      LOG.error("get NodeLabelsInfo From SubCluster = {} error.", subCluster, e);
+    }
+    return null;
+  }
+
+  private NodeLabelsInfo getYarnFederationNodeLabelsInfo(boolean isEnabled) {
+    if (isEnabled) {
+      String webAddress = WebAppUtils.getRouterWebAppURLWithScheme(this.router.getConfig());
+      return getSubClusterNodeLabelsByWebAddress(webAddress);
+    }
+    return null;
+  }
+
+  private NodeLabelsInfo getSubClusterNodeLabelsByWebAddress(String webAddress) {
+    Configuration conf = this.router.getConfig();
+    Client client = RouterWebServiceUtil.createJerseyClient(conf);
+    NodeLabelsInfo nodes = RouterWebServiceUtil
+        .genericForward(webAddress, null, NodeLabelsInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.GET_RM_NODE_LABELS, null, null, conf,
+        client);
+    client.destroy();
+    return nodes;
+  }
+
+  private void initYarnFederationNodeLabelsOfCluster(NodeLabelsInfo nodeLabelsInfo, Block html) {
+
+    Hamlet.TBODY<Hamlet.TABLE<Hamlet>> tbody = html.table("#nodelabels").
+        thead().
+        tr().
+        th(".name", "Label Name").
+        th(".type", "Label Type").
+        th(".numOfActiveNMs", "Num Of Active NMs").
+        th(".totalResource", "Total Resource").
+        __().__().
+        tbody();
+
+    if (nodeLabelsInfo != null) {
+      for (NodeLabelInfo info : nodeLabelsInfo.getNodeLabelsInfo()) {
+        Hamlet.TR<Hamlet.TBODY<Hamlet.TABLE<Hamlet>>> row =
+            tbody.tr().td(info.getName().isEmpty() ?
+            NodeLabel.DEFAULT_NODE_LABEL_PARTITION : info.getName());
+        String type = (info.getExclusivity()) ? "Exclusive Partition" : "Non Exclusive Partition";
+        row = row.td(type);
+        int nActiveNMs = info.getActiveNMs();
+        row = row.td(String.valueOf(nActiveNMs));
+        PartitionInfo partitionInfo = info.getPartitionInfo();
+        ResourceInfo available = partitionInfo.getResourceAvailable();
+        row.td(available.toFormattedString()).__();
+      }
+    }
+
+    tbody.__().__();
+  }
+}

+ 49 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/NodeLabelsPage.java

@@ -0,0 +1,49 @@
+/**
+ * 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.router.webapp;
+
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
+
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.NODE_SC;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
+
+/**
+ * Renders a block for the nodelabels with metrics information.
+ */
+public class NodeLabelsPage extends RouterView {
+
+  @Override
+  protected void preHead(Hamlet.HTML<__> html) {
+    commonPreHead(html);
+    String type = $(NODE_SC);
+    String title = "Node labels of the cluster";
+    if (type != null && !type.isEmpty()) {
+      title = title + " (" + type + ")";
+    }
+    setTitle(title);
+    set(DATATABLES_ID, "nodelabels");
+    setTableStyles(html, "nodelabels", ".healthStatus {width:10em}", ".healthReport {width:10em}");
+  }
+
+  @Override
+  protected Class<? extends SubView> content() {
+    return NodeLabelsBlock.class;
+  }
+}

+ 83 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterBlock.java

@@ -18,7 +18,9 @@
 package org.apache.hadoop.yarn.server.router.webapp;
 
 import com.sun.jersey.api.client.Client;
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
@@ -27,6 +29,7 @@ import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
 import org.apache.hadoop.yarn.server.router.Router;
+import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
@@ -174,4 +177,84 @@ public abstract class RouterBlock extends HtmlBlock {
   public FederationStateStoreFacade getFacade() {
     return facade;
   }
+
+  /**
+   * Initialize the Nodes menu.
+   *
+   * @param mainList HTML Object.
+   * @param subClusterIds subCluster List.
+   */
+  protected void initNodesMenu(Hamlet.UL<Hamlet.DIV<Hamlet>> mainList,
+      List<String> subClusterIds) {
+    if (CollectionUtils.isNotEmpty(subClusterIds)) {
+      Hamlet.UL<Hamlet.LI<Hamlet.UL<Hamlet.DIV<Hamlet>>>> nodesList =
+          mainList.li().a(url("nodes"), "Nodes").ul().
+          $style("padding:0.3em 1em 0.1em 2em");
+
+      // ### nodes info
+      nodesList.li().__();
+      for (String subClusterId : subClusterIds) {
+        nodesList.li().a(url("nodes", subClusterId), subClusterId).__();
+      }
+      nodesList.__().__();
+    } else {
+      mainList.li().a(url("nodes"), "Nodes").__();
+    }
+  }
+
+  /**
+   * Initialize the Applications menu.
+   *
+   * @param mainList HTML Object.
+   * @param subClusterIds subCluster List.
+   */
+  protected void initApplicationsMenu(Hamlet.UL<Hamlet.DIV<Hamlet>> mainList,
+      List<String> subClusterIds) {
+    if (CollectionUtils.isNotEmpty(subClusterIds)) {
+      Hamlet.UL<Hamlet.LI<Hamlet.UL<Hamlet.DIV<Hamlet>>>> apps =
+          mainList.li().a(url("apps"), "Applications").ul();
+      apps.li().__();
+      for (String subClusterId : subClusterIds) {
+        Hamlet.LI<Hamlet.UL<Hamlet.LI<Hamlet.UL<Hamlet.DIV<Hamlet>>>>> subClusterList = apps.
+            li().a(url("apps", subClusterId), subClusterId);
+        Hamlet.UL<Hamlet.LI<Hamlet.UL<Hamlet.LI<Hamlet.UL<Hamlet.DIV<Hamlet>>>>>> subAppStates =
+            subClusterList.ul().$style("padding:0.3em 1em 0.1em 2em");
+        subAppStates.li().__();
+        for (YarnApplicationState state : YarnApplicationState.values()) {
+          subAppStates.
+              li().a(url("apps", subClusterId, state.toString()), state.toString()).__();
+        }
+        subAppStates.li().__().__();
+        subClusterList.__();
+      }
+      apps.__().__();
+    } else {
+      mainList.li().a(url("apps"), "Applications").__();
+    }
+  }
+
+  /**
+   * Initialize the NodeLabels menu.
+   *
+   * @param mainList HTML Object.
+   * @param subClusterIds subCluster List.
+   */
+  protected void initNodeLabelsMenu(Hamlet.UL<Hamlet.DIV<Hamlet>> mainList,
+      List<String> subClusterIds) {
+
+    if (CollectionUtils.isNotEmpty(subClusterIds)) {
+      Hamlet.UL<Hamlet.LI<Hamlet.UL<Hamlet.DIV<Hamlet>>>> nodesList =
+          mainList.li().a(url("nodelabels"), "Node Labels").ul().
+          $style("padding:0.3em 1em 0.1em 2em");
+
+      // ### nodelabels info
+      nodesList.li().__();
+      for (String subClusterId : subClusterIds) {
+        nodesList.li().a(url("nodelabels", subClusterId), subClusterId).__();
+      }
+      nodesList.__().__();
+    } else {
+      mainList.li().a(url("nodelabels"), "Node Labels").__();
+    }
+  }
 }

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterController.java

@@ -56,4 +56,9 @@ public class RouterController extends Controller {
     setTitle("Nodes");
     render(NodesPage.class);
   }
+
+  public void nodeLabels() {
+    setTitle("Node Labels");
+    render(NodeLabelsPage.class);
+  }
 }

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebApp.java

@@ -52,5 +52,6 @@ public class RouterWebApp extends WebApp implements YarnWebParams {
     route(pajoin("/apps", APP_SC, APP_STATE), RouterController.class, "apps");
     route(pajoin("/nodes", NODE_SC), RouterController.class, "nodes");
     route("/federation", RouterController.class, "federation");
+    route(pajoin("/nodelabels", NODE_SC), RouterController.class, "nodeLabels");
   }
 }

+ 38 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServiceUtil.java

@@ -59,6 +59,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.StatisticsItemInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.PartitionInfo;
 import org.apache.hadoop.yarn.server.uam.UnmanagedApplicationManager;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 import org.apache.hadoop.yarn.webapp.ForbiddenException;
@@ -576,4 +578,40 @@ public final class RouterWebServiceUtil {
 
     return result;
   }
+
+  public static NodeLabelsInfo mergeNodeLabelsInfo(Map<SubClusterInfo, NodeLabelsInfo> paramMap) {
+    Map<String, NodeLabelInfo> resultMap = new HashMap<>();
+    paramMap.values().stream()
+        .flatMap(nodeLabelsInfo -> nodeLabelsInfo.getNodeLabelsInfo().stream())
+        .forEach(nodeLabelInfo -> {
+          String keyLabelName = nodeLabelInfo.getName();
+          if (resultMap.containsKey(keyLabelName)) {
+            NodeLabelInfo mapNodeLabelInfo = resultMap.get(keyLabelName);
+            mapNodeLabelInfo = mergeNodeLabelInfo(mapNodeLabelInfo, nodeLabelInfo);
+            resultMap.put(keyLabelName, mapNodeLabelInfo);
+          } else {
+            resultMap.put(keyLabelName, nodeLabelInfo);
+          }
+        });
+    NodeLabelsInfo nodeLabelsInfo = new NodeLabelsInfo();
+    nodeLabelsInfo.getNodeLabelsInfo().addAll(resultMap.values());
+    return nodeLabelsInfo;
+  }
+
+  private static NodeLabelInfo mergeNodeLabelInfo(NodeLabelInfo left, NodeLabelInfo right) {
+    NodeLabelInfo resultNodeLabelInfo = new NodeLabelInfo();
+    resultNodeLabelInfo.setName(left.getName());
+
+    int newActiveNMs = left.getActiveNMs() + right.getActiveNMs();
+    resultNodeLabelInfo.setActiveNMs(newActiveNMs);
+
+    boolean newExclusivity = left.getExclusivity() && right.getExclusivity();
+    resultNodeLabelInfo.setExclusivity(newExclusivity);
+
+    PartitionInfo leftPartition = left.getPartitionInfo();
+    PartitionInfo rightPartition = right.getPartitionInfo();
+    PartitionInfo newPartitionInfo = PartitionInfo.addTo(leftPartition, rightPartition);
+    resultNodeLabelInfo.setPartitionInfo(newPartitionInfo);
+    return resultNodeLabelInfo;
+  }
 }

+ 11 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java

@@ -943,4 +943,15 @@ public class RouterWebServices implements RMWebServiceProtocol {
     return pipeline.getRootInterceptor()
         .signalToContainer(containerId, command, req);
   }
+
+  @GET
+  @Path(RMWSConsts.GET_RM_NODE_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  public NodeLabelsInfo getRMNodeLabels(@Context HttpServletRequest hsr)
+      throws IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain(hsr);
+    return pipeline.getRootInterceptor().getRMNodeLabels(hsr);
+  }
 }

+ 34 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java

@@ -514,6 +514,11 @@ public class TestRouterMetrics {
       metrics.incrGetAppTimeoutsFailedRetrieved();
     }
 
+    public void getRMNodeLabelsFailed() {
+      LOG.info("Mocked: failed getRMNodeLabelsFailed call");
+      metrics.incrGetRMNodeLabelsFailedRetrieved();
+    }
+
     public void getCheckUserAccessToQueueRetrieved() {
       LOG.info("Mocked: failed checkUserAccessToQueueRetrieved call");
       metrics.incrCheckUserAccessToQueueFailedRetrieved();
@@ -729,6 +734,11 @@ public class TestRouterMetrics {
       metrics.succeededGetAppTimeoutsRetrieved(duration);
     }
 
+    public void getRMNodeLabelsRetrieved(long duration) {
+      LOG.info("Mocked: successful getRMNodeLabels call with duration {}", duration);
+      metrics.succeededGetRMNodeLabelsRetrieved(duration);
+    }
+
     public void getCheckUserAccessToQueueRetrieved(long duration) {
       LOG.info("Mocked: successful CheckUserAccessToQueue call with duration {}", duration);
       metrics.succeededCheckUserAccessToQueueRetrieved(duration);
@@ -1476,6 +1486,29 @@ public class TestRouterMetrics {
         metrics.getAppTimeoutsFailedRetrieved());
   }
 
+  @Test
+  public void testGetRMNodeLabelsRetrieved() {
+    long totalGoodBefore = metrics.getNumSucceededGetRMNodeLabelsRetrieved();
+    goodSubCluster.getRMNodeLabelsRetrieved(150);
+    Assert.assertEquals(totalGoodBefore + 1,
+        metrics.getNumSucceededGetRMNodeLabelsRetrieved());
+    Assert.assertEquals(150,
+        metrics.getLatencySucceededGetRMNodeLabelsRetrieved(), ASSERT_DOUBLE_DELTA);
+    goodSubCluster.getRMNodeLabelsRetrieved(300);
+    Assert.assertEquals(totalGoodBefore + 2,
+        metrics.getNumSucceededGetRMNodeLabelsRetrieved());
+    Assert.assertEquals(225,
+        metrics.getLatencySucceededGetRMNodeLabelsRetrieved(), ASSERT_DOUBLE_DELTA);
+  }
+
+  @Test
+  public void testGetRMNodeLabelsRetrievedFailed() {
+    long totalBadBefore = metrics.getRMNodeLabelsFailedRetrieved();
+    badSubCluster.getRMNodeLabelsFailed();
+    Assert.assertEquals(totalBadBefore + 1,
+        metrics.getRMNodeLabelsFailedRetrieved());
+  }
+
   @Test
   public void testCheckUserAccessToQueueRetrievedRetrieved() {
     long totalGoodBefore = metrics.getNumSucceededCheckUserAccessToQueueRetrievedRetrieved();
@@ -1498,4 +1531,4 @@ public class TestRouterMetrics {
     Assert.assertEquals(totalBadBefore + 1,
         metrics.getCheckUserAccessToQueueFailedRetrieved());
   }
-}
+}

+ 16 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java

@@ -118,6 +118,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.StatisticsItemIn
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationListInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.PartitionInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.RMQueueAclInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServices;
 import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
@@ -898,6 +899,21 @@ public class MockDefaultRequestInterceptorREST
     return Response.status(Status.OK).entity(resResponse).build();
   }
 
+  @Override
+  public NodeLabelsInfo getRMNodeLabels(HttpServletRequest hsr) {
+
+    NodeLabelInfo nodeLabelInfo = new NodeLabelInfo();
+    nodeLabelInfo.setExclusivity(true);
+    nodeLabelInfo.setName("Test-Label");
+    nodeLabelInfo.setActiveNMs(10);
+    PartitionInfo partitionInfo = new PartitionInfo();
+
+    NodeLabelsInfo nodeLabelsInfo = new NodeLabelsInfo();
+    nodeLabelsInfo.getNodeLabelsInfo().add(nodeLabelInfo);
+
+    return nodeLabelsInfo;
+  }
+
   private MockRM setupResourceManager() throws Exception {
     DefaultMetricsSystem.setMiniClusterMode(true);
 

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockRESTRequestInterceptor.java

@@ -185,6 +185,11 @@ public class MockRESTRequestInterceptor extends AbstractRESTRequestInterceptor {
     return new NodeToLabelsInfo();
   }
 
+  @Override
+  public NodeLabelsInfo getRMNodeLabels(HttpServletRequest hsr) throws IOException {
+    return new NodeLabelsInfo();
+  }
+
   @Override
   public LabelsToNodesInfo getLabelsToNodes(Set<String> labels)
       throws IOException {

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/PassThroughRESTRequestInterceptor.java

@@ -217,6 +217,11 @@ public class PassThroughRESTRequestInterceptor
     return getNextInterceptor().getNodeToLabels(hsr);
   }
 
+  @Override
+  public NodeLabelsInfo getRMNodeLabels(HttpServletRequest hsr) throws IOException {
+    return getNextInterceptor().getRMNodeLabels(hsr);
+  }
+
   @Override
   public LabelsToNodesInfo getLabelsToNodes(Set<String> labels)
       throws IOException {

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationWebApp.java

@@ -99,4 +99,22 @@ public class TestFederationWebApp extends TestRouterWebServicesREST {
     config.setBoolean(YarnConfiguration.FEDERATION_ENABLED, false);
     WebAppTests.testPage(AppsPage.class, Router.class, new MockRouter(config));
   }
+
+  @Test
+  public void testNodeLabelAppViewNotEnable()
+      throws InterruptedException, YarnException, IOException {
+    // Test Federation Not Enabled
+    Configuration config = new YarnConfiguration();
+    config.setBoolean(YarnConfiguration.FEDERATION_ENABLED, false);
+    WebAppTests.testPage(NodeLabelsPage.class, Router.class, new MockRouter(config));
+  }
+
+  @Test
+  public void testNodeLabelAppViewEnable()
+      throws InterruptedException, YarnException, IOException {
+    // Test Federation Not Enabled
+    Configuration config = new YarnConfiguration();
+    config.setBoolean(YarnConfiguration.FEDERATION_ENABLED, true);
+    WebAppTests.testPage(NodeLabelsPage.class, Router.class, new MockRouter(config));
+  }
 }