|
@@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
|
|
|
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.security.BaseContainerTokenSecretManager;
|
|
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
|
|
import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
|
|
@@ -174,17 +175,14 @@ public class OpportunisticContainerAllocator {
|
|
|
new DominantResourceCalculator();
|
|
|
|
|
|
private final BaseContainerTokenSecretManager tokenSecretManager;
|
|
|
- private int webpagePort;
|
|
|
|
|
|
/**
|
|
|
* Create a new Opportunistic Container Allocator.
|
|
|
* @param tokenSecretManager TokenSecretManager
|
|
|
- * @param webpagePort Webpage Port
|
|
|
*/
|
|
|
public OpportunisticContainerAllocator(
|
|
|
- BaseContainerTokenSecretManager tokenSecretManager, int webpagePort) {
|
|
|
+ BaseContainerTokenSecretManager tokenSecretManager) {
|
|
|
this.tokenSecretManager = tokenSecretManager;
|
|
|
- this.webpagePort = webpagePort;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -271,15 +269,15 @@ public class OpportunisticContainerAllocator {
|
|
|
private void allocateContainersInternal(long rmIdentifier,
|
|
|
AllocationParams appParams, ContainerIdGenerator idCounter,
|
|
|
Set<String> blacklist, ApplicationAttemptId id,
|
|
|
- Map<String, NodeId> allNodes, String userName,
|
|
|
+ Map<String, RemoteNode> allNodes, String userName,
|
|
|
Map<Resource, List<Container>> containers, ResourceRequest anyAsk)
|
|
|
throws YarnException {
|
|
|
int toAllocate = anyAsk.getNumContainers()
|
|
|
- (containers.isEmpty() ? 0 :
|
|
|
containers.get(anyAsk.getCapability()).size());
|
|
|
|
|
|
- List<NodeId> nodesForScheduling = new ArrayList<>();
|
|
|
- for (Entry<String, NodeId> nodeEntry : allNodes.entrySet()) {
|
|
|
+ List<RemoteNode> nodesForScheduling = new ArrayList<>();
|
|
|
+ for (Entry<String, RemoteNode> nodeEntry : allNodes.entrySet()) {
|
|
|
// Do not use blacklisted nodes for scheduling.
|
|
|
if (blacklist.contains(nodeEntry.getKey())) {
|
|
|
continue;
|
|
@@ -295,9 +293,9 @@ public class OpportunisticContainerAllocator {
|
|
|
for (int numCont = 0; numCont < toAllocate; numCont++) {
|
|
|
nextNodeToSchedule++;
|
|
|
nextNodeToSchedule %= nodesForScheduling.size();
|
|
|
- NodeId nodeId = nodesForScheduling.get(nextNodeToSchedule);
|
|
|
+ RemoteNode node = nodesForScheduling.get(nextNodeToSchedule);
|
|
|
Container container = buildContainer(rmIdentifier, appParams, idCounter,
|
|
|
- anyAsk, id, userName, nodeId);
|
|
|
+ anyAsk, id, userName, node);
|
|
|
List<Container> cList = containers.get(anyAsk.getCapability());
|
|
|
if (cList == null) {
|
|
|
cList = new ArrayList<>();
|
|
@@ -313,7 +311,7 @@ public class OpportunisticContainerAllocator {
|
|
|
private Container buildContainer(long rmIdentifier,
|
|
|
AllocationParams appParams, ContainerIdGenerator idCounter,
|
|
|
ResourceRequest rr, ApplicationAttemptId id, String userName,
|
|
|
- NodeId nodeId) throws YarnException {
|
|
|
+ RemoteNode node) throws YarnException {
|
|
|
ContainerId cId =
|
|
|
ContainerId.newContainerId(id, idCounter.generateContainerId());
|
|
|
|
|
@@ -324,7 +322,7 @@ public class OpportunisticContainerAllocator {
|
|
|
long currTime = System.currentTimeMillis();
|
|
|
ContainerTokenIdentifier containerTokenIdentifier =
|
|
|
new ContainerTokenIdentifier(
|
|
|
- cId, 0, nodeId.getHost() + ":" + nodeId.getPort(), userName,
|
|
|
+ cId, 0, node.getNodeId().toString(), userName,
|
|
|
capability, currTime + appParams.containerTokenExpiryInterval,
|
|
|
tokenSecretManager.getCurrentKey().getKeyId(), rmIdentifier,
|
|
|
rr.getPriority(), currTime,
|
|
@@ -332,10 +330,10 @@ public class OpportunisticContainerAllocator {
|
|
|
ExecutionType.OPPORTUNISTIC);
|
|
|
byte[] pwd =
|
|
|
tokenSecretManager.createPassword(containerTokenIdentifier);
|
|
|
- Token containerToken = newContainerToken(nodeId, pwd,
|
|
|
+ Token containerToken = newContainerToken(node.getNodeId(), pwd,
|
|
|
containerTokenIdentifier);
|
|
|
Container container = BuilderUtils.newContainer(
|
|
|
- cId, nodeId, nodeId.getHost() + ":" + webpagePort,
|
|
|
+ cId, node.getNodeId(), node.getHttpAddress(),
|
|
|
capability, rr.getPriority(), containerToken,
|
|
|
containerTokenIdentifier.getExecutionType(),
|
|
|
rr.getAllocationRequestId());
|