Browse Source

YARN-11236. Implement FederationReservationHomeSubClusterStore With MemoryStore. (#4711)

slfan1989 3 years ago
parent
commit
d383cc4525
27 changed files with 1975 additions and 11 deletions
  1. 15 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/AbstractRouterPolicy.java
  2. 89 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationReservationHomeSubClusterStore.java
  3. 5 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationStateStore.java
  4. 53 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
  5. 23 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java
  6. 25 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java
  7. 72 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddReservationHomeSubClusterRequest.java
  8. 65 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddReservationHomeSubClusterResponse.java
  9. 64 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetReservationHomeSubClusterRequest.java
  10. 72 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetReservationHomeSubClusterResponse.java
  11. 39 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetReservationsHomeSubClusterRequest.java
  12. 75 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetReservationsHomeSubClusterResponse.java
  13. 129 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/ReservationHomeSubCluster.java
  14. 130 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/AddReservationHomeSubClusterRequestPBImpl.java
  15. 115 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/AddReservationHomeSubClusterResponsePBImpl.java
  16. 138 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetReservationHomeSubClusterRequestPBImpl.java
  17. 130 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetReservationHomeSubClusterResponsePBImpl.java
  18. 77 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetReservationsHomeSubClusterRequestPBImpl.java
  19. 181 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetReservationsHomeSubClusterResponsePBImpl.java
  20. 168 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/ReservationHomeSubClusterPBImpl.java
  21. 125 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationReservationHomeSubClusterStoreInputValidator.java
  22. 36 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
  23. 50 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
  24. 68 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseRouterPoliciesTest.java
  25. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java
  26. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestRejectRouterPolicy.java
  27. 24 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java

+ 15 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/AbstractRouterPolicy.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.federation.policies.router;
 
 import java.util.List;
 import java.util.Map;
+import java.util.Collections;
 
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
@@ -34,6 +35,7 @@ import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPo
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
 
 /**
  * Base abstract class for {@link FederationRouterPolicy} implementations, that
@@ -98,7 +100,17 @@ public abstract class AbstractRouterPolicy extends
 
     // if a reservation exists limit scope to the sub-cluster this
     // reservation is mapped to
-    // TODO: Implemented in YARN-11236
+    if (reservationId != null) {
+      // note this might throw YarnException if the reservation is
+      // unknown. This is to be expected, and should be handled by
+      // policy invoker.
+      FederationStateStoreFacade stateStoreFacade =
+          getPolicyContext().getFederationStateStoreFacade();
+      SubClusterId resSubCluster = stateStoreFacade.getReservationHomeSubCluster(reservationId);
+      SubClusterInfo subClusterInfo = activeSubClusters.get(resSubCluster);
+      return Collections.singletonMap(resSubCluster, subClusterInfo);
+    }
+
     return activeSubClusters;
   }
 
@@ -129,7 +141,7 @@ public abstract class AbstractRouterPolicy extends
 
     // apply filtering based on reservation location and active sub-clusters
     Map<SubClusterId, SubClusterInfo> filteredSubClusters = prefilterSubClusters(
-            appContext.getReservationID(), getActiveSubclusters());
+        appContext.getReservationID(), getActiveSubclusters());
 
     FederationPolicyUtils.validateSubClusterAvailability(filteredSubClusters.keySet(), blackLists);
 
@@ -167,8 +179,7 @@ public abstract class AbstractRouterPolicy extends
     }
 
     // apply filtering based on reservation location and active sub-clusters
-    Map<SubClusterId, SubClusterInfo> filteredSubClusters = prefilterSubClusters(
-        request.getReservationId(), getActiveSubclusters());
+    Map<SubClusterId, SubClusterInfo> filteredSubClusters = getActiveSubclusters();
 
     // pick the chosen subCluster from the active ones
     return chooseSubCluster(request.getQueue(), filteredSubClusters);

+ 89 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationReservationHomeSubClusterStore.java

@@ -0,0 +1,89 @@
+/**
+ * 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.federation.store;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationsHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationsHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterRequest;
+
+/**
+ * FederationReservationHomeSubClusterStore maintains the state of all
+ * <em>Reservations</em> that have been submitted to the federated cluster.
+ *
+ * *
+ * <p>
+ * The mapping details contains:
+ * <ul>
+ * <li>{@code ReservationId}</li>
+ * <li>{@code SubClusterId}</li>
+ * </ul>
+ *
+ */
+@Private
+@Unstable
+public interface FederationReservationHomeSubClusterStore {
+
+  /**
+   * Register the home {@code SubClusterId} of the newly submitted
+   * {@code ReservationId}. Currently response is empty if the operation was
+   * successful, if not an exception reporting reason for a failure. If a
+   * mapping for the Reservation already existed, the {@code SubClusterId} in
+   * this response will return the existing mapping which might be different
+   * from that in the {@code AddReservationHomeSubClusterRequest}.
+   *
+   * @param request the request to register a new Reservation with its home
+   *          sub-cluster
+   * @return upon successful registration of the Reservation in the StateStore,
+   *         {@code AddReservationHomeSubClusterRequest} containing the home
+   *         sub-cluster of the Reservation. Otherwise, an exception reporting
+   *         reason for a failure
+   * @throws YarnException if the request is invalid/fails
+   */
+  AddReservationHomeSubClusterResponse addReservationHomeSubCluster(
+      AddReservationHomeSubClusterRequest request) throws YarnException;
+
+  /**
+   * Get information about the Reservation identified by the input
+   * {@code ReservationId}.
+   *
+   * @param request contains the Reservation queried
+   * @return {@code ReservationHomeSubCluster} containing the Reservation's home
+   *         subcluster
+   * @throws YarnException if the request is invalid/fails
+   */
+  GetReservationHomeSubClusterResponse getReservationHomeSubCluster(
+      GetReservationHomeSubClusterRequest request) throws YarnException;
+
+  /**
+   * Get the {@code ReservationHomeSubCluster} list representing the mapping of
+   * all submitted Reservations to it's home sub-cluster.
+   *
+   * @param request empty representing all Reservations
+   * @return the mapping of all submitted Reservation to it's home sub-cluster
+   * @throws YarnException if the request is invalid/fails
+   */
+  GetReservationsHomeSubClusterResponse getReservationsHomeSubCluster(
+      GetReservationsHomeSubClusterRequest request) throws YarnException;
+
+}

+ 5 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationStateStore.java

@@ -25,12 +25,13 @@ import org.apache.hadoop.yarn.server.records.Version;
 /**
  * FederationStore extends the three interfaces used to coordinate the state of
  * a federated cluster: {@link FederationApplicationHomeSubClusterStore},
- * {@link FederationMembershipStateStore}, and {@link FederationPolicyStore}.
+ * {@link FederationMembershipStateStore}, {@link FederationPolicyStore}, and
+ * {@link FederationReservationHomeSubClusterStore}.
  *
  */
-public interface FederationStateStore
-    extends FederationApplicationHomeSubClusterStore,
-    FederationMembershipStateStore, FederationPolicyStore {
+public interface FederationStateStore extends
+    FederationApplicationHomeSubClusterStore, FederationMembershipStateStore,
+    FederationPolicyStore, FederationReservationHomeSubClusterStore {
 
   /**
    * Initialize the FederationStore.

+ 53 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java

@@ -27,6 +27,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 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.AddApplicationHomeSubClusterRequest;
@@ -59,7 +60,15 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegister
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationsHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationsHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.ReservationHomeSubCluster;
 import org.apache.hadoop.yarn.server.federation.store.utils.FederationApplicationHomeSubClusterStoreInputValidator;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationReservationHomeSubClusterStoreInputValidator;
 import org.apache.hadoop.yarn.server.federation.store.utils.FederationMembershipStateStoreInputValidator;
 import org.apache.hadoop.yarn.server.federation.store.utils.FederationPolicyStoreInputValidator;
 import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils;
@@ -75,6 +84,7 @@ public class MemoryFederationStateStore implements FederationStateStore {
 
   private Map<SubClusterId, SubClusterInfo> membership;
   private Map<ApplicationId, SubClusterId> applications;
+  private Map<ReservationId, SubClusterId> reservations;
   private Map<String, SubClusterPolicyConfiguration> policies;
 
   private final MonotonicClock clock = new MonotonicClock();
@@ -86,6 +96,7 @@ public class MemoryFederationStateStore implements FederationStateStore {
   public void init(Configuration conf) {
     membership = new ConcurrentHashMap<SubClusterId, SubClusterInfo>();
     applications = new ConcurrentHashMap<ApplicationId, SubClusterId>();
+    reservations = new ConcurrentHashMap<ReservationId, SubClusterId>();
     policies = new ConcurrentHashMap<String, SubClusterPolicyConfiguration>();
   }
 
@@ -93,6 +104,7 @@ public class MemoryFederationStateStore implements FederationStateStore {
   public void close() {
     membership = null;
     applications = null;
+    reservations = null;
     policies = null;
   }
 
@@ -312,4 +324,45 @@ public class MemoryFederationStateStore implements FederationStateStore {
     return null;
   }
 
+  @Override
+  public AddReservationHomeSubClusterResponse addReservationHomeSubCluster(
+      AddReservationHomeSubClusterRequest request) throws YarnException {
+    FederationReservationHomeSubClusterStoreInputValidator.validate(request);
+    ReservationHomeSubCluster homeSubCluster = request.getReservationHomeSubCluster();
+    ReservationId reservationId = homeSubCluster.getReservationId();
+    if (!reservations.containsKey(reservationId)) {
+      reservations.put(reservationId, homeSubCluster.getHomeSubCluster());
+    }
+    return AddReservationHomeSubClusterResponse.newInstance(reservations.get(reservationId));
+  }
+
+  @Override
+  public GetReservationHomeSubClusterResponse getReservationHomeSubCluster(
+      GetReservationHomeSubClusterRequest request) throws YarnException {
+    FederationReservationHomeSubClusterStoreInputValidator.validate(request);
+    ReservationId reservationId = request.getReservationId();
+    if (!reservations.containsKey(reservationId)) {
+      throw new YarnException("Reservation " + reservationId + " does not exist");
+    }
+    SubClusterId subClusterId = reservations.get(reservationId);
+    ReservationHomeSubCluster homeSubCluster =
+        ReservationHomeSubCluster.newInstance(reservationId, subClusterId);
+    return GetReservationHomeSubClusterResponse.newInstance(homeSubCluster);
+  }
+
+  @Override
+  public GetReservationsHomeSubClusterResponse getReservationsHomeSubCluster(
+      GetReservationsHomeSubClusterRequest request) throws YarnException {
+    List<ReservationHomeSubCluster> result = new ArrayList<>();
+
+    for (Entry<ReservationId, SubClusterId> entry : reservations.entrySet()) {
+      ReservationId reservationId = entry.getKey();
+      SubClusterId subClusterId = entry.getValue();
+      ReservationHomeSubCluster homeSubCluster =
+          ReservationHomeSubCluster.newInstance(reservationId, subClusterId);
+      result.add(homeSubCluster);
+    }
+
+    return GetReservationsHomeSubClusterResponse.newInstance(result);
+  }
 }

+ 23 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java

@@ -68,6 +68,12 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegister
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
 import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationsHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationsHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.utils.FederationApplicationHomeSubClusterStoreInputValidator;
 import org.apache.hadoop.yarn.server.federation.store.utils.FederationMembershipStateStoreInputValidator;
 import org.apache.hadoop.yarn.server.federation.store.utils.FederationPolicyStoreInputValidator;
@@ -1004,4 +1010,21 @@ public class SQLFederationStateStore implements FederationStateStore {
     return ba;
   }
 
+  @Override
+  public AddReservationHomeSubClusterResponse addReservationHomeSubCluster(
+      AddReservationHomeSubClusterRequest request) throws YarnException {
+    throw new NotImplementedException("Code is not implemented");
+  }
+
+  @Override
+  public GetReservationHomeSubClusterResponse getReservationHomeSubCluster(
+      GetReservationHomeSubClusterRequest request) throws YarnException {
+    throw new NotImplementedException("Code is not implemented");
+  }
+
+  @Override
+  public GetReservationsHomeSubClusterResponse getReservationsHomeSubCluster(
+      GetReservationsHomeSubClusterRequest request) throws YarnException {
+    throw new NotImplementedException("Code is not implemented");
+  }
 }

+ 25 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java

@@ -25,6 +25,7 @@ import java.util.Calendar;
 import java.util.List;
 import java.util.TimeZone;
 
+import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.curator.ZKCuratorManager;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -65,6 +66,12 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegister
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
 import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationsHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationsHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterIdPBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterInfoPBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterPolicyConfigurationPBImpl;
@@ -637,4 +644,22 @@ public class ZookeeperFederationStateStore implements FederationStateStore {
     Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
     return cal.getTimeInMillis();
   }
+
+  @Override
+  public AddReservationHomeSubClusterResponse addReservationHomeSubCluster(
+      AddReservationHomeSubClusterRequest request) throws YarnException {
+    throw new NotImplementedException("Code is not implemented");
+  }
+
+  @Override
+  public GetReservationHomeSubClusterResponse getReservationHomeSubCluster(
+      GetReservationHomeSubClusterRequest request) throws YarnException {
+    throw new NotImplementedException("Code is not implemented");
+  }
+
+  @Override
+  public GetReservationsHomeSubClusterResponse getReservationsHomeSubCluster(
+      GetReservationsHomeSubClusterRequest request) throws YarnException {
+    throw new NotImplementedException("Code is not implemented");
+  }
 }

+ 72 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddReservationHomeSubClusterRequest.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
+ * <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.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * The request sent by the <code>Router</code> to <code>Federation state
+ * store</code> to map the home subcluster of a newly submitted Reservation.
+ *
+ * <p>
+ * The request includes the mapping details, i.e.:
+ * <ul>
+ * <li>{@code ReservationId}</li>
+ * <li>{@code SubClusterId}</li>
+ * </ul>
+ */
+@Private
+@Unstable
+public abstract class AddReservationHomeSubClusterRequest {
+
+  @Private
+  @Unstable
+  public static AddReservationHomeSubClusterRequest newInstance(
+      ReservationHomeSubCluster reservationHomeSubCluster) {
+    AddReservationHomeSubClusterRequest mapRequest =
+        Records.newRecord(AddReservationHomeSubClusterRequest.class);
+    mapRequest.setReservationHomeSubCluster(reservationHomeSubCluster);
+    return mapRequest;
+  }
+
+  /**
+   * Get the {@link ReservationHomeSubCluster} representing the mapping of the
+   * Reservation to it's home sub-cluster.
+   *
+   * @return the mapping of the Reservation to it's home sub-cluster.
+   */
+  @Public
+  @Unstable
+  public abstract ReservationHomeSubCluster getReservationHomeSubCluster();
+
+  /**
+   * Set the {@link ReservationHomeSubCluster} representing the mapping of the
+   * Reservation to it's home sub-cluster.
+   *
+   * @param reservationHomeSubCluster the mapping of the Reservation to it's
+   *          home sub-cluster.
+   */
+  @Private
+  @Unstable
+  public abstract void setReservationHomeSubCluster(
+      ReservationHomeSubCluster reservationHomeSubCluster);
+}

+ 65 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddReservationHomeSubClusterResponse.java

@@ -0,0 +1,65 @@
+/**
+ * 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.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * AddReservationHomeSubClusterResponse contains the answer from the
+ * {@code FederationReservationHomeSubClusterStore} to a request to insert a
+ * newly generated ReservationId and its owner.
+ *
+ * The response contains reservation's home sub-cluster as it is stored in the
+ * {@code FederationReservationHomeSubClusterStore}. If a mapping for the
+ * reservation already existed, the {@code SubClusterId} in this response will
+ * return the existing mapping which might be different from that in the
+ * {@code AddReservationHomeSubClusterRequest}.
+ */
+@Private
+@Unstable
+public abstract class AddReservationHomeSubClusterResponse {
+
+  @Private
+  @Unstable
+  public static AddReservationHomeSubClusterResponse newInstance(
+      SubClusterId homeSubCluster) {
+    AddReservationHomeSubClusterResponse response =
+        Records.newRecord(AddReservationHomeSubClusterResponse.class);
+    response.setHomeSubCluster(homeSubCluster);
+    return response;
+  }
+
+  /**
+   * Set the home sub-cluster that this Reservation has been assigned to.
+   *
+   * @param homeSubCluster the {@link SubClusterId} of this reservation's home
+   *          sub-cluster
+   */
+  public abstract void setHomeSubCluster(SubClusterId homeSubCluster);
+
+  /**
+   * Get the home sub-cluster that this Reservation has been assigned to. This
+   * may not match the {@link SubClusterId} in the corresponding response, if
+   * the mapping for the request's reservation already existed.
+   *
+   * @return the {@link SubClusterId} of this reservation's home sub-cluster
+   */
+  public abstract SubClusterId getHomeSubCluster();
+}

+ 64 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetReservationHomeSubClusterRequest.java

@@ -0,0 +1,64 @@
+/**
+ * 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.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Request class to obtain the home sub-cluster for the specified
+ * {@link ReservationId}.
+ */
+@Private
+@Unstable
+public abstract class GetReservationHomeSubClusterRequest {
+
+  @Private
+  @Unstable
+  public static GetReservationHomeSubClusterRequest newInstance(
+      ReservationId reservationId) {
+    GetReservationHomeSubClusterRequest appMapping =
+        Records.newRecord(GetReservationHomeSubClusterRequest.class);
+    appMapping.setReservationId(reservationId);
+    return appMapping;
+  }
+
+  /**
+   * Get the {@link ReservationId} representing the unique identifier of the
+   * application.
+   *
+   * @return the application identifier
+   */
+  @Public
+  @Unstable
+  public abstract ReservationId getReservationId();
+
+  /**
+   * Set the {@link ReservationId} representing the unique identifier of the
+   * application.
+   *
+   * @param reservationId the reservatopm identifier
+   */
+  @Private
+  @Unstable
+  public abstract void setReservationId(ReservationId reservationId);
+
+}

+ 72 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetReservationHomeSubClusterResponse.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
+ * <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.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * The response sent by <code>Federation state
+ * store</code> to a query for the home subcluster of a newly submitted
+ * reservation.
+ *
+ * <p>
+ * The request includes the mapping details, i.e.:
+ * <ul>
+ * <li>{@code ReservationId}</li>
+ * <li>{@code SubClusterId}</li>
+ * </ul>
+ */
+@Private
+@Unstable
+public abstract class GetReservationHomeSubClusterResponse {
+
+  @Private
+  @Unstable
+  public static GetReservationHomeSubClusterResponse newInstance(
+      ReservationHomeSubCluster reservationHomeSubCluster) {
+    GetReservationHomeSubClusterResponse mapResponse =
+        Records.newRecord(GetReservationHomeSubClusterResponse.class);
+    mapResponse.setReservationHomeSubCluster(reservationHomeSubCluster);
+    return mapResponse;
+  }
+
+  /**
+   * Get the {@link ReservationHomeSubCluster} representing the mapping of the
+   * Reservation to it's home sub-cluster.
+   *
+   * @return the mapping of the reservation to it's home sub-cluster.
+   */
+  @Public
+  @Unstable
+  public abstract ReservationHomeSubCluster getReservationHomeSubCluster();
+
+  /**
+   * Set the {@link ReservationHomeSubCluster} representing the mapping of the
+   * Reservation to it's home sub-cluster.
+   *
+   * @param reservationHomeSubCluster the mapping of the reservation to it's home sub-cluster.
+   */
+  @Private
+  @Unstable
+  public abstract void setReservationHomeSubCluster(
+      ReservationHomeSubCluster reservationHomeSubCluster);
+}

+ 39 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetReservationsHomeSubClusterRequest.java

@@ -0,0 +1,39 @@
+/**
+ * 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.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Request class to obtain the home sub-cluster mapping of all active
+ * reservations.
+ */
+@Private
+@Unstable
+public abstract class GetReservationsHomeSubClusterRequest {
+
+  @Private
+  @Unstable
+  public static GetReservationsHomeSubClusterRequest newInstance() {
+    GetReservationsHomeSubClusterRequest request =
+        Records.newRecord(GetReservationsHomeSubClusterRequest.class);
+    return request;
+  }
+}

+ 75 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetReservationsHomeSubClusterResponse.java

@@ -0,0 +1,75 @@
+/**
+ * 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.federation.store.records;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * The response sent by <code>Federation state
+ * store</code> to a query for the home subcluster of all submitted
+ * reservations.
+ *
+ * <p>
+ * The response includes the mapping details, i.e.:
+ * <ul>
+ * <li>{@code ReservationId}</li>
+ * <li>{@code SubClusterId}</li>
+ * </ul>
+ */
+@Private
+@Unstable
+public abstract class GetReservationsHomeSubClusterResponse {
+
+  @Private
+  @Unstable
+  public static GetReservationsHomeSubClusterResponse newInstance(
+      List<ReservationHomeSubCluster> appsHomeSubClusters) {
+    GetReservationsHomeSubClusterResponse mapResponse =
+        Records.newRecord(GetReservationsHomeSubClusterResponse.class);
+    mapResponse.setAppsHomeSubClusters(appsHomeSubClusters);
+    return mapResponse;
+  }
+
+  /**
+   * Get the {@link ReservationHomeSubCluster} list representing the mapping of
+   * all submitted reservations to it's home sub-cluster.
+   *
+   * @return the mapping of all submitted reservation to it's home sub-cluster.
+   */
+  @Public
+  @Unstable
+  public abstract List<ReservationHomeSubCluster> getAppsHomeSubClusters();
+
+  /**
+   * Set the {@link ReservationHomeSubCluster} list representing the mapping of
+   * all submitted reservations to it's home sub-cluster.
+   *
+   * @param reservationsHomeSubClusters the mapping of all submitted reservation
+   *          to it's home sub-cluster.
+   */
+  @Private
+  @Unstable
+  public abstract void setAppsHomeSubClusters(
+      List<ReservationHomeSubCluster> reservationsHomeSubClusters);
+}

+ 129 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/ReservationHomeSubCluster.java

@@ -0,0 +1,129 @@
+/**
+ * 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.federation.store.records;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * ReservationHomeSubCluster is a report of the runtime information of the
+ * reservation that is running in the federated cluster.
+ *
+ * <p>
+ * It includes information such as:
+ * <ul>
+ * <li>{@link ReservationId}</li>
+ * <li>{@link SubClusterId}</li>
+ * </ul>
+ *
+ */
+@Private
+@Unstable
+public abstract class ReservationHomeSubCluster {
+
+  @Private
+  @Unstable
+  public static ReservationHomeSubCluster newInstance(ReservationId resId,
+      SubClusterId homeSubCluster) {
+    ReservationHomeSubCluster appMapping = Records.newRecord(ReservationHomeSubCluster.class);
+    appMapping.setReservationId(resId);
+    appMapping.setHomeSubCluster(homeSubCluster);
+    return appMapping;
+  }
+
+  /**
+   * Get the {@link ReservationId} representing the unique identifier of the
+   * Reservation.
+   *
+   * @return the reservation identifier
+   */
+  @Public
+  @Unstable
+  public abstract ReservationId getReservationId();
+
+  /**
+   * Set the {@link ReservationId} representing the unique identifier of the
+   * Reservation.
+   *
+   * @param resId the reservation identifier
+   */
+  @Private
+  @Unstable
+  public abstract void setReservationId(ReservationId resId);
+
+  /**
+   * Get the {@link SubClusterId} representing the unique identifier of the home
+   * subcluster in which the reservation is mapped to.
+   *
+   * @return the home subcluster identifier
+   */
+  @Public
+  @Unstable
+  public abstract SubClusterId getHomeSubCluster();
+
+  /**
+   * Set the {@link SubClusterId} representing the unique identifier of the home
+   * subcluster in which the ReservationMaster of the reservation is running.
+   *
+   * @param subClusterId the home subcluster identifier
+   */
+  @Private
+  @Unstable
+  public abstract void setHomeSubCluster(SubClusterId subClusterId);
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    ReservationHomeSubCluster other = (ReservationHomeSubCluster) obj;
+
+    return new EqualsBuilder()
+        .append(this.getReservationId(), other.getReservationId())
+        .append(this.getHomeSubCluster(), other.getHomeSubCluster())
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder().
+        append(this.getReservationId()).
+        append(this.getHomeSubCluster()).
+        toHashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "ReservationHomeSubCluster [getReservationId()="
+        + getReservationId() + ", getApplicationHomeSubcluster()=" + getHomeSubCluster()
+        + "]";
+  }
+
+}

+ 130 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/AddReservationHomeSubClusterRequestPBImpl.java

@@ -0,0 +1,130 @@
+/**
+ * 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.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.AddReservationHomeSubClusterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.AddReservationHomeSubClusterRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.ReservationHomeSubClusterProto;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.ReservationHomeSubCluster;
+
+import org.apache.hadoop.thirdparty.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link AddReservationHomeSubClusterRequest}.
+ */
+@Private
+@Unstable
+public class AddReservationHomeSubClusterRequestPBImpl
+    extends AddReservationHomeSubClusterRequest {
+
+  private AddReservationHomeSubClusterRequestProto proto =
+      AddReservationHomeSubClusterRequestProto.getDefaultInstance();
+  private AddReservationHomeSubClusterRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public AddReservationHomeSubClusterRequestPBImpl() {
+    builder = AddReservationHomeSubClusterRequestProto.newBuilder();
+  }
+
+  public AddReservationHomeSubClusterRequestPBImpl(
+      AddReservationHomeSubClusterRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public AddReservationHomeSubClusterRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = AddReservationHomeSubClusterRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+  }
+
+  @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());
+  }
+
+  @Override
+  public ReservationHomeSubCluster getReservationHomeSubCluster() {
+    AddReservationHomeSubClusterRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasAppSubclusterMap()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getAppSubclusterMap());
+  }
+
+  @Override
+  public void setReservationHomeSubCluster(
+      ReservationHomeSubCluster reservationInfo) {
+    maybeInitBuilder();
+    if (reservationInfo == null) {
+      builder.clearAppSubclusterMap();
+      return;
+    }
+    builder.setAppSubclusterMap(convertToProtoFormat(reservationInfo));
+  }
+
+  private ReservationHomeSubCluster convertFromProtoFormat(
+      ReservationHomeSubClusterProto sc) {
+    return new ReservationHomeSubClusterPBImpl(sc);
+  }
+
+  private ReservationHomeSubClusterProto convertToProtoFormat(
+      ReservationHomeSubCluster sc) {
+    return ((ReservationHomeSubClusterPBImpl) sc).getProto();
+  }
+}

