Browse Source

YARN-11619. [Federation] Router CLI Supports List SubClusters. (#6304) Contributed by Shilun Fan.

Reviewed-by: Inigo Goiri <inigoiri@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 1 year ago
parent
commit
bd93b2007c
23 changed files with 857 additions and 45 deletions
  1. 15 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
  2. 66 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/FederationSubCluster.java
  3. 34 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetSubClustersRequest.java
  4. 46 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetSubClustersResponse.java
  5. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto
  6. 7 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
  7. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  8. 101 37
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java
  9. 36 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java
  10. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java
  11. 20 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java
  12. 127 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/FederationSubClusterPBImpl.java
  13. 72 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetSubClustersRequestPBImpl.java
  14. 131 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetSubClustersResponsePBImpl.java
  15. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
  16. 10 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
  17. 32 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java
  18. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java
  19. 50 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/FederationRMAdminInterceptor.java
  20. 9 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java
  21. 33 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java
  22. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java
  23. 19 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestFederationRMAdminInterceptor.java

+ 15 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java

@@ -66,6 +66,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePol
 import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesResponse;
 
@@ -236,6 +238,19 @@ public interface ResourceManagerAdministrationProtocol extends GetUserMappingsPr
   DeleteFederationApplicationResponse deleteFederationApplication(
       DeleteFederationApplicationRequest request) throws YarnException, IOException;
 
+  /**
+   * In YARN-Federation mode, this method provides a way to get federation subcluster list.
+   *
+   * @param request GetSubClustersRequest Request.
+   * @return Response from getFederationSubClusters.
+   * @throws YarnException exceptions from yarn servers.
+   * @throws IOException if an IO error occurred.
+   */
+  @Private
+  @Idempotent
+  GetSubClustersResponse getFederationSubClusters(GetSubClustersRequest request)
+      throws YarnException, IOException;
+
   /**
    * In YARN-Federation mode, this method provides a way to delete queue weight policies.
    *

+ 66 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/FederationSubCluster.java

@@ -0,0 +1,66 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Federation SubClusters.
+ */
+@Private
+@Unstable
+public abstract class FederationSubCluster {
+
+  @Private
+  @Unstable
+  public static FederationSubCluster newInstance(String subClusterId, String state,
+      String lastHeartBeatTime) {
+    FederationSubCluster subCluster = Records.newRecord(FederationSubCluster.class);
+    subCluster.setSubClusterId(subClusterId);
+    subCluster.setSubClusterState(state);
+    subCluster.setLastHeartBeatTime(lastHeartBeatTime);
+    return subCluster;
+  }
+
+  @Public
+  @Unstable
+  public abstract String getSubClusterId();
+
+  @Private
+  @Unstable
+  public abstract void setSubClusterId(String subClusterId);
+
+  @Public
+  @Unstable
+  public abstract String getSubClusterState();
+
+  @Public
+  @Unstable
+  public abstract void setSubClusterState(String state);
+
+  @Public
+  @Unstable
+  public abstract String getLastHeartBeatTime();
+
+  @Public
+  @Unstable
+  public abstract void setLastHeartBeatTime(String lastHeartBeatTime);
+}

+ 34 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetSubClustersRequest.java

@@ -0,0 +1,34 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+@Public
+@Unstable
+public abstract class GetSubClustersRequest {
+  @Public
+  @Stable
+  public static GetSubClustersRequest newInstance() {
+    GetSubClustersRequest request = Records.newRecord(GetSubClustersRequest.class);
+    return request;
+  }
+}

+ 46 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/GetSubClustersResponse.java

@@ -0,0 +1,46 @@
+/**
+ * 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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.util.List;
+
+@Public
+@Unstable
+public abstract class GetSubClustersResponse {
+  @Public
+  @Stable
+  public static GetSubClustersResponse newInstance(List<FederationSubCluster> subClustersList) {
+    GetSubClustersResponse response = Records.newRecord(GetSubClustersResponse.class);
+    response.setFederationSubClusters(subClustersList);
+    return response;
+  }
+
+  @Public
+  @Unstable
+  public abstract List<FederationSubCluster> getFederationSubClusters();
+
+  @Private
+  @Unstable
+  public abstract void setFederationSubClusters(List<FederationSubCluster> federationSubClusters);
+}

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto

@@ -52,5 +52,6 @@ service ResourceManagerAdministrationProtocolService {
   rpc batchSaveFederationQueuePolicies(BatchSaveFederationQueuePoliciesRequestProto) returns (BatchSaveFederationQueuePoliciesResponseProto);
   rpc listFederationQueuePolicies(QueryFederationQueuePoliciesRequestProto) returns (QueryFederationQueuePoliciesResponseProto);
   rpc deleteFederationApplication(DeleteFederationApplicationRequestProto) returns (DeleteFederationApplicationResponseProto);
+  rpc getFederationSubClusters(GetSubClustersRequestProto) returns (GetSubClustersResponseProto);
   rpc deleteFederationPoliciesByQueues(DeleteFederationQueuePoliciesRequestProto) returns (DeleteFederationQueuePoliciesResponseProto);
 }

+ 7 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto

@@ -211,6 +211,13 @@ message DeleteFederationApplicationResponseProto {
   optional string message = 1;
 }
 
+message GetSubClustersRequestProto {
+}
+
+message GetSubClustersResponseProto {
+  repeated FederationSubClusterProto subClusters = 1;
+}
+
 message DeleteFederationQueuePoliciesRequestProto {
   repeated string queues = 1;
 }

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto

@@ -448,6 +448,12 @@ message FederationQueueWeightProto {
   optional string policyManagerClassName = 5;
 }
 
+message FederationSubClusterProto {
+  optional string subClusterId = 1;
+  optional string lastHeartBeatTime = 2;
+  optional string subClusterState = 3;
+}
+
 ////////////////////////////////////////////////////////////////////////
 ////// From AM_RM_Protocol /////////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////

+ 101 - 37
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RouterCLI.java

@@ -54,6 +54,9 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueu
 import org.apache.hadoop.yarn.server.api.protocolrecords.FederationQueueWeight;
 import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.FederationSubCluster;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersResponse;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.w3c.dom.Document;
@@ -103,31 +106,54 @@ public class RouterCLI extends Configured implements Tool {
   // Constant
   private static final String OPTION_SC = "sc";
   private static final String OPTION_SUBCLUSTERID = "subClusterId";
-  private static final String CMD_DEREGISTERSUBCLUSTER = "-deregisterSubCluster";
+  private static final String OPTION_GET_SUBCLUSTERS = "getSubClusters";
+  private static final String OPTION_DEREGISTER_SUBCLUSTER = "deregisterSubCluster";
+  private static final String CMD_SUBCLUSTER = "-subCluster";
+  private static final String CMD_DEREGISTER_SUBCLUSTER = "-deregisterSubCluster";
 
   // DeregisterSubCluster Command Parameters
-  protected final static UsageInfo SUBCLUSTER_ID = new UsageInfo("<-sc|--subClusterId>",
-      "'-sc' option allows you to specify the sub-cluster to operate on, " +
-      "while the '--subClusterId' option is the long format of -sc and serves the same purpose.");
+  protected final static UsageInfo DEREGISTER_SUBCLUSTER_USAGE = new UsageInfo(
+      "-deregisterSubCluster <-sc|--subClusterId>",
+      "This command is used to deregister subCluster, " +
+      "If the interval between the heartbeat time of the subCluster and" +
+      "the current time exceeds the timeout period, set the state of the subCluster to SC_LOST.");
 
   // DeregisterSubCluster Command Examples
   protected final static String DEREGISTER_SUBCLUSTER_EXAMPLE_1 =
-      "yarn routeradmin -deregisterSubCluster -sc SC-1";
+      "yarn routeradmin -subCluster -deregisterSubCluster -sc SC-1";
   protected final static String DEREGISTER_SUBCLUSTER_EXAMPLE_2 =
-      "yarn routeradmin -deregisterSubCluster --subClusterId SC-1";
+      "yarn routeradmin -subCluster -deregisterSubCluster --subClusterId SC-1";
 
   // DeregisterSubCluster Command Help Information
   protected final static String DEREGISTER_SUBCLUSTER_HELP_INFO =
       "deregister subCluster, If the interval between the heartbeat time of the subCluster and" +
       "the current time exceeds the timeout period, set the state of the subCluster to SC_LOST.";
 
-  protected final static RouterCmdUsageInfos DEREGISTER_SUBCLUSTER_USAGEINFOS =
+  protected final static UsageInfo GET_SUBCLUSTER_USAGE = new UsageInfo("-getSubClusters",
+      "This command is used to get information about all subclusters.");
+
+  private static final String GET_SUBCLUSTER_TITLE = "Yarn Federation SubCluster";
+
+  // Columns information
+  private static final List<String> GET_SUBCLUSTER_HEADER = Arrays.asList(
+      "SubCluster Id", "SubCluster State", "Last HeartBeatTime");
+
+  // GetSubCluster Command Examples
+  protected final static String GET_SUBCLUSTER_EXAMPLE =
+      "yarn routeradmin -subCluster -getSubClusters";
+
+  protected final static RouterCmdUsageInfos SUBCLUSTER_USAGEINFOS =
       new RouterCmdUsageInfos()
-      .addUsageInfo(SUBCLUSTER_ID)
-      .addHelpInfo(DEREGISTER_SUBCLUSTER_HELP_INFO)
-      .addExampleDescs(CMD_DEREGISTERSUBCLUSTER, "If we want to deregisterSubCluster SC-1")
-      .addExample(CMD_DEREGISTERSUBCLUSTER, DEREGISTER_SUBCLUSTER_EXAMPLE_1)
-      .addExample(CMD_DEREGISTERSUBCLUSTER, DEREGISTER_SUBCLUSTER_EXAMPLE_2);
+      // deregisterSubCluster
+      .addUsageInfo(DEREGISTER_SUBCLUSTER_USAGE)
+      .addExampleDescs(DEREGISTER_SUBCLUSTER_USAGE.args, "If we want to deregisterSubCluster SC-1")
+      .addExample(DEREGISTER_SUBCLUSTER_USAGE.args, DEREGISTER_SUBCLUSTER_EXAMPLE_1)
+      .addExample(DEREGISTER_SUBCLUSTER_USAGE.args, DEREGISTER_SUBCLUSTER_EXAMPLE_2)
+      // getSubCluster
+      .addUsageInfo(GET_SUBCLUSTER_USAGE)
+      .addExampleDescs(GET_SUBCLUSTER_USAGE.args,
+      "If we want to get information about all subClusters in Federation")
+      .addExample(GET_SUBCLUSTER_USAGE.args, GET_SUBCLUSTER_EXAMPLE);
 
   // Command2: policy
 
@@ -279,8 +305,8 @@ public class RouterCLI extends Configured implements Tool {
 
   protected final static Map<String, RouterCmdUsageInfos> ADMIN_USAGE =
       ImmutableMap.<String, RouterCmdUsageInfos>builder()
-      // Command1: deregisterSubCluster
-      .put(CMD_DEREGISTERSUBCLUSTER, DEREGISTER_SUBCLUSTER_USAGEINFOS)
+      // Command1: subCluster
+      .put(CMD_SUBCLUSTER, SUBCLUSTER_USAGEINFOS)
       // Command2: policy
       .put(CMD_POLICY, POLICY_USAGEINFOS)
       // Command3: application
@@ -408,27 +434,19 @@ public class RouterCLI extends Configured implements Tool {
     ToolRunner.printGenericCommandUsage(System.err);
   }
 
-  /**
-   * According to the parameter Deregister SubCluster.
-   *
-   * @param args parameter array.
-   * @return If the Deregister SubCluster operation is successful,
-   * it will return 0. Otherwise, it will return -1.
-   *
-   * @throws IOException raised on errors performing I/O.
-   * @throws YarnException exceptions from yarn servers.
-   * @throws ParseException Exceptions thrown during parsing of a command-line.
-   */
-  private int handleDeregisterSubCluster(String[] args)
-      throws IOException, YarnException, ParseException {
-
+  private int handleSubCluster(String[] args) throws ParseException, IOException, YarnException {
     // Prepare Options.
     Options opts = new Options();
+    opts.addOption("subCluster", false,
+         "We provide a set of commands for SubCluster Include deregisterSubCluster, " +
+         "get SubClusters.");
     opts.addOption("deregisterSubCluster", false,
         "Deregister YARN subCluster, if subCluster Heartbeat Timeout.");
+    opts.addOption("getSubClusters", false,
+        "Get information about all subClusters of Federation.");
     Option subClusterOpt = new Option(OPTION_SC, OPTION_SUBCLUSTERID, true,
         "The subCluster can be specified using either the '-sc' or '--subCluster' option. " +
-         " If the subCluster's Heartbeat Timeout, it will be marked as 'SC_LOST'.");
+        " If the subCluster's Heartbeat Timeout, it will be marked as 'SC_LOST'.");
     subClusterOpt.setOptionalArg(true);
     opts.addOption(subClusterOpt);
 
@@ -442,15 +460,61 @@ public class RouterCLI extends Configured implements Tool {
       return EXIT_ERROR;
     }
 
-    // Try to parse the subClusterId.
-    String subClusterId = null;
-    if (cliParser.hasOption(OPTION_SC) || cliParser.hasOption(OPTION_SUBCLUSTERID)) {
-      subClusterId = cliParser.getOptionValue(OPTION_SC);
-      if (subClusterId == null) {
-        subClusterId = cliParser.getOptionValue(OPTION_SUBCLUSTERID);
+    // deregister subCluster
+    if (cliParser.hasOption(OPTION_DEREGISTER_SUBCLUSTER)) {
+      String subClusterId = null;
+      if (cliParser.hasOption(OPTION_SC) || cliParser.hasOption(OPTION_SUBCLUSTERID)) {
+        subClusterId = cliParser.getOptionValue(OPTION_SC);
+        if (subClusterId == null) {
+          subClusterId = cliParser.getOptionValue(OPTION_SUBCLUSTERID);
+        }
       }
+      return handleDeregisterSubCluster(subClusterId);
+    } else if (cliParser.hasOption(OPTION_GET_SUBCLUSTERS)) {
+      // get subClusters
+      return handleGetSubClusters();
+    } else {
+      // printUsage
+      printUsage(args[0]);
     }
 
+    return EXIT_ERROR;
+  }
+
+  private int handleGetSubClusters() throws IOException, YarnException {
+    PrintWriter writer = new PrintWriter(new OutputStreamWriter(
+        System.out, Charset.forName(StandardCharsets.UTF_8.name())));
+    ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol();
+    GetSubClustersRequest request = GetSubClustersRequest.newInstance();
+    GetSubClustersResponse response = adminProtocol.getFederationSubClusters(request);
+    FormattingCLIUtils formattingCLIUtils = new FormattingCLIUtils(GET_SUBCLUSTER_TITLE)
+        .addHeaders(GET_SUBCLUSTER_HEADER);
+    List<FederationSubCluster> federationSubClusters = response.getFederationSubClusters();
+    federationSubClusters.forEach(federationSubCluster -> {
+      String responseSubClusterId = federationSubCluster.getSubClusterId();
+      String state = federationSubCluster.getSubClusterState();
+      String lastHeartBeatTime = federationSubCluster.getLastHeartBeatTime();
+      formattingCLIUtils.addLine(responseSubClusterId, state, lastHeartBeatTime);
+    });
+    writer.print(formattingCLIUtils.render());
+    writer.flush();
+    return EXIT_SUCCESS;
+  }
+
+  /**
+   * According to the parameter Deregister SubCluster.
+   *
+   * @param subClusterId subClusterId.
+   * @return If the Deregister SubCluster operation is successful,
+   * it will return 0. Otherwise, it will return -1.
+   *
+   * @throws IOException raised on errors performing I/O.
+   * @throws YarnException exceptions from yarn servers.
+   * @throws ParseException Exceptions thrown during parsing of a command-line.
+   */
+  private int handleDeregisterSubCluster(String subClusterId)
+      throws IOException, YarnException, ParseException {
+
     // If subClusterId is not empty, try deregisterSubCluster subCluster,
     // otherwise try deregisterSubCluster all subCluster.
     if (StringUtils.isNotBlank(subClusterId)) {
@@ -965,8 +1029,8 @@ public class RouterCLI extends Configured implements Tool {
         printHelp();
       }
       return EXIT_SUCCESS;
-    } else if (CMD_DEREGISTERSUBCLUSTER.equals(cmd)) {
-      return handleDeregisterSubCluster(args);
+    } else if (CMD_SUBCLUSTER.equals(cmd)) {
+      return handleSubCluster(args);
     } else if (CMD_POLICY.equals(cmd)) {
       return handlePolicy(args);
     } else if (CMD_APPLICATION.equals(cmd)) {

+ 36 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRouterCLI.java

@@ -31,6 +31,9 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.SaveFederationQueuePoli
 import org.apache.hadoop.yarn.server.api.protocolrecords.FederationQueueWeight;
 import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.FederationSubCluster;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesResponse;
 import org.junit.Before;
@@ -96,6 +99,21 @@ public class TestRouterCLI {
           return QueryFederationQueuePoliciesResponse.newInstance(1, 1, 1, 10, weights);
         });
 
+    when(admin.getFederationSubClusters(any(GetSubClustersRequest.class)))
+        .thenAnswer((Answer<GetSubClustersResponse>) invocationOnMock -> {
+          // Step1. parse request.
+          List<FederationSubCluster> subClustersList = new ArrayList<>();
+          // Add SC-1
+          FederationSubCluster subCluster1 = FederationSubCluster.newInstance("SC-1",
+              "RUNNING", new Date().toString());
+          // Add SC-2
+          FederationSubCluster subCluster2 = FederationSubCluster.newInstance("SC-2",
+              "RUNNING", new Date().toString());
+          subClustersList.add(subCluster1);
+          subClustersList.add(subCluster2);
+          return GetSubClustersResponse.newInstance(subClustersList);
+        });
+
     when(admin.deleteFederationPoliciesByQueues(any(DeleteFederationQueuePoliciesRequest.class)))
         .thenAnswer((Answer<DeleteFederationQueuePoliciesResponse>) invocationOnMock -> {
           // Step1. parse request.
@@ -285,15 +303,17 @@ public class TestRouterCLI {
     Map<String, RouterCLI.RouterCmdUsageInfos> adminUsage = rmAdminCLI.getAdminUsage();
     assertEquals(3, adminUsage.size());
 
-    RouterCLI.RouterCmdUsageInfos deregisterSubClusterUsageInfos =
-        adminUsage.get("-deregisterSubCluster");
-    assertNotNull(deregisterSubClusterUsageInfos);
-    Map<String, List<String>> dsExamplesMap = deregisterSubClusterUsageInfos.getExamples();
+    RouterCLI.RouterCmdUsageInfos subClusterUsageInfos = adminUsage.get("-subCluster");
+    assertNotNull(subClusterUsageInfos);
+    Map<String, List<String>> dsExamplesMap = subClusterUsageInfos.getExamples();
     assertNotNull(dsExamplesMap);
-    assertEquals(1, dsExamplesMap.size());
-    List<String> dsExamples = dsExamplesMap.get("-deregisterSubCluster");
+    assertEquals(2, dsExamplesMap.size());
+    List<String> dsExamples = dsExamplesMap.get("-deregisterSubCluster <-sc|--subClusterId>");
     assertNotNull(dsExamples);
     assertEquals(2, dsExamples.size());
+    List<String> getSubClustersExamples = dsExamplesMap.get("-getSubClusters");
+    assertNotNull(getSubClustersExamples);
+    assertEquals(1, getSubClustersExamples.size());
 
     RouterCLI.RouterCmdUsageInfos policyUsageInfos = adminUsage.get("-policy");
     assertNotNull(policyUsageInfos);
@@ -312,12 +332,21 @@ public class TestRouterCLI {
   }
 
   @Test
-  public void testDeleteFederationPoliciesByQueues() throws Exception {
+  public void testGetSubClusters() throws Exception {
     PrintStream oldOutPrintStream = System.out;
     ByteArrayOutputStream dataOut = new ByteArrayOutputStream();
     System.setOut(new PrintStream(dataOut));
     oldOutPrintStream.println(dataOut);
+    String[] args = {"-subCluster", "-getSubClusters"};
+    assertEquals(0, rmAdminCLI.run(args));
+  }
 
+  @Test
+  public void testDeleteFederationPoliciesByQueues() throws Exception {
+    PrintStream oldOutPrintStream = System.out;
+    ByteArrayOutputStream dataOut = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(dataOut));
+    oldOutPrintStream.println(dataOut);
     String[] args = {"-policy", "-d", "--queue", "root.a"};
     assertEquals(0, rmAdminCLI.run(args));
   }

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java

@@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.SaveF
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.BatchSaveFederationQueuePoliciesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.QueryFederationQueuePoliciesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetSubClustersRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationQueuePoliciesRequestProto;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocolPB;
@@ -92,6 +93,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePo
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesRequestPBImpl;
@@ -130,6 +133,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.QueryFederation
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.QueryFederationQueuePoliciesResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationApplicationResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.GetSubClustersRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.GetSubClustersResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationQueuePoliciesRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationQueuePoliciesResponsePBImpl;
 
@@ -445,6 +450,19 @@ public class ResourceManagerAdministrationProtocolPBClientImpl implements Resour
     return null;
   }
 
+  @Override
+  public GetSubClustersResponse getFederationSubClusters(GetSubClustersRequest request)
+      throws YarnException, IOException {
+    GetSubClustersRequestProto requestProto = ((GetSubClustersRequestPBImpl) request).getProto();
+    try {
+      return new GetSubClustersResponsePBImpl(
+          proxy.getFederationSubClusters(null, requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+    }
+    return null;
+  }
+
   @Override
   public DeleteFederationQueuePoliciesResponse deleteFederationPoliciesByQueues(
       DeleteFederationQueuePoliciesRequest request) throws YarnException, IOException {

+ 20 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java

@@ -62,6 +62,8 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.SaveF
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.SaveFederationQueuePolicyResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationApplicationRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetSubClustersRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetSubClustersResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationQueuePoliciesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.DeleteFederationQueuePoliciesResponseProto;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
@@ -93,6 +95,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePol
 import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl;
@@ -133,6 +137,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.QueryFederation
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.QueryFederationQueuePoliciesResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationApplicationResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.GetSubClustersRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.GetSubClustersResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationQueuePoliciesRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DeleteFederationQueuePoliciesResponsePBImpl;
 
@@ -474,6 +480,20 @@ public class ResourceManagerAdministrationProtocolPBServiceImpl implements Resou
     }
   }
 
+  @Override
+  public GetSubClustersResponseProto getFederationSubClusters(
+      RpcController controller, GetSubClustersRequestProto proto) throws ServiceException {
+    GetSubClustersRequest request = new GetSubClustersRequestPBImpl(proto);
+    try {
+      GetSubClustersResponse response = real.getFederationSubClusters(request);
+      return ((GetSubClustersResponsePBImpl) response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
   @Override
   public DeleteFederationQueuePoliciesResponseProto deleteFederationPoliciesByQueues(
       RpcController controller, DeleteFederationQueuePoliciesRequestProto proto)

+ 127 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/FederationSubClusterPBImpl.java

@@ -0,0 +1,127 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.FederationSubClusterProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.FederationSubClusterProto;
+import org.apache.hadoop.yarn.server.api.protocolrecords.FederationSubCluster;
+
+public class FederationSubClusterPBImpl extends FederationSubCluster {
+
+  private FederationSubClusterProto proto = FederationSubClusterProto.getDefaultInstance();
+  private FederationSubClusterProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public FederationSubClusterPBImpl() {
+    this.builder = FederationSubClusterProto.newBuilder();
+  }
+
+  public FederationSubClusterPBImpl(FederationSubClusterProto proto) {
+    this.proto = proto;
+    this.viaProto = true;
+  }
+
+  private synchronized void maybeInitBuilder() {
+    if (this.viaProto || this.builder == null) {
+      this.builder = FederationSubClusterProto.newBuilder(proto);
+    }
+    this.viaProto = false;
+  }
+
+  public FederationSubClusterProto getProto() {
+    this.proto = this.viaProto ? this.proto : this.builder.build();
+    this.viaProto = true;
+    return this.proto;
+  }
+
+  @Override
+  public String getSubClusterId() {
+    FederationSubClusterProtoOrBuilder p = this.viaProto ? this.proto : this.builder;
+    boolean hasSubClusterId = p.hasSubClusterId();
+    if (hasSubClusterId) {
+      return p.getSubClusterId();
+    }
+    return null;
+  }
+
+  @Override
+  public void setSubClusterId(String subClusterId) {
+    maybeInitBuilder();
+    if (subClusterId == null) {
+      builder.clearSubClusterId();
+      return;
+    }
+    builder.setSubClusterId(subClusterId);
+  }
+
+  @Override
+  public String getSubClusterState() {
+    FederationSubClusterProtoOrBuilder p = this.viaProto ? this.proto : this.builder;
+    boolean hasSubClusterState = p.hasSubClusterState();
+    if (hasSubClusterState) {
+      return p.getSubClusterState();
+    }
+    return null;
+  }
+
+  @Override
+  public void setSubClusterState(String subClusterState) {
+    maybeInitBuilder();
+    if (subClusterState == null) {
+      builder.clearSubClusterState();
+      return;
+    }
+    builder.setSubClusterState(subClusterState);
+  }
+
+  @Override
+  public String getLastHeartBeatTime() {
+    FederationSubClusterProtoOrBuilder p = this.viaProto ? this.proto : this.builder;
+    boolean hasLastHeartBeatTime = p.hasLastHeartBeatTime();
+    if (hasLastHeartBeatTime) {
+      return p.getLastHeartBeatTime();
+    }
+    return null;
+  }
+
+  @Override
+  public void setLastHeartBeatTime(String lastHeartBeatTime) {
+    maybeInitBuilder();
+    if (lastHeartBeatTime == null) {
+      builder.clearLastHeartBeatTime();
+      return;
+    }
+    builder.setLastHeartBeatTime(lastHeartBeatTime);
+  }
+
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof FederationSubCluster)) {
+      return false;
+    }
+    FederationSubClusterPBImpl otherImpl = this.getClass().cast(other);
+    return new EqualsBuilder()
+        .append(this.getProto(), otherImpl.getProto())
+        .isEquals();
+  }
+}

+ 72 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetSubClustersRequestPBImpl.java

@@ -0,0 +1,72 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.thirdparty.protobuf.TextFormat;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetSubClustersRequestProto;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersRequest;
+
+/**
+ * The class is responsible for get subclusters requests.
+ */
+@Private
+@Unstable
+public class GetSubClustersRequestPBImpl extends GetSubClustersRequest {
+
+  private GetSubClustersRequestProto proto = GetSubClustersRequestProto.getDefaultInstance();
+  private GetSubClustersRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public GetSubClustersRequestPBImpl() {
+    builder = GetSubClustersRequestProto.newBuilder();
+  }
+
+  public GetSubClustersRequestPBImpl(GetSubClustersRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetSubClustersRequestProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+}

+ 131 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/GetSubClustersResponsePBImpl.java

@@ -0,0 +1,131 @@
+/**
+ * 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.api.protocolrecords.impl.pb;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.hadoop.thirdparty.protobuf.TextFormat;
+import org.apache.hadoop.yarn.proto.YarnProtos.FederationSubClusterProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetSubClustersResponseProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetSubClustersResponseProto;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.FederationSubCluster;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The class is responsible for get subclusters responses.
+ */
+public class GetSubClustersResponsePBImpl extends GetSubClustersResponse {
+
+  private GetSubClustersResponseProto proto = GetSubClustersResponseProto.getDefaultInstance();
+  private GetSubClustersResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+  private List<FederationSubCluster> federationSubClusters = null;
+
+  public GetSubClustersResponsePBImpl() {
+    this.builder = GetSubClustersResponseProto.newBuilder();
+  }
+
+  public GetSubClustersResponsePBImpl(GetSubClustersResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  private synchronized void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetSubClustersResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    if (this.federationSubClusters != null) {
+      for (FederationSubCluster federationSubCluster : federationSubClusters) {
+        FederationSubClusterPBImpl federationSubClusterPBImpl =
+            (FederationSubClusterPBImpl) federationSubCluster;
+        builder.addSubClusters(federationSubClusterPBImpl.getProto());
+      }
+    }
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  public GetSubClustersResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof GetSubClustersResponse)) {
+      return false;
+    }
+    GetSubClustersResponsePBImpl otherImpl = this.getClass().cast(other);
+    return new EqualsBuilder()
+        .append(this.getProto(), otherImpl.getProto())
+        .isEquals();
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public List<FederationSubCluster> getFederationSubClusters() {
+    initFederationSubClustersList();
+    return this.federationSubClusters;
+  }
+
+  private void initFederationSubClustersList() {
+    if (this.federationSubClusters != null) {
+      return;
+    }
+    GetSubClustersResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<FederationSubClusterProto> getSubClustersResponseProtoList = p.getSubClustersList();
+    List<FederationSubCluster> subClusters = new ArrayList<>();
+    if (getSubClustersResponseProtoList == null || getSubClustersResponseProtoList.size() == 0) {
+      this.federationSubClusters = subClusters;
+      return;
+    }
+    for (FederationSubClusterProto federationSubClusterProto : getSubClustersResponseProtoList) {
+      subClusters.add(new FederationSubClusterPBImpl(federationSubClusterProto));
+    }
+    this.federationSubClusters = subClusters;
+  }
+
+  @Override
+  public void setFederationSubClusters(List<FederationSubCluster> pFederationSubClusters) {
+    if (federationSubClusters == null) {
+      federationSubClusters = new ArrayList<>();
+    }
+    federationSubClusters.clear();
+    federationSubClusters.addAll(pFederationSubClusters);
+  }
+}

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java

@@ -167,6 +167,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequ
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersResponse;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.Assert;
 import org.slf4j.Logger;
@@ -998,6 +1000,12 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     return null;
   }
 
+  @Override
+  public GetSubClustersResponse getFederationSubClusters(GetSubClustersRequest request)
+      throws YarnException, IOException {
+    return null;
+  }
+
   @Override
   public DeleteFederationQueuePoliciesResponse deleteFederationPoliciesByQueues(
       DeleteFederationQueuePoliciesRequest request) throws YarnException, IOException {

+ 10 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java

@@ -105,6 +105,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePol
 import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesResponse;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils;
@@ -1117,6 +1119,14 @@ public class AdminService extends CompositeService implements
         " Please call Router's deleteFederationApplication to delete Application.");
   }
 
+  @Override
+  public GetSubClustersResponse getFederationSubClusters(
+      GetSubClustersRequest request) throws YarnException, IOException {
+    throw new YarnException("It is not allowed to call the RM's " +
+        " getFederationSubClusters. " +
+        " Please call Router's getFederationSubClusters to get SubClusters.");
+  }
+
   @Override
   public DeleteFederationQueuePoliciesResponse deleteFederationPoliciesByQueues(
       DeleteFederationQueuePoliciesRequest request) throws YarnException, IOException {

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

@@ -157,6 +157,8 @@ public final class RouterMetrics {
   private MutableGaugeInt numListFederationQueuePoliciesFailedRetrieved;
   @Metric("# of deleteFederationApplication failed to be retrieved")
   private MutableGaugeInt numDeleteFederationApplicationFailedRetrieved;
+  @Metric("# of getFederationSubClusters failed to be retrieved")
+  private MutableGaugeInt numGetFederationSubClustersFailedRetrieved;
   @Metric("# of deleteFederationPoliciesByQueues failed to be retrieved")
   private MutableGaugeInt numDeleteFederationPoliciesByQueuesRetrieved;
   @Metric("# of refreshAdminAcls failed to be retrieved")
@@ -313,6 +315,8 @@ public final class RouterMetrics {
   private MutableRate totalSucceededListFederationQueuePoliciesFailedRetrieved;
   @Metric("Total number of successful Retrieved DeleteFederationApplication and latency(ms)")
   private MutableRate totalSucceededDeleteFederationApplicationFailedRetrieved;
+  @Metric("Total number of successful Retrieved getFederationSubClusters and latency(ms)")
+  private MutableRate totalSucceededGetFederationSubClustersRetrieved;
   @Metric("Total number of successful Retrieved DeleteFederationPoliciesByQueues and latency(ms)")
   private MutableRate totalSucceededDeleteFederationPoliciesByQueuesRetrieved;
   @Metric("Total number of successful Retrieved RefreshAdminAcls and latency(ms)")
@@ -405,6 +409,7 @@ public final class RouterMetrics {
   private MutableQuantiles batchSaveFederationQueuePoliciesLatency;
   private MutableQuantiles listFederationQueuePoliciesLatency;
   private MutableQuantiles deleteFederationApplicationLatency;
+  private MutableQuantiles getFederationSubClustersLatency;
   private MutableQuantiles deleteFederationPoliciesByQueuesLatency;
   private MutableQuantiles refreshAdminAclsLatency;
   private MutableQuantiles refreshServiceAclsLatency;
@@ -632,6 +637,10 @@ public final class RouterMetrics {
         "deleteFederationApplicationLatency",
         "latency of delete FederationApplication timeouts", "ops", "latency", 10);
 
+    getFederationSubClustersLatency = registry.newQuantiles(
+        "getFederationSubClustersLatency",
+        "latency of get FederationSubClusters timeouts", "ops", "latency", 10);
+
     deleteFederationPoliciesByQueuesLatency = registry.newQuantiles(
         "deleteFederationPoliciesByQueuesLatency",
         "latency of delete FederationPoliciesByQueues timeouts", "ops", "latency", 10);
@@ -985,6 +994,11 @@ public final class RouterMetrics {
     return totalSucceededDeleteFederationApplicationFailedRetrieved.lastStat().numSamples();
   }
 
+  @VisibleForTesting
+  public long getNumSucceededGetFederationSubClustersRetrieved() {
+    return totalSucceededGetFederationSubClustersRetrieved.lastStat().numSamples();
+  }
+
   @VisibleForTesting
   public long getNumSucceededDeleteFederationPoliciesByQueuesRetrieved() {
     return totalSucceededDeleteFederationPoliciesByQueuesRetrieved.lastStat().numSamples();
@@ -1355,6 +1369,11 @@ public final class RouterMetrics {
     return totalSucceededDeleteFederationApplicationFailedRetrieved.lastStat().mean();
   }
 
+  @VisibleForTesting
+  public double getLatencySucceededGetFederationSubClustersRetrieved() {
+    return totalSucceededGetFederationSubClustersRetrieved.lastStat().mean();
+  }
+
   @VisibleForTesting
   public double getLatencySucceededDeleteFederationPoliciesByQueuesRetrieved() {
     return totalSucceededDeleteFederationPoliciesByQueuesRetrieved.lastStat().mean();
@@ -1671,6 +1690,10 @@ public final class RouterMetrics {
     return numDeleteFederationApplicationFailedRetrieved.value();
   }
 
+  public int getFederationSubClustersFailedRetrieved() {
+    return numGetFederationSubClustersFailedRetrieved.value();
+  }
+
   public int getDeleteFederationPoliciesByQueuesRetrieved() {
     return numDeleteFederationPoliciesByQueuesRetrieved.value();
   }
@@ -2047,6 +2070,11 @@ public final class RouterMetrics {
     deleteFederationApplicationLatency.add(duration);
   }
 
+  public void succeededGetFederationSubClustersRetrieved(long duration) {
+    totalSucceededGetFederationSubClustersRetrieved.add(duration);
+    getFederationSubClustersLatency.add(duration);
+  }
+
   public void succeededDeleteFederationPoliciesByQueuesRetrieved(long duration) {
     totalSucceededDeleteFederationPoliciesByQueuesRetrieved.add(duration);
     deleteFederationPoliciesByQueuesLatency.add(duration);
@@ -2346,6 +2374,10 @@ public final class RouterMetrics {
     numDeleteFederationApplicationFailedRetrieved.incr();
   }
 
+  public void incrGetFederationSubClustersFailedRetrieved() {
+    numGetFederationSubClustersFailedRetrieved.incr();
+  }
+
   public void incrDeleteFederationPoliciesByQueuesRetrieved() {
     numDeleteFederationPoliciesByQueuesRetrieved.incr();
   }

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

@@ -66,6 +66,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePol
 import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesResponse;
 import org.slf4j.Logger;
@@ -245,6 +247,12 @@ public class DefaultRMAdminRequestInterceptor
     return rmAdminProxy.deleteFederationApplication(request);
   }
 
+  @Override
+  public GetSubClustersResponse getFederationSubClusters(
+      GetSubClustersRequest request) throws YarnException, IOException {
+    return rmAdminProxy.getFederationSubClusters(request);
+  }
+
   @Override
   public DeleteFederationQueuePoliciesResponse deleteFederationPoliciesByQueues(
       DeleteFederationQueuePoliciesRequest request) throws YarnException, IOException {

+ 50 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/FederationRMAdminInterceptor.java

@@ -66,6 +66,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.DeregisterSubClusterReq
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeregisterSubClusterResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeregisterSubClusters;
 import org.apache.hadoop.yarn.server.api.protocolrecords.FederationQueueWeight;
+import org.apache.hadoop.yarn.server.api.protocolrecords.FederationSubCluster;
 import org.apache.hadoop.yarn.server.api.protocolrecords.SaveFederationQueuePolicyRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.SaveFederationQueuePolicyResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.BatchSaveFederationQueuePoliciesRequest;
@@ -76,6 +77,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePo
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersResponse;
 import org.apache.hadoop.yarn.server.federation.failover.FederationProxyProviderUtil;
 import org.apache.hadoop.yarn.server.federation.policies.manager.PriorityBroadcastPolicyManager;
 import org.apache.hadoop.yarn.server.federation.policies.manager.WeightedHomePolicyManager;
@@ -1142,6 +1145,53 @@ public class FederationRMAdminInterceptor extends AbstractRMAdminRequestIntercep
     throw new YarnException("Unable to deleteFederationApplication.");
   }
 
+  /**
+   * Get federation subcluster list.
+   *
+   * @param request GetSubClustersRequest Request.
+   * @return SubClusters Response.
+   * @throws YarnException exceptions from yarn servers.
+   * @throws IOException io error occurs.
+   */
+  @Override
+  public GetSubClustersResponse getFederationSubClusters(GetSubClustersRequest request)
+       throws YarnException, IOException {
+
+    // Parameter validation.
+    if (request == null) {
+      routerMetrics.incrGetFederationSubClustersFailedRetrieved();
+      RouterServerUtil.logAndThrowException(
+          "Missing getFederationSubClusters Request.", null);
+    }
+
+    // Step1. Get all subClusters of the cluster.
+    Map<SubClusterId, SubClusterInfo> subClusters =
+        federationFacade.getSubClusters(false);
+
+    // Step2. Get FederationSubCluster data.
+    List<FederationSubCluster> federationSubClusters = new ArrayList<>();
+    long startTime = clock.getTime();
+    for (Map.Entry<SubClusterId, SubClusterInfo> subCluster : subClusters.entrySet()) {
+      SubClusterId subClusterId = subCluster.getKey();
+      try {
+        SubClusterInfo subClusterInfo = subCluster.getValue();
+        long lastHeartBeat = subClusterInfo.getLastHeartBeat();
+        Date lastHeartBeatDate = new Date(lastHeartBeat);
+        FederationSubCluster federationSubCluster = FederationSubCluster.newInstance(
+            subClusterId.getId(), subClusterInfo.getState().name(), lastHeartBeatDate.toString());
+        federationSubClusters.add(federationSubCluster);
+      } catch (Exception e) {
+        routerMetrics.incrGetFederationSubClustersFailedRetrieved();
+        LOG.error("getSubClusters SubClusterId = [%s] error.", subClusterId, e);
+      }
+    }
+    long stopTime = clock.getTime();
+    routerMetrics.succeededGetFederationSubClustersRetrieved(stopTime - startTime);
+
+    // Step3. Return results.
+    return GetSubClustersResponse.newInstance(federationSubClusters);
+  }
+
   /**
    * Delete Policies based on the provided queue list.
    *

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java

@@ -73,6 +73,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePol
 import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesResponse;
 import org.apache.hadoop.yarn.server.router.RouterServerUtil;
@@ -431,6 +433,13 @@ public class RouterRMAdminService extends AbstractService
     return pipeline.getRootInterceptor().deleteFederationApplication(request);
   }
 
+  @Override
+  public GetSubClustersResponse getFederationSubClusters(
+      GetSubClustersRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getFederationSubClusters(request);
+  }
+
   @Override
   public DeleteFederationQueuePoliciesResponse deleteFederationPoliciesByQueues(
       DeleteFederationQueuePoliciesRequest request) throws YarnException, IOException {

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

@@ -639,6 +639,10 @@ public class TestRouterMetrics {
       metrics.incrListFederationQueuePoliciesFailedRetrieved();
     }
 
+    public void getFederationSubClustersFailedRetrieved() {
+      LOG.info("Mocked: failed GetFederationSubClusters call");
+      metrics.incrGetFederationSubClustersFailedRetrieved();
+    }
     public void getDeleteFederationPoliciesByQueuesFailedRetrieved() {
       LOG.info("Mocked: failed DeleteFederationPoliciesByQueues call");
       metrics.incrDeleteFederationPoliciesByQueuesRetrieved();
@@ -991,9 +995,14 @@ public class TestRouterMetrics {
       metrics.succeededListFederationQueuePoliciesRetrieved(duration);
     }
 
+    public void getFederationSubClustersRetrieved(long duration) {
+      LOG.info("Mocked: successful GetFederationSubClustersRetrieved " +
+          " call with duration {}", duration);
+      metrics.succeededGetFederationSubClustersRetrieved(duration);
+    }
     public void deleteFederationPoliciesByQueuesRetrieved(long duration) {
       LOG.info("Mocked: successful DeleteFederationPoliciesByQueuesRetrieved " +
-              " call with duration {}", duration);
+          " call with duration {}", duration);
       metrics.succeededDeleteFederationPoliciesByQueuesRetrieved(duration);
     }
   }
@@ -2323,6 +2332,29 @@ public class TestRouterMetrics {
         metrics.getLatencySucceededListFederationQueuePoliciesRetrieved(), ASSERT_DOUBLE_DELTA);
   }
 
+  @Test
+  public void testGetFederationSubClustersFailedRetrieved() {
+    long totalBadBefore = metrics.getFederationSubClustersFailedRetrieved();
+    badSubCluster.getFederationSubClustersFailedRetrieved();
+    Assert.assertEquals(totalBadBefore + 1,
+        metrics.getFederationSubClustersFailedRetrieved());
+  }
+
+  @Test
+  public void testGetFederationSubClustersRetrieved() {
+    long totalGoodBefore = metrics.getNumSucceededGetFederationSubClustersRetrieved();
+    goodSubCluster.getFederationSubClustersRetrieved(150);
+    Assert.assertEquals(totalGoodBefore + 1,
+        metrics.getNumSucceededGetFederationSubClustersRetrieved());
+    Assert.assertEquals(150,
+        metrics.getLatencySucceededGetFederationSubClustersRetrieved(), ASSERT_DOUBLE_DELTA);
+    goodSubCluster.getFederationSubClustersRetrieved(300);
+    Assert.assertEquals(totalGoodBefore + 2,
+        metrics.getNumSucceededGetFederationSubClustersRetrieved());
+    Assert.assertEquals(225,
+        metrics.getLatencySucceededGetFederationSubClustersRetrieved(), ASSERT_DOUBLE_DELTA);
+  }
+
   @Test
   public void testDeleteFederationPoliciesByQueuesFailedRetrieved() {
     long totalBadBefore = metrics.getDeleteFederationPoliciesByQueuesRetrieved();

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java

@@ -60,6 +60,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePol
 import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesResponse;
 
@@ -196,6 +198,12 @@ public class PassThroughRMAdminRequestInterceptor
     return getNextInterceptor().deleteFederationApplication(request);
   }
 
+  @Override
+  public GetSubClustersResponse getFederationSubClusters(GetSubClustersRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().getFederationSubClusters(request);
+  }
+
   @Override
   public DeleteFederationQueuePoliciesResponse deleteFederationPoliciesByQueues(
       DeleteFederationQueuePoliciesRequest request) throws YarnException, IOException {

+ 19 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestFederationRMAdminInterceptor.java

@@ -67,6 +67,9 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePol
 import org.apache.hadoop.yarn.server.api.protocolrecords.QueryFederationQueuePoliciesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationApplicationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetSubClustersResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.FederationSubCluster;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DeleteFederationQueuePoliciesResponse;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
@@ -1031,6 +1034,22 @@ public class TestFederationRMAdminInterceptor extends BaseRouterRMAdminTest {
         deleteFederationApplicationResponse.getMessage());
   }
 
+  @Test
+  public void testGetFederationSubClusters() throws Exception {
+    LambdaTestUtils.intercept(YarnException.class,
+        "Missing getFederationSubClusters Request.",
+        () -> interceptor.getFederationSubClusters(null));
+
+    GetSubClustersRequest request = GetSubClustersRequest.newInstance();
+    GetSubClustersResponse federationSubClusters = interceptor.getFederationSubClusters(request);
+    assertNotNull(federationSubClusters);
+
+    List<FederationSubCluster> federationSubClustersList =
+        federationSubClusters.getFederationSubClusters();
+    assertNotNull(federationSubClustersList);
+    assertEquals(4, federationSubClustersList.size());
+  }
+
   @Test
   public void testDeleteFederationPoliciesByQueues() throws IOException, YarnException {
     // subClusters