|
@@ -19,7 +19,6 @@
|
|
|
package org.apache.hadoop.yarn.server.scheduler;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
-import org.apache.commons.lang3.StringUtils;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
|
import org.apache.hadoop.util.Time;
|
|
@@ -38,21 +37,15 @@ import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.server.api.ContainerType;
|
|
|
|
|
|
import org.apache.hadoop.yarn.server.api.protocolrecords.RemoteNode;
|
|
|
-import org.apache.hadoop.yarn.server.metrics.OpportunisticSchedulerMetrics;
|
|
|
import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
|
|
|
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
-import org.slf4j.Logger;
|
|
|
-import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.util.ArrayList;
|
|
|
-import java.util.Collection;
|
|
|
import java.util.HashMap;
|
|
|
-import java.util.HashSet;
|
|
|
-import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
@@ -61,16 +54,11 @@ import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
|
|
/**
|
|
|
* <p>
|
|
|
- * The OpportunisticContainerAllocator allocates containers on a given list of
|
|
|
- * nodes, after modifying the container sizes to respect the limits set by the
|
|
|
- * ResourceManager. It tries to distribute the containers as evenly as possible.
|
|
|
+ * Base abstract class for Opportunistic container allocations, that provides
|
|
|
+ * common functions required for Opportunistic container allocation.
|
|
|
* </p>
|
|
|
*/
|
|
|
-public class OpportunisticContainerAllocator {
|
|
|
-
|
|
|
- private static final int NODE_LOCAL_LOOP = 0;
|
|
|
- private static final int RACK_LOCAL_LOOP = 1;
|
|
|
- private static final int OFF_SWITCH_LOOP = 2;
|
|
|
+public abstract class OpportunisticContainerAllocator {
|
|
|
|
|
|
private int maxAllocationsPerAMHeartbeat = -1;
|
|
|
|
|
@@ -212,9 +200,6 @@ public class OpportunisticContainerAllocator {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static final Logger LOG =
|
|
|
- LoggerFactory.getLogger(OpportunisticContainerAllocator.class);
|
|
|
-
|
|
|
private static final ResourceCalculator RESOURCE_CALCULATOR =
|
|
|
new DominantResourceCalculator();
|
|
|
|
|
@@ -238,26 +223,30 @@ public class OpportunisticContainerAllocator {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- static class EnrichedResourceRequest {
|
|
|
+ /**
|
|
|
+ * This class encapsulates Resource Request and provides requests per
|
|
|
+ * node and rack.
|
|
|
+ */
|
|
|
+ public static class EnrichedResourceRequest {
|
|
|
private final Map<String, AtomicInteger> nodeLocations = new HashMap<>();
|
|
|
private final Map<String, AtomicInteger> rackLocations = new HashMap<>();
|
|
|
private final ResourceRequest request;
|
|
|
private final long timestamp;
|
|
|
|
|
|
- EnrichedResourceRequest(ResourceRequest request) {
|
|
|
+ public EnrichedResourceRequest(ResourceRequest request) {
|
|
|
this.request = request;
|
|
|
timestamp = Time.monotonicNow();
|
|
|
}
|
|
|
|
|
|
- long getTimestamp() {
|
|
|
+ public long getTimestamp() {
|
|
|
return timestamp;
|
|
|
}
|
|
|
|
|
|
- ResourceRequest getRequest() {
|
|
|
+ public ResourceRequest getRequest() {
|
|
|
return request;
|
|
|
}
|
|
|
|
|
|
- void addLocation(String location, int count) {
|
|
|
+ public void addLocation(String location, int count) {
|
|
|
Map<String, AtomicInteger> m = rackLocations;
|
|
|
if (!location.startsWith("/")) {
|
|
|
m = nodeLocations;
|
|
@@ -269,7 +258,7 @@ public class OpportunisticContainerAllocator {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- void removeLocation(String location) {
|
|
|
+ public void removeLocation(String location) {
|
|
|
Map<String, AtomicInteger> m = rackLocations;
|
|
|
AtomicInteger count = m.get(location);
|
|
|
if (count == null) {
|
|
@@ -284,14 +273,15 @@ public class OpportunisticContainerAllocator {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- Set<String> getNodeLocations() {
|
|
|
+ public Set<String> getNodeLocations() {
|
|
|
return nodeLocations.keySet();
|
|
|
}
|
|
|
|
|
|
- Set<String> getRackLocations() {
|
|
|
+ public Set<String> getRackLocations() {
|
|
|
return rackLocations.keySet();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
/**
|
|
|
* Create a new Opportunistic Container Allocator.
|
|
|
* @param tokenSecretManager TokenSecretManager
|
|
@@ -319,6 +309,14 @@ public class OpportunisticContainerAllocator {
|
|
|
this.maxAllocationsPerAMHeartbeat = maxAllocationsPerAMHeartbeat;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Get the Max Allocations per AM heartbeat.
|
|
|
+ * @return maxAllocationsPerAMHeartbeat.
|
|
|
+ */
|
|
|
+ public int getMaxAllocationsPerAMHeartbeat() {
|
|
|
+ return this.maxAllocationsPerAMHeartbeat;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Allocate OPPORTUNISTIC containers.
|
|
|
* @param blackList Resource BlackList Request
|
|
@@ -330,72 +328,37 @@ public class OpportunisticContainerAllocator {
|
|
|
* @return List of Containers.
|
|
|
* @throws YarnException YarnException
|
|
|
*/
|
|
|
- public List<Container> allocateContainers(ResourceBlacklistRequest blackList,
|
|
|
+ public abstract List<Container> allocateContainers(
|
|
|
+ ResourceBlacklistRequest blackList,
|
|
|
List<ResourceRequest> oppResourceReqs,
|
|
|
ApplicationAttemptId applicationAttemptId,
|
|
|
OpportunisticContainerContext opportContext, long rmIdentifier,
|
|
|
- String appSubmitter) throws YarnException {
|
|
|
+ String appSubmitter) throws YarnException;
|
|
|
+
|
|
|
|
|
|
- // Update black list.
|
|
|
+ protected void updateBlacklist(ResourceBlacklistRequest blackList,
|
|
|
+ OpportunisticContainerContext oppContext) {
|
|
|
if (blackList != null) {
|
|
|
- opportContext.getBlacklist().removeAll(blackList.getBlacklistRemovals());
|
|
|
- opportContext.getBlacklist().addAll(blackList.getBlacklistAdditions());
|
|
|
+ oppContext.getBlacklist().removeAll(blackList.getBlacklistRemovals());
|
|
|
+ oppContext.getBlacklist().addAll(blackList.getBlacklistAdditions());
|
|
|
}
|
|
|
+ }
|
|
|
|
|
|
- // Add OPPORTUNISTIC requests to the outstanding ones.
|
|
|
- opportContext.addToOutstandingReqs(oppResourceReqs);
|
|
|
- Set<String> nodeBlackList = new HashSet<>(opportContext.getBlacklist());
|
|
|
- Set<String> allocatedNodes = new HashSet<>();
|
|
|
- List<Container> allocatedContainers = new ArrayList<>();
|
|
|
-
|
|
|
- // Satisfy the outstanding OPPORTUNISTIC requests.
|
|
|
- boolean continueLoop = true;
|
|
|
- while (continueLoop) {
|
|
|
- continueLoop = false;
|
|
|
- List<Map<Resource, List<Allocation>>> allocations = new ArrayList<>();
|
|
|
- for (SchedulerRequestKey schedulerKey :
|
|
|
- opportContext.getOutstandingOpReqs().descendingKeySet()) {
|
|
|
- // Allocated containers :
|
|
|
- // Key = Requested Capability,
|
|
|
- // Value = List of Containers of given cap (the actual container size
|
|
|
- // might be different than what is requested, which is why
|
|
|
- // we need the requested capability (key) to match against
|
|
|
- // the outstanding reqs)
|
|
|
- int remAllocs = -1;
|
|
|
- if (maxAllocationsPerAMHeartbeat > 0) {
|
|
|
- remAllocs =
|
|
|
- maxAllocationsPerAMHeartbeat - allocatedContainers.size()
|
|
|
- - getTotalAllocations(allocations);
|
|
|
- if (remAllocs <= 0) {
|
|
|
- LOG.info("Not allocating more containers as we have reached max "
|
|
|
- + "allocations per AM heartbeat {}",
|
|
|
- maxAllocationsPerAMHeartbeat);
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- Map<Resource, List<Allocation>> allocation = allocate(
|
|
|
- rmIdentifier, opportContext, schedulerKey, applicationAttemptId,
|
|
|
- appSubmitter, nodeBlackList, allocatedNodes, remAllocs);
|
|
|
- if (allocation.size() > 0) {
|
|
|
- allocations.add(allocation);
|
|
|
- continueLoop = true;
|
|
|
- }
|
|
|
- }
|
|
|
- for (Map<Resource, List<Allocation>> allocation : allocations) {
|
|
|
- for (Map.Entry<Resource, List<Allocation>> e : allocation.entrySet()) {
|
|
|
- opportContext.matchAllocationToOutstandingRequest(
|
|
|
- e.getKey(), e.getValue());
|
|
|
- for (Allocation alloc : e.getValue()) {
|
|
|
- allocatedContainers.add(alloc.getContainer());
|
|
|
- }
|
|
|
+ protected void matchAllocation(List<Map<Resource,
|
|
|
+ List<Allocation>>> allocations, List<Container> allocatedContainers,
|
|
|
+ OpportunisticContainerContext oppContext) {
|
|
|
+ for (Map<Resource, List<Allocation>> allocation : allocations) {
|
|
|
+ for (Map.Entry<Resource, List<Allocation>> e : allocation.entrySet()) {
|
|
|
+ oppContext.matchAllocationToOutstandingRequest(
|
|
|
+ e.getKey(), e.getValue());
|
|
|
+ for (Allocation alloc : e.getValue()) {
|
|
|
+ allocatedContainers.add(alloc.getContainer());
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- return allocatedContainers;
|
|
|
}
|
|
|
|
|
|
- private int getTotalAllocations(
|
|
|
+ protected int getTotalAllocations(
|
|
|
List<Map<Resource, List<Allocation>>> allocations) {
|
|
|
int totalAllocs = 0;
|
|
|
for (Map<Resource, List<Allocation>> allocation : allocations) {
|
|
@@ -406,223 +369,8 @@ public class OpportunisticContainerAllocator {
|
|
|
return totalAllocs;
|
|
|
}
|
|
|
|
|
|
- private Map<Resource, List<Allocation>> allocate(long rmIdentifier,
|
|
|
- OpportunisticContainerContext appContext, SchedulerRequestKey schedKey,
|
|
|
- ApplicationAttemptId appAttId, String userName, Set<String> blackList,
|
|
|
- Set<String> allocatedNodes, int maxAllocations)
|
|
|
- throws YarnException {
|
|
|
- Map<Resource, List<Allocation>> containers = new HashMap<>();
|
|
|
- for (EnrichedResourceRequest enrichedAsk :
|
|
|
- appContext.getOutstandingOpReqs().get(schedKey).values()) {
|
|
|
- int remainingAllocs = -1;
|
|
|
- if (maxAllocations > 0) {
|
|
|
- int totalAllocated = 0;
|
|
|
- for (List<Allocation> allocs : containers.values()) {
|
|
|
- totalAllocated += allocs.size();
|
|
|
- }
|
|
|
- remainingAllocs = maxAllocations - totalAllocated;
|
|
|
- if (remainingAllocs <= 0) {
|
|
|
- LOG.info("Not allocating more containers as max allocations per AM "
|
|
|
- + "heartbeat {} has reached", maxAllocationsPerAMHeartbeat);
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- allocateContainersInternal(rmIdentifier, appContext.getAppParams(),
|
|
|
- appContext.getContainerIdGenerator(), blackList, allocatedNodes,
|
|
|
- appAttId, appContext.getNodeMap(), userName, containers, enrichedAsk,
|
|
|
- remainingAllocs);
|
|
|
- ResourceRequest anyAsk = enrichedAsk.getRequest();
|
|
|
- if (!containers.isEmpty()) {
|
|
|
- LOG.info("Opportunistic allocation requested for [priority={}, "
|
|
|
- + "allocationRequestId={}, num_containers={}, capability={}] "
|
|
|
- + "allocated = {}", anyAsk.getPriority(),
|
|
|
- anyAsk.getAllocationRequestId(), anyAsk.getNumContainers(),
|
|
|
- anyAsk.getCapability(), containers.keySet());
|
|
|
- }
|
|
|
- }
|
|
|
- return containers;
|
|
|
- }
|
|
|
-
|
|
|
- private void allocateContainersInternal(long rmIdentifier,
|
|
|
- AllocationParams appParams, ContainerIdGenerator idCounter,
|
|
|
- Set<String> blacklist, Set<String> allocatedNodes,
|
|
|
- ApplicationAttemptId id, Map<String, RemoteNode> allNodes,
|
|
|
- String userName, Map<Resource, List<Allocation>> allocations,
|
|
|
- EnrichedResourceRequest enrichedAsk, int maxAllocations)
|
|
|
- throws YarnException {
|
|
|
- if (allNodes.size() == 0) {
|
|
|
- LOG.info("No nodes currently available to " +
|
|
|
- "allocate OPPORTUNISTIC containers.");
|
|
|
- return;
|
|
|
- }
|
|
|
- ResourceRequest anyAsk = enrichedAsk.getRequest();
|
|
|
- int toAllocate = anyAsk.getNumContainers()
|
|
|
- - (allocations.isEmpty() ? 0 :
|
|
|
- allocations.get(anyAsk.getCapability()).size());
|
|
|
- toAllocate = Math.min(toAllocate,
|
|
|
- appParams.getMaxAllocationsPerSchedulerKeyPerRound());
|
|
|
- if (maxAllocations >= 0) {
|
|
|
- toAllocate = Math.min(maxAllocations, toAllocate);
|
|
|
- }
|
|
|
- int numAllocated = 0;
|
|
|
- // Node Candidates are selected as follows:
|
|
|
- // * Node local candidates selected in loop == 0
|
|
|
- // * Rack local candidates selected in loop == 1
|
|
|
- // * From loop == 2 onwards, we revert to off switch allocations.
|
|
|
- int loopIndex = OFF_SWITCH_LOOP;
|
|
|
- if (enrichedAsk.getNodeLocations().size() > 0) {
|
|
|
- loopIndex = NODE_LOCAL_LOOP;
|
|
|
- }
|
|
|
- while (numAllocated < toAllocate) {
|
|
|
- Collection<RemoteNode> nodeCandidates =
|
|
|
- findNodeCandidates(loopIndex, allNodes, blacklist, allocatedNodes,
|
|
|
- enrichedAsk);
|
|
|
- for (RemoteNode rNode : nodeCandidates) {
|
|
|
- String rNodeHost = rNode.getNodeId().getHost();
|
|
|
- // Ignore black list
|
|
|
- if (blacklist.contains(rNodeHost)) {
|
|
|
- LOG.info("Nodes for scheduling has a blacklisted node" +
|
|
|
- " [" + rNodeHost + "]..");
|
|
|
- continue;
|
|
|
- }
|
|
|
- String location = ResourceRequest.ANY;
|
|
|
- if (loopIndex == NODE_LOCAL_LOOP) {
|
|
|
- if (enrichedAsk.getNodeLocations().contains(rNodeHost)) {
|
|
|
- location = rNodeHost;
|
|
|
- } else {
|
|
|
- continue;
|
|
|
- }
|
|
|
- } else if (allocatedNodes.contains(rNodeHost)) {
|
|
|
- LOG.info("Opportunistic container has already been allocated on {}.",
|
|
|
- rNodeHost);
|
|
|
- continue;
|
|
|
- }
|
|
|
- if (loopIndex == RACK_LOCAL_LOOP) {
|
|
|
- if (enrichedAsk.getRackLocations().contains(rNode.getRackName())) {
|
|
|
- location = rNode.getRackName();
|
|
|
- } else {
|
|
|
- continue;
|
|
|
- }
|
|
|
- }
|
|
|
- Container container = createContainer(rmIdentifier, appParams,
|
|
|
- idCounter, id, userName, allocations, location,
|
|
|
- anyAsk, rNode);
|
|
|
- numAllocated++;
|
|
|
- updateMetrics(loopIndex);
|
|
|
- allocatedNodes.add(rNodeHost);
|
|
|
- LOG.info("Allocated [" + container.getId() + "] as opportunistic at " +
|
|
|
- "location [" + location + "]");
|
|
|
- if (numAllocated >= toAllocate) {
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- if (loopIndex == NODE_LOCAL_LOOP &&
|
|
|
- enrichedAsk.getRackLocations().size() > 0) {
|
|
|
- loopIndex = RACK_LOCAL_LOOP;
|
|
|
- } else {
|
|
|
- loopIndex++;
|
|
|
- }
|
|
|
- // Handle case where there are no nodes remaining after blacklist is
|
|
|
- // considered.
|
|
|
- if (loopIndex > OFF_SWITCH_LOOP && numAllocated == 0) {
|
|
|
- LOG.warn("Unable to allocate any opportunistic containers.");
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void updateMetrics(int loopIndex) {
|
|
|
- OpportunisticSchedulerMetrics metrics =
|
|
|
- OpportunisticSchedulerMetrics.getMetrics();
|
|
|
- if (loopIndex == NODE_LOCAL_LOOP) {
|
|
|
- metrics.incrNodeLocalOppContainers();
|
|
|
- } else if (loopIndex == RACK_LOCAL_LOOP) {
|
|
|
- metrics.incrRackLocalOppContainers();
|
|
|
- } else {
|
|
|
- metrics.incrOffSwitchOppContainers();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private Collection<RemoteNode> findNodeCandidates(int loopIndex,
|
|
|
- Map<String, RemoteNode> allNodes, Set<String> blackList,
|
|
|
- Set<String> allocatedNodes, EnrichedResourceRequest enrichedRR) {
|
|
|
- LinkedList<RemoteNode> retList = new LinkedList<>();
|
|
|
- String partition = getRequestPartition(enrichedRR);
|
|
|
- if (loopIndex > 1) {
|
|
|
- for (RemoteNode remoteNode : allNodes.values()) {
|
|
|
- if (StringUtils.equals(partition, getRemoteNodePartition(remoteNode))) {
|
|
|
- retList.add(remoteNode);
|
|
|
- }
|
|
|
- }
|
|
|
- return retList;
|
|
|
- } else {
|
|
|
-
|
|
|
- int numContainers = enrichedRR.getRequest().getNumContainers();
|
|
|
- while (numContainers > 0) {
|
|
|
- if (loopIndex == 0) {
|
|
|
- // Node local candidates
|
|
|
- numContainers = collectNodeLocalCandidates(
|
|
|
- allNodes, enrichedRR, retList, numContainers);
|
|
|
- } else {
|
|
|
- // Rack local candidates
|
|
|
- numContainers =
|
|
|
- collectRackLocalCandidates(allNodes, enrichedRR, retList,
|
|
|
- blackList, allocatedNodes, numContainers);
|
|
|
- }
|
|
|
- if (numContainers == enrichedRR.getRequest().getNumContainers()) {
|
|
|
- // If there is no change in numContainers, then there is no point
|
|
|
- // in looping again.
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- return retList;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private int collectRackLocalCandidates(Map<String, RemoteNode> allNodes,
|
|
|
- EnrichedResourceRequest enrichedRR, LinkedList<RemoteNode> retList,
|
|
|
- Set<String> blackList, Set<String> allocatedNodes, int numContainers) {
|
|
|
- String partition = getRequestPartition(enrichedRR);
|
|
|
- for (RemoteNode rNode : allNodes.values()) {
|
|
|
- if (StringUtils.equals(partition, getRemoteNodePartition(rNode)) &&
|
|
|
- enrichedRR.getRackLocations().contains(rNode.getRackName())) {
|
|
|
- String rHost = rNode.getNodeId().getHost();
|
|
|
- if (blackList.contains(rHost)) {
|
|
|
- continue;
|
|
|
- }
|
|
|
- if (allocatedNodes.contains(rHost)) {
|
|
|
- retList.addLast(rNode);
|
|
|
- } else {
|
|
|
- retList.addFirst(rNode);
|
|
|
- numContainers--;
|
|
|
- }
|
|
|
- }
|
|
|
- if (numContainers == 0) {
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- return numContainers;
|
|
|
- }
|
|
|
-
|
|
|
- private int collectNodeLocalCandidates(Map<String, RemoteNode> allNodes,
|
|
|
- EnrichedResourceRequest enrichedRR, List<RemoteNode> retList,
|
|
|
- int numContainers) {
|
|
|
- String partition = getRequestPartition(enrichedRR);
|
|
|
- for (String nodeName : enrichedRR.getNodeLocations()) {
|
|
|
- RemoteNode remoteNode = allNodes.get(nodeName);
|
|
|
- if (remoteNode != null &&
|
|
|
- StringUtils.equals(partition, getRemoteNodePartition(remoteNode))) {
|
|
|
- retList.add(remoteNode);
|
|
|
- numContainers--;
|
|
|
- }
|
|
|
- if (numContainers == 0) {
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- return numContainers;
|
|
|
- }
|
|
|
-
|
|
|
- private Container createContainer(long rmIdentifier,
|
|
|
+ @SuppressWarnings("checkstyle:parameternumber")
|
|
|
+ protected Container createContainer(long rmIdentifier,
|
|
|
AllocationParams appParams, ContainerIdGenerator idCounter,
|
|
|
ApplicationAttemptId id, String userName,
|
|
|
Map<Resource, List<Allocation>> allocations, String location,
|
|
@@ -654,6 +402,7 @@ public class OpportunisticContainerAllocator {
|
|
|
SchedulerRequestKey.create(rr), userName, node, cId, capability);
|
|
|
}
|
|
|
|
|
|
+ @SuppressWarnings("checkstyle:parameternumber")
|
|
|
private Container createContainer(long rmIdentifier, long tokenExpiry,
|
|
|
SchedulerRequestKey schedulerKey, String userName, RemoteNode node,
|
|
|
ContainerId cId, Resource capability) {
|
|
@@ -718,7 +467,7 @@ public class OpportunisticContainerAllocator {
|
|
|
return partitionedRequests;
|
|
|
}
|
|
|
|
|
|
- private String getRequestPartition(EnrichedResourceRequest enrichedRR) {
|
|
|
+ protected String getRequestPartition(EnrichedResourceRequest enrichedRR) {
|
|
|
String partition = enrichedRR.getRequest().getNodeLabelExpression();
|
|
|
if (partition == null) {
|
|
|
partition = CommonNodeLabelsManager.NO_LABEL;
|
|
@@ -726,7 +475,7 @@ public class OpportunisticContainerAllocator {
|
|
|
return partition;
|
|
|
}
|
|
|
|
|
|
- private String getRemoteNodePartition(RemoteNode node) {
|
|
|
+ protected String getRemoteNodePartition(RemoteNode node) {
|
|
|
String partition = node.getNodePartition();
|
|
|
if (partition == null) {
|
|
|
partition = CommonNodeLabelsManager.NO_LABEL;
|