+ 115 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/AddReservationHomeSubClusterResponsePBImpl.java

@@ -0,0 +1,115 @@
+/**
+ * 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.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.AddReservationHomeSubClusterResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.AddReservationHomeSubClusterResponseProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterIdProto;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+
+import org.apache.hadoop.thirdparty.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link AddReservationHomeSubClusterResponse}.
+ */
+@Private
+@Unstable
+public class AddReservationHomeSubClusterResponsePBImpl
+    extends AddReservationHomeSubClusterResponse {
+
+  private AddReservationHomeSubClusterResponseProto proto =
+      AddReservationHomeSubClusterResponseProto.getDefaultInstance();
+  private AddReservationHomeSubClusterResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public AddReservationHomeSubClusterResponsePBImpl() {
+    builder = AddReservationHomeSubClusterResponseProto.newBuilder();
+  }
+
+  public AddReservationHomeSubClusterResponsePBImpl(
+      AddReservationHomeSubClusterResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = AddReservationHomeSubClusterResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  public AddReservationHomeSubClusterResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public void setHomeSubCluster(SubClusterId homeSubCluster) {
+    maybeInitBuilder();
+    if (homeSubCluster == null) {
+      builder.clearHomeSubCluster();
+      return;
+    }
+    builder.setHomeSubCluster(convertToProtoFormat(homeSubCluster));
+  }
+
+  @Override
+  public SubClusterId getHomeSubCluster() {
+    AddReservationHomeSubClusterResponseProtoOrBuilder p = viaProto ? proto : builder;
+
+    if (!p.hasHomeSubCluster()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getHomeSubCluster());
+  }
+
+  @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());
+  }
+
+  private SubClusterId convertFromProtoFormat(SubClusterIdProto sc) {
+    return new SubClusterIdPBImpl(sc);
+  }
+
+  private SubClusterIdProto convertToProtoFormat(SubClusterId sc) {
+    return ((SubClusterIdPBImpl) sc).getProto();
+  }
+}

