|
@@ -19,6 +19,7 @@
|
|
|
package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
|
|
|
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Collection;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.Iterator;
|
|
@@ -31,6 +32,9 @@ import java.util.TreeMap;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
|
|
+import org.apache.commons.collections.MapUtils;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
|
|
import org.apache.hadoop.yarn.api.records.EnhancedHeadroom;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
@@ -54,6 +58,19 @@ import org.slf4j.LoggerFactory;
|
|
|
import org.apache.hadoop.classification.VisibleForTesting;
|
|
|
import org.apache.hadoop.util.Preconditions;
|
|
|
|
|
|
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOAD_BASED_SC_SELECTOR_ENABLED;
|
|
|
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_LOAD_BASED_SC_SELECTOR_ENABLED;
|
|
|
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOAD_BASED_SC_SELECTOR_THRESHOLD;
|
|
|
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_LOAD_BASED_SC_SELECTOR_THRESHOLD;
|
|
|
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOAD_BASED_SC_SELECTOR_USE_ACTIVE_CORE;
|
|
|
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_LOAD_BASED_SC_SELECTOR_USE_ACTIVE_CORE;
|
|
|
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOAD_BASED_SC_SELECTOR_MULTIPLIER;
|
|
|
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_LOAD_BASED_SC_SELECTOR_MULTIPLIER;
|
|
|
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOAD_BASED_SC_SELECTOR_FAIL_ON_ERROR;
|
|
|
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_LOAD_BASED_SC_SELECTOR_FAIL_ON_ERROR;
|
|
|
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.FEDERATION_BLACKLIST_SUBCLUSTERS;
|
|
|
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_FEDERATION_BLACKLIST_SUBCLUSTERS;
|
|
|
+
|
|
|
/**
|
|
|
* An implementation of the {@link FederationAMRMProxyPolicy} interface that
|
|
|
* carefully multicasts the requests with the following behavior:
|
|
@@ -131,11 +148,44 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
|
|
|
private Map<SubClusterId, Float> weights;
|
|
|
private SubClusterResolver resolver;
|
|
|
|
|
|
+ private Configuration conf;
|
|
|
private Map<SubClusterId, Resource> headroom;
|
|
|
private Map<SubClusterId, EnhancedHeadroom> enhancedHeadroom;
|
|
|
private float hrAlpha;
|
|
|
private FederationStateStoreFacade federationFacade;
|
|
|
private SubClusterId homeSubcluster;
|
|
|
+ private int printRRMax;
|
|
|
+ public static final String PRINT_RR_MAX =
|
|
|
+ "yarn.nodemanager.amrmproxy.address.splitmerge.printmaxrrcount";
|
|
|
+ public static final int DEFAULT_PRINT_RR_MAX = 1000;
|
|
|
+ private boolean failOnError = DEFAULT_LOAD_BASED_SC_SELECTOR_FAIL_ON_ERROR;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Print a list of Resource Requests into a one line string.
|
|
|
+ *
|
|
|
+ * @param response list of ResourceRequest
|
|
|
+ * @param max number of ResourceRequest to print
|
|
|
+ * @return the printed one line string
|
|
|
+ */
|
|
|
+ public static String prettyPrintRequests(List<ResourceRequest> response, int max) {
|
|
|
+ StringBuilder builder = new StringBuilder();
|
|
|
+ for (ResourceRequest rr : response) {
|
|
|
+ builder.append("[id:").append(rr.getAllocationRequestId())
|
|
|
+ .append(" loc:")
|
|
|
+ .append(rr.getResourceName())
|
|
|
+ .append(" num:")
|
|
|
+ .append(rr.getNumContainers())
|
|
|
+ .append(" pri:")
|
|
|
+ .append(((rr.getPriority() != null) ? rr.getPriority().getPriority() : -1))
|
|
|
+ .append("], ");
|
|
|
+ if (max != -1) {
|
|
|
+ if (max-- <= 0) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return builder.toString();
|
|
|
+ }
|
|
|
|
|
|
@Override
|
|
|
public void reinitialize(
|
|
@@ -182,6 +232,7 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
|
|
|
weights = newWeightsConverted;
|
|
|
resolver = policyContext.getFederationSubclusterResolver();
|
|
|
|
|
|
+ // Data structures that only need to initialize once
|
|
|
if (headroom == null) {
|
|
|
headroom = new ConcurrentHashMap<>();
|
|
|
enhancedHeadroom = new ConcurrentHashMap<>();
|
|
@@ -191,6 +242,11 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
|
|
|
this.federationFacade =
|
|
|
policyContext.getFederationStateStoreFacade();
|
|
|
this.homeSubcluster = policyContext.getHomeSubcluster();
|
|
|
+
|
|
|
+ this.conf = this.federationFacade.getConf();
|
|
|
+ this.printRRMax = this.conf.getInt(PRINT_RR_MAX, DEFAULT_PRINT_RR_MAX);
|
|
|
+ this.failOnError = this.conf.getBoolean(LOAD_BASED_SC_SELECTOR_FAIL_ON_ERROR,
|
|
|
+ DEFAULT_LOAD_BASED_SC_SELECTOR_FAIL_ON_ERROR);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -217,10 +273,9 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
|
|
|
// active subclusters. Create a new instance per call because this method
|
|
|
// can be called concurrently.
|
|
|
AllocationBookkeeper bookkeeper = new AllocationBookkeeper();
|
|
|
- bookkeeper.reinitialize(getActiveSubclusters(), timedOutSubClusters);
|
|
|
+ bookkeeper.reinitialize(getActiveSubclusters(), timedOutSubClusters, conf);
|
|
|
|
|
|
- List<ResourceRequest> nonLocalizedRequests =
|
|
|
- new ArrayList<ResourceRequest>();
|
|
|
+ List<ResourceRequest> nonLocalizedRequests = new ArrayList<>();
|
|
|
|
|
|
SubClusterId targetId = null;
|
|
|
Set<SubClusterId> targetIds = null;
|
|
@@ -240,6 +295,17 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
|
|
|
// Handle "node" requests
|
|
|
try {
|
|
|
targetId = resolver.getSubClusterForNode(rr.getResourceName());
|
|
|
+
|
|
|
+ // If needed, re-reroute node requests base on SC load
|
|
|
+ boolean loadBasedSCSelectorEnabled =
|
|
|
+ conf.getBoolean(LOAD_BASED_SC_SELECTOR_ENABLED, DEFAULT_LOAD_BASED_SC_SELECTOR_ENABLED);
|
|
|
+ if (loadBasedSCSelectorEnabled) {
|
|
|
+ int maxPendingThreshold = conf.getInt(LOAD_BASED_SC_SELECTOR_THRESHOLD,
|
|
|
+ DEFAULT_LOAD_BASED_SC_SELECTOR_THRESHOLD);
|
|
|
+ targetId = routeNodeRequestIfNeeded(targetId, maxPendingThreshold,
|
|
|
+ bookkeeper.getActiveAndEnabledSC());
|
|
|
+ }
|
|
|
+ LOG.debug("Node request {}", rr.getResourceName());
|
|
|
} catch (YarnException e) {
|
|
|
// this might happen as we can't differentiate node from rack names
|
|
|
// we log altogether later
|
|
@@ -285,7 +351,16 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
|
|
|
// handle all non-localized requests (ANY)
|
|
|
splitAnyRequests(nonLocalizedRequests, bookkeeper);
|
|
|
|
|
|
- return bookkeeper.getAnswer();
|
|
|
+ // Take the split result, feed into the askBalancer
|
|
|
+ Map<SubClusterId, List<ResourceRequest>> answer = bookkeeper.getAnswer();
|
|
|
+ LOG.info("Before split {} RRs: {}", resourceRequests.size(),
|
|
|
+ prettyPrintRequests(resourceRequests, this.printRRMax));
|
|
|
+
|
|
|
+ for (Map.Entry<SubClusterId, List<ResourceRequest>> entry : bookkeeper.getAnswer().entrySet()) {
|
|
|
+ LOG.info("After split {} has {} RRs: {}", entry.getKey(), entry.getValue().size(),
|
|
|
+ prettyPrintRequests(entry.getValue(), this.printRRMax));
|
|
|
+ }
|
|
|
+ return answer;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -495,6 +570,120 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
|
|
|
return headroomWeighting;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * When certain subcluster is too loaded, reroute Node requests going there.
|
|
|
+ *
|
|
|
+ * @param targetId current subClusterId where request is sent
|
|
|
+ * @param maxThreshold threshold for Pending count
|
|
|
+ * @param activeAndEnabledSCs list of active sc
|
|
|
+ * @return subClusterId target sc id
|
|
|
+ */
|
|
|
+ protected SubClusterId routeNodeRequestIfNeeded(SubClusterId targetId,
|
|
|
+ int maxThreshold, Set<SubClusterId> activeAndEnabledSCs) {
|
|
|
+ // If targetId is not in the active and enabled SC list, reroute the traffic
|
|
|
+ if (activeAndEnabledSCs.contains(targetId)) {
|
|
|
+ int targetPendingCount = getSubClusterLoad(targetId);
|
|
|
+ if (targetPendingCount == -1 || targetPendingCount < maxThreshold) {
|
|
|
+ return targetId;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ SubClusterId scId = chooseSubClusterIdForMaxLoadSC(targetId, maxThreshold, activeAndEnabledSCs);
|
|
|
+ return scId;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Check if the current target subcluster is over max load, and if it is
|
|
|
+ * reroute it.
|
|
|
+ *
|
|
|
+ * @param targetId the original target subcluster id
|
|
|
+ * @param maxThreshold the max load threshold to reroute
|
|
|
+ * @param activeAndEnabledSCs the list of active and enabled subclusters
|
|
|
+ * @return targetId if it is within maxThreshold, otherwise a new id
|
|
|
+ */
|
|
|
+ private SubClusterId chooseSubClusterIdForMaxLoadSC(SubClusterId targetId,
|
|
|
+ int maxThreshold, Set<SubClusterId> activeAndEnabledSCs) {
|
|
|
+ ArrayList<Float> weight = new ArrayList<>();
|
|
|
+ ArrayList<SubClusterId> scIds = new ArrayList<>();
|
|
|
+ int targetLoad = getSubClusterLoad(targetId);
|
|
|
+ if (targetLoad == -1 || !activeAndEnabledSCs.contains(targetId)) {
|
|
|
+ // Probably a SC that's not active and enabled. Forcing a reroute
|
|
|
+ targetLoad = Integer.MAX_VALUE;
|
|
|
+ }
|
|
|
+
|
|
|
+ /*
|
|
|
+ * Prepare the weight for a random draw among all known SCs.
|
|
|
+ *
|
|
|
+ * For SC with pending bigger than maxThreshold / 2, use maxThreshold /
|
|
|
+ * pending as weight. We multiplied by maxThreshold so that the weight
|
|
|
+ * won't be too small in value.
|
|
|
+ *
|
|
|
+ * For SC with pending less than maxThreshold / 2, we cap the weight at 2
|
|
|
+ * = (maxThreshold / (maxThreshold / 2)) so that SC with small pending
|
|
|
+ * will not get a huge weight and thus get swamped.
|
|
|
+ */
|
|
|
+ for (SubClusterId sc : activeAndEnabledSCs) {
|
|
|
+ int scLoad = getSubClusterLoad(sc);
|
|
|
+ if (scLoad > targetLoad) {
|
|
|
+ // Never mind if it is not the most loaded SC
|
|
|
+ return targetId;
|
|
|
+ }
|
|
|
+ if (scLoad <= maxThreshold / 2) {
|
|
|
+ weight.add(2f);
|
|
|
+ } else {
|
|
|
+ weight.add((float) maxThreshold / scLoad);
|
|
|
+ }
|
|
|
+ scIds.add(sc);
|
|
|
+ }
|
|
|
+ if (weights.size() == 0) {
|
|
|
+ return targetId;
|
|
|
+ }
|
|
|
+ return scIds.get(FederationPolicyUtils.getWeightedRandom(weight));
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * get the Load data of the subCluster.
|
|
|
+ *
|
|
|
+ * @param subClusterId subClusterId.
|
|
|
+ * @return The number of pending containers for the subCluster.
|
|
|
+ */
|
|
|
+ private int getSubClusterLoad(SubClusterId subClusterId) {
|
|
|
+ EnhancedHeadroom headroomData = this.enhancedHeadroom.get(subClusterId);
|
|
|
+ if (headroomData == null) {
|
|
|
+ return -1;
|
|
|
+ }
|
|
|
+
|
|
|
+ // Use new data from enhanced headroom
|
|
|
+ boolean useActiveCoreEnabled = conf.getBoolean(LOAD_BASED_SC_SELECTOR_USE_ACTIVE_CORE,
|
|
|
+ DEFAULT_LOAD_BASED_SC_SELECTOR_USE_ACTIVE_CORE);
|
|
|
+
|
|
|
+ // If we consider the number of vCores in the subCluster
|
|
|
+ if (useActiveCoreEnabled) {
|
|
|
+
|
|
|
+ // If the vcore of the subCluster is less than or equal to 0,
|
|
|
+ // it means that containers cannot be scheduled to this subCluster,
|
|
|
+ // and we will return a very large number, indicating that the subCluster is unavailable.
|
|
|
+ if (headroomData.getTotalActiveCores() <= 0) {
|
|
|
+ return Integer.MAX_VALUE;
|
|
|
+ }
|
|
|
+
|
|
|
+ // Multiply by a constant factor, to ensure the numerator > denominator.
|
|
|
+ // We will normalize the PendingCount, using PendingCount * multiplier / TotalActiveCores.
|
|
|
+ long multiplier = conf.getLong(LOAD_BASED_SC_SELECTOR_MULTIPLIER,
|
|
|
+ DEFAULT_LOAD_BASED_SC_SELECTOR_MULTIPLIER);
|
|
|
+ double value =
|
|
|
+ headroomData.getNormalizedPendingCount(multiplier) / headroomData.getTotalActiveCores();
|
|
|
+ if (value > Integer.MAX_VALUE) {
|
|
|
+ return Integer.MAX_VALUE;
|
|
|
+ } else {
|
|
|
+ return (int) value;
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ // If the number of vcores in the subCluster is not considered,
|
|
|
+ // we directly return the number of pending containers in the subCluster.
|
|
|
+ return headroomData.getTotalPendingCount();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* This helper class is used to book-keep the requests made to each
|
|
|
* subcluster, and maintain useful statistics to split ANY requests.
|
|
@@ -523,8 +712,9 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
|
|
|
|
|
|
private void reinitialize(
|
|
|
Map<SubClusterId, SubClusterInfo> activeSubclusters,
|
|
|
- Set<SubClusterId> timedOutSubClusters) throws YarnException {
|
|
|
- if (activeSubclusters == null) {
|
|
|
+ Set<SubClusterId> timedOutSubClusters, Configuration pConf) throws YarnException {
|
|
|
+
|
|
|
+ if (MapUtils.isEmpty(activeSubclusters)) {
|
|
|
throw new YarnRuntimeException("null activeSubclusters received");
|
|
|
}
|
|
|
|
|
@@ -548,10 +738,28 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ // subCluster blacklisting from configuration
|
|
|
+ String blacklistedSubClusters = pConf.get(FEDERATION_BLACKLIST_SUBCLUSTERS,
|
|
|
+ DEFAULT_FEDERATION_BLACKLIST_SUBCLUSTERS);
|
|
|
+ if (blacklistedSubClusters != null) {
|
|
|
+ Collection<String> tempList = StringUtils.getStringCollection(blacklistedSubClusters);
|
|
|
+ for (String item : tempList) {
|
|
|
+ activeAndEnabledSC.remove(SubClusterId.newInstance(item.trim()));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
if (activeAndEnabledSC.size() < 1) {
|
|
|
- throw new NoActiveSubclustersException(
|
|
|
- "None of the subclusters enabled in this policy (weight>0) are "
|
|
|
- + "currently active we cannot forward the ResourceRequest(s)");
|
|
|
+ String errorMsg = "None of the subClusters enabled in this Policy (weight > 0) are "
|
|
|
+ + "currently active we cannot forward the ResourceRequest(s)";
|
|
|
+ if (failOnError) {
|
|
|
+ throw new NoActiveSubclustersException(errorMsg);
|
|
|
+ } else {
|
|
|
+ LOG.error(errorMsg + ", continuing by enabling all active subClusters.");
|
|
|
+ activeAndEnabledSC.addAll(activeSubclusters.keySet());
|
|
|
+ for (SubClusterId sc : activeSubclusters.keySet()) {
|
|
|
+ policyWeights.put(sc, 1.0f);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
Set<SubClusterId> tmpSCSet = new HashSet<>(activeAndEnabledSC);
|
|
@@ -559,7 +767,7 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
|
|
|
|
|
|
if (tmpSCSet.size() < 1) {
|
|
|
LOG.warn("All active and enabled subclusters have expired last "
|
|
|
- + "heartbeat time. Ignore the expiry check for this request");
|
|
|
+ + "heartbeat time. Ignore the expiry check for this request.");
|
|
|
} else {
|
|
|
activeAndEnabledSC = tmpSCSet;
|
|
|
}
|