From eb0e35e5e5a849f20ff83ca209158d7e36516073 Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Fri, 5 Aug 2022 08:58:52 -0700 Subject: [PATCH 1/9] YARN-11236. Implement FederationReservationHomeSubClusterStore With MemoryStore. --- .../policies/router/AbstractRouterPolicy.java | 15 +- ...erationReservationHomeSubClusterStore.java | 89 ++++++++ .../store/FederationStateStore.java | 9 +- .../impl/MemoryFederationStateStore.java | 49 +++++ .../store/impl/SQLFederationStateStore.java | 23 +++ .../impl/ZookeeperFederationStateStore.java | 24 +++ .../AddReservationHomeSubClusterRequest.java | 72 +++++++ .../AddReservationHomeSubClusterResponse.java | 65 ++++++ .../GetReservationHomeSubClusterRequest.java | 64 ++++++ .../GetReservationHomeSubClusterResponse.java | 73 +++++++ .../GetReservationsHomeSubClusterRequest.java | 39 ++++ ...GetReservationsHomeSubClusterResponse.java | 75 +++++++ .../records/ReservationHomeSubCluster.java | 124 ++++++++++++ ...eservationHomeSubClusterRequestPBImpl.java | 132 ++++++++++++ ...servationHomeSubClusterResponsePBImpl.java | 117 +++++++++++ ...eservationHomeSubClusterRequestPBImpl.java | 139 +++++++++++++ ...servationHomeSubClusterResponsePBImpl.java | 132 ++++++++++++ ...servationsHomeSubClusterRequestPBImpl.java | 78 +++++++ ...ervationsHomeSubClusterResponsePBImpl.java | 190 ++++++++++++++++++ .../pb/ReservationHomeSubClusterPBImpl.java | 167 +++++++++++++++ ...tionHomeSubClusterStoreInputValidator.java | 134 ++++++++++++ .../utils/FederationStateStoreFacade.java | 42 +++- .../proto/yarn_server_federation_protos.proto | 50 +++++ .../router/BaseRouterPoliciesTest.java | 58 ++++++ .../router/TestPriorityRouterPolicy.java | 3 +- .../router/TestRejectRouterPolicy.java | 6 + .../FederationStateStoreService.java | 31 ++- 27 files changed, 1987 insertions(+), 13 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationReservationHomeSubClusterStore.java create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 diff --git a/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 b/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 index dddc5384fc49d..473c4cec4814d 100644 --- a/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 +++ b/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 @@ 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; @@ -98,7 +99,16 @@ protected Map prefilterSubClusters( // 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. + SubClusterId resSubCluster = getPolicyContext().getFederationStateStoreFacade(). + getReservationHomeSubCluster(reservationId); + + return Collections.singletonMap(resSubCluster, activeSubClusters.get(resSubCluster)); + } + return activeSubClusters; } @@ -167,8 +177,7 @@ public SubClusterId getReservationHomeSubcluster(ReservationSubmissionRequest re } // apply filtering based on reservation location and active sub-clusters - Map filteredSubClusters = prefilterSubClusters( - request.getReservationId(), getActiveSubclusters()); + Map filteredSubClusters = getActiveSubclusters(); // pick the chosen subCluster from the active ones return chooseSubCluster(request.getQueue(), filteredSubClusters); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationReservationHomeSubClusterStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationReservationHomeSubClusterStore.java new file mode 100644 index 0000000000000..6ebb22c3ac864 --- /dev/null +++ b/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 + *

+ * 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; + +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 + * Reservations that have been submitted to the federated cluster. + * + * * + *

+ * The mapping details contains: + *

    + *
  • {@code ReservationId}
  • + *
  • {@code SubClusterId}
  • + *
+ * + */ +@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; + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationStateStore.java index 9397e9c240473..67461e6c30b86 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationStateStore.java +++ b/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 @@ /** * 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. diff --git a/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 b/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 index 7c06256a41364..e667a393829e5 100644 --- a/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 +++ b/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 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.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 membership; private Map applications; + private Map reservations; private Map policies; private final MonotonicClock clock = new MonotonicClock(); @@ -86,6 +96,7 @@ public class MemoryFederationStateStore implements FederationStateStore { public void init(Configuration conf) { membership = new ConcurrentHashMap(); applications = new ConcurrentHashMap(); + reservations = new ConcurrentHashMap(); policies = new ConcurrentHashMap(); } @@ -93,6 +104,7 @@ public void init(Configuration conf) { public void close() { membership = null; applications = null; + reservations = null; policies = null; } @@ -312,4 +324,41 @@ public Version loadVersion() { return null; } + @Override + public AddReservationHomeSubClusterResponse addReservationHomeSubCluster( + AddReservationHomeSubClusterRequest request) throws YarnException { + FederationReservationHomeSubClusterStoreInputValidator + .validateAddReservationHomeSubClusterRequest(request); + ReservationId reservationId = + request.getReservationHomeSubCluster().getReservationId(); + if (!reservations.containsKey(reservationId)) { + reservations.put(reservationId, + request.getReservationHomeSubCluster().getHomeSubCluster()); + } + return AddReservationHomeSubClusterResponse.newInstance(reservations.get(reservationId)); + } + + @Override + public GetReservationHomeSubClusterResponse getReservationHomeSubCluster( + GetReservationHomeSubClusterRequest request) throws YarnException { + FederationReservationHomeSubClusterStoreInputValidator + .validateGetReservationHomeSubClusterRequest(request); + ReservationId reservationId = request.getReservationId(); + if (!reservations.containsKey(reservationId)) { + throw new YarnException("Reservation " + reservationId + " does not exist"); + } + return GetReservationHomeSubClusterResponse.newInstance( + ReservationHomeSubCluster.newInstance(reservationId, reservations.get(reservationId))); + } + + @Override + public GetReservationsHomeSubClusterResponse getReservationsHomeSubCluster( + GetReservationsHomeSubClusterRequest request) throws YarnException { + List result = new ArrayList<>(); + for (Entry e : reservations.entrySet()) { + result.add(ReservationHomeSubCluster.newInstance(e.getKey(), e.getValue())); + } + GetReservationsHomeSubClusterResponse.newInstance(result); + return GetReservationsHomeSubClusterResponse.newInstance(result); + } } diff --git a/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 b/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 index 2b3fea5609b2e..dfcfb06bb46b8 100644 --- a/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 +++ b/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.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 @@ private static byte[] getByteArray(ByteBuffer bb) { 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"); + } } diff --git a/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 b/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 index c9b5849ad68d2..4b2ff934f0a27 100644 --- a/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 +++ b/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 @@ -65,6 +65,12 @@ 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 +643,22 @@ private static long getCurrentTime() { Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC")); return cal.getTimeInMillis(); } + + @Override + public AddReservationHomeSubClusterResponse addReservationHomeSubCluster( + AddReservationHomeSubClusterRequest request) throws YarnException { + return null; + } + + @Override + public GetReservationHomeSubClusterResponse getReservationHomeSubCluster( + GetReservationHomeSubClusterRequest request) throws YarnException { + return null; + } + + @Override + public GetReservationsHomeSubClusterResponse getReservationsHomeSubCluster( + GetReservationsHomeSubClusterRequest request) throws YarnException { + return null; + } } diff --git a/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 b/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 new file mode 100644 index 0000000000000..e51bce342849c --- /dev/null +++ b/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 + *

+ * 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.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; + +/** + *

+ * The request sent by the Router to Federation state + * store to map the home subcluster of a newly submitted Reservation. + * + *

+ * The request includes the mapping details, i.e.: + *

    + *
  • {@code ReservationId}
  • + *
  • {@code SubClusterId}
  • + *
+ */ +@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); +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..0486d889ac19f --- /dev/null +++ b/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 + *