+ 138 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetReservationHomeSubClusterRequestPBImpl.java

@@ -0,0 +1,138 @@
+/**
+ * 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.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationIdPBImpl;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetReservationHomeSubClusterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetReservationHomeSubClusterRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationIdProto;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterRequest;
+
+import org.apache.hadoop.thirdparty.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link GetReservationHomeSubClusterRequest}.
+ */
+@Private
+@Unstable
+public class GetReservationHomeSubClusterRequestPBImpl
+    extends GetReservationHomeSubClusterRequest {
+
+  private GetReservationHomeSubClusterRequestProto proto =
+      GetReservationHomeSubClusterRequestProto.getDefaultInstance();
+  private GetReservationHomeSubClusterRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private ReservationId reservationId = null;
+
+  public GetReservationHomeSubClusterRequestPBImpl() {
+    builder = GetReservationHomeSubClusterRequestProto.newBuilder();
+  }
+
+  public GetReservationHomeSubClusterRequestPBImpl(
+      GetReservationHomeSubClusterRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetReservationHomeSubClusterRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetReservationHomeSubClusterRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationId != null) {
+      builder.setReservationId(convertToProtoFormat(this.reservationId));
+    }
+  }
+
+  @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());
+  }
+
+  @Override
+  public ReservationId getReservationId() {
+    GetReservationHomeSubClusterRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (reservationId != null) {
+      return reservationId;
+    }
+
+    if (!p.hasReservationId()) {
+      return null;
+    }
+    this.reservationId = convertFromProtoFormat(p.getReservationId());
+    return reservationId;
+  }
+
+  @Override
+  public void setReservationId(ReservationId reservationId) {
+    maybeInitBuilder();
+    if (reservationId == null) {
+      builder.clearReservationId();
+      return;
+    }
+    this.reservationId = reservationId;
+  }
+
+  private ReservationId convertFromProtoFormat(ReservationIdProto appId) {
+    return new ReservationIdPBImpl(appId);
+  }
+
+  private ReservationIdProto convertToProtoFormat(ReservationId appId) {
+    return ((ReservationIdPBImpl) appId).getProto();
+  }
+}

