|
@@ -318,6 +318,7 @@ public class OpportunisticContainerAllocator {
|
|
opportContext.addToOutstandingReqs(oppResourceReqs);
|
|
opportContext.addToOutstandingReqs(oppResourceReqs);
|
|
|
|
|
|
Set<String> nodeBlackList = new HashSet<>(opportContext.getBlacklist());
|
|
Set<String> nodeBlackList = new HashSet<>(opportContext.getBlacklist());
|
|
|
|
+ Set<String> allocatedNodes = new HashSet<>();
|
|
List<Container> allocatedContainers = new ArrayList<>();
|
|
List<Container> allocatedContainers = new ArrayList<>();
|
|
|
|
|
|
// Satisfy the outstanding OPPORTUNISTIC requests.
|
|
// Satisfy the outstanding OPPORTUNISTIC requests.
|
|
@@ -335,7 +336,7 @@ public class OpportunisticContainerAllocator {
|
|
// the outstanding reqs)
|
|
// the outstanding reqs)
|
|
Map<Resource, List<Allocation>> allocation = allocate(
|
|
Map<Resource, List<Allocation>> allocation = allocate(
|
|
rmIdentifier, opportContext, schedulerKey, applicationAttemptId,
|
|
rmIdentifier, opportContext, schedulerKey, applicationAttemptId,
|
|
- appSubmitter, nodeBlackList);
|
|
|
|
|
|
+ appSubmitter, nodeBlackList, allocatedNodes);
|
|
if (allocation.size() > 0) {
|
|
if (allocation.size() > 0) {
|
|
allocations.add(allocation);
|
|
allocations.add(allocation);
|
|
continueLoop = true;
|
|
continueLoop = true;
|
|
@@ -357,14 +358,15 @@ public class OpportunisticContainerAllocator {
|
|
|
|
|
|
private Map<Resource, List<Allocation>> allocate(long rmIdentifier,
|
|
private Map<Resource, List<Allocation>> allocate(long rmIdentifier,
|
|
OpportunisticContainerContext appContext, SchedulerRequestKey schedKey,
|
|
OpportunisticContainerContext appContext, SchedulerRequestKey schedKey,
|
|
- ApplicationAttemptId appAttId, String userName, Set<String> blackList)
|
|
|
|
|
|
+ ApplicationAttemptId appAttId, String userName, Set<String> blackList,
|
|
|
|
+ Set<String> allocatedNodes)
|
|
throws YarnException {
|
|
throws YarnException {
|
|
Map<Resource, List<Allocation>> containers = new HashMap<>();
|
|
Map<Resource, List<Allocation>> containers = new HashMap<>();
|
|
for (EnrichedResourceRequest enrichedAsk :
|
|
for (EnrichedResourceRequest enrichedAsk :
|
|
appContext.getOutstandingOpReqs().get(schedKey).values()) {
|
|
appContext.getOutstandingOpReqs().get(schedKey).values()) {
|
|
allocateContainersInternal(rmIdentifier, appContext.getAppParams(),
|
|
allocateContainersInternal(rmIdentifier, appContext.getAppParams(),
|
|
- appContext.getContainerIdGenerator(), blackList, appAttId,
|
|
|
|
- appContext.getNodeMap(), userName, containers, enrichedAsk);
|
|
|
|
|
|
+ appContext.getContainerIdGenerator(), blackList, allocatedNodes,
|
|
|
|
+ appAttId, appContext.getNodeMap(), userName, containers, enrichedAsk);
|
|
ResourceRequest anyAsk = enrichedAsk.getRequest();
|
|
ResourceRequest anyAsk = enrichedAsk.getRequest();
|
|
if (!containers.isEmpty()) {
|
|
if (!containers.isEmpty()) {
|
|
LOG.info("Opportunistic allocation requested for [priority={}, "
|
|
LOG.info("Opportunistic allocation requested for [priority={}, "
|
|
@@ -379,9 +381,9 @@ public class OpportunisticContainerAllocator {
|
|
|
|
|
|
private void allocateContainersInternal(long rmIdentifier,
|
|
private void allocateContainersInternal(long rmIdentifier,
|
|
AllocationParams appParams, ContainerIdGenerator idCounter,
|
|
AllocationParams appParams, ContainerIdGenerator idCounter,
|
|
- Set<String> blacklist, ApplicationAttemptId id,
|
|
|
|
- Map<String, RemoteNode> allNodes, String userName,
|
|
|
|
- Map<Resource, List<Allocation>> allocations,
|
|
|
|
|
|
+ Set<String> blacklist, Set<String> allocatedNodes,
|
|
|
|
+ ApplicationAttemptId id, Map<String, RemoteNode> allNodes,
|
|
|
|
+ String userName, Map<Resource, List<Allocation>> allocations,
|
|
EnrichedResourceRequest enrichedAsk)
|
|
EnrichedResourceRequest enrichedAsk)
|
|
throws YarnException {
|
|
throws YarnException {
|
|
if (allNodes.size() == 0) {
|
|
if (allNodes.size() == 0) {
|
|
@@ -406,7 +408,8 @@ public class OpportunisticContainerAllocator {
|
|
}
|
|
}
|
|
while (numAllocated < toAllocate) {
|
|
while (numAllocated < toAllocate) {
|
|
Collection<RemoteNode> nodeCandidates =
|
|
Collection<RemoteNode> nodeCandidates =
|
|
- findNodeCandidates(loopIndex, allNodes, blacklist, enrichedAsk);
|
|
|
|
|
|
+ findNodeCandidates(loopIndex, allNodes, blacklist, allocatedNodes,
|
|
|
|
+ enrichedAsk);
|
|
for (RemoteNode rNode : nodeCandidates) {
|
|
for (RemoteNode rNode : nodeCandidates) {
|
|
String rNodeHost = rNode.getNodeId().getHost();
|
|
String rNodeHost = rNode.getNodeId().getHost();
|
|
// Ignore black list
|
|
// Ignore black list
|
|
@@ -422,6 +425,10 @@ public class OpportunisticContainerAllocator {
|
|
} else {
|
|
} else {
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
|
|
+ } else if (allocatedNodes.contains(rNodeHost)) {
|
|
|
|
+ LOG.info("Opportunistic container has already been allocated on {}.",
|
|
|
|
+ rNodeHost);
|
|
|
|
+ continue;
|
|
}
|
|
}
|
|
if (loopIndex == RACK_LOCAL_LOOP) {
|
|
if (loopIndex == RACK_LOCAL_LOOP) {
|
|
if (enrichedAsk.getRackLocations().contains(rNode.getRackName())) {
|
|
if (enrichedAsk.getRackLocations().contains(rNode.getRackName())) {
|
|
@@ -435,11 +442,7 @@ public class OpportunisticContainerAllocator {
|
|
anyAsk, rNode);
|
|
anyAsk, rNode);
|
|
numAllocated++;
|
|
numAllocated++;
|
|
updateMetrics(loopIndex);
|
|
updateMetrics(loopIndex);
|
|
- // Try to spread the allocations across the nodes.
|
|
|
|
- // But don't add if it is a node local request.
|
|
|
|
- if (loopIndex != NODE_LOCAL_LOOP) {
|
|
|
|
- blacklist.add(rNode.getNodeId().getHost());
|
|
|
|
- }
|
|
|
|
|
|
+ allocatedNodes.add(rNodeHost);
|
|
LOG.info("Allocated [" + container.getId() + "] as opportunistic at " +
|
|
LOG.info("Allocated [" + container.getId() + "] as opportunistic at " +
|
|
"location [" + location + "]");
|
|
"location [" + location + "]");
|
|
if (numAllocated >= toAllocate) {
|
|
if (numAllocated >= toAllocate) {
|
|
@@ -475,7 +478,7 @@ public class OpportunisticContainerAllocator {
|
|
|
|
|
|
private Collection<RemoteNode> findNodeCandidates(int loopIndex,
|
|
private Collection<RemoteNode> findNodeCandidates(int loopIndex,
|
|
Map<String, RemoteNode> allNodes, Set<String> blackList,
|
|
Map<String, RemoteNode> allNodes, Set<String> blackList,
|
|
- EnrichedResourceRequest enrichedRR) {
|
|
|
|
|
|
+ Set<String> allocatedNodes, EnrichedResourceRequest enrichedRR) {
|
|
LinkedList<RemoteNode> retList = new LinkedList<>();
|
|
LinkedList<RemoteNode> retList = new LinkedList<>();
|
|
String partition = getRequestPartition(enrichedRR);
|
|
String partition = getRequestPartition(enrichedRR);
|
|
if (loopIndex > 1) {
|
|
if (loopIndex > 1) {
|
|
@@ -495,8 +498,9 @@ public class OpportunisticContainerAllocator {
|
|
allNodes, enrichedRR, retList, numContainers);
|
|
allNodes, enrichedRR, retList, numContainers);
|
|
} else {
|
|
} else {
|
|
// Rack local candidates
|
|
// Rack local candidates
|
|
- numContainers = collectRackLocalCandidates(
|
|
|
|
- allNodes, enrichedRR, retList, blackList, numContainers);
|
|
|
|
|
|
+ numContainers =
|
|
|
|
+ collectRackLocalCandidates(allNodes, enrichedRR, retList,
|
|
|
|
+ blackList, allocatedNodes, numContainers);
|
|
}
|
|
}
|
|
if (numContainers == enrichedRR.getRequest().getNumContainers()) {
|
|
if (numContainers == enrichedRR.getRequest().getNumContainers()) {
|
|
// If there is no change in numContainers, then there is no point
|
|
// If there is no change in numContainers, then there is no point
|
|
@@ -510,12 +514,16 @@ public class OpportunisticContainerAllocator {
|
|
|
|
|
|
private int collectRackLocalCandidates(Map<String, RemoteNode> allNodes,
|
|
private int collectRackLocalCandidates(Map<String, RemoteNode> allNodes,
|
|
EnrichedResourceRequest enrichedRR, LinkedList<RemoteNode> retList,
|
|
EnrichedResourceRequest enrichedRR, LinkedList<RemoteNode> retList,
|
|
- Set<String> blackList, int numContainers) {
|
|
|
|
|
|
+ Set<String> blackList, Set<String> allocatedNodes, int numContainers) {
|
|
String partition = getRequestPartition(enrichedRR);
|
|
String partition = getRequestPartition(enrichedRR);
|
|
for (RemoteNode rNode : allNodes.values()) {
|
|
for (RemoteNode rNode : allNodes.values()) {
|
|
if (StringUtils.equals(partition, getRemoteNodePartition(rNode)) &&
|
|
if (StringUtils.equals(partition, getRemoteNodePartition(rNode)) &&
|
|
enrichedRR.getRackLocations().contains(rNode.getRackName())) {
|
|
enrichedRR.getRackLocations().contains(rNode.getRackName())) {
|
|
- if (blackList.contains(rNode.getNodeId().getHost())) {
|
|
|
|
|
|
+ String rHost = rNode.getNodeId().getHost();
|
|
|
|
+ if (blackList.contains(rHost)) {
|
|
|
|
+ continue;
|
|
|
|
+ }
|
|
|
|
+ if (allocatedNodes.contains(rHost)) {
|
|
retList.addLast(rNode);
|
|
retList.addLast(rNode);
|
|
} else {
|
|
} else {
|
|
retList.addFirst(rNode);
|
|
retList.addFirst(rNode);
|