瀏覽代碼

YARN-11310. [Federation] Refactoring Yarn Router's Federation Web Page. (#4924)

slfan1989 2 年之前
父節點
當前提交
42d883937d

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java

@@ -31,6 +31,8 @@ import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
 public abstract class HtmlBlock extends TextView implements SubView {
 
   protected static final String UNAVAILABLE = "N/A";
+  protected static final long BYTES_IN_MB = 1024 * 1024;
+  protected static final String DATE_PATTERN = "yyyy-MM-dd HH:mm:ss";
 
   public class Block extends Hamlet {
     Block(PrintWriter out, int level, boolean wasInline) {

+ 77 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/federation/federation.js

@@ -0,0 +1,77 @@
+/**
+ * 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.
+ */
+
+$(document).ready(function() {
+    var table = $('#rms').DataTable();
+    $('#rms tbody').on('click', 'td.details-control', function () {
+        var tr = $(this).closest('tr');
+        var row = table.row(tr);
+        if (row.child.isShown()) {
+            row.child.hide();
+            tr.removeClass('shown');
+        } else {
+            var capabilityArr = scTableData.filter(item => (item.subcluster === row.id()));
+            var capabilityObj = JSON.parse(capabilityArr[0].capability).clusterMetrics;
+            row.child(
+                '<table>' +
+                '   <tr>' +
+                '      <td>' +
+                '         <h3>Application Metrics</h3>' +
+                '         ApplicationSubmitted* : ' + capabilityObj.appsSubmitted + ' </p>' +
+                '         ApplicationCompleted* : ' + capabilityObj.appsCompleted + ' </p>' +
+                '         ApplicationPending*   : ' + capabilityObj.appsPending + ' </p>' +
+                '         ApplicationRunning*   : ' + capabilityObj.appsRunning + ' </p>' +
+                '         ApplicationFailed*    : ' + capabilityObj.appsFailed + ' </p>' +
+                '         ApplicationKilled*    : ' + capabilityObj.appsKilled + ' </p>' +
+                '      </td>' +
+                '      <td>' +
+                '        <h3>Resource Metrics</h3>' +
+                '        <h4>Memory</h4>' +
+                '        TotalMB : ' + capabilityObj.totalMB + ' </p>' +
+                '        ReservedMB : ' + capabilityObj.reservedMB + ' </p>' +
+                '        AvailableMB : ' + capabilityObj.availableMB + ' </p>' +
+                '        AllocatedMB : ' + capabilityObj.allocatedMB + ' </p>' +
+                '        PendingMB : ' + capabilityObj.pendingMB + ' </p>' +
+                '        <h4>VirtualCores</h4>' +
+                '        TotalVirtualCores : ' + capabilityObj.totalVirtualCores + ' </p>' +
+                '        ReservedVirtualCores : ' + capabilityObj.reservedVirtualCores + ' </p>' +
+                '        AvailableVirtualCore : ' + capabilityObj.availableVirtualCores + ' </p>' +
+                '        AllocatedVirtualCores : '+ capabilityObj.allocatedVirtualCores + ' </p>' +
+                '        PendingVirtualCores : ' + capabilityObj.pendingVirtualCores + ' </p>' +
+                '        <h4>Containers</h4>' +
+                '        ContainersAllocated : ' + capabilityObj.containersAllocated + ' </p>' +
+                '        ContainersReserved : ' + capabilityObj.containersReserved + ' </p>' +
+                '        ContainersPending : ' + capabilityObj.containersPending + ' </p>' +
+                '     </td>' +
+                '     <td>' +
+                '        <h3>Node Metrics</h3>' +
+                '         TotalNodes : ' + capabilityObj.totalNodes + ' </p>' +
+                '         LostNodes : ' + capabilityObj.lostNodes + ' </p>' +
+                '         UnhealthyNodes : ' + capabilityObj.unhealthyNodes + ' </p>' +
+                '         DecommissioningNodes : ' + capabilityObj.decommissioningNodes + ' </p>' +
+                '         DecommissionedNodes : ' + capabilityObj.decommissionedNodes + ' </p>' +
+                '         RebootedNodes : ' + capabilityObj.rebootedNodes + ' </p>' +
+                '         ActiveNodes : ' + capabilityObj.activeNodes + ' </p>' +
+                '         ShutdownNodes : ' + capabilityObj.shutdownNodes + ' </p>' +
+                '     </td>' +
+                '  </tr>' +
+                '</table>').show();
+            tr.addClass('shown');
+        }
+    });
+});

+ 156 - 106
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationBlock.java

@@ -24,11 +24,13 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
+import java.util.HashMap;
 
+import com.google.gson.Gson;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.commons.lang3.time.DateFormatUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
 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;
@@ -37,6 +39,7 @@ import org.apache.hadoop.yarn.server.router.Router;
 import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TABLE;
 import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.inject.Inject;
@@ -46,8 +49,6 @@ import com.sun.jersey.api.json.JSONUnmarshaller;
 
 class FederationBlock extends HtmlBlock {
 
-  private static final long BYTES_IN_MB = 1024 * 1024;
-
   private final Router router;
 
   @Inject
@@ -58,119 +59,168 @@ class FederationBlock extends HtmlBlock {
 
   @Override
   public void render(Block html) {
+
     Configuration conf = this.router.getConfig();
     boolean isEnabled = conf.getBoolean(
         YarnConfiguration.FEDERATION_ENABLED,
         YarnConfiguration.DEFAULT_FEDERATION_ENABLED);
-    if (isEnabled) {
-      setTitle("Federation");
-
-      // Table header
-      TBODY<TABLE<Hamlet>> tbody = html.table("#rms").thead().tr()
-          .th(".id", "SubCluster")
-          .th(".submittedA", "Applications Submitted*")
-          .th(".pendingA", "Applications Pending*")
-          .th(".runningA", "Applications Running*")
-          .th(".failedA", "Applications Failed*")
-          .th(".killedA", "Applications Killed*")
-          .th(".completedA", "Applications Completed*")
-          .th(".contAllocated", "Containers Allocated")
-          .th(".contReserved", "Containers Reserved")
-          .th(".contPending", "Containers Pending")
-          .th(".availableM", "Available Memory")
-          .th(".allocatedM", "Allocated Memory")
-          .th(".reservedM", "Reserved Memory")
-          .th(".totalM", "Total Memory")
-          .th(".availableVC", "Available VirtualCores")
-          .th(".allocatedVC", "Allocated VirtualCores")
-          .th(".reservedVC", "Reserved VirtualCores")
-          .th(".totalVC", "Total VirtualCores")
-          .th(".activeN", "Active Nodes")
-          .th(".lostN", "Lost Nodes")
-          .th(".availableN", "Available Nodes")
-          .th(".unhealtyN", "Unhealthy Nodes")
-          .th(".rebootedN", "Rebooted Nodes")
-          .th(".totalN", "Total Nodes")
-          .__().__().tbody();
-
-      try {
-        // Binding to the FederationStateStore
-        FederationStateStoreFacade facade =
-            FederationStateStoreFacade.getInstance();
-        Map<SubClusterId, SubClusterInfo> subClustersInfo =
-            facade.getSubClusters(true);
-
-        // Sort the SubClusters
-        List<SubClusterInfo> subclusters = new ArrayList<>();
-        subclusters.addAll(subClustersInfo.values());
-        Comparator<? super SubClusterInfo> cmp =
-            new Comparator<SubClusterInfo>() {
-              @Override
-              public int compare(SubClusterInfo o1, SubClusterInfo o2) {
-                return o1.getSubClusterId().compareTo(o2.getSubClusterId());
-              }
-            };
-        Collections.sort(subclusters, cmp);
-
-        for (SubClusterInfo subcluster : subclusters) {
-          SubClusterId subClusterId = subcluster.getSubClusterId();
-          String webAppAddress = subcluster.getRMWebServiceAddress();
-          String capability = subcluster.getCapability();
-          ClusterMetricsInfo subClusterInfo = getClusterMetricsInfo(capability);
-
-          // Building row per SubCluster
-          tbody.tr().td().a("//" + webAppAddress, subClusterId.toString()).__()
-              .td(Integer.toString(subClusterInfo.getAppsSubmitted()))
-              .td(Integer.toString(subClusterInfo.getAppsPending()))
-              .td(Integer.toString(subClusterInfo.getAppsRunning()))
-              .td(Integer.toString(subClusterInfo.getAppsFailed()))
-              .td(Integer.toString(subClusterInfo.getAppsKilled()))
-              .td(Integer.toString(subClusterInfo.getAppsCompleted()))
-              .td(Integer.toString(subClusterInfo.getContainersAllocated()))
-              .td(Integer.toString(subClusterInfo.getReservedContainers()))
-              .td(Integer.toString(subClusterInfo.getPendingContainers()))
-              .td(StringUtils.byteDesc(
-                  subClusterInfo.getAvailableMB() * BYTES_IN_MB))
-              .td(StringUtils.byteDesc(
-                  subClusterInfo.getAllocatedMB() * BYTES_IN_MB))
-              .td(StringUtils.byteDesc(
-                  subClusterInfo.getReservedMB() * BYTES_IN_MB))
-              .td(StringUtils.byteDesc(
-                  subClusterInfo.getTotalMB() * BYTES_IN_MB))
-              .td(Long.toString(subClusterInfo.getAvailableVirtualCores()))
-              .td(Long.toString(subClusterInfo.getAllocatedVirtualCores()))
-              .td(Long.toString(subClusterInfo.getReservedVirtualCores()))
-              .td(Long.toString(subClusterInfo.getTotalVirtualCores()))
-              .td(Integer.toString(subClusterInfo.getActiveNodes()))
-              .td(Integer.toString(subClusterInfo.getLostNodes()))
-              .td(Integer.toString(subClusterInfo.getDecommissionedNodes()))
-              .td(Integer.toString(subClusterInfo.getUnhealthyNodes()))
-              .td(Integer.toString(subClusterInfo.getRebootedNodes()))
-              .td(Integer.toString(subClusterInfo.getTotalNodes())).__();
-        }
-      } catch (YarnException e) {
-        LOG.error("Cannot render ResourceManager", e);
-      }
 
-      tbody.__().__().div()
-          .p().__("*The application counts are local per subcluster").__().__();
-    } else {
-      setTitle("Federation is not Enabled!");
-    }
+    // init Html Page Federation
+    initHtmlPageFederation(html, isEnabled);
   }
 
-  private static ClusterMetricsInfo getClusterMetricsInfo(String capability) {
-    ClusterMetricsInfo clusterMetrics = null;
+  /**
+   * Parse the capability and obtain the metric information of the cluster.
+   *
+   * @param capability metric json obtained from RM.
+   * @return ClusterMetricsInfo Object
+   */
+  private ClusterMetricsInfo getClusterMetricsInfo(String capability) {
     try {
-      JSONJAXBContext jc = new JSONJAXBContext(
-          JSONConfiguration.mapped().rootUnwrapping(false).build(),
-          ClusterMetricsInfo.class);
-      JSONUnmarshaller unmarshaller = jc.createJSONUnmarshaller();
-      clusterMetrics = unmarshaller.unmarshalFromJSON(
-          new StringReader(capability), ClusterMetricsInfo.class);
+      if (capability != null && !capability.isEmpty()) {
+        JSONJAXBContext jc = new JSONJAXBContext(
+            JSONConfiguration.mapped().rootUnwrapping(false).build(), ClusterMetricsInfo.class);
+        JSONUnmarshaller unmarShaller = jc.createJSONUnmarshaller();
+        StringReader stringReader = new StringReader(capability);
+        ClusterMetricsInfo clusterMetrics =
+            unmarShaller.unmarshalFromJSON(stringReader, ClusterMetricsInfo.class);
+        return clusterMetrics;
+      }
     } catch (Exception e) {
       LOG.error("Cannot parse SubCluster info", e);
     }
-    return clusterMetrics;
+    return null;
+  }
+
+  /**
+   * Initialize the subCluster details JavaScript of the Federation page.
+   *
+   * This part of the js script will control to display or hide the detailed information
+   * of the subCluster when the user clicks on the subClusterId.
+   *
+   * We will obtain the specific information of a SubCluster,
+   * including the information of Applications, Resources, and Nodes.
+   *
+   * @param html html object
+   * @param subClusterDetailMap subCluster Detail Map
+   */
+  private void initFederationSubClusterDetailTableJs(Block html,
+      List<Map<String, String>> subClusterDetailMap) {
+    Gson gson = new Gson();
+    html.script().$type("text/javascript").
+        __(" var scTableData = " + gson.toJson(subClusterDetailMap) + "; ")
+        .__();
+    html.script(root_url("static/federation/federation.js"));
+  }
+
+  /**
+   * Initialize the Html page.
+   *
+   * @param html html object
+   */
+  private void initHtmlPageFederation(Block html, boolean isEnabled) {
+    List<Map<String, String>> lists = new ArrayList<>();
+
+    // If Yarn Federation is not enabled, the user needs to be prompted.
+    if (!isEnabled) {
+      html.style(".alert {padding: 15px; margin-bottom: 20px; " +
+          " border: 1px solid transparent; border-radius: 4px;}");
+      html.style(".alert-dismissable {padding-right: 35px;}");
+      html.style(".alert-info {color: #856404;background-color: #fff3cd;border-color: #ffeeba;}");
+
+      Hamlet.DIV<Hamlet> div = html.div("#div_id").$class("alert alert-dismissable alert-info");
+      div.p().$style("color:red").__("Federation is not Enabled.").__()
+          .p().__()
+          .p().__("We can refer to the following documents to configure Yarn Federation. ").__()
+          .p().__()
+          .a("https://hadoop.apache.org/docs/stable/hadoop-yarn/hadoop-yarn-site/Federation.html",
+          "Hadoop: YARN Federation").
+          __();
+    }
+
+    // Table header
+    TBODY<TABLE<Hamlet>> tbody =
+        html.table("#rms").$class("cell-border").$style("width:100%").thead().tr()
+        .th(".id", "SubCluster")
+        .th(".state", "State")
+        .th(".lastStartTime", "LastStartTime")
+        .th(".lastHeartBeat", "LastHeartBeat")
+        .th(".resources", "Resources")
+        .th(".nodes", "Nodes")
+        .__().__().tbody();
+
+    try {
+      // Binding to the FederationStateStore
+      FederationStateStoreFacade facade = FederationStateStoreFacade.getInstance();
+
+      Map<SubClusterId, SubClusterInfo> subClustersInfo = facade.getSubClusters(true);
+
+      // Sort the SubClusters
+      List<SubClusterInfo> subclusters = new ArrayList<>();
+      subclusters.addAll(subClustersInfo.values());
+      Comparator<? super SubClusterInfo> cmp = Comparator.comparing(o -> o.getSubClusterId());
+      Collections.sort(subclusters, cmp);
+
+      for (SubClusterInfo subcluster : subclusters) {
+
+        Map<String, String> subclusterMap = new HashMap<>();
+
+        // Prepare subCluster
+        SubClusterId subClusterId = subcluster.getSubClusterId();
+        String subClusterIdText = subClusterId.getId();
+
+        // Prepare WebAppAddress
+        String webAppAddress = subcluster.getRMWebServiceAddress();
+        String herfWebAppAddress = "";
+        if (webAppAddress != null && !webAppAddress.isEmpty()) {
+          herfWebAppAddress =
+              WebAppUtils.getHttpSchemePrefix(this.router.getConfig()) + webAppAddress;
+        }
+
+        // Prepare Capability
+        String capability = subcluster.getCapability();
+        ClusterMetricsInfo subClusterInfo = getClusterMetricsInfo(capability);
+
+        // Prepare LastStartTime & LastHeartBeat
+        String lastStartTime =
+            DateFormatUtils.format(subcluster.getLastStartTime(), DATE_PATTERN);
+        String lastHeartBeat =
+            DateFormatUtils.format(subcluster.getLastHeartBeat(), DATE_PATTERN);
+
+        // Prepare Resource
+        long totalMB = subClusterInfo.getTotalMB();
+        String totalMBDesc = StringUtils.byteDesc(totalMB * BYTES_IN_MB);
+        long totalVirtualCores = subClusterInfo.getTotalVirtualCores();
+        String resources = String.format("<Memory:%s, VCore:%s>", totalMBDesc, totalVirtualCores);
+
+        // Prepare Node
+        long totalNodes = subClusterInfo.getTotalNodes();
+        long activeNodes = subClusterInfo.getActiveNodes();
+        String nodes = String.format("<Total Nodes:%s, Active Nodes:%s>", totalNodes, activeNodes);
+
+        // Prepare HTML Table
+        tbody.tr().$id(subClusterIdText)
+            .td().$class("details-control").a(herfWebAppAddress, subClusterIdText).__()
+            .td(subcluster.getState().name())
+            .td(lastStartTime)
+            .td(lastHeartBeat)
+            .td(resources)
+            .td(nodes)
+            .__();
+
+        subclusterMap.put("subcluster", subClusterId.getId());
+        subclusterMap.put("capability", capability);
+        lists.add(subclusterMap);
+      }
+    } catch (Exception e) {
+      LOG.error("Cannot render Router Federation.", e);
+    }
+
+    // Init FederationBlockTableJs
+    initFederationSubClusterDetailTableJs(html, lists);
+
+    // Tips
+    tbody.__().__().div().p().$style("color:red")
+        .__("*The application counts are local per subcluster").__().__();
   }
 }

+ 10 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationPage.java

@@ -33,8 +33,9 @@ class FederationPage extends RouterView {
   @Override
   protected void preHead(Page.HTML<__> html) {
     commonPreHead(html);
-    setTitle("Federation");
+    setTitle("About The YARN Federation");
     set(DATATABLES_ID, "rms");
+    set("ui.div.id", "div_id");
     set(initID(DATATABLES, "rms"), rmsTableInit());
     setTableStyles(html, "rms", ".healthStatus {width:10em}",
         ".healthReport {width:10em}");
@@ -46,12 +47,14 @@ class FederationPage extends RouterView {
   }
 
   private String rmsTableInit() {
-    StringBuilder b = tableInit().append(", aoColumnDefs: [");
-    b.append("{'bSearchable': false, 'aTargets': [ 7 ]}")
-        .append(", {'sType': 'title-numeric', 'bSearchable': false, "
-            + "'aTargets': [ 8, 9 ] }")
-        .append(", {'sType': 'title-numeric', 'aTargets': [ 5 ]}")
+    StringBuilder builder = tableInit().append(", aoColumnDefs: [");
+    builder
+        .append("{'sName':'State', 'sType':'string', 'bSearchable':false, 'aTargets':[1]},")
+        .append("{'sName':'LastStartTime', 'sType':'string', 'bSearchable':false, 'aTargets':[2]},")
+        .append("{'sName':'lastHeartBeat', 'sType':'string', 'bSearchable':false, 'aTargets':[3]},")
+        .append("{'sName':'resource', 'sType':'string', 'bSearchable':false, 'aTargets':[4]},")
+        .append("{'sName':'nodes', 'sType':'string', 'bSearchable':false, 'aTargets':[5]}")
         .append("]}");
-    return b.toString();
+    return builder.toString();
   }
 }

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

@@ -34,7 +34,7 @@ public class RouterController extends Controller {
 
   @Override
   public void index() {
-    setTitle("Router");
+    setTitle("About the YARN Router");
     render(AboutPage.class);
   }
 
@@ -44,7 +44,6 @@ public class RouterController extends Controller {
   }
 
   public void federation() {
-    setTitle("Federation");
     render(FederationPage.class);
   }
 

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

@@ -0,0 +1,99 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+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.store.records.SubClusterRegisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.apache.hadoop.yarn.server.router.Router;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class MockRouter extends Router {
+
+  private FederationStateStoreFacade facade;
+
+  public MockRouter(Configuration configuration)
+      throws InterruptedException, YarnException, IOException {
+    this.setConfig(configuration);
+
+    boolean isEnabled = configuration.getBoolean(
+        YarnConfiguration.FEDERATION_ENABLED,
+        YarnConfiguration.DEFAULT_FEDERATION_ENABLED);
+
+    if (isEnabled) {
+      facade = FederationStateStoreFacade.getInstance();
+      initTestFederationSubCluster();
+    }
+  }
+
+  public void initTestFederationSubCluster()
+      throws IOException, InterruptedException, YarnException {
+
+    // Initialize subcluster information
+    String scAmRMAddress = "5.6.7.8:5";
+    String scClientRMAddress = "5.6.7.8:6";
+    String scRmAdminAddress = "5.6.7.8:7";
+    String scWebAppAddress = "127.0.0.1:8080";
+
+    // Initialize subcluster capability
+    String[] capabilityPathItems = new String[] {".", "target", "test-classes", "capability"};
+    String capabilityPath = StringUtils.join(capabilityPathItems, File.separator);
+    String capabilityJson =
+        FileUtils.readFileToString(new File(capabilityPath), StandardCharsets.UTF_8);
+
+    // capability json needs to remove asflicense
+    String regex = "\"___asflicense__.*\\n(.*,\\n){1,15}.*\\n.*";
+    Pattern p = Pattern.compile(regex);
+    Matcher m = p.matcher(capabilityJson);
+    capabilityJson = m.replaceAll("").trim();
+
+    // Initialize subcluster sc1
+    SubClusterInfo sc1 =
+        SubClusterInfo.newInstance(SubClusterId.newInstance("SC-1"),
+        scAmRMAddress, scClientRMAddress, scRmAdminAddress, scWebAppAddress,
+        SubClusterState.SC_RUNNING, Time.now(), capabilityJson);
+    Thread.sleep(5000);
+    sc1.setLastHeartBeat(Time.now());
+
+    // Initialize subcluster sc2
+    SubClusterInfo sc2 =
+        SubClusterInfo.newInstance(SubClusterId.newInstance("SC-2"),
+        scAmRMAddress, scClientRMAddress, scRmAdminAddress, scWebAppAddress,
+        SubClusterState.SC_RUNNING, Time.now(), capabilityJson);
+    Thread.sleep(5000);
+    sc2.setLastHeartBeat(Time.now());
+
+    FederationStateStore stateStore = facade.getStateStore();
+    stateStore.registerSubCluster(SubClusterRegisterRequest.newInstance(sc1));
+    stateStore.registerSubCluster(SubClusterRegisterRequest.newInstance(sc2));
+  }
+}

+ 48 - 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

@@ -0,0 +1,48 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.router.Router;
+import org.apache.hadoop.yarn.webapp.test.WebAppTests;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TestFederationWebApp {
+
+  @Test
+  public void testFederationWebViewNotEnable()
+      throws InterruptedException, YarnException, IOException {
+    // Test Federation is not Enabled
+    Configuration config = new YarnConfiguration();
+    config.setBoolean(YarnConfiguration.FEDERATION_ENABLED, false);
+    WebAppTests.testPage(FederationPage.class, Router.class, new MockRouter(config));
+  }
+
+  @Test
+  public void testFederationWebViewEnable()
+      throws InterruptedException, YarnException, IOException {
+    // Test Federation Enabled
+    Configuration config = new YarnConfiguration();
+    config.setBoolean(YarnConfiguration.FEDERATION_ENABLED, true);
+    WebAppTests.testPage(FederationPage.class, Router.class, new MockRouter(config));
+  }
+}

文件差異過大導致無法顯示
+ 19 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/capability


部分文件因文件數量過多而無法顯示