+ 130 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetReservationHomeSubClusterResponsePBImpl.java

@@ -0,0 +1,130 @@
+/**
+ * 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.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.ReservationHomeSubClusterProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetReservationHomeSubClusterResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetReservationHomeSubClusterResponseProtoOrBuilder;
+import org.apache.hadoop.yarn.server.federation.store.records.ReservationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterResponse;
+
+import org.apache.hadoop.thirdparty.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link GetReservationHomeSubClusterResponse}.
+ */
+@Private
+@Unstable
+public class GetReservationHomeSubClusterResponsePBImpl
+    extends GetReservationHomeSubClusterResponse {
+
+  private GetReservationHomeSubClusterResponseProto proto =
+      GetReservationHomeSubClusterResponseProto.getDefaultInstance();
+  private GetReservationHomeSubClusterResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public GetReservationHomeSubClusterResponsePBImpl() {
+    builder = GetReservationHomeSubClusterResponseProto.newBuilder();
+  }
+
+  public GetReservationHomeSubClusterResponsePBImpl(
+      GetReservationHomeSubClusterResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetReservationHomeSubClusterResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetReservationHomeSubClusterResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+  }
+
+  @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());
+  }
+
+  @Override
+  public ReservationHomeSubCluster getReservationHomeSubCluster() {
+    GetReservationHomeSubClusterResponseProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasAppSubclusterMap()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getAppSubclusterMap());
+  }
+
+  @Override
+  public void setReservationHomeSubCluster(
+      ReservationHomeSubCluster reservationInfo) {
+    maybeInitBuilder();
+    if (reservationInfo == null) {
+      builder.clearAppSubclusterMap();
+      return;
+    }
+    builder.setAppSubclusterMap(convertToProtoFormat(reservationInfo));
+  }
+
+  private ReservationHomeSubCluster convertFromProtoFormat(
+      ReservationHomeSubClusterProto sc) {
+    return new ReservationHomeSubClusterPBImpl(sc);
+  }
+
+  private ReservationHomeSubClusterProto convertToProtoFormat(
+      ReservationHomeSubCluster sc) {
+    return ((ReservationHomeSubClusterPBImpl) sc).getProto();
+  }
+}

