|
@@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
|
|
@@ -136,7 +137,7 @@ public class RouterPolicyFacade {
|
|
|
|
|
|
if (appSubmissionContext == null) {
|
|
|
throw new FederationPolicyException(
|
|
|
- "The ApplicationSubmissionContext " + "cannot be null.");
|
|
|
+ "The ApplicationSubmissionContext cannot be null.");
|
|
|
}
|
|
|
|
|
|
String queue = appSubmissionContext.getQueue();
|
|
@@ -148,51 +149,7 @@ public class RouterPolicyFacade {
|
|
|
queue = YarnConfiguration.DEFAULT_QUEUE_NAME;
|
|
|
}
|
|
|
|
|
|
- // the facade might cache this request, based on its parameterization
|
|
|
- SubClusterPolicyConfiguration configuration = null;
|
|
|
-
|
|
|
- try {
|
|
|
- configuration = federationFacade.getPolicyConfiguration(queue);
|
|
|
- } catch (YarnException e) {
|
|
|
- String errMsg = "There is no policy configured for the queue: " + queue
|
|
|
- + ", falling back to defaults.";
|
|
|
- LOG.warn(errMsg, e);
|
|
|
- }
|
|
|
-
|
|
|
- // If there is no policy configured for this queue, fallback to the baseline
|
|
|
- // policy that is configured either in the store or via XML config (and
|
|
|
- // cached)
|
|
|
- if (configuration == null) {
|
|
|
- LOG.warn("There is no policies configured for queue: " + queue + " we"
|
|
|
- + " fallback to default policy for: "
|
|
|
- + YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY);
|
|
|
-
|
|
|
- queue = YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY;
|
|
|
- try {
|
|
|
- configuration = federationFacade.getPolicyConfiguration(queue);
|
|
|
- } catch (YarnException e) {
|
|
|
- String errMsg = "Cannot retrieve policy configured for the queue: "
|
|
|
- + queue + ", falling back to defaults.";
|
|
|
- LOG.warn(errMsg, e);
|
|
|
-
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // the fallback is not configure via store, but via XML, using
|
|
|
- // previously loaded configuration.
|
|
|
- if (configuration == null) {
|
|
|
- configuration =
|
|
|
- cachedConfs.get(YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY);
|
|
|
- }
|
|
|
-
|
|
|
- // if the configuration has changed since last loaded, reinit the policy
|
|
|
- // based on current configuration
|
|
|
- if (!cachedConfs.containsKey(queue)
|
|
|
- || !cachedConfs.get(queue).equals(configuration)) {
|
|
|
- singlePolicyReinit(policyMap, cachedConfs, queue, configuration);
|
|
|
- }
|
|
|
-
|
|
|
- FederationRouterPolicy policy = policyMap.get(queue);
|
|
|
+ FederationRouterPolicy policy = getFederationRouterPolicy(cachedConfs, policyMap, queue);
|
|
|
if (policy == null) {
|
|
|
// this should never happen, as the to maps are updated together
|
|
|
throw new FederationPolicyException("No FederationRouterPolicy found "
|
|
@@ -262,4 +219,92 @@ public class RouterPolicyFacade {
|
|
|
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * This method provides a wrapper of all policy functionalities for routing a
|
|
|
+ * reservation. Internally it manages configuration changes, and policy
|
|
|
+ * init/reinit.
|
|
|
+ *
|
|
|
+ * @param request the reservation to route.
|
|
|
+ *
|
|
|
+ * @return the id of the subcluster that will be the "home" for this
|
|
|
+ * reservation.
|
|
|
+ *
|
|
|
+ * @throws YarnException if there are issues initializing policies, or no
|
|
|
+ * valid sub-cluster id could be found for this reservation.
|
|
|
+ */
|
|
|
+ public SubClusterId getReservationHomeSubCluster(
|
|
|
+ ReservationSubmissionRequest request) throws YarnException {
|
|
|
+
|
|
|
+ // the maps are concurrent, but we need to protect from reset()
|
|
|
+ // reinitialization mid-execution by creating a new reference local to this
|
|
|
+ // method.
|
|
|
+ Map<String, SubClusterPolicyConfiguration> cachedConfs = globalConfMap;
|
|
|
+ Map<String, FederationRouterPolicy> policyMap = globalPolicyMap;
|
|
|
+
|
|
|
+ if (request == null) {
|
|
|
+ throw new FederationPolicyException(
|
|
|
+ "The ReservationSubmissionRequest cannot be null.");
|
|
|
+ }
|
|
|
+
|
|
|
+ String queue = request.getQueue();
|
|
|
+ FederationRouterPolicy policy = getFederationRouterPolicy(cachedConfs, policyMap, queue);
|
|
|
+
|
|
|
+ if (policy == null) {
|
|
|
+ // this should never happen, as the to maps are updated together
|
|
|
+ throw new FederationPolicyException("No FederationRouterPolicy found "
|
|
|
+ + "for queue: " + request.getQueue() + " (while routing "
|
|
|
+ + "reservation: " + request.getReservationId() + ") "
|
|
|
+ + "and no default specified.");
|
|
|
+ }
|
|
|
+
|
|
|
+ return policy.getReservationHomeSubcluster(request);
|
|
|
+ }
|
|
|
+
|
|
|
+ private FederationRouterPolicy getFederationRouterPolicy(
|
|
|
+ Map<String, SubClusterPolicyConfiguration> cachedConfiguration,
|
|
|
+ Map<String, FederationRouterPolicy> policyMap, String queue)
|
|
|
+ throws FederationPolicyInitializationException {
|
|
|
+
|
|
|
+ // the facade might cache this request, based on its parameterization
|
|
|
+ SubClusterPolicyConfiguration configuration = null;
|
|
|
+ String copyQueue = queue;
|
|
|
+
|
|
|
+ try {
|
|
|
+ configuration = federationFacade.getPolicyConfiguration(copyQueue);
|
|
|
+ } catch (YarnException e) {
|
|
|
+ LOG.warn("There is no policy configured for the queue: {}, falling back to defaults.",
|
|
|
+ copyQueue, e);
|
|
|
+ }
|
|
|
+
|
|
|
+ // If there is no policy configured for this queue, fallback to the baseline
|
|
|
+ // policy that is configured either in the store or via XML config (and cached)
|
|
|
+ if (configuration == null) {
|
|
|
+ final String policyKey = YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY;
|
|
|
+ LOG.warn("There is no policies configured for queue: {} " +
|
|
|
+ "we fallback to default policy for: {}. ", copyQueue, policyKey);
|
|
|
+ copyQueue = YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY;
|
|
|
+ try {
|
|
|
+ configuration = federationFacade.getPolicyConfiguration(copyQueue);
|
|
|
+ } catch (YarnException e) {
|
|
|
+ LOG.warn("Cannot retrieve policy configured for the queue: {}, falling back to defaults.",
|
|
|
+ copyQueue, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // the fallback is not configure via store, but via XML, using
|
|
|
+ // previously loaded configuration.
|
|
|
+ if (configuration == null) {
|
|
|
+ configuration = cachedConfiguration.get(YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY);
|
|
|
+ }
|
|
|
+
|
|
|
+ // if the configuration has changed since last loaded, reinit the policy
|
|
|
+ // based on current configuration
|
|
|
+ SubClusterPolicyConfiguration policyConfiguration =
|
|
|
+ cachedConfiguration.getOrDefault(copyQueue, null);
|
|
|
+ if (policyConfiguration == null || !policyConfiguration.equals(configuration)) {
|
|
|
+ singlePolicyReinit(policyMap, cachedConfiguration, copyQueue, configuration);
|
|
|
+ }
|
|
|
+
|
|
|
+ return policyMap.get(copyQueue);
|
|
|
+ }
|
|
|
}
|