+ * 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.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(); +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..f82fec6206440 --- /dev/null +++ b/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 + *

+ * 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.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); + +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..a1e1502e8d8b5 --- /dev/null +++ b/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,73 @@ +/** + * 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.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; + +/** + *

+ * The response sent by Federation state + * store to a query for the home subcluster of a newly submitted + * reservation. + * + *

+ * The request includes the mapping details, i.e.: + *

    + *
  • {@code ReservationId}
  • + *
  • {@code SubClusterId}
  • + *
+ */ +@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); +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..c91d25dcd4930 --- /dev/null +++ b/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 + *

+ * 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.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; + } +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..f39430dbe6322 --- /dev/null +++ b/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 + *

+ * 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.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; + +/** + *

+ * The response sent by Federation state + * store to a query for the home subcluster of all submitted + * reservations. + * + *

+ * The response includes the mapping details, i.e.: + *

    + *
  • {@code ReservationId}
  • + *
  • {@code SubClusterId}
  • + *
+ */ +@Private +@Unstable +public abstract class GetReservationsHomeSubClusterResponse { + + @Private + @Unstable + public static GetReservationsHomeSubClusterResponse newInstance( + List 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 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 reservationsHomeSubClusters); +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..98a4ed0e6e1cc --- /dev/null +++ b/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,124 @@ +/** + * 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.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; + +/** + *

+ * ReservationHomeSubCluster is a report of the runtime information of the + * reservation that is running in the federated cluster. + * + *

+ * It includes information such as: + *

    + *
  • {@link ReservationId}
  • + *
  • {@link SubClusterId}
  • + *
+ * + */ +@Private +@Unstable +public abstract class ReservationHomeSubCluster { + + @Private + @Unstable + public static ReservationHomeSubCluster newInstance(ReservationId appId, + SubClusterId homeSubCluster) { + ReservationHomeSubCluster appMapping = + Records.newRecord(ReservationHomeSubCluster.class); + appMapping.setReservationId(appId); + 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 reservationId the reservation identifier + */ + @Private + @Unstable + public abstract void setReservationId(ReservationId reservationId); + + /** + * 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 homeSubCluster the home subcluster identifier + */ + @Private + @Unstable + public abstract void setHomeSubCluster(SubClusterId homeSubCluster); + + @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; + if (!this.getReservationId().equals(other.getReservationId())) { + return false; + } + return this.getHomeSubCluster().equals(other.getHomeSubCluster()); + } + + @Override + public int hashCode() { + return getReservationId().hashCode() * 31 + getHomeSubCluster().hashCode(); + } + + @Override + public String toString() { + return "ReservationHomeSubCluster [getReservationId()=" + + getReservationId() + ", getApplicationHomeSubcluster()=" + getHomeSubCluster() + + "]"; + } + +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..6f7d3cc352e72 --- /dev/null +++ b/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,132 @@ +/** + * 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.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(); + } + +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..6a53abc0ed276 --- /dev/null +++ b/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,117 @@ +/** + * 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.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(); + } + +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..c3ec6d1f152a0 --- /dev/null +++ b/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,139 @@ +/** + * 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.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(); + } +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..70bdab1063f88 --- /dev/null +++ b/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,132 @@ +/** + * 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.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(); + } + +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..8f3b1a298af6c --- /dev/null +++ b/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,78 @@ +/** + * 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.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()); + } + +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..de2a3cf307729 --- /dev/null +++ b/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,190 @@ +/** + * 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.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 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 getAppsHomeSubClusters() { + initSubClustersInfoList(); + return appsHomeSubCluster; + } + + @Override + public void setAppsHomeSubClusters( + List 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 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 iterable = + new Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + + private Iterator 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(); + } + +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..64ef0d762b6d6 --- /dev/null +++ b/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,167 @@ +/** + * 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.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 reservationId; + } + + @Override + public void setReservationId(ReservationId reservationId) { + maybeInitBuilder(); + if (reservationId == null) { + builder.clearReservationId(); + return; + } + this.reservationId = reservationId; + } + + @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 homeSubCluster) { + maybeInitBuilder(); + if (homeSubCluster == null) { + builder.clearHomeSubCluster(); + } + this.homeSubCluster = homeSubCluster; + } + + 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(); + } +} diff --git a/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 b/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 new file mode 100644 index 0000000000000..ac56e6007477b --- /dev/null +++ b/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,134 @@ +/** + * 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 validateAddReservationHomeSubClusterRequest( + 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 validateGetReservationHomeSubClusterRequest( + 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); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java index fa3c8b8d9819f..b2c4c393d0bfd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java +++ b/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.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.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.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,40 @@ public Configuration getConf() { 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 sub cluster 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 +600,7 @@ private static class CacheRequest { private K key; private Func func; - public CacheRequest(K key, Func func) { + CacheRequest(K key, Func func) { this.key = key; this.func = func; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto index 114a60df87b6a..33f5cb3fc14e0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto +++ b/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; diff --git a/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 b/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 index afa46b358cf92..fe1056eb599ea 100644 --- a/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 +++ b/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 @@ -24,7 +24,11 @@ import java.util.Random; import org.apache.hadoop.test.LambdaTestUtils; +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.FederationPolicyUtils; @@ -32,11 +36,14 @@ 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.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 +133,55 @@ public void testNullReservationContext() throws Exception { () -> policy.getReservationHomeSubcluster(null)); } + @Test + public void testUnknownReservation() throws Exception { + ReservationSubmissionRequest resReq = getReservationSubmissionRequest(); + ReservationId reservationId = ReservationId.newInstance(System.currentTimeMillis(), 1); + when(resReq.getQueue()).thenReturn("queue1"); + when(resReq.getReservationId()).thenReturn(reservationId); + + // route an application that uses this app + ApplicationSubmissionContext applicationSubmissionContext = + ApplicationSubmissionContext.newInstance( + ApplicationId.newInstance(System.currentTimeMillis(), 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 { + ReservationSubmissionRequest resReq = getReservationSubmissionRequest(); + when(resReq.getQueue()).thenReturn("queue1"); + when(resReq.getReservationId()) + .thenReturn(ReservationId.newInstance(System.currentTimeMillis(), 1)); + + // first we invoke a reservation placement + SubClusterId chosen = ((FederationRouterPolicy) getPolicy()) + .getReservationHomeSubcluster(resReq); + + // add this to the store + this.getFederationPolicyContext().getFederationStateStoreFacade() + .addReservationHomeSubCluster(ReservationHomeSubCluster.newInstance( + resReq.getReservationId(), chosen)); + + // route an application that uses this app + ApplicationSubmissionContext applicationSubmissionContext = + ApplicationSubmissionContext.newInstance( + ApplicationId.newInstance(System.currentTimeMillis(), 1), "app1", + "queue1", Priority.newInstance(1), null, false, false, 1, null, + null, false); + applicationSubmissionContext.setReservationID(resReq.getReservationId()); + SubClusterId chosen2 = ((FederationRouterPolicy) getPolicy()) + .getHomeSubcluster(applicationSubmissionContext,new ArrayList<>()); + + // application follows reservation + Assert.assertEquals(chosen, chosen2); + } + } diff --git a/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 b/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 index ea03905110273..0b78a10c0a18e 100644 --- a/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 +++ b/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 void setUp() throws Exception { } getPolicyInfo().setRouterPolicyWeights(routerWeights); getPolicyInfo().setAMRMPolicyWeights(amrmWeights); - FederationPoliciesTestUtil.initializePolicyContext(getPolicy(), - getPolicyInfo(), getActiveSubclusters()); + setupContext(); } @Test diff --git a/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 b/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 index a3816b6d08777..b38d6651a60a3 100644 --- a/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 +++ b/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 void testNullQueueRouting() throws YarnException { 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(); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java index c4dae7d4f7d3f..8b77e3877f52c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java +++ b/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.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; @@ -115,10 +121,11 @@ protected void serviceInit(Configuration conf) throws Exception { heartbeatInterval = conf.getLong( YarnConfiguration.FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS, - YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS); + YarnConfiguration + .DEFAULT_FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS); if (heartbeatInterval <= 0) { - heartbeatInterval = - YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS; + heartbeatInterval = YarnConfiguration + .DEFAULT_FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS; } LOG.info("Initialized federation membership service."); @@ -301,4 +308,22 @@ public DeleteApplicationHomeSubClusterResponse deleteApplicationHomeSubCluster( 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); + } } From 8e63615447d74945f2ec2f11287f1b5f2867a29c Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Fri, 5 Aug 2022 18:22:36 -0700 Subject: [PATCH 2/9] YARN-11236. Fix CheckStyle. --- .../policies/router/AbstractRouterPolicy.java | 12 +++++++----- .../store/impl/MemoryFederationStateStore.java | 6 ++---- ...ReservationHomeSubClusterStoreInputValidator.java | 6 ++---- .../policies/router/BaseRouterPoliciesTest.java | 3 ++- 4 files changed, 13 insertions(+), 14 deletions(-) diff --git a/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 b/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 index 473c4cec4814d..3bab20cef1661 100644 --- a/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 +++ b/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 @@ -35,6 +35,7 @@ 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 @@ -103,10 +104,11 @@ protected Map prefilterSubClusters( // note this might throw YarnException if the reservation is // unknown. This is to be expected, and should be handled by // policy invoker. - SubClusterId resSubCluster = getPolicyContext().getFederationStateStoreFacade(). - getReservationHomeSubCluster(reservationId); - - return Collections.singletonMap(resSubCluster, activeSubClusters.get(resSubCluster)); + FederationStateStoreFacade stateStoreFacade = + getPolicyContext().getFederationStateStoreFacade(); + SubClusterId resSubCluster = stateStoreFacade.getReservationHomeSubCluster(reservationId); + SubClusterInfo subClusterInfo = activeSubClusters.get(resSubCluster); + return Collections.singletonMap(resSubCluster, subClusterInfo); } return activeSubClusters; @@ -139,7 +141,7 @@ public SubClusterId getHomeSubcluster(ApplicationSubmissionContext appContext, // apply filtering based on reservation location and active sub-clusters Map filteredSubClusters = prefilterSubClusters( - appContext.getReservationID(), getActiveSubclusters()); + appContext.getReservationID(), getActiveSubclusters()); FederationPolicyUtils.validateSubClusterAvailability(filteredSubClusters.keySet(), blackLists); diff --git a/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 b/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 index e667a393829e5..f1cb58aa7af6d 100644 --- a/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 +++ b/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 @@ -327,8 +327,7 @@ public Version loadVersion() { @Override public AddReservationHomeSubClusterResponse addReservationHomeSubCluster( AddReservationHomeSubClusterRequest request) throws YarnException { - FederationReservationHomeSubClusterStoreInputValidator - .validateAddReservationHomeSubClusterRequest(request); + FederationReservationHomeSubClusterStoreInputValidator.validate(request); ReservationId reservationId = request.getReservationHomeSubCluster().getReservationId(); if (!reservations.containsKey(reservationId)) { @@ -341,8 +340,7 @@ public AddReservationHomeSubClusterResponse addReservationHomeSubCluster( @Override public GetReservationHomeSubClusterResponse getReservationHomeSubCluster( GetReservationHomeSubClusterRequest request) throws YarnException { - FederationReservationHomeSubClusterStoreInputValidator - .validateGetReservationHomeSubClusterRequest(request); + FederationReservationHomeSubClusterStoreInputValidator.validate(request); ReservationId reservationId = request.getReservationId(); if (!reservations.containsKey(reservationId)) { throw new YarnException("Reservation " + reservationId + " does not exist"); diff --git a/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 b/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 index ac56e6007477b..1647e180c1fe2 100644 --- a/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 +++ b/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 @@ -49,8 +49,7 @@ private FederationReservationHomeSubClusterStoreInputValidator() { * against * @throws FederationStateStoreInvalidInputException if the request is invalid */ - public static void validateAddReservationHomeSubClusterRequest( - AddReservationHomeSubClusterRequest request) + public static void validate(AddReservationHomeSubClusterRequest request) throws FederationStateStoreInvalidInputException { if (request == null) { String message = "Missing AddReservationHomeSubCluster Request." @@ -73,8 +72,7 @@ public static void validateAddReservationHomeSubClusterRequest( * against * @throws FederationStateStoreInvalidInputException if the request is invalid */ - public static void validateGetReservationHomeSubClusterRequest( - GetReservationHomeSubClusterRequest request) + public static void validate(GetReservationHomeSubClusterRequest request) throws FederationStateStoreInvalidInputException { if (request == null) { String message = "Missing GetReservationHomeSubCluster Request." diff --git a/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 b/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 index fe1056eb599ea..9bc15ae1f85bd 100644 --- a/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 +++ b/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,6 +19,7 @@ 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; @@ -178,7 +179,7 @@ public void testFollowReservation() throws YarnException { null, false); applicationSubmissionContext.setReservationID(resReq.getReservationId()); SubClusterId chosen2 = ((FederationRouterPolicy) getPolicy()) - .getHomeSubcluster(applicationSubmissionContext,new ArrayList<>()); + .getHomeSubcluster(applicationSubmissionContext, Collections.emptyList()); // application follows reservation Assert.assertEquals(chosen, chosen2); From 41da933f3e38150a114dbbfea1ec826fde60aeba Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Sat, 6 Aug 2022 03:17:14 -0700 Subject: [PATCH 3/9] YARN-11236. Fix CheckStyle. --- ...rationReservationHomeSubClusterStoreInputValidator.java | 3 +-- .../federation/FederationStateStoreService.java | 7 +++---- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/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 b/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 index 1647e180c1fe2..c23ec1ef0299a 100644 --- a/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 +++ b/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 @@ -123,8 +123,7 @@ private static void checkReservationHomeSubCluster( private static void checkReservationId(ReservationId reservationId) throws FederationStateStoreInvalidInputException { if (reservationId == null) { - String message = "Missing ReservationId." - + " Please try again by specifying an ReservationId."; + String message = "Missing ReservationId. Please try again by specifying an ReservationId."; LOG.warn(message); throw new FederationStateStoreInvalidInputException(message); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java index 8b77e3877f52c..d2d69231f5ef6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java @@ -121,11 +121,10 @@ protected void serviceInit(Configuration conf) throws Exception { heartbeatInterval = conf.getLong( YarnConfiguration.FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS, - YarnConfiguration - .DEFAULT_FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS); + YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS); if (heartbeatInterval <= 0) { - heartbeatInterval = YarnConfiguration - .DEFAULT_FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS; + heartbeatInterval = + YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS; } LOG.info("Initialized federation membership service."); From 294f8b2b71290f595043b5457b6930b0981f205e Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Sun, 7 Aug 2022 00:23:08 -0700 Subject: [PATCH 4/9] YARN-11236. Fix Code Style. --- .../impl/MemoryFederationStateStore.java | 14 +++++-- .../impl/ZookeeperFederationStateStore.java | 7 ++-- .../records/ReservationHomeSubCluster.java | 16 +++++--- .../utils/FederationStateStoreFacade.java | 5 +-- .../router/BaseRouterPoliciesTest.java | 37 ++++++++++++------- 5 files changed, 50 insertions(+), 29 deletions(-) diff --git a/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 b/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 index f1cb58aa7af6d..7471444f2bac1 100644 --- a/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 +++ b/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 @@ -345,18 +345,24 @@ public GetReservationHomeSubClusterResponse getReservationHomeSubCluster( if (!reservations.containsKey(reservationId)) { throw new YarnException("Reservation " + reservationId + " does not exist"); } + SubClusterId subClusterId = reservations.get(reservationId); return GetReservationHomeSubClusterResponse.newInstance( - ReservationHomeSubCluster.newInstance(reservationId, reservations.get(reservationId))); + ReservationHomeSubCluster.newInstance(reservationId, subClusterId)); } @Override public GetReservationsHomeSubClusterResponse getReservationsHomeSubCluster( GetReservationsHomeSubClusterRequest request) throws YarnException { List result = new ArrayList<>(); - for (Entry e : reservations.entrySet()) { - result.add(ReservationHomeSubCluster.newInstance(e.getKey(), e.getValue())); + + for (Entry entry : reservations.entrySet()) { + ReservationId key = entry.getKey(); + SubClusterId value = entry.getValue(); + ReservationHomeSubCluster homeSubCluster = + ReservationHomeSubCluster.newInstance(key, value); + result.add(homeSubCluster); } - GetReservationsHomeSubClusterResponse.newInstance(result); + return GetReservationsHomeSubClusterResponse.newInstance(result); } } diff --git a/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 b/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 index 4b2ff934f0a27..686ba077c7da5 100644 --- a/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 +++ b/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.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; @@ -647,18 +648,18 @@ private static long getCurrentTime() { @Override public AddReservationHomeSubClusterResponse addReservationHomeSubCluster( AddReservationHomeSubClusterRequest request) throws YarnException { - return null; + throw new NotImplementedException("Code is not implemented"); } @Override public GetReservationHomeSubClusterResponse getReservationHomeSubCluster( GetReservationHomeSubClusterRequest request) throws YarnException { - return null; + throw new NotImplementedException("Code is not implemented"); } @Override public GetReservationsHomeSubClusterResponse getReservationsHomeSubCluster( GetReservationsHomeSubClusterRequest request) throws YarnException { - return null; + throw new NotImplementedException("Code is not implemented"); } } diff --git a/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 b/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 index 98a4ed0e6e1cc..a107633df8817 100644 --- a/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 +++ b/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 @@ -17,6 +17,8 @@ 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; @@ -103,15 +105,19 @@ public boolean equals(Object obj) { return false; } ReservationHomeSubCluster other = (ReservationHomeSubCluster) obj; - if (!this.getReservationId().equals(other.getReservationId())) { - return false; - } - return this.getHomeSubCluster().equals(other.getHomeSubCluster()); + + return new EqualsBuilder() + .append(this.getReservationId(), other.getReservationId()) + .append(this.getHomeSubCluster(), other.getHomeSubCluster()) + .isEquals(); } @Override public int hashCode() { - return getReservationId().hashCode() * 31 + getHomeSubCluster().hashCode(); + return new HashCodeBuilder(). + append(this.getReservationId()). + append(this.getHomeSubCluster()). + toHashCode(); } @Override diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java index b2c4c393d0bfd..c628008f5be14 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java @@ -434,9 +434,8 @@ public SubClusterId addReservationHomeSubCluster( */ public SubClusterId getReservationHomeSubCluster(ReservationId reservationId) throws YarnException { - GetReservationHomeSubClusterResponse response = - stateStore.getReservationHomeSubCluster( - GetReservationHomeSubClusterRequest.newInstance(reservationId)); + GetReservationHomeSubClusterResponse response = stateStore.getReservationHomeSubCluster( + GetReservationHomeSubClusterRequest.newInstance(reservationId)); return response.getReservationHomeSubCluster().getHomeSubCluster(); } diff --git a/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 b/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 index 9bc15ae1f85bd..b930a9d979de4 100644 --- a/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 +++ b/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,6 +25,7 @@ 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; @@ -39,6 +40,7 @@ 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; @@ -136,16 +138,18 @@ public void testNullReservationContext() throws Exception { @Test public void testUnknownReservation() throws Exception { + + long now = Time.now(); ReservationSubmissionRequest resReq = getReservationSubmissionRequest(); - ReservationId reservationId = ReservationId.newInstance(System.currentTimeMillis(), 1); + 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(System.currentTimeMillis(), 1), "app1", - "queue1", Priority.newInstance(1), null, false, false, 1, null, null, false); + ApplicationId.newInstance(now, 1), "app1", "queue1", Priority.newInstance(1), + null, false, false, 1, null, null, false); applicationSubmissionContext.setReservationID(resReq.getReservationId()); FederationRouterPolicy policy = (FederationRouterPolicy) getPolicy(); @@ -157,29 +161,34 @@ public void testUnknownReservation() throws Exception { @Test public void testFollowReservation() throws YarnException { + + long now = Time.now(); ReservationSubmissionRequest resReq = getReservationSubmissionRequest(); when(resReq.getQueue()).thenReturn("queue1"); when(resReq.getReservationId()) - .thenReturn(ReservationId.newInstance(System.currentTimeMillis(), 1)); + .thenReturn(ReservationId.newInstance(now, 1)); + + FederationRouterPolicy routerPolicy = (FederationRouterPolicy) getPolicy(); + FederationStateStoreFacade storeFacade = + getFederationPolicyContext().getFederationStateStoreFacade(); // first we invoke a reservation placement - SubClusterId chosen = ((FederationRouterPolicy) getPolicy()) - .getReservationHomeSubcluster(resReq); + SubClusterId chosen = routerPolicy.getReservationHomeSubcluster(resReq); // add this to the store - this.getFederationPolicyContext().getFederationStateStoreFacade() - .addReservationHomeSubCluster(ReservationHomeSubCluster.newInstance( - resReq.getReservationId(), chosen)); + ReservationHomeSubCluster homeSubCluster = + ReservationHomeSubCluster.newInstance(resReq.getReservationId(), chosen); + storeFacade.addReservationHomeSubCluster(homeSubCluster); // route an application that uses this app ApplicationSubmissionContext applicationSubmissionContext = ApplicationSubmissionContext.newInstance( - ApplicationId.newInstance(System.currentTimeMillis(), 1), "app1", - "queue1", Priority.newInstance(1), null, false, false, 1, null, - null, false); + ApplicationId.newInstance(now, 1), "app1", "queue1", Priority.newInstance(1), + null, false, false, 1, null, null, false); + applicationSubmissionContext.setReservationID(resReq.getReservationId()); - SubClusterId chosen2 = ((FederationRouterPolicy) getPolicy()) - .getHomeSubcluster(applicationSubmissionContext, Collections.emptyList()); + SubClusterId chosen2 = routerPolicy.getHomeSubcluster(applicationSubmissionContext, + Collections.emptyList()); // application follows reservation Assert.assertEquals(chosen, chosen2); From e0050456d8ec10de45ce8ad24618b0ee7e5d4611 Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Wed, 10 Aug 2022 07:39:29 -0700 Subject: [PATCH 5/9] YARN-11236. Fix CheckStyle. --- .../store/impl/MemoryFederationStateStore.java | 18 +++++++++--------- .../records/ReservationHomeSubCluster.java | 11 +++++------ .../pb/ReservationHomeSubClusterPBImpl.java | 15 ++++++++------- .../router/BaseRouterPoliciesTest.java | 3 +-- 4 files changed, 23 insertions(+), 24 deletions(-) diff --git a/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 b/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 index 7471444f2bac1..24ed54d63668b 100644 --- a/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 +++ b/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 @@ -328,11 +328,10 @@ public Version loadVersion() { public AddReservationHomeSubClusterResponse addReservationHomeSubCluster( AddReservationHomeSubClusterRequest request) throws YarnException { FederationReservationHomeSubClusterStoreInputValidator.validate(request); - ReservationId reservationId = - request.getReservationHomeSubCluster().getReservationId(); + ReservationHomeSubCluster homeSubCluster = request.getReservationHomeSubCluster(); + ReservationId reservationId = homeSubCluster.getReservationId(); if (!reservations.containsKey(reservationId)) { - reservations.put(reservationId, - request.getReservationHomeSubCluster().getHomeSubCluster()); + reservations.put(reservationId, homeSubCluster.getHomeSubCluster()); } return AddReservationHomeSubClusterResponse.newInstance(reservations.get(reservationId)); } @@ -346,8 +345,9 @@ public GetReservationHomeSubClusterResponse getReservationHomeSubCluster( throw new YarnException("Reservation " + reservationId + " does not exist"); } SubClusterId subClusterId = reservations.get(reservationId); - return GetReservationHomeSubClusterResponse.newInstance( - ReservationHomeSubCluster.newInstance(reservationId, subClusterId)); + ReservationHomeSubCluster homeSubCluster = + ReservationHomeSubCluster.newInstance(reservationId, subClusterId); + return GetReservationHomeSubClusterResponse.newInstance(homeSubCluster); } @Override @@ -356,10 +356,10 @@ public GetReservationsHomeSubClusterResponse getReservationsHomeSubCluster( List result = new ArrayList<>(); for (Entry entry : reservations.entrySet()) { - ReservationId key = entry.getKey(); - SubClusterId value = entry.getValue(); + ReservationId reservationId = entry.getKey(); + SubClusterId subClusterId = entry.getValue(); ReservationHomeSubCluster homeSubCluster = - ReservationHomeSubCluster.newInstance(key, value); + ReservationHomeSubCluster.newInstance(reservationId, subClusterId); result.add(homeSubCluster); } diff --git a/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 b/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 index a107633df8817..6606b4ae4037c 100644 --- a/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 +++ b/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 @@ -46,8 +46,7 @@ public abstract class ReservationHomeSubCluster { @Unstable public static ReservationHomeSubCluster newInstance(ReservationId appId, SubClusterId homeSubCluster) { - ReservationHomeSubCluster appMapping = - Records.newRecord(ReservationHomeSubCluster.class); + ReservationHomeSubCluster appMapping = Records.newRecord(ReservationHomeSubCluster.class); appMapping.setReservationId(appId); appMapping.setHomeSubCluster(homeSubCluster); return appMapping; @@ -67,11 +66,11 @@ public static ReservationHomeSubCluster newInstance(ReservationId appId, * Set the {@link ReservationId} representing the unique identifier of the * Reservation. * - * @param reservationId the reservation identifier + * @param resId the reservation identifier */ @Private @Unstable - public abstract void setReservationId(ReservationId reservationId); + public abstract void setReservationId(ReservationId resId); /** * Get the {@link SubClusterId} representing the unique identifier of the home @@ -87,11 +86,11 @@ public static ReservationHomeSubCluster newInstance(ReservationId appId, * Set the {@link SubClusterId} representing the unique identifier of the home * subcluster in which the ReservationMaster of the reservation is running. * - * @param homeSubCluster the home subcluster identifier + * @param subClusterId the home subcluster identifier */ @Private @Unstable - public abstract void setHomeSubCluster(SubClusterId homeSubCluster); + public abstract void setHomeSubCluster(SubClusterId subClusterId); @Override public boolean equals(Object obj) { diff --git a/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 b/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 index 64ef0d762b6d6..ba7f9d185ffca 100644 --- a/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 +++ b/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 @@ -114,17 +114,17 @@ public ReservationId getReservationId() { return null; } this.reservationId = convertFromProtoFormat(p.getReservationId()); - return reservationId; + return this.reservationId; } @Override - public void setReservationId(ReservationId reservationId) { + public void setReservationId(ReservationId resId) { maybeInitBuilder(); - if (reservationId == null) { + if (resId == null) { builder.clearReservationId(); return; } - this.reservationId = reservationId; + this.reservationId = resId; } @Override @@ -141,12 +141,13 @@ public SubClusterId getHomeSubCluster() { } @Override - public void setHomeSubCluster(SubClusterId homeSubCluster) { + public void setHomeSubCluster(SubClusterId subClusterId) { maybeInitBuilder(); - if (homeSubCluster == null) { + if (subClusterId == null) { builder.clearHomeSubCluster(); + return; } - this.homeSubCluster = homeSubCluster; + this.homeSubCluster = subClusterId; } private SubClusterId convertFromProtoFormat(SubClusterIdProto subClusterId) { diff --git a/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 b/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 index b930a9d979de4..5f77d4575f559 100644 --- a/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 +++ b/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 @@ -165,8 +165,7 @@ 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)); + when(resReq.getReservationId()).thenReturn(ReservationId.newInstance(now, 1)); FederationRouterPolicy routerPolicy = (FederationRouterPolicy) getPolicy(); FederationStateStoreFacade storeFacade = From 51d243c2aa430472a03272a5c628defeaaad4cf6 Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Wed, 10 Aug 2022 07:57:49 -0700 Subject: [PATCH 6/9] YARN-11236. Fix CheckStyle. --- .../GetReservationHomeSubClusterResponse.java | 3 +-- .../records/ReservationHomeSubCluster.java | 4 ++-- ...servationsHomeSubClusterResponsePBImpl.java | 15 ++++----------- ...ationHomeSubClusterStoreInputValidator.java | 18 ++++++------------ .../utils/FederationStateStoreFacade.java | 16 ++++++---------- .../router/BaseRouterPoliciesTest.java | 9 +++++---- 6 files changed, 24 insertions(+), 41 deletions(-) diff --git a/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 b/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 index a1e1502e8d8b5..50328a176b100 100644 --- a/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 +++ b/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 @@ -63,8 +63,7 @@ public static GetReservationHomeSubClusterResponse newInstance( * 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. + * @param reservationHomeSubCluster the mapping of the reservation to it's home sub-cluster. */ @Private @Unstable diff --git a/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 b/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 index 6606b4ae4037c..e080d115716dd 100644 --- a/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 +++ b/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 @@ -44,10 +44,10 @@ public abstract class ReservationHomeSubCluster { @Private @Unstable - public static ReservationHomeSubCluster newInstance(ReservationId appId, + public static ReservationHomeSubCluster newInstance(ReservationId resId, SubClusterId homeSubCluster) { ReservationHomeSubCluster appMapping = Records.newRecord(ReservationHomeSubCluster.class); - appMapping.setReservationId(appId); + appMapping.setReservationId(resId); appMapping.setHomeSubCluster(homeSubCluster); return appMapping; } diff --git a/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 b/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 index de2a3cf307729..19fd5c0e8af1d 100644 --- a/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 +++ b/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 @@ -128,11 +128,9 @@ private void initSubClustersInfoList() { if (this.appsHomeSubCluster != null) { return; } - GetReservationsHomeSubClusterResponseProtoOrBuilder p = - viaProto ? proto : builder; - List subClusterInfosList = - p.getAppSubclusterMapList(); - appsHomeSubCluster = new ArrayList(); + GetReservationsHomeSubClusterResponseProtoOrBuilder p = viaProto ? proto : builder; + List subClusterInfosList = p.getAppSubclusterMapList(); + appsHomeSubCluster = new ArrayList<>(); for (ReservationHomeSubClusterProto r : subClusterInfosList) { appsHomeSubCluster.add(convertFromProtoFormat(r)); @@ -151,8 +149,7 @@ private void addSubClustersInfoToProto() { public Iterator iterator() { return new Iterator() { - private Iterator iter = - appsHomeSubCluster.iterator(); + private Iterator iter = appsHomeSubCluster.iterator(); @Override public boolean hasNext() { @@ -168,11 +165,8 @@ public ReservationHomeSubClusterProto next() { public void remove() { throw new UnsupportedOperationException(); } - }; - } - }; builder.addAllAppSubclusterMap(iterable); } @@ -186,5 +180,4 @@ private ReservationHomeSubClusterProto convertToProtoFormat( ReservationHomeSubCluster sc) { return ((ReservationHomeSubClusterPBImpl) sc).getProto(); } - } diff --git a/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 b/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 index c23ec1ef0299a..ff6ed9aeb0600 100644 --- a/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 +++ b/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 @@ -45,8 +45,7 @@ private FederationReservationHomeSubClusterStoreInputValidator() { * fast). Check if the provided {@link AddReservationHomeSubClusterRequest} * for adding a new reservation is valid or not. * - * @param request the {@link AddReservationHomeSubClusterRequest} to validate - * against + * @param request the {@link AddReservationHomeSubClusterRequest} to validate against * @throws FederationStateStoreInvalidInputException if the request is invalid */ public static void validate(AddReservationHomeSubClusterRequest request) @@ -68,8 +67,7 @@ public static void validate(AddReservationHomeSubClusterRequest request) * fast). Check if the provided {@link GetReservationHomeSubClusterRequest} * for querying reservation's information is valid or not. * - * @param request the {@link GetReservationHomeSubClusterRequest} to validate - * against + * @param request the {@link GetReservationHomeSubClusterRequest} to validate against * @throws FederationStateStoreInvalidInputException if the request is invalid */ public static void validate(GetReservationHomeSubClusterRequest request) @@ -88,14 +86,11 @@ public static void validate(GetReservationHomeSubClusterRequest request) /** * 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 + * @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." @@ -104,21 +99,20 @@ private static void checkReservationHomeSubCluster( 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 + * @throws FederationStateStoreInvalidInputException if the Reservation Id is invalid */ private static void checkReservationId(ReservationId reservationId) throws FederationStateStoreInvalidInputException { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java index c628008f5be14..ebde56d2b1da8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java @@ -406,30 +406,26 @@ public Configuration getConf() { 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 + * @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( + 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}. + * Returns the home {@link SubClusterId} for the specified {@link ReservationId}. * * @param reservationId the identifier of the reservation - * @return the home sub cluster identifier + * @return the home subCluster identifier * @throws YarnException if the call to the state store is unsuccessful */ public SubClusterId getReservationHomeSubCluster(ReservationId reservationId) diff --git a/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 b/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 index 5f77d4575f559..26a70a534a1b8 100644 --- a/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 +++ b/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 @@ -33,6 +33,7 @@ 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; @@ -168,8 +169,8 @@ public void testFollowReservation() throws YarnException { when(resReq.getReservationId()).thenReturn(ReservationId.newInstance(now, 1)); FederationRouterPolicy routerPolicy = (FederationRouterPolicy) getPolicy(); - FederationStateStoreFacade storeFacade = - getFederationPolicyContext().getFederationStateStoreFacade(); + FederationPolicyInitializationContext fdContext = getFederationPolicyContext(); + FederationStateStoreFacade storeFacade = fdContext.getFederationStateStoreFacade(); // first we invoke a reservation placement SubClusterId chosen = routerPolicy.getReservationHomeSubcluster(resReq); @@ -186,8 +187,8 @@ public void testFollowReservation() throws YarnException { null, false, false, 1, null, null, false); applicationSubmissionContext.setReservationID(resReq.getReservationId()); - SubClusterId chosen2 = routerPolicy.getHomeSubcluster(applicationSubmissionContext, - Collections.emptyList()); + SubClusterId chosen2 = routerPolicy.getHomeSubcluster( + applicationSubmissionContext, Collections.emptyList()); // application follows reservation Assert.assertEquals(chosen, chosen2); From 1e5aa27e31c5cb429c595d2df323b9b80668be75 Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Wed, 10 Aug 2022 08:06:45 -0700 Subject: [PATCH 7/9] YARN-11236. Fix CheckStyle. --- .../impl/pb/AddReservationHomeSubClusterRequestPBImpl.java | 4 +--- .../impl/pb/AddReservationHomeSubClusterResponsePBImpl.java | 4 +--- .../impl/pb/GetReservationHomeSubClusterRequestPBImpl.java | 3 +-- .../impl/pb/GetReservationHomeSubClusterResponsePBImpl.java | 4 +--- .../impl/pb/GetReservationsHomeSubClusterRequestPBImpl.java | 1 - .../pb/GetReservationsHomeSubClusterResponsePBImpl.java | 6 ++---- 6 files changed, 6 insertions(+), 16 deletions(-) diff --git a/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 b/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 index 6f7d3cc352e72..e6facbec9c937 100644 --- a/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 +++ b/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 @@ -100,8 +100,7 @@ public String toString() { @Override public ReservationHomeSubCluster getReservationHomeSubCluster() { - AddReservationHomeSubClusterRequestProtoOrBuilder p = - viaProto ? proto : builder; + AddReservationHomeSubClusterRequestProtoOrBuilder p = viaProto ? proto : builder; if (!p.hasAppSubclusterMap()) { return null; } @@ -128,5 +127,4 @@ private ReservationHomeSubClusterProto convertToProtoFormat( ReservationHomeSubCluster sc) { return ((ReservationHomeSubClusterPBImpl) sc).getProto(); } - } diff --git a/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 b/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 index 6a53abc0ed276..e64e740634dec 100644 --- a/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 +++ b/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 @@ -76,8 +76,7 @@ public void setHomeSubCluster(SubClusterId homeSubCluster) { @Override public SubClusterId getHomeSubCluster() { - AddReservationHomeSubClusterResponseProtoOrBuilder p = - viaProto ? proto : builder; + AddReservationHomeSubClusterResponseProtoOrBuilder p = viaProto ? proto : builder; if (!p.hasHomeSubCluster()) { return null; @@ -113,5 +112,4 @@ private SubClusterId convertFromProtoFormat(SubClusterIdProto sc) { private SubClusterIdProto convertToProtoFormat(SubClusterId sc) { return ((SubClusterIdPBImpl) sc).getProto(); } - } diff --git a/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 b/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 index c3ec6d1f152a0..d3011d84a65f3 100644 --- a/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 +++ b/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 @@ -106,8 +106,7 @@ public String toString() { @Override public ReservationId getReservationId() { - GetReservationHomeSubClusterRequestProtoOrBuilder p = - viaProto ? proto : builder; + GetReservationHomeSubClusterRequestProtoOrBuilder p = viaProto ? proto : builder; if (reservationId != null) { return reservationId; } diff --git a/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 b/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 index 70bdab1063f88..f72b61b650dcb 100644 --- a/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 +++ b/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 @@ -100,8 +100,7 @@ public String toString() { @Override public ReservationHomeSubCluster getReservationHomeSubCluster() { - GetReservationHomeSubClusterResponseProtoOrBuilder p = - viaProto ? proto : builder; + GetReservationHomeSubClusterResponseProtoOrBuilder p = viaProto ? proto : builder; if (!p.hasAppSubclusterMap()) { return null; } @@ -128,5 +127,4 @@ private ReservationHomeSubClusterProto convertToProtoFormat( ReservationHomeSubCluster sc) { return ((ReservationHomeSubClusterPBImpl) sc).getProto(); } - } diff --git a/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 b/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 index 8f3b1a298af6c..889941ede1b68 100644 --- a/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 +++ b/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 @@ -74,5 +74,4 @@ public boolean equals(Object other) { public String toString() { return TextFormat.shortDebugString(getProto()); } - } diff --git a/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 b/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 index 19fd5c0e8af1d..c380997fa1db9 100644 --- a/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 +++ b/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 @@ -171,13 +171,11 @@ public void remove() { builder.addAllAppSubclusterMap(iterable); } - private ReservationHomeSubCluster convertFromProtoFormat( - ReservationHomeSubClusterProto sc) { + private ReservationHomeSubCluster convertFromProtoFormat(ReservationHomeSubClusterProto sc) { return new ReservationHomeSubClusterPBImpl(sc); } - private ReservationHomeSubClusterProto convertToProtoFormat( - ReservationHomeSubCluster sc) { + private ReservationHomeSubClusterProto convertToProtoFormat(ReservationHomeSubCluster sc) { return ((ReservationHomeSubClusterPBImpl) sc).getProto(); } } From cbf67f5d5bee94ef2d42ab99bff3cf2ef5eb6fee Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Fri, 12 Aug 2022 18:14:32 -0700 Subject: [PATCH 8/9] YARN-11236. Fix Proto Warning. --- .../hadoop-yarn-server-common/pom.xml | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml index 3bf70bf7560e5..c023ffa72c544 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml @@ -260,6 +260,20 @@ + + maven-compiler-plugin + + + org/apache/hadoop/yarn/federation/proto/YarnServerFederationProtos.java + org/apache/hadoop/yarn/proto/CollectorNodemanagerProtocol.java + org/apache/hadoop/yarn/proto/DistributedSchedulingAMProtocol.java + org/apache/hadoop/yarn/proto/ResourceTracker.java + org/apache/hadoop/yarn/proto/SCMUploaderProtocol.java + org/apache/hadoop/yarn/proto/YarnServerCommonProtos.java + org/apache/hadoop/yarn/proto/YarnServerCommonServiceProtos.java + + + From 360acc4141a9a99459595fac0272151115c7be70 Mon Sep 17 00:00:00 2001 From: slfan1989 Date: Fri, 12 Aug 2022 22:12:30 -0700 Subject: [PATCH 9/9] Revert "YARN-11236. Fix Proto Warning." This reverts commit cbf67f5d5bee94ef2d42ab99bff3cf2ef5eb6fee. --- .../hadoop-yarn-server-common/pom.xml | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml index c023ffa72c544..3bf70bf7560e5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml @@ -260,20 +260,6 @@ - - maven-compiler-plugin - - - org/apache/hadoop/yarn/federation/proto/YarnServerFederationProtos.java - org/apache/hadoop/yarn/proto/CollectorNodemanagerProtocol.java - org/apache/hadoop/yarn/proto/DistributedSchedulingAMProtocol.java - org/apache/hadoop/yarn/proto/ResourceTracker.java - org/apache/hadoop/yarn/proto/SCMUploaderProtocol.java - org/apache/hadoop/yarn/proto/YarnServerCommonProtos.java - org/apache/hadoop/yarn/proto/YarnServerCommonServiceProtos.java - - -