+ 77 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetReservationsHomeSubClusterRequestPBImpl.java

@@ -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
+ * <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.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetReservationsHomeSubClusterRequestProto;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationsHomeSubClusterRequest;
+
+import org.apache.hadoop.thirdparty.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link GetReservationsHomeSubClusterRequest}.
+ */
+@Private
+@Unstable
+public class GetReservationsHomeSubClusterRequestPBImpl
+    extends GetReservationsHomeSubClusterRequest {
+
+  private GetReservationsHomeSubClusterRequestProto proto =
+      GetReservationsHomeSubClusterRequestProto.getDefaultInstance();
+  private GetReservationsHomeSubClusterRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public GetReservationsHomeSubClusterRequestPBImpl() {
+    builder = GetReservationsHomeSubClusterRequestProto.newBuilder();
+  }
+
+  public GetReservationsHomeSubClusterRequestPBImpl(
+      GetReservationsHomeSubClusterRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetReservationsHomeSubClusterRequestProto 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());
+  }
+}

+ 181 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetReservationsHomeSubClusterResponsePBImpl.java

@@ -0,0 +1,181 @@
+/**
+ * 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.federation.store.records.impl.pb;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.ReservationHomeSubClusterProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetReservationsHomeSubClusterResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetReservationsHomeSubClusterResponseProtoOrBuilder;
+import org.apache.hadoop.yarn.server.federation.store.records.ReservationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationsHomeSubClusterResponse;
+
+import org.apache.hadoop.thirdparty.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link GetReservationsHomeSubClusterResponse}.
+ */
+@Private
+@Unstable
+public class GetReservationsHomeSubClusterResponsePBImpl
+    extends GetReservationsHomeSubClusterResponse {
+
+  private GetReservationsHomeSubClusterResponseProto proto =
+      GetReservationsHomeSubClusterResponseProto.getDefaultInstance();
+  private GetReservationsHomeSubClusterResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private List<ReservationHomeSubCluster> appsHomeSubCluster;
+
+  public GetReservationsHomeSubClusterResponsePBImpl() {
+    builder = GetReservationsHomeSubClusterResponseProto.newBuilder();
+  }
+
+  public GetReservationsHomeSubClusterResponsePBImpl(
+      GetReservationsHomeSubClusterResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetReservationsHomeSubClusterResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetReservationsHomeSubClusterResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.appsHomeSubCluster != null) {
+      addSubClustersInfoToProto();
+    }
+  }
+
+  @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());
+  }
+
+  @Override
+  public List<ReservationHomeSubCluster> getAppsHomeSubClusters() {
+    initSubClustersInfoList();
+    return appsHomeSubCluster;
+  }
+
+  @Override
+  public void setAppsHomeSubClusters(
+      List<ReservationHomeSubCluster> appsHomeSubClusters) {
+    maybeInitBuilder();
+    if (appsHomeSubClusters == null) {
+      builder.clearAppSubclusterMap();
+      return;
+    }
+    this.appsHomeSubCluster = appsHomeSubClusters;
+  }
+
+  private void initSubClustersInfoList() {
+    if (this.appsHomeSubCluster != null) {
+      return;
+    }
+    GetReservationsHomeSubClusterResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ReservationHomeSubClusterProto> subClusterInfosList = p.getAppSubclusterMapList();
+    appsHomeSubCluster = new ArrayList<>();
+
+    for (ReservationHomeSubClusterProto r : subClusterInfosList) {
+      appsHomeSubCluster.add(convertFromProtoFormat(r));
+    }
+  }
+
+  private void addSubClustersInfoToProto() {
+    maybeInitBuilder();
+    builder.clearAppSubclusterMap();
+    if (appsHomeSubCluster == null) {
+      return;
+    }
+    Iterable<ReservationHomeSubClusterProto> iterable =
+        new Iterable<ReservationHomeSubClusterProto>() {
+          @Override
+          public Iterator<ReservationHomeSubClusterProto> iterator() {
+            return new Iterator<ReservationHomeSubClusterProto>() {
+
+              private Iterator<ReservationHomeSubCluster> iter = appsHomeSubCluster.iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iter.hasNext();
+              }
+
+              @Override
+              public ReservationHomeSubClusterProto next() {
+                return convertToProtoFormat(iter.next());
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+            };
+          }
+        };
+    builder.addAllAppSubclusterMap(iterable);
+  }
+
+  private ReservationHomeSubCluster convertFromProtoFormat(ReservationHomeSubClusterProto sc) {
+    return new ReservationHomeSubClusterPBImpl(sc);
+  }
+
+  private ReservationHomeSubClusterProto convertToProtoFormat(ReservationHomeSubCluster sc) {
+    return ((ReservationHomeSubClusterPBImpl) sc).getProto();
+  }
+}

