|
@@ -32,34 +32,23 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterReque
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
|
-import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|
|
import org.apache.hadoop.yarn.api.records.NMToken;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
-import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
-import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
-import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
|
|
|
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.AMRMProxyApplicationContext;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.AbstractRequestInterceptor;
|
|
|
-
|
|
|
-
|
|
|
-
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
|
|
|
+
|
|
|
+import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator;
|
|
|
+import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
-import java.util.HashMap;
|
|
|
-import java.util.HashSet;
|
|
|
-import java.util.LinkedHashMap;
|
|
|
import java.util.List;
|
|
|
-import java.util.Map;
|
|
|
-import java.util.Set;
|
|
|
-import java.util.TreeMap;
|
|
|
|
|
|
/**
|
|
|
* <p>The DistributedScheduler runs on the NodeManager and is modeled as an
|
|
@@ -76,74 +65,49 @@ import java.util.TreeMap;
|
|
|
*/
|
|
|
public final class DistributedScheduler extends AbstractRequestInterceptor {
|
|
|
|
|
|
- static class PartitionedResourceRequests {
|
|
|
- private List<ResourceRequest> guaranteed = new ArrayList<>();
|
|
|
- private List<ResourceRequest> opportunistic = new ArrayList<>();
|
|
|
- public List<ResourceRequest> getGuaranteed() {
|
|
|
- return guaranteed;
|
|
|
- }
|
|
|
- public List<ResourceRequest> getOpportunistic() {
|
|
|
- return opportunistic;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- static class DistributedSchedulerParams {
|
|
|
- Resource maxResource;
|
|
|
- Resource minResource;
|
|
|
- Resource incrementResource;
|
|
|
- int containerTokenExpiryInterval;
|
|
|
- }
|
|
|
-
|
|
|
private static final Logger LOG = LoggerFactory
|
|
|
.getLogger(DistributedScheduler.class);
|
|
|
|
|
|
private final static RecordFactory RECORD_FACTORY =
|
|
|
RecordFactoryProvider.getRecordFactory(null);
|
|
|
|
|
|
- // Currently just used to keep track of allocated containers.
|
|
|
- // Can be used for reporting stats later.
|
|
|
- private Set<ContainerId> containersAllocated = new HashSet<>();
|
|
|
-
|
|
|
- private DistributedSchedulerParams appParams =
|
|
|
- new DistributedSchedulerParams();
|
|
|
- private final OpportunisticContainerAllocator.ContainerIdCounter
|
|
|
- containerIdCounter =
|
|
|
- new OpportunisticContainerAllocator.ContainerIdCounter();
|
|
|
- private Map<String, NodeId> nodeList = new LinkedHashMap<>();
|
|
|
-
|
|
|
- // Mapping of NodeId to NodeTokens. Populated either from RM response or
|
|
|
- // generated locally if required.
|
|
|
- private Map<NodeId, NMToken> nodeTokens = new HashMap<>();
|
|
|
- final Set<String> blacklist = new HashSet<>();
|
|
|
-
|
|
|
- // This maintains a map of outstanding OPPORTUNISTIC Reqs. Key-ed by Priority,
|
|
|
- // Resource Name (Host/rack/any) and capability. This mapping is required
|
|
|
- // to match a received Container to an outstanding OPPORTUNISTIC
|
|
|
- // ResourceRequest (ask).
|
|
|
- final TreeMap<Priority, Map<Resource, ResourceRequest>>
|
|
|
- outstandingOpReqs = new TreeMap<>();
|
|
|
+ private OpportunisticContainerContext oppContainerContext =
|
|
|
+ new OpportunisticContainerContext();
|
|
|
|
|
|
private ApplicationAttemptId applicationAttemptId;
|
|
|
private OpportunisticContainerAllocator containerAllocator;
|
|
|
private NMTokenSecretManagerInNM nmSecretManager;
|
|
|
private String appSubmitter;
|
|
|
-
|
|
|
- public void init(AMRMProxyApplicationContext appContext) {
|
|
|
- super.init(appContext);
|
|
|
- initLocal(appContext.getApplicationAttemptId(),
|
|
|
- appContext.getNMCotext().getContainerAllocator(),
|
|
|
- appContext.getNMCotext().getNMTokenSecretManager(),
|
|
|
- appContext.getUser());
|
|
|
+ private long rmIdentifier;
|
|
|
+
|
|
|
+ public void init(AMRMProxyApplicationContext applicationContext) {
|
|
|
+ super.init(applicationContext);
|
|
|
+ initLocal(applicationContext.getNMCotext().getNodeStatusUpdater()
|
|
|
+ .getRMIdentifier(),
|
|
|
+ applicationContext.getApplicationAttemptId(),
|
|
|
+ applicationContext.getNMCotext().getContainerAllocator(),
|
|
|
+ applicationContext.getNMCotext().getNMTokenSecretManager(),
|
|
|
+ applicationContext.getUser());
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- void initLocal(ApplicationAttemptId applicationAttemptId,
|
|
|
- OpportunisticContainerAllocator containerAllocator,
|
|
|
+ void initLocal(long rmId, ApplicationAttemptId appAttemptId,
|
|
|
+ OpportunisticContainerAllocator oppContainerAllocator,
|
|
|
NMTokenSecretManagerInNM nmSecretManager, String appSubmitter) {
|
|
|
- this.applicationAttemptId = applicationAttemptId;
|
|
|
- this.containerAllocator = containerAllocator;
|
|
|
+ this.rmIdentifier = rmId;
|
|
|
+ this.applicationAttemptId = appAttemptId;
|
|
|
+ this.containerAllocator = oppContainerAllocator;
|
|
|
this.nmSecretManager = nmSecretManager;
|
|
|
this.appSubmitter = appSubmitter;
|
|
|
+
|
|
|
+ // Overrides the Generator to decrement container id.
|
|
|
+ this.oppContainerContext.setContainerIdGenerator(
|
|
|
+ new OpportunisticContainerAllocator.ContainerIdGenerator() {
|
|
|
+ @Override
|
|
|
+ public long generateContainerId() {
|
|
|
+ return this.containerIdCounter.decrementAndGet();
|
|
|
+ }
|
|
|
+ });
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -202,7 +166,8 @@ public final class DistributedScheduler extends AbstractRequestInterceptor {
|
|
|
if (allocatedContainers.size() > 0) {
|
|
|
response.getAllocatedContainers().addAll(allocatedContainers);
|
|
|
for (Container alloc : allocatedContainers) {
|
|
|
- if (!nodeTokens.containsKey(alloc.getNodeId())) {
|
|
|
+ if (!oppContainerContext.getNodeTokens().containsKey(
|
|
|
+ alloc.getNodeId())) {
|
|
|
newTokens.add(nmSecretManager.generateNMToken(appSubmitter, alloc));
|
|
|
}
|
|
|
}
|
|
@@ -212,115 +177,34 @@ public final class DistributedScheduler extends AbstractRequestInterceptor {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private PartitionedResourceRequests partitionAskList(List<ResourceRequest>
|
|
|
- askList) {
|
|
|
- PartitionedResourceRequests partitionedRequests =
|
|
|
- new PartitionedResourceRequests();
|
|
|
- for (ResourceRequest rr : askList) {
|
|
|
- if (rr.getExecutionTypeRequest().getExecutionType() ==
|
|
|
- ExecutionType.OPPORTUNISTIC) {
|
|
|
- partitionedRequests.getOpportunistic().add(rr);
|
|
|
- } else {
|
|
|
- partitionedRequests.getGuaranteed().add(rr);
|
|
|
- }
|
|
|
- }
|
|
|
- return partitionedRequests;
|
|
|
- }
|
|
|
-
|
|
|
private void updateParameters(
|
|
|
RegisterDistributedSchedulingAMResponse registerResponse) {
|
|
|
- appParams.minResource = registerResponse.getMinContainerResource();
|
|
|
- appParams.maxResource = registerResponse.getMaxContainerResource();
|
|
|
- appParams.incrementResource =
|
|
|
- registerResponse.getIncrContainerResource();
|
|
|
- if (appParams.incrementResource == null) {
|
|
|
- appParams.incrementResource = appParams.minResource;
|
|
|
+ oppContainerContext.getAppParams().setMinResource(
|
|
|
+ registerResponse.getMinContainerResource());
|
|
|
+ oppContainerContext.getAppParams().setMaxResource(
|
|
|
+ registerResponse.getMaxContainerResource());
|
|
|
+ oppContainerContext.getAppParams().setIncrementResource(
|
|
|
+ registerResponse.getIncrContainerResource());
|
|
|
+ if (oppContainerContext.getAppParams().getIncrementResource() == null) {
|
|
|
+ oppContainerContext.getAppParams().setIncrementResource(
|
|
|
+ oppContainerContext.getAppParams().getMinResource());
|
|
|
}
|
|
|
- appParams.containerTokenExpiryInterval = registerResponse
|
|
|
- .getContainerTokenExpiryInterval();
|
|
|
+ oppContainerContext.getAppParams().setContainerTokenExpiryInterval(
|
|
|
+ registerResponse.getContainerTokenExpiryInterval());
|
|
|
|
|
|
- containerIdCounter
|
|
|
+ oppContainerContext.getContainerIdGenerator()
|
|
|
.resetContainerIdCounter(registerResponse.getContainerIdStart());
|
|
|
setNodeList(registerResponse.getNodesForScheduling());
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Takes a list of ResourceRequests (asks), extracts the key information viz.
|
|
|
- * (Priority, ResourceName, Capability) and adds to the outstanding
|
|
|
- * OPPORTUNISTIC outstandingOpReqs map. The nested map is required to enforce
|
|
|
- * the current YARN constraint that only a single ResourceRequest can exist at
|
|
|
- * a give Priority and Capability.
|
|
|
- *
|
|
|
- * @param resourceAsks the list with the {@link ResourceRequest}s
|
|
|
- */
|
|
|
- public void addToOutstandingReqs(List<ResourceRequest> resourceAsks) {
|
|
|
- for (ResourceRequest request : resourceAsks) {
|
|
|
- Priority priority = request.getPriority();
|
|
|
-
|
|
|
- // TODO: Extend for Node/Rack locality. We only handle ANY requests now
|
|
|
- if (!ResourceRequest.isAnyLocation(request.getResourceName())) {
|
|
|
- continue;
|
|
|
- }
|
|
|
-
|
|
|
- if (request.getNumContainers() == 0) {
|
|
|
- continue;
|
|
|
- }
|
|
|
-
|
|
|
- Map<Resource, ResourceRequest> reqMap =
|
|
|
- this.outstandingOpReqs.get(priority);
|
|
|
- if (reqMap == null) {
|
|
|
- reqMap = new HashMap<>();
|
|
|
- this.outstandingOpReqs.put(priority, reqMap);
|
|
|
- }
|
|
|
-
|
|
|
- ResourceRequest resourceRequest = reqMap.get(request.getCapability());
|
|
|
- if (resourceRequest == null) {
|
|
|
- resourceRequest = request;
|
|
|
- reqMap.put(request.getCapability(), request);
|
|
|
- } else {
|
|
|
- resourceRequest.setNumContainers(
|
|
|
- resourceRequest.getNumContainers() + request.getNumContainers());
|
|
|
- }
|
|
|
- if (ResourceRequest.isAnyLocation(request.getResourceName())) {
|
|
|
- LOG.info("# of outstandingOpReqs in ANY (at priority = " + priority
|
|
|
- + ", with capability = " + request.getCapability() + " ) : "
|
|
|
- + resourceRequest.getNumContainers());
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * This method matches a returned list of Container Allocations to any
|
|
|
- * outstanding OPPORTUNISTIC ResourceRequest.
|
|
|
- */
|
|
|
- private void matchAllocationToOutstandingRequest(Resource capability,
|
|
|
- List<Container> allocatedContainers) {
|
|
|
- for (Container c : allocatedContainers) {
|
|
|
- containersAllocated.add(c.getId());
|
|
|
- Map<Resource, ResourceRequest> asks =
|
|
|
- outstandingOpReqs.get(c.getPriority());
|
|
|
-
|
|
|
- if (asks == null)
|
|
|
- continue;
|
|
|
-
|
|
|
- ResourceRequest rr = asks.get(capability);
|
|
|
- if (rr != null) {
|
|
|
- rr.setNumContainers(rr.getNumContainers() - 1);
|
|
|
- if (rr.getNumContainers() == 0) {
|
|
|
- asks.remove(capability);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
private void setNodeList(List<NodeId> nodeList) {
|
|
|
- this.nodeList.clear();
|
|
|
+ oppContainerContext.getNodeMap().clear();
|
|
|
addToNodeList(nodeList);
|
|
|
}
|
|
|
|
|
|
private void addToNodeList(List<NodeId> nodes) {
|
|
|
for (NodeId n : nodes) {
|
|
|
- this.nodeList.put(n.getHost(), n);
|
|
|
+ oppContainerContext.getNodeMap().put(n.getHost(), n);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -345,52 +229,13 @@ public final class DistributedScheduler extends AbstractRequestInterceptor {
|
|
|
LOG.debug("Forwarding allocate request to the" +
|
|
|
"Distributed Scheduler Service on YARN RM");
|
|
|
}
|
|
|
- // Partition requests into GUARANTEED and OPPORTUNISTIC reqs
|
|
|
- PartitionedResourceRequests partitionedAsks =
|
|
|
- partitionAskList(request.getAllocateRequest().getAskList());
|
|
|
-
|
|
|
- List<ContainerId> releasedContainers =
|
|
|
- request.getAllocateRequest().getReleaseList();
|
|
|
- int numReleasedContainers = releasedContainers.size();
|
|
|
- if (numReleasedContainers > 0) {
|
|
|
- LOG.info("AttemptID: " + applicationAttemptId + " released: "
|
|
|
- + numReleasedContainers);
|
|
|
- containersAllocated.removeAll(releasedContainers);
|
|
|
- }
|
|
|
-
|
|
|
- // Also, update black list
|
|
|
- ResourceBlacklistRequest rbr =
|
|
|
- request.getAllocateRequest().getResourceBlacklistRequest();
|
|
|
- if (rbr != null) {
|
|
|
- blacklist.removeAll(rbr.getBlacklistRemovals());
|
|
|
- blacklist.addAll(rbr.getBlacklistAdditions());
|
|
|
- }
|
|
|
-
|
|
|
- // Add OPPORTUNISTIC reqs to the outstanding reqs
|
|
|
- addToOutstandingReqs(partitionedAsks.getOpportunistic());
|
|
|
-
|
|
|
- List<Container> allocatedContainers = new ArrayList<>();
|
|
|
- for (Priority priority : outstandingOpReqs.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)
|
|
|
- Map<Resource, List<Container>> allocated =
|
|
|
- containerAllocator.allocate(this.appParams, containerIdCounter,
|
|
|
- outstandingOpReqs.get(priority).values(), blacklist,
|
|
|
- applicationAttemptId, nodeList, appSubmitter);
|
|
|
- for (Map.Entry<Resource, List<Container>> e : allocated.entrySet()) {
|
|
|
- matchAllocationToOutstandingRequest(e.getKey(), e.getValue());
|
|
|
- allocatedContainers.addAll(e.getValue());
|
|
|
- }
|
|
|
- }
|
|
|
+ List<Container> allocatedContainers =
|
|
|
+ containerAllocator.allocateContainers(
|
|
|
+ request.getAllocateRequest(), applicationAttemptId,
|
|
|
+ oppContainerContext, rmIdentifier, appSubmitter);
|
|
|
|
|
|
request.setAllocatedContainers(allocatedContainers);
|
|
|
|
|
|
- // Send all the GUARANTEED Reqs to RM
|
|
|
- request.getAllocateRequest().setAskList(partitionedAsks.getGuaranteed());
|
|
|
DistributedSchedulingAllocateResponse dsResp =
|
|
|
getNextInterceptor().allocateForDistributedScheduling(request);
|
|
|
|
|
@@ -398,7 +243,7 @@ public final class DistributedScheduler extends AbstractRequestInterceptor {
|
|
|
setNodeList(dsResp.getNodesForScheduling());
|
|
|
List<NMToken> nmTokens = dsResp.getAllocateResponse().getNMTokens();
|
|
|
for (NMToken nmToken : nmTokens) {
|
|
|
- nodeTokens.put(nmToken.getNodeId(), nmToken);
|
|
|
+ oppContainerContext.getNodeTokens().put(nmToken.getNodeId(), nmToken);
|
|
|
}
|
|
|
|
|
|
List<ContainerStatus> completedContainers =
|
|
@@ -407,7 +252,8 @@ public final class DistributedScheduler extends AbstractRequestInterceptor {
|
|
|
// Only account for opportunistic containers
|
|
|
for (ContainerStatus cs : completedContainers) {
|
|
|
if (cs.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
|
|
|
- containersAllocated.remove(cs.getContainerId());
|
|
|
+ oppContainerContext.getContainersAllocated()
|
|
|
+ .remove(cs.getContainerId());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -417,9 +263,9 @@ public final class DistributedScheduler extends AbstractRequestInterceptor {
|
|
|
dsResp.getAllocateResponse(), nmTokens, allocatedContainers);
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug(
|
|
|
- "Number of opportunistic containers currently allocated by" +
|
|
|
- "application: " + containersAllocated.size());
|
|
|
+ LOG.debug("Number of opportunistic containers currently" +
|
|
|
+ "allocated by application: " + oppContainerContext
|
|
|
+ .getContainersAllocated().size());
|
|
|
}
|
|
|
return dsResp;
|
|
|
}
|