|
@@ -18,6 +18,7 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.scheduler;
|
|
|
|
|
|
+import org.apache.commons.lang.StringUtils;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
@@ -461,10 +462,17 @@ public class OpportunisticContainerAllocator {
|
|
|
private Collection<RemoteNode> findNodeCandidates(int loopIndex,
|
|
|
Map<String, RemoteNode> allNodes, Set<String> blackList,
|
|
|
EnrichedResourceRequest enrichedRR) {
|
|
|
+ LinkedList<RemoteNode> retList = new LinkedList<>();
|
|
|
+ String partition = getRequestPartition(enrichedRR);
|
|
|
if (loopIndex > 1) {
|
|
|
- return allNodes.values();
|
|
|
+ for (RemoteNode remoteNode : allNodes.values()) {
|
|
|
+ if (StringUtils.equals(partition, getRemoteNodePartition(remoteNode))) {
|
|
|
+ retList.add(remoteNode);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return retList;
|
|
|
} else {
|
|
|
- LinkedList<RemoteNode> retList = new LinkedList<>();
|
|
|
+
|
|
|
int numContainers = enrichedRR.getRequest().getNumContainers();
|
|
|
while (numContainers > 0) {
|
|
|
if (loopIndex == 0) {
|
|
@@ -489,8 +497,10 @@ public class OpportunisticContainerAllocator {
|
|
|
private int collectRackLocalCandidates(Map<String, RemoteNode> allNodes,
|
|
|
EnrichedResourceRequest enrichedRR, LinkedList<RemoteNode> retList,
|
|
|
Set<String> blackList, int numContainers) {
|
|
|
+ String partition = getRequestPartition(enrichedRR);
|
|
|
for (RemoteNode rNode : allNodes.values()) {
|
|
|
- if (enrichedRR.getRackLocations().contains(rNode.getRackName())) {
|
|
|
+ if (StringUtils.equals(partition, getRemoteNodePartition(rNode)) &&
|
|
|
+ enrichedRR.getRackLocations().contains(rNode.getRackName())) {
|
|
|
if (blackList.contains(rNode.getNodeId().getHost())) {
|
|
|
retList.addLast(rNode);
|
|
|
} else {
|
|
@@ -508,9 +518,11 @@ public class OpportunisticContainerAllocator {
|
|
|
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) {
|
|
|
+ if (remoteNode != null &&
|
|
|
+ StringUtils.equals(partition, getRemoteNodePartition(remoteNode))) {
|
|
|
retList.add(remoteNode);
|
|
|
numContainers--;
|
|
|
}
|
|
@@ -563,7 +575,7 @@ public class OpportunisticContainerAllocator {
|
|
|
capability, currTime + tokenExpiry,
|
|
|
tokenSecretManager.getCurrentKey().getKeyId(), rmIdentifier,
|
|
|
schedulerKey.getPriority(), currTime,
|
|
|
- null, CommonNodeLabelsManager.NO_LABEL, ContainerType.TASK,
|
|
|
+ null, getRemoteNodePartition(node), ContainerType.TASK,
|
|
|
ExecutionType.OPPORTUNISTIC, schedulerKey.getAllocationRequestId());
|
|
|
byte[] pwd =
|
|
|
tokenSecretManager.createPassword(containerTokenIdentifier);
|
|
@@ -616,4 +628,20 @@ public class OpportunisticContainerAllocator {
|
|
|
}
|
|
|
return partitionedRequests;
|
|
|
}
|
|
|
+
|
|
|
+ private String getRequestPartition(EnrichedResourceRequest enrichedRR) {
|
|
|
+ String partition = enrichedRR.getRequest().getNodeLabelExpression();
|
|
|
+ if (partition == null) {
|
|
|
+ partition = CommonNodeLabelsManager.NO_LABEL;
|
|
|
+ }
|
|
|
+ return partition;
|
|
|
+ }
|
|
|
+
|
|
|
+ private String getRemoteNodePartition(RemoteNode node) {
|
|
|
+ String partition = node.getNodePartition();
|
|
|
+ if (partition == null) {
|
|
|
+ partition = CommonNodeLabelsManager.NO_LABEL;
|
|
|
+ }
|
|
|
+ return partition;
|
|
|
+ }
|
|
|
}
|