+ 168 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/ReservationHomeSubClusterPBImpl.java

@@ -0,0 +1,168 @@
+/**
+ * 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.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ReservationIdPBImpl;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.ReservationHomeSubClusterProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.ReservationHomeSubClusterProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ReservationIdProto;
+import org.apache.hadoop.yarn.server.federation.store.records.ReservationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+
+import org.apache.hadoop.thirdparty.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of {@link ReservationHomeSubCluster}.
+ */
+@Private
+@Unstable
+public class ReservationHomeSubClusterPBImpl extends ReservationHomeSubCluster {
+
+  private ReservationHomeSubClusterProto proto =
+      ReservationHomeSubClusterProto.getDefaultInstance();
+  private ReservationHomeSubClusterProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private ReservationId reservationId = null;
+  private SubClusterId homeSubCluster = null;
+
+  public ReservationHomeSubClusterPBImpl() {
+    builder = ReservationHomeSubClusterProto.newBuilder();
+  }
+
+  public ReservationHomeSubClusterPBImpl(ReservationHomeSubClusterProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ReservationHomeSubClusterProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ReservationHomeSubClusterProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.reservationId != null) {
+      builder.setReservationId(convertToProtoFormat(this.reservationId));
+    }
+    if (this.homeSubCluster != null) {
+      builder.setHomeSubCluster(convertToProtoFormat(this.homeSubCluster));
+    }
+  }
+
+  @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());
+  }
+
+  @Override
+  public ReservationId getReservationId() {
+    ReservationHomeSubClusterProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasReservationId()) {
+      return null;
+    }
+    this.reservationId = convertFromProtoFormat(p.getReservationId());
+    return this.reservationId;
+  }
+
+  @Override
+  public void setReservationId(ReservationId resId) {
+    maybeInitBuilder();
+    if (resId == null) {
+      builder.clearReservationId();
+      return;
+    }
+    this.reservationId = resId;
+  }
+
+  @Override
+  public SubClusterId getHomeSubCluster() {
+    ReservationHomeSubClusterProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.homeSubCluster != null) {
+      return this.homeSubCluster;
+    }
+    if (!p.hasHomeSubCluster()) {
+      return null;
+    }
+    this.homeSubCluster = convertFromProtoFormat(p.getHomeSubCluster());
+    return this.homeSubCluster;
+  }
+
+  @Override
+  public void setHomeSubCluster(SubClusterId subClusterId) {
+    maybeInitBuilder();
+    if (subClusterId == null) {
+      builder.clearHomeSubCluster();
+      return;
+    }
+    this.homeSubCluster = subClusterId;
+  }
+
+  private SubClusterId convertFromProtoFormat(SubClusterIdProto subClusterId) {
+    return new SubClusterIdPBImpl(subClusterId);
+  }
+
+  private SubClusterIdProto convertToProtoFormat(SubClusterId subClusterId) {
+    return ((SubClusterIdPBImpl) subClusterId).getProto();
+  }
+
+  private ReservationId convertFromProtoFormat(ReservationIdProto appId) {
+    return new ReservationIdPBImpl(appId);
+  }
+
+  private ReservationIdProto convertToProtoFormat(ReservationId appId) {
+    return ((ReservationIdPBImpl) appId).getProto();
+  }
+}

+ 125 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationReservationHomeSubClusterStoreInputValidator.java

@@ -0,0 +1,125 @@
+/**
+ * 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.federation.store.utils;
+
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.ReservationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class to validate the inputs to
+ * {@code FederationReservationHomeSubClusterStore}, allows a fail fast
+ * mechanism for invalid user inputs.
+ *
+ */
+public final class FederationReservationHomeSubClusterStoreInputValidator {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(FederationReservationHomeSubClusterStoreInputValidator.class);
+
+  private FederationReservationHomeSubClusterStoreInputValidator() {
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast). Check if the provided {@link AddReservationHomeSubClusterRequest}
+   * for adding a new reservation is valid or not.
+   *
+   * @param request the {@link AddReservationHomeSubClusterRequest} to validate against
+   * @throws FederationStateStoreInvalidInputException if the request is invalid
+   */
+  public static void validate(AddReservationHomeSubClusterRequest request)
+      throws FederationStateStoreInvalidInputException {
+    if (request == null) {
+      String message = "Missing AddReservationHomeSubCluster Request."
+          + " Please try again by specifying"
+          + " an AddReservationHomeSubCluster information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate ReservationHomeSubCluster info
+    checkReservationHomeSubCluster(request.getReservationHomeSubCluster());
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast). Check if the provided {@link GetReservationHomeSubClusterRequest}
+   * for querying reservation's information is valid or not.
+   *
+   * @param request the {@link GetReservationHomeSubClusterRequest} to validate against
+   * @throws FederationStateStoreInvalidInputException if the request is invalid
+   */
+  public static void validate(GetReservationHomeSubClusterRequest request)
+      throws FederationStateStoreInvalidInputException {
+    if (request == null) {
+      String message = "Missing GetReservationHomeSubCluster Request."
+          + " Please try again by specifying an Reservation Id information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate Reservation Id
+    checkReservationId(request.getReservationId());
+  }
+
+  /**
+   * Validate if the ReservationHomeSubCluster info are present or not.
+   *
+   * @param reservationHomeSubCluster the information of the Reservation to be verified
+   * @throws FederationStateStoreInvalidInputException if the SubCluster Info are invalid
+   */
+  private static void checkReservationHomeSubCluster(
+      ReservationHomeSubCluster reservationHomeSubCluster)
+      throws FederationStateStoreInvalidInputException {
+    if (reservationHomeSubCluster == null) {
+      String message = "Missing ReservationHomeSubCluster Info."
+          + " Please try again by specifying"
+          + " an ReservationHomeSubCluster information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate Reservation Id
+    checkReservationId(reservationHomeSubCluster.getReservationId());
+
+    // validate subcluster Id
+    FederationMembershipStateStoreInputValidator
+        .checkSubClusterId(reservationHomeSubCluster.getHomeSubCluster());
+  }
+
+  /**
+   * Validate if the Reservation id is present or not.
+   *
+   * @param reservationId the id of the Reservation to be verified
+   * @throws FederationStateStoreInvalidInputException if the Reservation Id is invalid
+   */
+  private static void checkReservationId(ReservationId reservationId)
+      throws FederationStateStoreInvalidInputException {
+    if (reservationId == null) {
+      String message = "Missing ReservationId. Please try again by specifying an ReservationId.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+  }
+}

+ 36 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -51,9 +52,13 @@ import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreRetriableException;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
 import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsRequest;
@@ -62,6 +67,7 @@ import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolic
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.ReservationHomeSubCluster;
 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.SubClusterPolicyConfiguration;
@@ -400,6 +406,35 @@ public final class FederationStateStoreFacade {
     return this.conf;
   }
 
+  /**
+   * Adds the home {@link SubClusterId} for the specified {@link ReservationId}.
+   *
+   * @param appHomeSubCluster the mapping of the reservation to it's home
+   *          sub-cluster
+   * @return the stored subCluster from StateStore
+   * @throws YarnException if the call to the state store is unsuccessful
+   */
+  public SubClusterId addReservationHomeSubCluster(ReservationHomeSubCluster appHomeSubCluster)
+      throws YarnException {
+    AddReservationHomeSubClusterResponse response = stateStore.addReservationHomeSubCluster(
+        AddReservationHomeSubClusterRequest.newInstance(appHomeSubCluster));
+    return response.getHomeSubCluster();
+  }
+
+  /**
+   * Returns the home {@link SubClusterId} for the specified {@link ReservationId}.
+   *
+   * @param reservationId the identifier of the reservation
+   * @return the home subCluster identifier
+   * @throws YarnException if the call to the state store is unsuccessful
+   */
+  public SubClusterId getReservationHomeSubCluster(ReservationId reservationId)
+      throws YarnException {
+    GetReservationHomeSubClusterResponse response = stateStore.getReservationHomeSubCluster(
+         GetReservationHomeSubClusterRequest.newInstance(reservationId));
+    return response.getReservationHomeSubCluster().getHomeSubCluster();
+  }
+
   /**
    * Helper method to create instances of Object using the class name defined in
    * the configuration object. The instances creates {@link RetryProxy} using
@@ -560,7 +595,7 @@ public final class FederationStateStoreFacade {
     private K key;
     private Func<K, V> func;
 
-    public CacheRequest(K key, Func<K, V> func) {
+    CacheRequest(K key, Func<K, V> func) {
       this.key = key;
       this.func = func;
     }

+ 50 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto

@@ -137,6 +137,56 @@ message DeleteApplicationHomeSubClusterRequestProto {
 message DeleteApplicationHomeSubClusterResponseProto {
 }
 
+
+// ---- reservations ----
+
+message ReservationHomeSubClusterProto {
+  optional ReservationIdProto reservation_id = 1;
+  optional SubClusterIdProto home_sub_cluster = 2;
+}
+
+message AddReservationHomeSubClusterRequestProto {
+  optional ReservationHomeSubClusterProto app_subcluster_map = 1;
+}
+
+message AddReservationHomeSubClusterResponseProto {
+  optional SubClusterIdProto home_sub_cluster = 1;
+}
+
+message UpdateReservationHomeSubClusterRequestProto {
+  optional ReservationHomeSubClusterProto app_subcluster_map = 1;
+}
+
+message UpdateReservationHomeSubClusterResponseProto {
+}
+
+message GetReservationHomeSubClusterRequestProto {
+  optional ReservationIdProto reservation_id = 1;
+}
+
+message GetReservationHomeSubClusterResponseProto {
+  optional ReservationHomeSubClusterProto app_subcluster_map = 1;
+}
+
+message GetReservationsHomeSubClusterRequestProto {
+
+}
+
+message GetReservationsHomeSubClusterResponseProto {
+  repeated ReservationHomeSubClusterProto app_subcluster_map = 1;
+}
+
+
+message DeleteReservationHomeSubClusterRequestProto {
+  optional ReservationIdProto reservation_id = 1;
+}
+
+message DeleteReservationHomeSubClusterResponseProto {
+}
+
+
+//----- configurations ---
+
 message SubClusterPolicyConfigurationProto {
   optional string queue = 1;
   optional string type = 2;

+ 68 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseRouterPoliciesTest.java

@@ -19,24 +19,35 @@
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
 import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyUtils;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.ReservationHomeSubCluster;
 import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
 
+import static org.mockito.Mockito.when;
+
 /**
  * Base class for router policies tests, tests for null input cases.
  */
@@ -126,4 +137,61 @@ public abstract class BaseRouterPoliciesTest
         () -> policy.getReservationHomeSubcluster(null));
   }
 
+  @Test
+  public void testUnknownReservation() throws Exception {
+
+    long now = Time.now();
+    ReservationSubmissionRequest resReq = getReservationSubmissionRequest();
+    ReservationId reservationId = ReservationId.newInstance(now, 1);
+    when(resReq.getQueue()).thenReturn("queue1");
+    when(resReq.getReservationId()).thenReturn(reservationId);
+
+    // route an application that uses this app
+    ApplicationSubmissionContext applicationSubmissionContext =
+        ApplicationSubmissionContext.newInstance(
+            ApplicationId.newInstance(now, 1), "app1", "queue1", Priority.newInstance(1),
+                null, false, false, 1, null, null, false);
+
+    applicationSubmissionContext.setReservationID(resReq.getReservationId());
+    FederationRouterPolicy policy = (FederationRouterPolicy) getPolicy();
+
+    LambdaTestUtils.intercept(YarnException.class,
+        "Reservation " + reservationId + " does not exist",
+        () -> policy.getHomeSubcluster(applicationSubmissionContext, new ArrayList<>()));
+  }
+
+  @Test
+  public void testFollowReservation() throws YarnException {
+
+    long now = Time.now();
+    ReservationSubmissionRequest resReq = getReservationSubmissionRequest();
+    when(resReq.getQueue()).thenReturn("queue1");
+    when(resReq.getReservationId()).thenReturn(ReservationId.newInstance(now, 1));
+
+    FederationRouterPolicy routerPolicy = (FederationRouterPolicy) getPolicy();
+    FederationPolicyInitializationContext fdContext = getFederationPolicyContext();
+    FederationStateStoreFacade storeFacade = fdContext.getFederationStateStoreFacade();
+
+    // first we invoke a reservation placement
+    SubClusterId chosen = routerPolicy.getReservationHomeSubcluster(resReq);
+
+    // add this to the store
+    ReservationHomeSubCluster homeSubCluster =
+        ReservationHomeSubCluster.newInstance(resReq.getReservationId(), chosen);
+    storeFacade.addReservationHomeSubCluster(homeSubCluster);
+
+    // route an application that uses this app
+    ApplicationSubmissionContext applicationSubmissionContext =
+        ApplicationSubmissionContext.newInstance(
+            ApplicationId.newInstance(now, 1), "app1", "queue1", Priority.newInstance(1),
+                null, false, false, 1, null, null, false);
+
+    applicationSubmissionContext.setReservationID(resReq.getReservationId());
+    SubClusterId chosen2 = routerPolicy.getHomeSubcluster(
+        applicationSubmissionContext, Collections.emptyList());
+
+    // application follows reservation
+    Assert.assertEquals(chosen, chosen2);
+  }
+
 }

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java

@@ -74,9 +74,8 @@ public class TestPriorityRouterPolicy extends BaseRouterPoliciesTest {
     }
     getPolicyInfo().setRouterPolicyWeights(routerWeights);
     getPolicyInfo().setAMRMPolicyWeights(amrmWeights);
-    FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
-        getPolicyInfo(), getActiveSubclusters());
 
+    setupContext();
   }
 
   @Test

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestRejectRouterPolicy.java

@@ -57,4 +57,10 @@ public class TestRejectRouterPolicy extends BaseRouterPoliciesTest {
             false, false, 0, Resources.none(), null, false, null, null);
     localPolicy.getHomeSubcluster(applicationSubmissionContext, null);
   }
+
+  @Override
+  @Test(expected = FederationPolicyException.class)
+  public void testFollowReservation() throws YarnException {
+    super.testFollowReservation();
+  }
 }

+ 24 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java

@@ -33,12 +33,18 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationsHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationsHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationsHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetReservationsHomeSubClusterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsRequest;
@@ -301,4 +307,22 @@ public class FederationStateStoreService extends AbstractService
       DeleteApplicationHomeSubClusterRequest request) throws YarnException {
     return stateStoreClient.deleteApplicationHomeSubCluster(request);
   }
+
+  @Override
+  public AddReservationHomeSubClusterResponse addReservationHomeSubCluster(
+      AddReservationHomeSubClusterRequest request) throws YarnException {
+    return stateStoreClient.addReservationHomeSubCluster(request);
+  }
+
+  @Override
+  public GetReservationHomeSubClusterResponse getReservationHomeSubCluster(
+      GetReservationHomeSubClusterRequest request) throws YarnException {
+    return stateStoreClient.getReservationHomeSubCluster(request);
+  }
+
+  @Override
+  public GetReservationsHomeSubClusterResponse getReservationsHomeSubCluster(
+      GetReservationsHomeSubClusterRequest request) throws YarnException {
+    return stateStoreClient.getReservationsHomeSubCluster(request);
+  }
 }