YARN-5716. Add global scheduler interface definition and update CapacityScheduler to use it. Contributed by Wangda Tan
This commit is contained in:
parent
acd509dc57
commit
de3b4aac56
|
@ -574,4 +574,13 @@
|
|||
</Or>
|
||||
<Bug pattern="VO_VOLATILE_INCREMENT" />
|
||||
</Match>
|
||||
|
||||
|
||||
<!-- Ignore false alert for UL_UNRELEASED_LOCK_EXCEPTION_PATH -->
|
||||
<Match>
|
||||
<Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler$ResourceCommitterService"/>
|
||||
<Method name="run" />
|
||||
<Bug pattern="UL_UNRELEASED_LOCK_EXCEPTION_PATH" />
|
||||
</Match>
|
||||
|
||||
</FindBugsFilter>
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.api.records.Container;
|
|||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerReport;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerState;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.api.records.Priority;
|
||||
|
@ -46,6 +47,8 @@ public interface RMContainer extends EventHandler<RMContainerEvent> {
|
|||
|
||||
ContainerId getContainerId();
|
||||
|
||||
void setContainerId(ContainerId containerId);
|
||||
|
||||
ApplicationAttemptId getApplicationAttemptId();
|
||||
|
||||
RMContainerState getState();
|
||||
|
@ -105,4 +108,14 @@ public interface RMContainer extends EventHandler<RMContainerEvent> {
|
|||
* @return If the container was allocated remotely.
|
||||
*/
|
||||
boolean isRemotelyAllocated();
|
||||
|
||||
/*
|
||||
* Return reserved resource for reserved containers, return allocated resource
|
||||
* for other container
|
||||
*/
|
||||
Resource getAllocatedOrReservedResource();
|
||||
|
||||
boolean completed();
|
||||
|
||||
NodeId getNodeId();
|
||||
}
|
||||
|
|
|
@ -161,7 +161,6 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
RMContainerEvent> stateMachine;
|
||||
private final ReadLock readLock;
|
||||
private final WriteLock writeLock;
|
||||
private final ContainerId containerId;
|
||||
private final ApplicationAttemptId appAttemptId;
|
||||
private final NodeId nodeId;
|
||||
private final Container container;
|
||||
|
@ -224,7 +223,6 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
RMContext rmContext, long creationTime, String nodeLabelExpression,
|
||||
boolean isExternallyAllocated) {
|
||||
this.stateMachine = stateMachineFactory.make(this);
|
||||
this.containerId = container.getId();
|
||||
this.nodeId = nodeId;
|
||||
this.container = container;
|
||||
this.allocatedSchedulerKey = SchedulerRequestKey.extractFrom(container);
|
||||
|
@ -255,7 +253,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
// containers. If false, and if this container is marked as the AM, metrics
|
||||
// will still be published for this container, but that calculation happens
|
||||
// later.
|
||||
if (saveNonAMContainerMetaInfo) {
|
||||
if (saveNonAMContainerMetaInfo && null != container.getId()) {
|
||||
rmContext.getSystemMetricsPublisher().containerCreated(
|
||||
this, this.creationTime);
|
||||
}
|
||||
|
@ -263,7 +261,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
|
||||
@Override
|
||||
public ContainerId getContainerId() {
|
||||
return this.containerId;
|
||||
return this.container.getId();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -356,8 +354,8 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
public String getDiagnosticsInfo() {
|
||||
try {
|
||||
readLock.lock();
|
||||
if (getFinishedStatus() != null) {
|
||||
return getFinishedStatus().getDiagnostics();
|
||||
if (finishedStatus != null) {
|
||||
return finishedStatus.getDiagnostics();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
@ -374,7 +372,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
logURL.append(WebAppUtils.getHttpSchemePrefix(rmContext
|
||||
.getYarnConfiguration()));
|
||||
logURL.append(WebAppUtils.getRunningLogURL(
|
||||
container.getNodeHttpAddress(), containerId.toString(),
|
||||
container.getNodeHttpAddress(), getContainerId().toString(),
|
||||
user));
|
||||
return logURL.toString();
|
||||
} finally {
|
||||
|
@ -386,8 +384,8 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
public int getContainerExitStatus() {
|
||||
try {
|
||||
readLock.lock();
|
||||
if (getFinishedStatus() != null) {
|
||||
return getFinishedStatus().getExitStatus();
|
||||
if (finishedStatus != null) {
|
||||
return finishedStatus.getExitStatus();
|
||||
} else {
|
||||
return 0;
|
||||
}
|
||||
|
@ -400,8 +398,8 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
public ContainerState getContainerState() {
|
||||
try {
|
||||
readLock.lock();
|
||||
if (getFinishedStatus() != null) {
|
||||
return getFinishedStatus().getState();
|
||||
if (finishedStatus != null) {
|
||||
return finishedStatus.getState();
|
||||
} else {
|
||||
return ContainerState.RUNNING;
|
||||
}
|
||||
|
@ -431,7 +429,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return containerId.toString();
|
||||
return getContainerId().toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -476,7 +474,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
} catch (InvalidStateTransitionException e) {
|
||||
LOG.error("Can't handle this event at current state", e);
|
||||
LOG.error("Invalid event " + event.getType() +
|
||||
" on container " + this.containerId);
|
||||
" on container " + this.getContainerId());
|
||||
}
|
||||
if (oldState != getState()) {
|
||||
LOG.info(event.getContainerId() + " Container Transitioned from "
|
||||
|
@ -489,10 +487,15 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
}
|
||||
}
|
||||
|
||||
public ContainerStatus getFinishedStatus() {
|
||||
return finishedStatus;
|
||||
public boolean completed() {
|
||||
return finishedStatus != null;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public NodeId getNodeId() {
|
||||
return nodeId;
|
||||
}
|
||||
|
||||
private static class BaseTransition implements
|
||||
SingleArcTransition<RMContainerImpl, RMContainerEvent> {
|
||||
|
||||
|
@ -517,7 +520,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
report.getContainerExitStatus());
|
||||
|
||||
new FinishedTransition().transition(container,
|
||||
new RMContainerFinishedEvent(container.containerId, status,
|
||||
new RMContainerFinishedEvent(container.getContainerId(), status,
|
||||
RMContainerEventType.FINISHED));
|
||||
return RMContainerState.COMPLETED;
|
||||
} else if (report.getContainerState().equals(ContainerState.RUNNING)) {
|
||||
|
@ -654,11 +657,11 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
} else {
|
||||
// Something wrong happened, kill the container
|
||||
LOG.warn("Something wrong happened, container size reported by NM"
|
||||
+ " is not expected, ContainerID=" + container.containerId
|
||||
+ " is not expected, ContainerID=" + container.getContainerId()
|
||||
+ " rm-size-resource:" + rmContainerResource + " nm-size-reosurce:"
|
||||
+ nmContainerResource);
|
||||
container.eventHandler.handle(new RMNodeCleanContainerEvent(
|
||||
container.nodeId, container.containerId));
|
||||
container.nodeId, container.getContainerId()));
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -761,7 +764,7 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
|
||||
// Inform node
|
||||
container.eventHandler.handle(new RMNodeCleanContainerEvent(
|
||||
container.nodeId, container.containerId));
|
||||
container.nodeId, container.getContainerId()));
|
||||
|
||||
// Inform appAttempt
|
||||
super.transition(container, event);
|
||||
|
@ -831,8 +834,8 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
|
||||
@Override
|
||||
public int compareTo(RMContainer o) {
|
||||
if (containerId != null && o.getContainerId() != null) {
|
||||
return containerId.compareTo(o.getContainerId());
|
||||
if (getContainerId() != null && o.getContainerId() != null) {
|
||||
return getContainerId().compareTo(o.getContainerId());
|
||||
}
|
||||
return -1;
|
||||
}
|
||||
|
@ -865,4 +868,35 @@ public class RMContainerImpl implements RMContainer, Comparable<RMContainer> {
|
|||
public boolean isRemotelyAllocated() {
|
||||
return isExternallyAllocated;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Resource getAllocatedOrReservedResource() {
|
||||
try {
|
||||
readLock.lock();
|
||||
if (getState().equals(RMContainerState.RESERVED)) {
|
||||
return getReservedResource();
|
||||
} else {
|
||||
return getAllocatedResource();
|
||||
}
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setContainerId(ContainerId containerId) {
|
||||
// In some cases, for example, global scheduling. It is possible that
|
||||
// container created without container-id assigned, so we will publish
|
||||
// container creation event to timeline service when id assigned.
|
||||
container.setId(containerId);
|
||||
|
||||
// If saveNonAMContainerMetaInfo is true, store system metrics for all
|
||||
// containers. If false, and if this container is marked as the AM, metrics
|
||||
// will still be published for this container, but that calculation happens
|
||||
// later.
|
||||
if (saveNonAMContainerMetaInfo && null != container.getId()) {
|
||||
rmContext.getSystemMetricsPublisher().containerCreated(
|
||||
this, this.creationTime);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -32,6 +33,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import org.apache.commons.collections.IteratorUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
|
@ -49,6 +51,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.ResourceRequestUpdateResult;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
|
||||
/**
|
||||
|
@ -691,6 +696,25 @@ public class AppSchedulingInfo {
|
|||
}
|
||||
}
|
||||
|
||||
public List<ResourceRequest> allocate(NodeType type,
|
||||
SchedulerNode node, SchedulerRequestKey schedulerKey,
|
||||
Container containerAllocated) {
|
||||
try {
|
||||
writeLock.lock();
|
||||
ResourceRequest request;
|
||||
if (type == NodeType.NODE_LOCAL) {
|
||||
request = resourceRequestMap.get(schedulerKey).get(node.getNodeName());
|
||||
} else if (type == NodeType.RACK_LOCAL) {
|
||||
request = resourceRequestMap.get(schedulerKey).get(node.getRackName());
|
||||
} else{
|
||||
request = resourceRequestMap.get(schedulerKey).get(ResourceRequest.ANY);
|
||||
}
|
||||
return allocate(type, node, schedulerKey, request, containerAllocated);
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Resources have been allocated to this application by the resource
|
||||
* scheduler. Track them.
|
||||
|
@ -701,40 +725,26 @@ public class AppSchedulingInfo {
|
|||
* @param containerAllocated Container Allocated
|
||||
* @return List of ResourceRequests
|
||||
*/
|
||||
public List<ResourceRequest> allocate(NodeType type, SchedulerNode node,
|
||||
SchedulerRequestKey schedulerKey, ResourceRequest request,
|
||||
Container containerAllocated) {
|
||||
List<ResourceRequest> resourceRequests = new ArrayList<>();
|
||||
public List<ResourceRequest> allocate(NodeType type,
|
||||
SchedulerNode node, SchedulerRequestKey schedulerKey,
|
||||
ResourceRequest request, Container containerAllocated) {
|
||||
try {
|
||||
this.writeLock.lock();
|
||||
writeLock.lock();
|
||||
List<ResourceRequest> resourceRequests = new ArrayList<>();
|
||||
if (type == NodeType.NODE_LOCAL) {
|
||||
allocateNodeLocal(node, schedulerKey, request, resourceRequests);
|
||||
} else if (type == NodeType.RACK_LOCAL) {
|
||||
allocateRackLocal(node, schedulerKey, request, resourceRequests);
|
||||
} else {
|
||||
} else{
|
||||
allocateOffSwitch(request, resourceRequests, schedulerKey);
|
||||
}
|
||||
QueueMetrics metrics = queue.getMetrics();
|
||||
if (pending) {
|
||||
// once an allocation is done we assume the application is
|
||||
// running from scheduler's POV.
|
||||
pending = false;
|
||||
metrics.runAppAttempt(applicationId, user);
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("allocate: applicationId=" + applicationId
|
||||
+ " container=" + containerAllocated.getId()
|
||||
+ " host=" + containerAllocated.getNodeId().toString()
|
||||
+ " user=" + user
|
||||
+ " resource=" + request.getCapability()
|
||||
+ " type=" + type);
|
||||
if (null != containerAllocated) {
|
||||
updateMetricsForAllocatedContainer(request, type, containerAllocated);
|
||||
}
|
||||
metrics.allocateResources(user, 1, request.getCapability(), true);
|
||||
metrics.incrNodeTypeAggregations(user, type);
|
||||
return resourceRequests;
|
||||
} finally {
|
||||
this.writeLock.unlock();
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -942,4 +952,116 @@ public class AppSchedulingInfo {
|
|||
request.getRelaxLocality(), request.getNodeLabelExpression());
|
||||
return newRequest;
|
||||
}
|
||||
|
||||
/*
|
||||
* In async environment, pending resource request could be updated during
|
||||
* scheduling, this method checks pending request before allocating
|
||||
*/
|
||||
public boolean checkAllocation(NodeType type, SchedulerNode node,
|
||||
SchedulerRequestKey schedulerKey) {
|
||||
try {
|
||||
readLock.lock();
|
||||
ResourceRequest r = resourceRequestMap.get(schedulerKey).get(
|
||||
ResourceRequest.ANY);
|
||||
if (r == null || r.getNumContainers() <= 0) {
|
||||
return false;
|
||||
}
|
||||
if (type == NodeType.RACK_LOCAL || type == NodeType.NODE_LOCAL) {
|
||||
r = resourceRequestMap.get(schedulerKey).get(node.getRackName());
|
||||
if (r == null || r.getNumContainers() <= 0) {
|
||||
return false;
|
||||
}
|
||||
if (type == NodeType.NODE_LOCAL) {
|
||||
r = resourceRequestMap.get(schedulerKey).get(node.getNodeName());
|
||||
if (r == null || r.getNumContainers() <= 0) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
public void updateMetricsForAllocatedContainer(
|
||||
ResourceRequest request, NodeType type, Container containerAllocated) {
|
||||
try {
|
||||
writeLock.lock();
|
||||
QueueMetrics metrics = queue.getMetrics();
|
||||
if (pending) {
|
||||
// once an allocation is done we assume the application is
|
||||
// running from scheduler's POV.
|
||||
pending = false;
|
||||
metrics.runAppAttempt(applicationId, user);
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("allocate: applicationId=" + applicationId + " container="
|
||||
+ containerAllocated.getId() + " host=" + containerAllocated
|
||||
.getNodeId().toString() + " user=" + user + " resource=" + request
|
||||
.getCapability() + " type=" + type);
|
||||
}
|
||||
metrics.allocateResources(user, 1, request.getCapability(), true);
|
||||
metrics.incrNodeTypeAggregations(user, type);
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
// Get placement-set by specified schedulerKey
|
||||
// Now simply return all node of the input clusterPlacementSet
|
||||
// TODO, need update this when we support global scheduling
|
||||
public <N extends SchedulerNode> SchedulingPlacementSet<N> getSchedulingPlacementSet(
|
||||
SchedulerRequestKey schedulerkey) {
|
||||
return new SchedulingPlacementSet<N>() {
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public Iterator<N> getPreferredNodeIterator(
|
||||
PlacementSet<N> clusterPlacementSet) {
|
||||
return IteratorUtils.singletonIterator(
|
||||
clusterPlacementSet.getAllNodes().values().iterator().next());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResourceRequestUpdateResult updateResourceRequests(
|
||||
List<ResourceRequest> requests,
|
||||
boolean recoverPreemptedRequestForAContainer) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, ResourceRequest> getResourceRequests() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ResourceRequest getResourceRequest(String resourceName,
|
||||
SchedulerRequestKey requestKey) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ResourceRequest> allocate(NodeType type, SchedulerNode node,
|
||||
ResourceRequest request) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<NodeId, N> getAllNodes() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getVersion() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartition() {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,10 +28,12 @@ import java.util.Map;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import com.google.common.collect.ConcurrentHashMultiset;
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.commons.lang.time.DateUtils;
|
||||
import org.apache.commons.lang.time.FastDateFormat;
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -49,12 +51,14 @@ import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|||
import org.apache.hadoop.yarn.api.records.LogAggregationContext;
|
||||
import org.apache.hadoop.yarn.api.records.NMToken;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.api.records.NodeLabel;
|
||||
import org.apache.hadoop.yarn.api.records.Priority;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
|
||||
import org.apache.hadoop.yarn.server.api.ContainerType;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AggregateAppResourceUsage;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
||||
|
@ -69,6 +73,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerUpdatesAcquiredEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity;
|
||||
|
||||
import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
|
||||
|
@ -178,6 +185,11 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
|||
protected ReentrantReadWriteLock.ReadLock readLock;
|
||||
protected ReentrantReadWriteLock.WriteLock writeLock;
|
||||
|
||||
// Not confirmed allocation resource, will be used to avoid too many proposal
|
||||
// rejected because of duplicated allocation
|
||||
private AtomicLong unconfirmedAllocatedMem = new AtomicLong();
|
||||
private AtomicInteger unconfirmedAllocatedVcores = new AtomicInteger();
|
||||
|
||||
public SchedulerApplicationAttempt(ApplicationAttemptId applicationAttemptId,
|
||||
String user, Queue queue, ActiveUsersManager activeUsersManager,
|
||||
RMContext rmContext) {
|
||||
|
@ -529,6 +541,8 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
|||
if (rmContainer == null) {
|
||||
rmContainer = new RMContainerImpl(container, getApplicationAttemptId(),
|
||||
node.getNodeID(), appSchedulingInfo.getUser(), rmContext);
|
||||
}
|
||||
if (rmContainer.getState() == RMContainerState.NEW) {
|
||||
attemptResourceUsage.incReserved(node.getPartition(),
|
||||
container.getResource());
|
||||
((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
|
||||
|
@ -839,16 +853,10 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
|||
}
|
||||
|
||||
// used for continuous scheduling
|
||||
public void resetSchedulingOpportunities(
|
||||
SchedulerRequestKey schedulerKey, long currentTimeMs) {
|
||||
try {
|
||||
writeLock.lock();
|
||||
lastScheduledContainer.put(schedulerKey, currentTimeMs);
|
||||
schedulingOpportunities.setCount(schedulerKey, 0);
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
|
||||
public void resetSchedulingOpportunities(SchedulerRequestKey schedulerKey,
|
||||
long currentTimeMs) {
|
||||
lastScheduledContainer.put(schedulerKey, currentTimeMs);
|
||||
schedulingOpportunities.setCount(schedulerKey, 0);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
|
@ -998,6 +1006,11 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
|||
|
||||
public void incNumAllocatedContainers(NodeType containerType,
|
||||
NodeType requestType) {
|
||||
if (containerType == null || requestType == null) {
|
||||
// Sanity check
|
||||
return;
|
||||
}
|
||||
|
||||
RMAppAttempt attempt =
|
||||
rmContext.getRMApps().get(attemptId.getApplicationId())
|
||||
.getCurrentAppAttempt();
|
||||
|
@ -1039,9 +1052,27 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
|||
public boolean hasPendingResourceRequest(ResourceCalculator rc,
|
||||
String nodePartition, Resource cluster,
|
||||
SchedulingMode schedulingMode) {
|
||||
return SchedulerUtils.hasPendingResourceRequest(rc,
|
||||
this.attemptResourceUsage, nodePartition, cluster,
|
||||
schedulingMode);
|
||||
// We need to consider unconfirmed allocations
|
||||
if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) {
|
||||
nodePartition = RMNodeLabelsManager.NO_LABEL;
|
||||
}
|
||||
|
||||
Resource pending = attemptResourceUsage.getPending(nodePartition);
|
||||
|
||||
// TODO, need consider node partition here
|
||||
// To avoid too many allocation-proposals rejected for non-default
|
||||
// partition allocation
|
||||
if (StringUtils.equals(nodePartition, RMNodeLabelsManager.NO_LABEL)) {
|
||||
pending = Resources.subtract(pending, Resources
|
||||
.createResource(unconfirmedAllocatedMem.get(),
|
||||
unconfirmedAllocatedVcores.get()));
|
||||
}
|
||||
|
||||
if (Resources.greaterThan(rc, cluster, pending, Resources.none())) {
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
|
@ -1206,6 +1237,22 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
|
|||
this.isAttemptRecovering = isRecovering;
|
||||
}
|
||||
|
||||
public <N extends SchedulerNode> SchedulingPlacementSet<N> getSchedulingPlacementSet(
|
||||
SchedulerRequestKey schedulerRequestKey) {
|
||||
return appSchedulingInfo.getSchedulingPlacementSet(schedulerRequestKey);
|
||||
}
|
||||
|
||||
|
||||
public void incUnconfirmedRes(Resource res) {
|
||||
unconfirmedAllocatedMem.addAndGet(res.getMemorySize());
|
||||
unconfirmedAllocatedVcores.addAndGet(res.getVirtualCores());
|
||||
}
|
||||
|
||||
public void decUnconfirmedRes(Resource res) {
|
||||
unconfirmedAllocatedMem.addAndGet(-res.getMemorySize());
|
||||
unconfirmedAllocatedVcores.addAndGet(-res.getVirtualCores());
|
||||
}
|
||||
|
||||
/**
|
||||
* Different state for Application Master, user can see this state from web UI
|
||||
*/
|
||||
|
|
|
@ -25,12 +25,14 @@ import org.apache.hadoop.yarn.api.records.Container;
|
|||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.api.records.Priority;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||
|
||||
/**
|
||||
* Utility for logging scheduler activities
|
||||
*/
|
||||
// FIXME: make sure PlacementSet works with this class
|
||||
public class ActivitiesLogger {
|
||||
private static final Log LOG = LogFactory.getLog(ActivitiesLogger.class);
|
||||
|
||||
|
@ -112,7 +114,7 @@ public class ActivitiesLogger {
|
|||
*/
|
||||
public static void recordAppActivityWithAllocation(
|
||||
ActivitiesManager activitiesManager, SchedulerNode node,
|
||||
SchedulerApplicationAttempt application, Container updatedContainer,
|
||||
SchedulerApplicationAttempt application, RMContainer updatedContainer,
|
||||
ActivityState activityState) {
|
||||
if (activitiesManager == null) {
|
||||
return;
|
||||
|
@ -122,9 +124,9 @@ public class ActivitiesLogger {
|
|||
// Add application-container activity into specific node allocation.
|
||||
activitiesManager.addSchedulingActivityForNode(node.getNodeID(),
|
||||
application.getApplicationId().toString(),
|
||||
updatedContainer.getId().toString(),
|
||||
updatedContainer.getPriority().toString(), activityState,
|
||||
ActivityDiagnosticConstant.EMPTY, type);
|
||||
updatedContainer.getContainer().toString(),
|
||||
updatedContainer.getContainer().getPriority().toString(),
|
||||
activityState, ActivityDiagnosticConstant.EMPTY, type);
|
||||
type = "app";
|
||||
// Add queue-application activity into specific node allocation.
|
||||
activitiesManager.addSchedulingActivityForNode(node.getNodeID(),
|
||||
|
@ -138,9 +140,10 @@ public class ActivitiesLogger {
|
|||
application.getApplicationId())) {
|
||||
String type = "container";
|
||||
activitiesManager.addSchedulingActivityForApp(
|
||||
application.getApplicationId(), updatedContainer.getId().toString(),
|
||||
updatedContainer.getPriority().toString(), activityState,
|
||||
ActivityDiagnosticConstant.EMPTY, type);
|
||||
application.getApplicationId(),
|
||||
updatedContainer.getContainerId(),
|
||||
updatedContainer.getContainer().getPriority().toString(),
|
||||
activityState, ActivityDiagnosticConstant.EMPTY, type);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -200,12 +200,13 @@ public class ActivitiesManager extends AbstractService {
|
|||
// Add queue, application or container activity into specific application
|
||||
// allocation.
|
||||
void addSchedulingActivityForApp(ApplicationId applicationId,
|
||||
String containerId, String priority, ActivityState state,
|
||||
ContainerId containerId, String priority, ActivityState state,
|
||||
String diagnostic, String type) {
|
||||
if (shouldRecordThisApp(applicationId)) {
|
||||
AppAllocation appAllocation = appsAllocation.get(applicationId);
|
||||
appAllocation.addAppAllocationActivity(containerId, priority, state,
|
||||
diagnostic, type);
|
||||
appAllocation.addAppAllocationActivity(containerId == null ?
|
||||
"Container-Id-Not-Assigned" :
|
||||
containerId.toString(), priority, state, diagnostic, type);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -54,6 +55,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimplePlacementSet;
|
||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
|
||||
|
@ -738,4 +745,68 @@ public abstract class AbstractCSQueue implements CSQueue {
|
|||
return csContext.getPreemptionManager().getKillableContainers(queueName,
|
||||
partition);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
@Override
|
||||
public CSAssignment assignContainers(Resource clusterResource,
|
||||
FiCaSchedulerNode node, ResourceLimits resourceLimits,
|
||||
SchedulingMode schedulingMode) {
|
||||
return assignContainers(clusterResource, new SimplePlacementSet<>(node),
|
||||
resourceLimits, schedulingMode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean accept(Resource cluster,
|
||||
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request) {
|
||||
// Do we need to check parent queue before making this decision?
|
||||
boolean checkParentQueue = false;
|
||||
|
||||
ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode> allocation =
|
||||
request.getFirstAllocatedOrReservedContainer();
|
||||
SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> schedulerContainer =
|
||||
allocation.getAllocatedOrReservedContainer();
|
||||
|
||||
// Do not check when allocating new container from a reserved container
|
||||
if (allocation.getAllocateFromReservedContainer() == null) {
|
||||
Resource required = allocation.getAllocatedOrReservedResource();
|
||||
Resource netAllocated = Resources.subtract(required,
|
||||
request.getTotalReleasedResource());
|
||||
|
||||
try {
|
||||
readLock.lock();
|
||||
|
||||
String partition = schedulerContainer.getNodePartition();
|
||||
Resource maxResourceLimit;
|
||||
if (allocation.getSchedulingMode()
|
||||
== SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
|
||||
maxResourceLimit = getQueueMaxResource(partition, cluster);
|
||||
} else{
|
||||
maxResourceLimit = labelManager.getResourceByLabel(
|
||||
schedulerContainer.getNodePartition(), cluster);
|
||||
}
|
||||
if (!Resources.fitsIn(resourceCalculator, cluster,
|
||||
Resources.add(queueUsage.getUsed(partition), netAllocated),
|
||||
maxResourceLimit)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Used resource=" + queueUsage.getUsed(partition)
|
||||
+ " exceeded maxResourceLimit of the queue ="
|
||||
+ maxResourceLimit);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
|
||||
// Only check parent queue when something new allocated or reserved.
|
||||
checkParentQueue = true;
|
||||
}
|
||||
|
||||
|
||||
if (parent != null && checkParentQueue) {
|
||||
return parent.accept(cluster, request);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,7 +38,11 @@ public class CSAssignment {
|
|||
new CSAssignment(SkippedType.OTHER);
|
||||
|
||||
private Resource resource;
|
||||
// Container allocation locality type
|
||||
private NodeType type;
|
||||
|
||||
// Pending request locality type
|
||||
private NodeType requestLocalityType;
|
||||
private RMContainer excessReservation;
|
||||
private FiCaSchedulerApp application;
|
||||
private SkippedType skipped;
|
||||
|
@ -57,6 +61,10 @@ public class CSAssignment {
|
|||
private boolean increaseAllocation;
|
||||
private List<RMContainer> containersToKill;
|
||||
|
||||
// Set when fulfilledReservation = true
|
||||
private RMContainer fulfilledReservedContainer;
|
||||
private SchedulingMode schedulingMode;
|
||||
|
||||
public CSAssignment(Resource resource, NodeType type) {
|
||||
this(resource, type, null, null, SkippedType.NONE, false);
|
||||
}
|
||||
|
@ -173,4 +181,29 @@ public class CSAssignment {
|
|||
public List<RMContainer> getContainersToKill() {
|
||||
return containersToKill;
|
||||
}
|
||||
|
||||
public RMContainer getFulfilledReservedContainer() {
|
||||
return fulfilledReservedContainer;
|
||||
}
|
||||
|
||||
public void setFulfilledReservedContainer(
|
||||
RMContainer fulfilledReservedContainer) {
|
||||
this.fulfilledReservedContainer = fulfilledReservedContainer;
|
||||
}
|
||||
|
||||
public SchedulingMode getSchedulingMode() {
|
||||
return schedulingMode;
|
||||
}
|
||||
|
||||
public void setSchedulingMode(SchedulingMode schedulingMode) {
|
||||
this.schedulingMode = schedulingMode;
|
||||
}
|
||||
|
||||
public NodeType getRequestLocalityType() {
|
||||
return requestLocalityType;
|
||||
}
|
||||
|
||||
public void setRequestLocalityType(NodeType requestLocalityType) {
|
||||
this.requestLocalityType = requestLocalityType;
|
||||
}
|
||||
}
|
|
@ -23,6 +23,7 @@ import java.util.Collection;
|
|||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
|
@ -42,8 +43,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimplePlacementSet;
|
||||
|
||||
/**
|
||||
* <code>CSQueue</code> represents a node in the tree of
|
||||
|
@ -195,14 +199,14 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
|
|||
/**
|
||||
* Assign containers to applications in the queue or it's children (if any).
|
||||
* @param clusterResource the resource of the cluster.
|
||||
* @param node node on which resources are available
|
||||
* @param ps {@link PlacementSet} of nodes which resources are available
|
||||
* @param resourceLimits how much overall resource of this queue can use.
|
||||
* @param schedulingMode Type of exclusive check when assign container on a
|
||||
* NodeManager, see {@link SchedulingMode}.
|
||||
* @return the assignment
|
||||
*/
|
||||
public CSAssignment assignContainers(Resource clusterResource,
|
||||
FiCaSchedulerNode node, ResourceLimits resourceLimits,
|
||||
PlacementSet<FiCaSchedulerNode> ps, ResourceLimits resourceLimits,
|
||||
SchedulingMode schedulingMode);
|
||||
|
||||
/**
|
||||
|
@ -340,4 +344,15 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
|
|||
* @return valid node labels
|
||||
*/
|
||||
public Set<String> getNodeLabelsForQueue();
|
||||
|
||||
@VisibleForTesting
|
||||
CSAssignment assignContainers(Resource clusterResource,
|
||||
FiCaSchedulerNode node, ResourceLimits resourceLimits,
|
||||
SchedulingMode schedulingMode);
|
||||
|
||||
boolean accept(Resource cluster,
|
||||
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request);
|
||||
|
||||
void apply(Resource cluster,
|
||||
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request);
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -208,6 +208,10 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
|
|||
public static final String SCHEDULE_ASYNCHRONOUSLY_ENABLE =
|
||||
SCHEDULE_ASYNCHRONOUSLY_PREFIX + ".enable";
|
||||
|
||||
@Private
|
||||
public static final String SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_THREAD =
|
||||
SCHEDULE_ASYNCHRONOUSLY_PREFIX + ".maximum-threads";
|
||||
|
||||
@Private
|
||||
public static final boolean DEFAULT_SCHEDULE_ASYNCHRONOUSLY_ENABLE = false;
|
||||
|
||||
|
|
|
@ -67,8 +67,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.Activi
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityState;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderingPolicyForPendingApps;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
|
||||
import org.apache.hadoop.yarn.server.utils.Lock;
|
||||
|
@ -914,54 +919,6 @@ public class LeafQueue extends AbstractCSQueue {
|
|||
ApplicationAttemptId applicationAttemptId) {
|
||||
return applicationAttemptMap.get(applicationAttemptId);
|
||||
}
|
||||
|
||||
private void handleExcessReservedContainer(Resource clusterResource,
|
||||
CSAssignment assignment, FiCaSchedulerNode node, FiCaSchedulerApp app) {
|
||||
if (assignment.getExcessReservation() != null) {
|
||||
RMContainer excessReservedContainer = assignment.getExcessReservation();
|
||||
|
||||
if (excessReservedContainer.hasIncreaseReservation()) {
|
||||
unreserveIncreasedContainer(clusterResource,
|
||||
app, node, excessReservedContainer);
|
||||
} else {
|
||||
completedContainer(clusterResource, assignment.getApplication(),
|
||||
scheduler.getNode(excessReservedContainer.getAllocatedNode()),
|
||||
excessReservedContainer,
|
||||
SchedulerUtils.createAbnormalContainerStatus(
|
||||
excessReservedContainer.getContainerId(),
|
||||
SchedulerUtils.UNRESERVED_CONTAINER),
|
||||
RMContainerEventType.RELEASED, null, false);
|
||||
}
|
||||
|
||||
assignment.setExcessReservation(null);
|
||||
}
|
||||
}
|
||||
|
||||
private void killToPreemptContainers(Resource clusterResource,
|
||||
FiCaSchedulerNode node,
|
||||
CSAssignment assignment) {
|
||||
if (assignment.getContainersToKill() != null) {
|
||||
StringBuilder sb = new StringBuilder("Killing containers: [");
|
||||
|
||||
for (RMContainer c : assignment.getContainersToKill()) {
|
||||
FiCaSchedulerApp application = csContext.getApplicationAttempt(
|
||||
c.getApplicationAttemptId());
|
||||
LeafQueue q = application.getCSLeafQueue();
|
||||
q.completedContainer(clusterResource, application, node, c, SchedulerUtils
|
||||
.createPreemptedContainerStatus(c.getContainerId(),
|
||||
SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL,
|
||||
null, false);
|
||||
sb.append("(container=" + c.getContainerId() + " resource=" + c
|
||||
.getAllocatedResource() + ")");
|
||||
}
|
||||
|
||||
sb.append("] for container=" + assignment.getAssignmentInformation()
|
||||
.getFirstAllocatedOrReservedContainerId() + " resource=" + assignment
|
||||
.getResource());
|
||||
LOG.info(sb.toString());
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
private void setPreemptionAllowed(ResourceLimits limits, String nodePartition) {
|
||||
// Set preemption-allowed:
|
||||
|
@ -971,174 +928,312 @@ public class LeafQueue extends AbstractCSQueue {
|
|||
limits.setIsAllowPreemption(usedCapacity < guaranteedCapacity);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CSAssignment assignContainers(Resource clusterResource,
|
||||
FiCaSchedulerNode node, ResourceLimits currentResourceLimits,
|
||||
SchedulingMode schedulingMode) {
|
||||
try {
|
||||
writeLock.lock();
|
||||
updateCurrentResourceLimits(currentResourceLimits, clusterResource);
|
||||
private CSAssignment allocateFromReservedContainer(
|
||||
Resource clusterResource, PlacementSet<FiCaSchedulerNode> ps,
|
||||
ResourceLimits currentResourceLimits, SchedulingMode schedulingMode) {
|
||||
FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps);
|
||||
if (null == node) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(
|
||||
"assignContainers: node=" + node.getNodeName() + " #applications="
|
||||
+ orderingPolicy.getNumSchedulableEntities());
|
||||
}
|
||||
|
||||
setPreemptionAllowed(currentResourceLimits, node.getPartition());
|
||||
|
||||
// Check for reserved resources
|
||||
RMContainer reservedContainer = node.getReservedContainer();
|
||||
if (reservedContainer != null) {
|
||||
FiCaSchedulerApp application = getApplication(
|
||||
reservedContainer.getApplicationAttemptId());
|
||||
RMContainer reservedContainer = node.getReservedContainer();
|
||||
if (reservedContainer != null) {
|
||||
FiCaSchedulerApp application = getApplication(
|
||||
reservedContainer.getApplicationAttemptId());
|
||||
|
||||
if (null != application) {
|
||||
ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager,
|
||||
node.getNodeID(), SystemClock.getInstance().getTime(), application);
|
||||
|
||||
CSAssignment assignment = application.assignContainers(clusterResource,
|
||||
node, currentResourceLimits, schedulingMode, reservedContainer);
|
||||
handleExcessReservedContainer(clusterResource, assignment, node,
|
||||
application);
|
||||
killToPreemptContainers(clusterResource, node, assignment);
|
||||
ps, currentResourceLimits, schedulingMode, reservedContainer);
|
||||
return assignment;
|
||||
}
|
||||
}
|
||||
|
||||
// if our queue cannot access this node, just return
|
||||
if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
|
||||
&& !accessibleToPartition(node.getPartition())) {
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParent().getQueueName(), getQueueName(), ActivityState.REJECTED,
|
||||
ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION + node
|
||||
.getPartition());
|
||||
return CSAssignment.NULL_ASSIGNMENT;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
// Check if this queue need more resource, simply skip allocation if this
|
||||
// queue doesn't need more resources.
|
||||
if (!hasPendingResourceRequest(node.getPartition(), clusterResource,
|
||||
schedulingMode)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Skip this queue=" + getQueuePath()
|
||||
+ ", because it doesn't need more resource, schedulingMode="
|
||||
+ schedulingMode.name() + " node-partition=" + node
|
||||
@Override
|
||||
public CSAssignment assignContainers(Resource clusterResource,
|
||||
PlacementSet<FiCaSchedulerNode> ps, ResourceLimits currentResourceLimits,
|
||||
SchedulingMode schedulingMode) {
|
||||
updateCurrentResourceLimits(currentResourceLimits, clusterResource);
|
||||
FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("assignContainers: partition=" + ps.getPartition()
|
||||
+ " #applications=" + orderingPolicy.getNumSchedulableEntities());
|
||||
}
|
||||
|
||||
setPreemptionAllowed(currentResourceLimits, ps.getPartition());
|
||||
|
||||
// Check for reserved resources, try to allocate reserved container first.
|
||||
CSAssignment assignment = allocateFromReservedContainer(clusterResource,
|
||||
ps, currentResourceLimits, schedulingMode);
|
||||
if (null != assignment) {
|
||||
return assignment;
|
||||
}
|
||||
|
||||
// if our queue cannot access this node, just return
|
||||
if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
|
||||
&& !accessibleToPartition(ps.getPartition())) {
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParent().getQueueName(), getQueueName(), ActivityState.REJECTED,
|
||||
ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION + ps
|
||||
.getPartition());
|
||||
}
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED,
|
||||
ActivityDiagnosticConstant.QUEUE_DO_NOT_NEED_MORE_RESOURCE);
|
||||
return CSAssignment.NULL_ASSIGNMENT;
|
||||
}
|
||||
return CSAssignment.NULL_ASSIGNMENT;
|
||||
}
|
||||
|
||||
for (Iterator<FiCaSchedulerApp> assignmentIterator =
|
||||
orderingPolicy.getAssignmentIterator();
|
||||
assignmentIterator.hasNext(); ) {
|
||||
FiCaSchedulerApp application = assignmentIterator.next();
|
||||
|
||||
ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager,
|
||||
node.getNodeID(), SystemClock.getInstance().getTime(), application);
|
||||
|
||||
// Check queue max-capacity limit
|
||||
if (!super.canAssignToThisQueue(clusterResource, node.getPartition(),
|
||||
currentResourceLimits, application.getCurrentReservation(),
|
||||
schedulingMode)) {
|
||||
ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue(
|
||||
activitiesManager, node, application, application.getPriority(),
|
||||
ActivityDiagnosticConstant.QUEUE_MAX_CAPACITY_LIMIT);
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED,
|
||||
ActivityDiagnosticConstant.EMPTY);
|
||||
return CSAssignment.NULL_ASSIGNMENT;
|
||||
}
|
||||
|
||||
Resource userLimit = computeUserLimitAndSetHeadroom(application,
|
||||
clusterResource, node.getPartition(), schedulingMode);
|
||||
|
||||
// Check user limit
|
||||
if (!canAssignToUser(clusterResource, application.getUser(), userLimit,
|
||||
application, node.getPartition(), currentResourceLimits)) {
|
||||
application.updateAMContainerDiagnostics(AMState.ACTIVATED,
|
||||
"User capacity has reached its maximum limit.");
|
||||
ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue(
|
||||
activitiesManager, node, application, application.getPriority(),
|
||||
ActivityDiagnosticConstant.USER_CAPACITY_MAXIMUM_LIMIT);
|
||||
continue;
|
||||
}
|
||||
|
||||
// Try to schedule
|
||||
CSAssignment assignment = application.assignContainers(clusterResource,
|
||||
node, currentResourceLimits, schedulingMode, null);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("post-assignContainers for application " + application
|
||||
.getApplicationId());
|
||||
application.showRequests();
|
||||
}
|
||||
|
||||
// Did we schedule or reserve a container?
|
||||
Resource assigned = assignment.getResource();
|
||||
|
||||
handleExcessReservedContainer(clusterResource, assignment, node,
|
||||
application);
|
||||
killToPreemptContainers(clusterResource, node, assignment);
|
||||
|
||||
if (Resources.greaterThan(resourceCalculator, clusterResource, assigned,
|
||||
Resources.none())) {
|
||||
// Get reserved or allocated container from application
|
||||
RMContainer reservedOrAllocatedRMContainer =
|
||||
application.getRMContainer(assignment.getAssignmentInformation()
|
||||
.getFirstAllocatedOrReservedContainerId());
|
||||
|
||||
// Book-keeping
|
||||
// Note: Update headroom to account for current allocation too...
|
||||
allocateResource(clusterResource, application, assigned,
|
||||
node.getPartition(), reservedOrAllocatedRMContainer,
|
||||
assignment.isIncreasedAllocation());
|
||||
|
||||
// Update reserved metrics
|
||||
Resource reservedRes =
|
||||
assignment.getAssignmentInformation().getReserved();
|
||||
if (reservedRes != null && !reservedRes.equals(Resources.none())) {
|
||||
incReservedResource(node.getPartition(), reservedRes);
|
||||
}
|
||||
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParent().getQueueName(), getQueueName(),
|
||||
ActivityState.ACCEPTED, ActivityDiagnosticConstant.EMPTY);
|
||||
|
||||
// Done
|
||||
return assignment;
|
||||
} else if (assignment.getSkippedType()
|
||||
== CSAssignment.SkippedType.OTHER) {
|
||||
ActivitiesLogger.APP.finishSkippedAppAllocationRecording(
|
||||
activitiesManager, application.getApplicationId(),
|
||||
ActivityState.SKIPPED, ActivityDiagnosticConstant.EMPTY);
|
||||
application.updateNodeInfoForAMDiagnostics(node);
|
||||
} else if (assignment.getSkippedType()
|
||||
== CSAssignment.SkippedType.QUEUE_LIMIT) {
|
||||
return assignment;
|
||||
} else{
|
||||
// If we don't allocate anything, and it is not skipped by application,
|
||||
// we will return to respect FIFO of applications
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED,
|
||||
ActivityDiagnosticConstant.RESPECT_FIFO);
|
||||
ActivitiesLogger.APP.finishSkippedAppAllocationRecording(
|
||||
activitiesManager, application.getApplicationId(),
|
||||
ActivityState.SKIPPED, ActivityDiagnosticConstant.EMPTY);
|
||||
return CSAssignment.NULL_ASSIGNMENT;
|
||||
}
|
||||
// Check if this queue need more resource, simply skip allocation if this
|
||||
// queue doesn't need more resources.
|
||||
if (!hasPendingResourceRequest(ps.getPartition(), clusterResource,
|
||||
schedulingMode)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Skip this queue=" + getQueuePath()
|
||||
+ ", because it doesn't need more resource, schedulingMode="
|
||||
+ schedulingMode.name() + " node-partition=" + ps.getPartition());
|
||||
}
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED,
|
||||
ActivityDiagnosticConstant.EMPTY);
|
||||
|
||||
ActivityDiagnosticConstant.QUEUE_DO_NOT_NEED_MORE_RESOURCE);
|
||||
return CSAssignment.NULL_ASSIGNMENT;
|
||||
}
|
||||
|
||||
for (Iterator<FiCaSchedulerApp> assignmentIterator =
|
||||
orderingPolicy.getAssignmentIterator();
|
||||
assignmentIterator.hasNext(); ) {
|
||||
FiCaSchedulerApp application = assignmentIterator.next();
|
||||
|
||||
ActivitiesLogger.APP.startAppAllocationRecording(activitiesManager,
|
||||
node.getNodeID(), SystemClock.getInstance().getTime(), application);
|
||||
|
||||
// Check queue max-capacity limit
|
||||
if (!super.canAssignToThisQueue(clusterResource, ps.getPartition(),
|
||||
currentResourceLimits, application.getCurrentReservation(),
|
||||
schedulingMode)) {
|
||||
ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue(
|
||||
activitiesManager, node, application, application.getPriority(),
|
||||
ActivityDiagnosticConstant.QUEUE_MAX_CAPACITY_LIMIT);
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED,
|
||||
ActivityDiagnosticConstant.EMPTY);
|
||||
return CSAssignment.NULL_ASSIGNMENT;
|
||||
}
|
||||
|
||||
Resource userLimit = computeUserLimitAndSetHeadroom(application,
|
||||
clusterResource, ps.getPartition(), schedulingMode);
|
||||
|
||||
// Check user limit
|
||||
if (!canAssignToUser(clusterResource, application.getUser(), userLimit,
|
||||
application, ps.getPartition(), currentResourceLimits)) {
|
||||
application.updateAMContainerDiagnostics(AMState.ACTIVATED,
|
||||
"User capacity has reached its maximum limit.");
|
||||
ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue(
|
||||
activitiesManager, node, application, application.getPriority(),
|
||||
ActivityDiagnosticConstant.USER_CAPACITY_MAXIMUM_LIMIT);
|
||||
continue;
|
||||
}
|
||||
|
||||
// Try to schedule
|
||||
assignment = application.assignContainers(clusterResource,
|
||||
ps, currentResourceLimits, schedulingMode, null);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("post-assignContainers for application " + application
|
||||
.getApplicationId());
|
||||
application.showRequests();
|
||||
}
|
||||
|
||||
// Did we schedule or reserve a container?
|
||||
Resource assigned = assignment.getResource();
|
||||
|
||||
if (Resources.greaterThan(resourceCalculator, clusterResource, assigned,
|
||||
Resources.none())) {
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParent().getQueueName(), getQueueName(),
|
||||
ActivityState.ACCEPTED, ActivityDiagnosticConstant.EMPTY);
|
||||
return assignment;
|
||||
} else if (assignment.getSkippedType()
|
||||
== CSAssignment.SkippedType.OTHER) {
|
||||
ActivitiesLogger.APP.finishSkippedAppAllocationRecording(
|
||||
activitiesManager, application.getApplicationId(),
|
||||
ActivityState.SKIPPED, ActivityDiagnosticConstant.EMPTY);
|
||||
application.updateNodeInfoForAMDiagnostics(node);
|
||||
} else if (assignment.getSkippedType()
|
||||
== CSAssignment.SkippedType.QUEUE_LIMIT) {
|
||||
return assignment;
|
||||
} else{
|
||||
// If we don't allocate anything, and it is not skipped by application,
|
||||
// we will return to respect FIFO of applications
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED,
|
||||
ActivityDiagnosticConstant.RESPECT_FIFO);
|
||||
ActivitiesLogger.APP.finishSkippedAppAllocationRecording(
|
||||
activitiesManager, application.getApplicationId(),
|
||||
ActivityState.SKIPPED, ActivityDiagnosticConstant.EMPTY);
|
||||
return CSAssignment.NULL_ASSIGNMENT;
|
||||
}
|
||||
}
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED,
|
||||
ActivityDiagnosticConstant.EMPTY);
|
||||
|
||||
return CSAssignment.NULL_ASSIGNMENT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean accept(Resource cluster,
|
||||
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request) {
|
||||
ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode> allocation =
|
||||
request.getFirstAllocatedOrReservedContainer();
|
||||
SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> schedulerContainer =
|
||||
allocation.getAllocatedOrReservedContainer();
|
||||
|
||||
// Do not check limits when allocation from a reserved container
|
||||
if (allocation.getAllocateFromReservedContainer() == null) {
|
||||
try {
|
||||
readLock.lock();
|
||||
FiCaSchedulerApp app =
|
||||
schedulerContainer.getSchedulerApplicationAttempt();
|
||||
String username = app.getUser();
|
||||
String p = schedulerContainer.getNodePartition();
|
||||
|
||||
// check user-limit
|
||||
Resource userLimit = computeUserLimitAndSetHeadroom(app, cluster, p,
|
||||
allocation.getSchedulingMode());
|
||||
|
||||
// Deduct resources that we can release
|
||||
Resource usedResource = Resources.clone(getUser(username).getUsed(p));
|
||||
Resources.subtractFrom(usedResource,
|
||||
request.getTotalReleasedResource());
|
||||
|
||||
if (Resources.greaterThan(resourceCalculator, cluster, usedResource,
|
||||
userLimit)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Used resource=" + usedResource + " exceeded user-limit="
|
||||
+ userLimit);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
return super.accept(cluster, request);
|
||||
}
|
||||
|
||||
private void internalReleaseContainer(Resource clusterResource,
|
||||
SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> schedulerContainer) {
|
||||
RMContainer rmContainer = schedulerContainer.getRmContainer();
|
||||
|
||||
LeafQueue targetLeafQueue =
|
||||
schedulerContainer.getSchedulerApplicationAttempt().getCSLeafQueue();
|
||||
|
||||
if (targetLeafQueue == this) {
|
||||
// When trying to preempt containers from the same queue
|
||||
if (rmContainer.hasIncreaseReservation()) {
|
||||
// Increased container reservation
|
||||
unreserveIncreasedContainer(clusterResource,
|
||||
schedulerContainer.getSchedulerApplicationAttempt(),
|
||||
schedulerContainer.getSchedulerNode(), rmContainer);
|
||||
} else if (rmContainer.getState() == RMContainerState.RESERVED) {
|
||||
// For other reserved containers
|
||||
// This is a reservation exchange, complete previous reserved container
|
||||
completedContainer(clusterResource,
|
||||
schedulerContainer.getSchedulerApplicationAttempt(),
|
||||
schedulerContainer.getSchedulerNode(), rmContainer, SchedulerUtils
|
||||
.createAbnormalContainerStatus(rmContainer.getContainerId(),
|
||||
SchedulerUtils.UNRESERVED_CONTAINER),
|
||||
RMContainerEventType.RELEASED, null, false);
|
||||
}
|
||||
} else{
|
||||
// When trying to preempt containers from different queue -- this
|
||||
// is for lazy preemption feature (kill preemption candidate in scheduling
|
||||
// cycle).
|
||||
targetLeafQueue.completedContainer(clusterResource,
|
||||
schedulerContainer.getSchedulerApplicationAttempt(),
|
||||
schedulerContainer.getSchedulerNode(),
|
||||
schedulerContainer.getRmContainer(), SchedulerUtils
|
||||
.createPreemptedContainerStatus(rmContainer.getContainerId(),
|
||||
SchedulerUtils.PREEMPTED_CONTAINER),
|
||||
RMContainerEventType.KILL, null, false);
|
||||
}
|
||||
}
|
||||
|
||||
private void releaseContainers(Resource clusterResource,
|
||||
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request) {
|
||||
for (SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> c : request
|
||||
.getContainersToRelease()) {
|
||||
internalReleaseContainer(clusterResource, c);
|
||||
}
|
||||
|
||||
// Handle container reservation looking, or lazy preemption case:
|
||||
if (null != request.getContainersToAllocate() && !request
|
||||
.getContainersToAllocate().isEmpty()) {
|
||||
for (ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode> context : request
|
||||
.getContainersToAllocate()) {
|
||||
if (null != context.getToRelease()) {
|
||||
for (SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> c : context
|
||||
.getToRelease()) {
|
||||
internalReleaseContainer(clusterResource, c);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void apply(Resource cluster,
|
||||
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request) {
|
||||
// Do we need to call parent queue's apply?
|
||||
boolean applyToParentQueue = false;
|
||||
|
||||
releaseContainers(cluster, request);
|
||||
|
||||
try {
|
||||
writeLock.lock();
|
||||
|
||||
if (request.anythingAllocatedOrReserved()) {
|
||||
ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode>
|
||||
allocation = request.getFirstAllocatedOrReservedContainer();
|
||||
SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode>
|
||||
schedulerContainer = allocation.getAllocatedOrReservedContainer();
|
||||
|
||||
// Do not modify queue when allocation from reserved container
|
||||
if (allocation.getAllocateFromReservedContainer() == null) {
|
||||
// Only invoke apply() of ParentQueue when new allocation /
|
||||
// reservation happen.
|
||||
applyToParentQueue = true;
|
||||
// Book-keeping
|
||||
// Note: Update headroom to account for current allocation too...
|
||||
allocateResource(cluster,
|
||||
schedulerContainer.getSchedulerApplicationAttempt(),
|
||||
allocation.getAllocatedOrReservedResource(),
|
||||
schedulerContainer.getNodePartition(),
|
||||
schedulerContainer.getRmContainer(),
|
||||
allocation.isIncreasedAllocation());
|
||||
orderingPolicy.containerAllocated(
|
||||
schedulerContainer.getSchedulerApplicationAttempt(),
|
||||
schedulerContainer.getRmContainer());
|
||||
}
|
||||
|
||||
// Update reserved resource
|
||||
if (Resources.greaterThan(resourceCalculator, cluster,
|
||||
request.getTotalReservedResource(), Resources.none())) {
|
||||
incReservedResource(schedulerContainer.getNodePartition(),
|
||||
request.getTotalReservedResource());
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
|
||||
if (parent != null && applyToParentQueue) {
|
||||
parent.apply(cluster, request);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
protected Resource getHeadroom(User user, Resource queueCurrentLimit,
|
||||
Resource clusterResource, FiCaSchedulerApp application) {
|
||||
return getHeadroom(user, queueCurrentLimit, clusterResource, application,
|
||||
|
|
|
@ -47,8 +47,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.Activi
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityState;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.AllocationState;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -71,12 +76,10 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
|
||||
protected final Set<CSQueue> childQueues;
|
||||
private final boolean rootQueue;
|
||||
final Comparator<CSQueue> nonPartitionedQueueComparator;
|
||||
final PartitionedQueueComparator partitionQueueComparator;
|
||||
volatile int numApplications;
|
||||
private final Comparator<CSQueue> nonPartitionedQueueComparator;
|
||||
private final PartitionedQueueComparator partitionQueueComparator;
|
||||
private volatile int numApplications;
|
||||
private final CapacitySchedulerContext scheduler;
|
||||
private boolean needToResortQueuesAtNextAllocation = false;
|
||||
private int offswitchPerHeartbeatLimit;
|
||||
|
||||
private final RecordFactory recordFactory =
|
||||
RecordFactoryProvider.getRecordFactory(null);
|
||||
|
@ -86,7 +89,7 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
super(cs, queueName, parent, old);
|
||||
this.scheduler = cs;
|
||||
this.nonPartitionedQueueComparator = cs.getNonPartitionedQueueComparator();
|
||||
this.partitionQueueComparator = cs.getPartitionedQueueComparator();
|
||||
this.partitionQueueComparator = new PartitionedQueueComparator();
|
||||
|
||||
this.rootQueue = (parent == null);
|
||||
|
||||
|
@ -126,16 +129,12 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
}
|
||||
}
|
||||
|
||||
offswitchPerHeartbeatLimit =
|
||||
csContext.getConfiguration().getOffSwitchPerHeartbeatLimit();
|
||||
|
||||
LOG.info(queueName + ", capacity=" + this.queueCapacities.getCapacity()
|
||||
+ ", absoluteCapacity=" + this.queueCapacities.getAbsoluteCapacity()
|
||||
+ ", maxCapacity=" + this.queueCapacities.getMaximumCapacity()
|
||||
+ ", absoluteMaxCapacity=" + this.queueCapacities
|
||||
.getAbsoluteMaximumCapacity() + ", state=" + state + ", acls="
|
||||
+ aclsString + ", labels=" + labelStrBuilder.toString() + "\n"
|
||||
+ ", offswitchPerHeartbeatLimit = " + getOffSwitchPerHeartbeatLimit()
|
||||
+ ", reservationsContinueLooking=" + reservationsContinueLooking);
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
|
@ -215,11 +214,6 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
|
||||
}
|
||||
|
||||
@Private
|
||||
public int getOffSwitchPerHeartbeatLimit() {
|
||||
return offswitchPerHeartbeatLimit;
|
||||
}
|
||||
|
||||
private QueueUserACLInfo getUserAclInfo(
|
||||
UserGroupInformation user) {
|
||||
try {
|
||||
|
@ -435,156 +429,145 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
|
||||
@Override
|
||||
public CSAssignment assignContainers(Resource clusterResource,
|
||||
FiCaSchedulerNode node, ResourceLimits resourceLimits,
|
||||
SchedulingMode schedulingMode) {
|
||||
int offswitchCount = 0;
|
||||
try {
|
||||
writeLock.lock();
|
||||
// if our queue cannot access this node, just return
|
||||
if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
|
||||
&& !accessibleToPartition(node.getPartition())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Skip this queue=" + getQueuePath()
|
||||
+ ", because it is not able to access partition=" + node
|
||||
.getPartition());
|
||||
}
|
||||
PlacementSet<FiCaSchedulerNode> ps, ResourceLimits resourceLimits,
|
||||
SchedulingMode schedulingMode) {
|
||||
FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps);
|
||||
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParentName(), getQueueName(), ActivityState.REJECTED,
|
||||
ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION + node
|
||||
.getPartition());
|
||||
if (rootQueue) {
|
||||
ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager,
|
||||
node);
|
||||
}
|
||||
|
||||
return CSAssignment.NULL_ASSIGNMENT;
|
||||
// if our queue cannot access this node, just return
|
||||
if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
|
||||
&& !accessibleToPartition(ps.getPartition())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Skip this queue=" + getQueuePath()
|
||||
+ ", because it is not able to access partition=" + ps
|
||||
.getPartition());
|
||||
}
|
||||
|
||||
// Check if this queue need more resource, simply skip allocation if this
|
||||
// queue doesn't need more resources.
|
||||
if (!super.hasPendingResourceRequest(node.getPartition(), clusterResource,
|
||||
schedulingMode)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Skip this queue=" + getQueuePath()
|
||||
+ ", because it doesn't need more resource, schedulingMode="
|
||||
+ schedulingMode.name() + " node-partition=" + node
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParentName(), getQueueName(), ActivityState.REJECTED,
|
||||
ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION + node
|
||||
.getPartition());
|
||||
}
|
||||
if (rootQueue) {
|
||||
ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager,
|
||||
node);
|
||||
}
|
||||
|
||||
return CSAssignment.NULL_ASSIGNMENT;
|
||||
}
|
||||
|
||||
// Check if this queue need more resource, simply skip allocation if this
|
||||
// queue doesn't need more resources.
|
||||
if (!super.hasPendingResourceRequest(ps.getPartition(), clusterResource,
|
||||
schedulingMode)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Skip this queue=" + getQueuePath()
|
||||
+ ", because it doesn't need more resource, schedulingMode="
|
||||
+ schedulingMode.name() + " node-partition=" + ps
|
||||
.getPartition());
|
||||
}
|
||||
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParentName(), getQueueName(), ActivityState.SKIPPED,
|
||||
ActivityDiagnosticConstant.QUEUE_DO_NOT_NEED_MORE_RESOURCE);
|
||||
if (rootQueue) {
|
||||
ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager,
|
||||
node);
|
||||
}
|
||||
|
||||
return CSAssignment.NULL_ASSIGNMENT;
|
||||
}
|
||||
|
||||
CSAssignment assignment = new CSAssignment(Resources.createResource(0, 0),
|
||||
NodeType.NODE_LOCAL);
|
||||
|
||||
while (canAssign(clusterResource, node)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Trying to assign containers to child-queue of "
|
||||
+ getQueueName());
|
||||
}
|
||||
|
||||
// Are we over maximum-capacity for this queue?
|
||||
// This will also consider parent's limits and also continuous reservation
|
||||
// looking
|
||||
if (!super.canAssignToThisQueue(clusterResource, ps.getPartition(),
|
||||
resourceLimits, Resources
|
||||
.createResource(getMetrics().getReservedMB(),
|
||||
getMetrics().getReservedVirtualCores()), schedulingMode)) {
|
||||
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParentName(), getQueueName(), ActivityState.SKIPPED,
|
||||
ActivityDiagnosticConstant.QUEUE_DO_NOT_NEED_MORE_RESOURCE);
|
||||
ActivityDiagnosticConstant.QUEUE_MAX_CAPACITY_LIMIT);
|
||||
if (rootQueue) {
|
||||
ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager,
|
||||
node);
|
||||
}
|
||||
|
||||
return CSAssignment.NULL_ASSIGNMENT;
|
||||
break;
|
||||
}
|
||||
|
||||
CSAssignment assignment = new CSAssignment(Resources.createResource(0, 0),
|
||||
NodeType.NODE_LOCAL);
|
||||
// Schedule
|
||||
CSAssignment assignedToChild = assignContainersToChildQueues(
|
||||
clusterResource, ps, resourceLimits, schedulingMode);
|
||||
assignment.setType(assignedToChild.getType());
|
||||
assignment.setRequestLocalityType(
|
||||
assignedToChild.getRequestLocalityType());
|
||||
assignment.setExcessReservation(assignedToChild.getExcessReservation());
|
||||
assignment.setContainersToKill(assignedToChild.getContainersToKill());
|
||||
|
||||
while (canAssign(clusterResource, node)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Trying to assign containers to child-queue of "
|
||||
+ getQueueName());
|
||||
}
|
||||
// Done if no child-queue assigned anything
|
||||
if (Resources.greaterThan(resourceCalculator, clusterResource,
|
||||
assignedToChild.getResource(), Resources.none())) {
|
||||
|
||||
// Are we over maximum-capacity for this queue?
|
||||
// This will also consider parent's limits and also continuous reservation
|
||||
// looking
|
||||
if (!super.canAssignToThisQueue(clusterResource, node.getPartition(),
|
||||
resourceLimits, Resources
|
||||
.createResource(getMetrics().getReservedMB(),
|
||||
getMetrics().getReservedVirtualCores()), schedulingMode)) {
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParentName(), getQueueName(), ActivityState.ACCEPTED,
|
||||
ActivityDiagnosticConstant.EMPTY);
|
||||
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParentName(), getQueueName(), ActivityState.SKIPPED,
|
||||
ActivityDiagnosticConstant.QUEUE_MAX_CAPACITY_LIMIT);
|
||||
boolean isReserved =
|
||||
assignedToChild.getAssignmentInformation().getReservationDetails()
|
||||
!= null && !assignedToChild.getAssignmentInformation()
|
||||
.getReservationDetails().isEmpty();
|
||||
if (node != null && !isReserved) {
|
||||
if (rootQueue) {
|
||||
ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager,
|
||||
node);
|
||||
ActivitiesLogger.NODE.finishAllocatedNodeAllocation(
|
||||
activitiesManager, node,
|
||||
assignedToChild.getAssignmentInformation()
|
||||
.getFirstAllocatedOrReservedContainerId(),
|
||||
AllocationState.ALLOCATED);
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
// Schedule
|
||||
CSAssignment assignedToChild = assignContainersToChildQueues(
|
||||
clusterResource, node, resourceLimits, schedulingMode);
|
||||
assignment.setType(assignedToChild.getType());
|
||||
|
||||
// Done if no child-queue assigned anything
|
||||
if (Resources.greaterThan(resourceCalculator, clusterResource,
|
||||
assignedToChild.getResource(), Resources.none())) {
|
||||
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParentName(), getQueueName(), ActivityState.ACCEPTED,
|
||||
ActivityDiagnosticConstant.EMPTY);
|
||||
|
||||
if (node.getReservedContainer() == null) {
|
||||
if (rootQueue) {
|
||||
ActivitiesLogger.NODE.finishAllocatedNodeAllocation(
|
||||
activitiesManager, node,
|
||||
assignedToChild.getAssignmentInformation()
|
||||
.getFirstAllocatedOrReservedContainerId(),
|
||||
AllocationState.ALLOCATED);
|
||||
}
|
||||
} else{
|
||||
if (rootQueue) {
|
||||
ActivitiesLogger.NODE.finishAllocatedNodeAllocation(
|
||||
activitiesManager, node,
|
||||
assignedToChild.getAssignmentInformation()
|
||||
.getFirstAllocatedOrReservedContainerId(),
|
||||
AllocationState.RESERVED);
|
||||
}
|
||||
}
|
||||
|
||||
// Track resource utilization for the parent-queue
|
||||
allocateResource(clusterResource, assignedToChild.getResource(),
|
||||
node.getPartition(), assignedToChild.isIncreasedAllocation());
|
||||
|
||||
// Track resource utilization in this pass of the scheduler
|
||||
Resources.addTo(assignment.getResource(),
|
||||
assignedToChild.getResource());
|
||||
Resources.addTo(assignment.getAssignmentInformation().getAllocated(),
|
||||
assignedToChild.getAssignmentInformation().getAllocated());
|
||||
Resources.addTo(assignment.getAssignmentInformation().getReserved(),
|
||||
assignedToChild.getAssignmentInformation().getReserved());
|
||||
assignment.getAssignmentInformation().incrAllocations(
|
||||
assignedToChild.getAssignmentInformation().getNumAllocations());
|
||||
assignment.getAssignmentInformation().incrReservations(
|
||||
assignedToChild.getAssignmentInformation().getNumReservations());
|
||||
assignment.getAssignmentInformation().getAllocationDetails().addAll(
|
||||
assignedToChild.getAssignmentInformation()
|
||||
.getAllocationDetails());
|
||||
assignment.getAssignmentInformation().getReservationDetails().addAll(
|
||||
assignedToChild.getAssignmentInformation()
|
||||
.getReservationDetails());
|
||||
assignment.setIncreasedAllocation(
|
||||
assignedToChild.isIncreasedAllocation());
|
||||
|
||||
LOG.info("assignedContainer" + " queue=" + getQueueName()
|
||||
+ " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
|
||||
+ getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed()
|
||||
+ " cluster=" + clusterResource);
|
||||
|
||||
} else{
|
||||
assignment.setSkippedType(assignedToChild.getSkippedType());
|
||||
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParentName(), getQueueName(), ActivityState.SKIPPED,
|
||||
ActivityDiagnosticConstant.EMPTY);
|
||||
if (rootQueue) {
|
||||
ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager,
|
||||
node);
|
||||
ActivitiesLogger.NODE.finishAllocatedNodeAllocation(
|
||||
activitiesManager, node,
|
||||
assignedToChild.getAssignmentInformation()
|
||||
.getFirstAllocatedOrReservedContainerId(),
|
||||
AllocationState.RESERVED);
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
// Track resource utilization in this pass of the scheduler
|
||||
Resources.addTo(assignment.getResource(),
|
||||
assignedToChild.getResource());
|
||||
Resources.addTo(assignment.getAssignmentInformation().getAllocated(),
|
||||
assignedToChild.getAssignmentInformation().getAllocated());
|
||||
Resources.addTo(assignment.getAssignmentInformation().getReserved(),
|
||||
assignedToChild.getAssignmentInformation().getReserved());
|
||||
assignment.getAssignmentInformation().incrAllocations(
|
||||
assignedToChild.getAssignmentInformation().getNumAllocations());
|
||||
assignment.getAssignmentInformation().incrReservations(
|
||||
assignedToChild.getAssignmentInformation().getNumReservations());
|
||||
assignment.getAssignmentInformation().getAllocationDetails().addAll(
|
||||
assignedToChild.getAssignmentInformation()
|
||||
.getAllocationDetails());
|
||||
assignment.getAssignmentInformation().getReservationDetails().addAll(
|
||||
assignedToChild.getAssignmentInformation()
|
||||
.getReservationDetails());
|
||||
assignment.setIncreasedAllocation(
|
||||
assignedToChild.isIncreasedAllocation());
|
||||
|
||||
LOG.info("assignedContainer" + " queue=" + getQueueName()
|
||||
+ " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
|
||||
+ getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed()
|
||||
+ " cluster=" + clusterResource);
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(
|
||||
"ParentQ=" + getQueueName() + " assignedSoFarInThisIteration="
|
||||
|
@ -592,39 +575,47 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
+ getUsedCapacity() + " absoluteUsedCapacity="
|
||||
+ getAbsoluteUsedCapacity());
|
||||
}
|
||||
} else{
|
||||
assignment.setSkippedType(assignedToChild.getSkippedType());
|
||||
|
||||
if (assignment.getType() == NodeType.OFF_SWITCH) {
|
||||
offswitchCount++;
|
||||
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
|
||||
getParentName(), getQueueName(), ActivityState.SKIPPED,
|
||||
ActivityDiagnosticConstant.EMPTY);
|
||||
if (rootQueue) {
|
||||
ActivitiesLogger.NODE.finishSkippedNodeAllocation(activitiesManager,
|
||||
node);
|
||||
}
|
||||
|
||||
// Do not assign more containers if this isn't the root queue
|
||||
// or if we've already assigned enough OFF_SWITCH containers in
|
||||
// this pass
|
||||
if (!rootQueue || offswitchCount >= getOffSwitchPerHeartbeatLimit()) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
if (rootQueue && offswitchCount >= getOffSwitchPerHeartbeatLimit()) {
|
||||
LOG.debug("Assigned maximum number of off-switch containers: " +
|
||||
offswitchCount + ", assignments so far: " + assignment);
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
return assignment;
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
/*
|
||||
* Previously here, we can allocate more than one container for each
|
||||
* allocation under rootQ. Now this logic is not proper any more
|
||||
* in global scheduling world.
|
||||
*
|
||||
* So here do not try to allocate more than one container for each
|
||||
* allocation, let top scheduler make the decision.
|
||||
*/
|
||||
break;
|
||||
}
|
||||
|
||||
return assignment;
|
||||
}
|
||||
|
||||
private boolean canAssign(Resource clusterResource, FiCaSchedulerNode node) {
|
||||
// When node == null means global scheduling is enabled, always return true
|
||||
if (null == node) {
|
||||
return true;
|
||||
}
|
||||
|
||||
// Two conditions need to meet when trying to allocate:
|
||||
// 1) Node doesn't have reserved container
|
||||
// 2) Node's available-resource + killable-resource should > 0
|
||||
return node.getReservedContainer() == null && Resources.greaterThanOrEqual(
|
||||
resourceCalculator, clusterResource, Resources
|
||||
.add(node.getUnallocatedResource(), node.getTotalKillableResources()),
|
||||
minimumAllocation);
|
||||
.add(node.getUnallocatedResource(),
|
||||
node.getTotalKillableResources()), minimumAllocation);
|
||||
}
|
||||
|
||||
private ResourceLimits getResourceLimitsOfChild(CSQueue child,
|
||||
|
@ -662,28 +653,20 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
|
||||
return new ResourceLimits(childLimit);
|
||||
}
|
||||
|
||||
private Iterator<CSQueue> sortAndGetChildrenAllocationIterator(FiCaSchedulerNode node) {
|
||||
if (node.getPartition().equals(RMNodeLabelsManager.NO_LABEL)) {
|
||||
if (needToResortQueuesAtNextAllocation) {
|
||||
// If we skipped resort queues last time, we need to re-sort queue
|
||||
// before allocation
|
||||
List<CSQueue> childrenList = new ArrayList<>(childQueues);
|
||||
childQueues.clear();
|
||||
childQueues.addAll(childrenList);
|
||||
needToResortQueuesAtNextAllocation = false;
|
||||
}
|
||||
return childQueues.iterator();
|
||||
}
|
||||
|
||||
partitionQueueComparator.setPartitionToLookAt(node.getPartition());
|
||||
private Iterator<CSQueue> sortAndGetChildrenAllocationIterator(
|
||||
String partition) {
|
||||
// Previously we keep a sorted list for default partition, it is not good
|
||||
// when multi-threading scheduler is enabled, so to make a simpler code
|
||||
// now re-sort queue every time irrespective to node partition.
|
||||
partitionQueueComparator.setPartitionToLookAt(partition);
|
||||
List<CSQueue> childrenList = new ArrayList<>(childQueues);
|
||||
Collections.sort(childrenList, partitionQueueComparator);
|
||||
return childrenList.iterator();
|
||||
}
|
||||
|
||||
private CSAssignment assignContainersToChildQueues(
|
||||
Resource cluster, FiCaSchedulerNode node, ResourceLimits limits,
|
||||
|
||||
private CSAssignment assignContainersToChildQueues(Resource cluster,
|
||||
PlacementSet<FiCaSchedulerNode> ps, ResourceLimits limits,
|
||||
SchedulingMode schedulingMode) {
|
||||
CSAssignment assignment = CSAssignment.NULL_ASSIGNMENT;
|
||||
|
||||
|
@ -691,8 +674,8 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
printChildQueues();
|
||||
|
||||
// Try to assign to most 'under-served' sub-queue
|
||||
for (Iterator<CSQueue> iter = sortAndGetChildrenAllocationIterator(node); iter
|
||||
.hasNext();) {
|
||||
for (Iterator<CSQueue> iter = sortAndGetChildrenAllocationIterator(
|
||||
ps.getPartition()); iter.hasNext(); ) {
|
||||
CSQueue childQueue = iter.next();
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("Trying to assign to queue: " + childQueue.getQueuePath()
|
||||
|
@ -702,9 +685,9 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
// Get ResourceLimits of child queue before assign containers
|
||||
ResourceLimits childLimits =
|
||||
getResourceLimitsOfChild(childQueue, cluster, parentLimits,
|
||||
node.getPartition());
|
||||
ps.getPartition());
|
||||
|
||||
CSAssignment childAssignment = childQueue.assignContainers(cluster, node,
|
||||
CSAssignment childAssignment = childQueue.assignContainers(cluster, ps,
|
||||
childLimits, schedulingMode);
|
||||
if(LOG.isDebugEnabled()) {
|
||||
LOG.debug("Assigned to queue: " + childQueue.getQueuePath() +
|
||||
|
@ -712,22 +695,9 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
childAssignment.getResource() + ", " + childAssignment.getType());
|
||||
}
|
||||
|
||||
// If we do assign, remove the queue and re-insert in-order to re-sort
|
||||
if (Resources.greaterThan(
|
||||
resourceCalculator, cluster,
|
||||
childAssignment.getResource(), Resources.none())) {
|
||||
// Only update childQueues when we doing non-partitioned node
|
||||
// allocation.
|
||||
if (RMNodeLabelsManager.NO_LABEL.equals(node.getPartition())) {
|
||||
// Remove and re-insert to sort
|
||||
iter.remove();
|
||||
LOG.info("Re-sorting assigned queue: " + childQueue.getQueuePath()
|
||||
+ " stats: " + childQueue);
|
||||
childQueues.add(childQueue);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
printChildQueues();
|
||||
}
|
||||
}
|
||||
assignment = childAssignment;
|
||||
break;
|
||||
} else if (childAssignment.getSkippedType() ==
|
||||
|
@ -770,10 +740,10 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
+ " child-queues: " + getChildQueuesToPrint());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void internalReleaseResource(Resource clusterResource,
|
||||
FiCaSchedulerNode node, Resource releasedResource, boolean changeResource,
|
||||
CSQueue completedChildQueue, boolean sortQueues) {
|
||||
FiCaSchedulerNode node, Resource releasedResource,
|
||||
boolean changeResource) {
|
||||
try {
|
||||
writeLock.lock();
|
||||
super.releaseResource(clusterResource, releasedResource,
|
||||
|
@ -784,29 +754,6 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
"completedContainer " + this + ", cluster=" + clusterResource);
|
||||
}
|
||||
|
||||
// Note that this is using an iterator on the childQueues so this can't
|
||||
// be called if already within an iterator for the childQueues. Like
|
||||
// from assignContainersToChildQueues.
|
||||
if (sortQueues) {
|
||||
// reinsert the updated queue
|
||||
for (Iterator<CSQueue> iter = childQueues.iterator();
|
||||
iter.hasNext(); ) {
|
||||
CSQueue csqueue = iter.next();
|
||||
if (csqueue.equals(completedChildQueue)) {
|
||||
iter.remove();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Re-sorting completed queue: " + csqueue);
|
||||
}
|
||||
childQueues.add(csqueue);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// If we skipped sort queue this time, we need to resort queues to make
|
||||
// sure we allocate from least usage (or order defined by queue policy)
|
||||
// queues.
|
||||
needToResortQueuesAtNextAllocation = !sortQueues;
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
|
@ -821,8 +768,7 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
Resources.negate(decreaseRequest.getDeltaCapacity());
|
||||
|
||||
internalReleaseResource(clusterResource,
|
||||
csContext.getNode(decreaseRequest.getNodeId()), absDeltaCapacity, false,
|
||||
null, false);
|
||||
csContext.getNode(decreaseRequest.getNodeId()), absDeltaCapacity, false);
|
||||
|
||||
// Inform the parent
|
||||
if (parent != null) {
|
||||
|
@ -835,7 +781,7 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
FiCaSchedulerApp app, FiCaSchedulerNode node, RMContainer rmContainer) {
|
||||
if (app != null) {
|
||||
internalReleaseResource(clusterResource, node,
|
||||
rmContainer.getReservedResource(), false, null, false);
|
||||
rmContainer.getReservedResource(), false);
|
||||
|
||||
// Inform the parent
|
||||
if (parent != null) {
|
||||
|
@ -853,8 +799,7 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
boolean sortQueues) {
|
||||
if (application != null) {
|
||||
internalReleaseResource(clusterResource, node,
|
||||
rmContainer.getContainer().getResource(), false, completedChildQueue,
|
||||
sortQueues);
|
||||
rmContainer.getContainer().getResource(), false);
|
||||
|
||||
// Inform the parent
|
||||
if (parent != null) {
|
||||
|
@ -1062,4 +1007,37 @@ public class ParentQueue extends AbstractCSQueue {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void apply(Resource cluster,
|
||||
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request) {
|
||||
if (request.anythingAllocatedOrReserved()) {
|
||||
ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode>
|
||||
allocation = request.getFirstAllocatedOrReservedContainer();
|
||||
SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode>
|
||||
schedulerContainer = allocation.getAllocatedOrReservedContainer();
|
||||
|
||||
// Do not modify queue when allocation from reserved container
|
||||
if (allocation.getAllocateFromReservedContainer() == null) {
|
||||
try {
|
||||
writeLock.lock();
|
||||
// Book-keeping
|
||||
// Note: Update headroom to account for current allocation too...
|
||||
allocateResource(cluster, allocation.getAllocatedOrReservedResource(),
|
||||
schedulerContainer.getNodePartition(),
|
||||
allocation.isIncreasedAllocation());
|
||||
|
||||
LOG.info("assignedContainer" + " queue=" + getQueueName()
|
||||
+ " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
|
||||
+ getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed()
|
||||
+ " cluster=" + cluster);
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (parent != null) {
|
||||
parent.apply(cluster, request);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssign
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
|
||||
|
@ -77,11 +78,13 @@ public abstract class AbstractContainerAllocator {
|
|||
// Handle excess reservation
|
||||
assignment.setExcessReservation(result.getContainerToBeUnreserved());
|
||||
|
||||
assignment.setRequestLocalityType(result.requestLocalityType);
|
||||
|
||||
// If we allocated something
|
||||
if (Resources.greaterThan(rc, clusterResource,
|
||||
result.getResourceToBeAllocated(), Resources.none())) {
|
||||
Resource allocatedResource = result.getResourceToBeAllocated();
|
||||
Container updatedContainer = result.getUpdatedContainer();
|
||||
RMContainer updatedContainer = result.getUpdatedContainer();
|
||||
|
||||
assignment.setResource(allocatedResource);
|
||||
assignment.setType(result.getContainerNodeType());
|
||||
|
@ -92,8 +95,7 @@ public abstract class AbstractContainerAllocator {
|
|||
+ application.getApplicationId() + " resource=" + allocatedResource
|
||||
+ " queue=" + this.toString() + " cluster=" + clusterResource);
|
||||
assignment.getAssignmentInformation().addReservationDetails(
|
||||
updatedContainer.getId(),
|
||||
application.getCSLeafQueue().getQueuePath());
|
||||
updatedContainer, application.getCSLeafQueue().getQueuePath());
|
||||
assignment.getAssignmentInformation().incrReservations();
|
||||
Resources.addTo(assignment.getAssignmentInformation().getReserved(),
|
||||
allocatedResource);
|
||||
|
@ -111,41 +113,37 @@ public abstract class AbstractContainerAllocator {
|
|||
ActivityState.RESERVED);
|
||||
ActivitiesLogger.APP.finishAllocatedAppAllocationRecording(
|
||||
activitiesManager, application.getApplicationId(),
|
||||
updatedContainer.getId(), ActivityState.RESERVED,
|
||||
updatedContainer.getContainerId(), ActivityState.RESERVED,
|
||||
ActivityDiagnosticConstant.EMPTY);
|
||||
}
|
||||
} else if (result.getAllocationState() == AllocationState.ALLOCATED){
|
||||
// This is a new container
|
||||
// Inform the ordering policy
|
||||
LOG.info("assignedContainer" + " application attempt="
|
||||
+ application.getApplicationAttemptId() + " container="
|
||||
+ updatedContainer.getId() + " queue=" + this + " clusterResource="
|
||||
LOG.info("assignedContainer" + " application attempt=" + application
|
||||
.getApplicationAttemptId() + " container=" + updatedContainer
|
||||
.getContainerId() + " queue=" + this + " clusterResource="
|
||||
+ clusterResource + " type=" + assignment.getType());
|
||||
|
||||
application
|
||||
.getCSLeafQueue()
|
||||
.getOrderingPolicy()
|
||||
.containerAllocated(application,
|
||||
application.getRMContainer(updatedContainer.getId()));
|
||||
|
||||
assignment.getAssignmentInformation().addAllocationDetails(
|
||||
updatedContainer.getId(),
|
||||
application.getCSLeafQueue().getQueuePath());
|
||||
updatedContainer, application.getCSLeafQueue().getQueuePath());
|
||||
assignment.getAssignmentInformation().incrAllocations();
|
||||
Resources.addTo(assignment.getAssignmentInformation().getAllocated(),
|
||||
allocatedResource);
|
||||
|
||||
if (rmContainer != null) {
|
||||
assignment.setFulfilledReservation(true);
|
||||
assignment.setFulfilledReservedContainer(rmContainer);
|
||||
}
|
||||
|
||||
ActivitiesLogger.APP.recordAppActivityWithAllocation(activitiesManager,
|
||||
node, application, updatedContainer, ActivityState.ALLOCATED);
|
||||
ActivitiesLogger.APP.finishAllocatedAppAllocationRecording(
|
||||
activitiesManager, application.getApplicationId(),
|
||||
updatedContainer.getId(), ActivityState.ACCEPTED,
|
||||
updatedContainer.getContainerId(), ActivityState.ACCEPTED,
|
||||
ActivityDiagnosticConstant.EMPTY);
|
||||
|
||||
// Update unformed resource
|
||||
application.incUnconfirmedRes(allocatedResource);
|
||||
}
|
||||
|
||||
assignment.setContainersToKill(result.getToKillContainers());
|
||||
|
@ -170,8 +168,15 @@ public abstract class AbstractContainerAllocator {
|
|||
* <li>Do allocation: this will decide/create allocated/reserved
|
||||
* container, this will also update metrics</li>
|
||||
* </ul>
|
||||
*
|
||||
* @param clusterResource clusterResource
|
||||
* @param ps PlacementSet
|
||||
* @param schedulingMode scheduling mode (exclusive or nonexclusive)
|
||||
* @param resourceLimits resourceLimits
|
||||
* @param reservedContainer reservedContainer
|
||||
* @return CSAssignemnt proposal
|
||||
*/
|
||||
public abstract CSAssignment assignContainers(Resource clusterResource,
|
||||
FiCaSchedulerNode node, SchedulingMode schedulingMode,
|
||||
PlacementSet<FiCaSchedulerNode> ps, SchedulingMode schedulingMode,
|
||||
ResourceLimits resourceLimits, RMContainer reservedContainer);
|
||||
}
|
|
@ -23,6 +23,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
|
|||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
|
||||
import java.util.List;
|
||||
|
@ -57,8 +58,13 @@ public class ContainerAllocation {
|
|||
private Resource resourceToBeAllocated = Resources.none();
|
||||
AllocationState state;
|
||||
NodeType containerNodeType = NodeType.NODE_LOCAL;
|
||||
NodeType requestNodeType = NodeType.NODE_LOCAL;
|
||||
Container updatedContainer;
|
||||
NodeType requestLocalityType = null;
|
||||
|
||||
/**
|
||||
* When some (new) container allocated/reserved or some increase container
|
||||
* request allocated/reserved, updatedContainer will be set.
|
||||
*/
|
||||
RMContainer updatedContainer;
|
||||
private List<RMContainer> toKillContainers;
|
||||
|
||||
public ContainerAllocation(RMContainer containerToBeUnreserved,
|
||||
|
@ -87,7 +93,7 @@ public class ContainerAllocation {
|
|||
return containerNodeType;
|
||||
}
|
||||
|
||||
public Container getUpdatedContainer() {
|
||||
public RMContainer getUpdatedContainer() {
|
||||
return updatedContainer;
|
||||
}
|
||||
|
||||
|
|
|
@ -28,12 +28,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssign
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
|
||||
public class ContainerAllocator extends AbstractContainerAllocator {
|
||||
AbstractContainerAllocator increaseContainerAllocator;
|
||||
AbstractContainerAllocator regularContainerAllocator;
|
||||
private AbstractContainerAllocator increaseContainerAllocator;
|
||||
private AbstractContainerAllocator regularContainerAllocator;
|
||||
|
||||
public ContainerAllocator(FiCaSchedulerApp application,
|
||||
ResourceCalculator rc, RMContext rmContext) {
|
||||
|
@ -52,17 +53,17 @@ public class ContainerAllocator extends AbstractContainerAllocator {
|
|||
|
||||
@Override
|
||||
public CSAssignment assignContainers(Resource clusterResource,
|
||||
FiCaSchedulerNode node, SchedulingMode schedulingMode,
|
||||
PlacementSet<FiCaSchedulerNode> ps, SchedulingMode schedulingMode,
|
||||
ResourceLimits resourceLimits, RMContainer reservedContainer) {
|
||||
if (reservedContainer != null) {
|
||||
if (reservedContainer.getState() == RMContainerState.RESERVED) {
|
||||
// It's a regular container
|
||||
return regularContainerAllocator.assignContainers(clusterResource,
|
||||
node, schedulingMode, resourceLimits, reservedContainer);
|
||||
ps, schedulingMode, resourceLimits, reservedContainer);
|
||||
} else {
|
||||
// It's a increase container
|
||||
return increaseContainerAllocator.assignContainers(clusterResource,
|
||||
node, schedulingMode, resourceLimits, reservedContainer);
|
||||
ps, schedulingMode, resourceLimits, reservedContainer);
|
||||
}
|
||||
} else {
|
||||
/*
|
||||
|
@ -70,14 +71,14 @@ public class ContainerAllocator extends AbstractContainerAllocator {
|
|||
* anything, we will try to allocate regular container
|
||||
*/
|
||||
CSAssignment assign =
|
||||
increaseContainerAllocator.assignContainers(clusterResource, node,
|
||||
increaseContainerAllocator.assignContainers(clusterResource, ps,
|
||||
schedulingMode, resourceLimits, null);
|
||||
if (Resources.greaterThan(rc, clusterResource, assign.getResource(),
|
||||
Resources.none())) {
|
||||
return assign;
|
||||
}
|
||||
|
||||
return regularContainerAllocator.assignContainers(clusterResource, node,
|
||||
return regularContainerAllocator.assignContainers(clusterResource, ps,
|
||||
schedulingMode, resourceLimits, null);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,12 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
|
@ -36,16 +30,21 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
|
||||
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils;
|
||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
public class IncreaseContainerAllocator extends AbstractContainerAllocator {
|
||||
private static final Log LOG =
|
||||
LogFactory.getLog(IncreaseContainerAllocator.class);
|
||||
|
@ -76,7 +75,7 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
|
|||
request.getDeltaCapacity());
|
||||
assignment.getAssignmentInformation().incrReservations();
|
||||
assignment.getAssignmentInformation().addReservationDetails(
|
||||
request.getContainerId(), application.getCSLeafQueue().getQueuePath());
|
||||
request.getRMContainer(), application.getCSLeafQueue().getQueuePath());
|
||||
assignment.setIncreasedAllocation(true);
|
||||
|
||||
LOG.info("Reserved increase container request:" + request.toString());
|
||||
|
@ -93,8 +92,12 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
|
|||
request.getDeltaCapacity());
|
||||
assignment.getAssignmentInformation().incrAllocations();
|
||||
assignment.getAssignmentInformation().addAllocationDetails(
|
||||
request.getContainerId(), application.getCSLeafQueue().getQueuePath());
|
||||
request.getRMContainer(), application.getCSLeafQueue().getQueuePath());
|
||||
assignment.setIncreasedAllocation(true);
|
||||
|
||||
if (fromReservation) {
|
||||
assignment.setFulfilledReservedContainer(request.getRMContainer());
|
||||
}
|
||||
|
||||
// notify application
|
||||
application
|
||||
|
@ -114,19 +117,6 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
|
|||
SchedContainerChangeRequest increaseRequest) {
|
||||
if (Resources.fitsIn(rc, cluster, increaseRequest.getDeltaCapacity(),
|
||||
node.getUnallocatedResource())) {
|
||||
// OK, we can allocate this increase request
|
||||
// Unreserve it first
|
||||
application.unreserve(
|
||||
increaseRequest.getRMContainer().getAllocatedSchedulerKey(),
|
||||
(FiCaSchedulerNode) node, increaseRequest.getRMContainer());
|
||||
|
||||
// Notify application
|
||||
application.increaseContainer(increaseRequest);
|
||||
|
||||
// Notify node
|
||||
node.increaseContainer(increaseRequest.getContainerId(),
|
||||
increaseRequest.getDeltaCapacity());
|
||||
|
||||
return createSuccessfullyIncreasedCSAssignment(increaseRequest, true);
|
||||
} else {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
@ -144,40 +134,26 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
|
|||
Resource cluster, SchedContainerChangeRequest increaseRequest) {
|
||||
if (Resources.fitsIn(rc, cluster, increaseRequest.getDeltaCapacity(),
|
||||
node.getUnallocatedResource())) {
|
||||
// Notify node
|
||||
node.increaseContainer(increaseRequest.getContainerId(),
|
||||
increaseRequest.getDeltaCapacity());
|
||||
|
||||
// OK, we can allocate this increase request
|
||||
// Notify application
|
||||
application.increaseContainer(increaseRequest);
|
||||
return createSuccessfullyIncreasedCSAssignment(increaseRequest, false);
|
||||
} else {
|
||||
boolean reservationSucceeded =
|
||||
application.reserveIncreasedContainer(
|
||||
increaseRequest.getRMContainer().getAllocatedSchedulerKey(),
|
||||
node, increaseRequest.getRMContainer(),
|
||||
increaseRequest.getDeltaCapacity());
|
||||
|
||||
if (reservationSucceeded) {
|
||||
// We cannot allocate this container, but since queue capacity /
|
||||
// user-limit matches, we can reserve this container on this node.
|
||||
return createReservedIncreasedCSAssignment(increaseRequest);
|
||||
} else {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Reserve increase request=" + increaseRequest.toString()
|
||||
+ " failed. Skipping..");
|
||||
}
|
||||
return CSAssignment.SKIP_ASSIGNMENT;
|
||||
}
|
||||
} else{
|
||||
// We cannot allocate this container, but since queue capacity /
|
||||
// user-limit matches, we can reserve this container on this node.
|
||||
return createReservedIncreasedCSAssignment(increaseRequest);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CSAssignment assignContainers(Resource clusterResource,
|
||||
FiCaSchedulerNode node, SchedulingMode schedulingMode,
|
||||
PlacementSet<FiCaSchedulerNode> ps, SchedulingMode schedulingMode,
|
||||
ResourceLimits resourceLimits, RMContainer reservedContainer) {
|
||||
AppSchedulingInfo sinfo = application.getAppSchedulingInfo();
|
||||
FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps);
|
||||
|
||||
if (null == node) {
|
||||
// This is global scheduling enabled
|
||||
// FIXME, support container increase when global scheduling enabled
|
||||
return CSAssignment.SKIP_ASSIGNMENT;
|
||||
}
|
||||
NodeId nodeId = node.getNodeID();
|
||||
|
||||
if (reservedContainer == null) {
|
||||
|
@ -258,8 +234,6 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
|
|||
}
|
||||
Iterator<Entry<ContainerId, SchedContainerChangeRequest>> iter =
|
||||
increaseRequestMap.entrySet().iterator();
|
||||
List<SchedContainerChangeRequest> toBeRemovedRequests =
|
||||
new ArrayList<>();
|
||||
|
||||
while (iter.hasNext()) {
|
||||
Entry<ContainerId, SchedContainerChangeRequest> entry =
|
||||
|
@ -289,7 +263,7 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
|
|||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(" Container is not running any more, skip...");
|
||||
}
|
||||
toBeRemovedRequests.add(increaseRequest);
|
||||
application.addToBeRemovedIncreaseRequest(increaseRequest);
|
||||
continue;
|
||||
}
|
||||
|
||||
|
@ -304,7 +278,7 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
|
|||
LOG.debug(" Target capacity is more than what node can offer,"
|
||||
+ " node.resource=" + node.getTotalResource());
|
||||
}
|
||||
toBeRemovedRequests.add(increaseRequest);
|
||||
application.addToBeRemovedIncreaseRequest(increaseRequest);
|
||||
continue;
|
||||
}
|
||||
|
||||
|
@ -318,15 +292,6 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
|
|||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// Remove invalid in request requests
|
||||
if (!toBeRemovedRequests.isEmpty()) {
|
||||
for (SchedContainerChangeRequest req : toBeRemovedRequests) {
|
||||
sinfo.removeIncreaseRequest(req.getNodeId(),
|
||||
req.getRMContainer().getAllocatedSchedulerKey(),
|
||||
req.getContainerId());
|
||||
}
|
||||
}
|
||||
|
||||
// We may have allocated something
|
||||
if (assigned != null && assigned.getSkippedType()
|
||||
|
|
|
@ -19,13 +19,13 @@
|
|||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.yarn.api.records.Container;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.api.records.Priority;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
|
@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils;
|
||||
|
@ -50,6 +51,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssign
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
|
||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
|
@ -71,12 +75,12 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
|
||||
private boolean checkHeadroom(Resource clusterResource,
|
||||
ResourceLimits currentResourceLimits, Resource required,
|
||||
FiCaSchedulerNode node) {
|
||||
String nodePartition) {
|
||||
// If headroom + currentReservation < required, we cannot allocate this
|
||||
// require
|
||||
Resource resourceCouldBeUnReserved = application.getCurrentReservation();
|
||||
if (!application.getCSLeafQueue().getReservationContinueLooking()
|
||||
|| !node.getPartition().equals(RMNodeLabelsManager.NO_LABEL)) {
|
||||
|| !nodePartition.equals(RMNodeLabelsManager.NO_LABEL)) {
|
||||
// If we don't allow reservation continuous looking, OR we're looking at
|
||||
// non-default node partition, we won't allow to unreserve before
|
||||
// allocation.
|
||||
|
@ -87,20 +91,17 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
required);
|
||||
}
|
||||
|
||||
|
||||
private ContainerAllocation preCheckForNewContainer(Resource clusterResource,
|
||||
FiCaSchedulerNode node, SchedulingMode schedulingMode,
|
||||
/*
|
||||
* Pre-check if we can allocate a pending resource request
|
||||
* (given schedulerKey) to a given PlacementSet.
|
||||
* We will consider stuffs like exclusivity, pending resource, node partition,
|
||||
* headroom, etc.
|
||||
*/
|
||||
private ContainerAllocation preCheckForPlacementSet(Resource clusterResource,
|
||||
PlacementSet<FiCaSchedulerNode> ps, SchedulingMode schedulingMode,
|
||||
ResourceLimits resourceLimits, SchedulerRequestKey schedulerKey) {
|
||||
Priority priority = schedulerKey.getPriority();
|
||||
|
||||
if (SchedulerAppUtils.isPlaceBlacklisted(application, node, LOG)) {
|
||||
application.updateAppSkipNodeDiagnostics(
|
||||
CSAMContainerLaunchDiagnosticsConstants.SKIP_AM_ALLOCATION_IN_BLACK_LISTED_NODE);
|
||||
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
||||
activitiesManager, node, application, priority,
|
||||
ActivityDiagnosticConstant.SKIP_BLACK_LISTED_NODE);
|
||||
return ContainerAllocation.APP_SKIPPED;
|
||||
}
|
||||
FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps);
|
||||
|
||||
ResourceRequest anyRequest =
|
||||
application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
|
||||
|
@ -144,7 +145,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
// matches the node's label?
|
||||
// If not match, jump to next priority.
|
||||
if (!SchedulerUtils.checkResourceRequestMatchingNodePartition(
|
||||
anyRequest.getNodeLabelExpression(), node.getPartition(),
|
||||
anyRequest.getNodeLabelExpression(), ps.getPartition(),
|
||||
schedulingMode)) {
|
||||
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
||||
activitiesManager, node, application, priority,
|
||||
|
@ -165,7 +166,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
}
|
||||
}
|
||||
|
||||
if (!checkHeadroom(clusterResource, resourceLimits, required, node)) {
|
||||
if (!checkHeadroom(clusterResource, resourceLimits, required,
|
||||
ps.getPartition())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("cannot allocate required resource=" + required
|
||||
+ " because of headroom");
|
||||
|
@ -176,9 +178,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
return ContainerAllocation.QUEUE_SKIPPED;
|
||||
}
|
||||
|
||||
// Inform the application it is about to get a scheduling opportunity
|
||||
application.addSchedulingOpportunity(schedulerKey);
|
||||
|
||||
// Increase missed-non-partitioned-resource-request-opportunity.
|
||||
// This is to make sure non-partitioned-resource-request will prefer
|
||||
// to be allocated to non-partitioned nodes
|
||||
|
@ -210,32 +209,43 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
return ContainerAllocation.APP_SKIPPED;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
ContainerAllocation preAllocation(Resource clusterResource,
|
||||
private ContainerAllocation checkIfNodeBlackListed(FiCaSchedulerNode node,
|
||||
SchedulerRequestKey schedulerKey) {
|
||||
Priority priority = schedulerKey.getPriority();
|
||||
|
||||
if (SchedulerAppUtils.isPlaceBlacklisted(application, node, LOG)) {
|
||||
application.updateAppSkipNodeDiagnostics(
|
||||
CSAMContainerLaunchDiagnosticsConstants.SKIP_AM_ALLOCATION_IN_BLACK_LISTED_NODE);
|
||||
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
||||
activitiesManager, node, application, priority,
|
||||
ActivityDiagnosticConstant.SKIP_BLACK_LISTED_NODE);
|
||||
return ContainerAllocation.APP_SKIPPED;
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
ContainerAllocation tryAllocateOnNode(Resource clusterResource,
|
||||
FiCaSchedulerNode node, SchedulingMode schedulingMode,
|
||||
ResourceLimits resourceLimits, SchedulerRequestKey schedulerKey,
|
||||
RMContainer reservedContainer) {
|
||||
ContainerAllocation result;
|
||||
if (null == reservedContainer) {
|
||||
// pre-check when allocating new container
|
||||
result =
|
||||
preCheckForNewContainer(clusterResource, node, schedulingMode,
|
||||
resourceLimits, schedulerKey);
|
||||
if (null != result) {
|
||||
return result;
|
||||
}
|
||||
} else {
|
||||
// pre-check when allocating reserved container
|
||||
if (application.getTotalRequiredResources(schedulerKey) == 0) {
|
||||
// Release
|
||||
return new ContainerAllocation(reservedContainer, null,
|
||||
AllocationState.QUEUE_SKIPPED);
|
||||
}
|
||||
|
||||
// Sanity checks before assigning to this node
|
||||
result = checkIfNodeBlackListed(node, schedulerKey);
|
||||
if (null != result) {
|
||||
return result;
|
||||
}
|
||||
|
||||
// Inform the application it is about to get a scheduling opportunity
|
||||
// TODO, we may need to revisit here to see if we should add scheduling
|
||||
// opportunity here
|
||||
application.addSchedulingOpportunity(schedulerKey);
|
||||
|
||||
// Try to allocate containers on node
|
||||
result =
|
||||
assignContainersOnNode(clusterResource, node, schedulerKey,
|
||||
|
@ -383,20 +393,20 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
Priority priority = schedulerKey.getPriority();
|
||||
|
||||
ContainerAllocation allocation;
|
||||
NodeType requestLocalityType = null;
|
||||
|
||||
NodeType requestType = null;
|
||||
// Data-local
|
||||
ResourceRequest nodeLocalResourceRequest =
|
||||
application.getResourceRequest(schedulerKey, node.getNodeName());
|
||||
if (nodeLocalResourceRequest != null) {
|
||||
requestType = NodeType.NODE_LOCAL;
|
||||
requestLocalityType = NodeType.NODE_LOCAL;
|
||||
allocation =
|
||||
assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest,
|
||||
node, schedulerKey, reservedContainer, schedulingMode,
|
||||
currentResoureLimits);
|
||||
if (Resources.greaterThan(rc, clusterResource,
|
||||
allocation.getResourceToBeAllocated(), Resources.none())) {
|
||||
allocation.requestNodeType = requestType;
|
||||
allocation.requestLocalityType = requestLocalityType;
|
||||
return allocation;
|
||||
}
|
||||
}
|
||||
|
@ -412,9 +422,9 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
return ContainerAllocation.PRIORITY_SKIPPED;
|
||||
}
|
||||
|
||||
if (requestType != NodeType.NODE_LOCAL) {
|
||||
requestType = NodeType.RACK_LOCAL;
|
||||
}
|
||||
requestLocalityType = requestLocalityType == null ?
|
||||
NodeType.RACK_LOCAL :
|
||||
requestLocalityType;
|
||||
|
||||
allocation =
|
||||
assignRackLocalContainers(clusterResource, rackLocalResourceRequest,
|
||||
|
@ -422,7 +432,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
currentResoureLimits);
|
||||
if (Resources.greaterThan(rc, clusterResource,
|
||||
allocation.getResourceToBeAllocated(), Resources.none())) {
|
||||
allocation.requestNodeType = requestType;
|
||||
allocation.requestLocalityType = requestLocalityType;
|
||||
return allocation;
|
||||
}
|
||||
}
|
||||
|
@ -437,22 +447,22 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
ActivityDiagnosticConstant.SKIP_PRIORITY_BECAUSE_OF_RELAX_LOCALITY);
|
||||
return ContainerAllocation.PRIORITY_SKIPPED;
|
||||
}
|
||||
if (requestType != NodeType.NODE_LOCAL
|
||||
&& requestType != NodeType.RACK_LOCAL) {
|
||||
requestType = NodeType.OFF_SWITCH;
|
||||
}
|
||||
|
||||
requestLocalityType = requestLocalityType == null ?
|
||||
NodeType.OFF_SWITCH :
|
||||
requestLocalityType;
|
||||
|
||||
allocation =
|
||||
assignOffSwitchContainers(clusterResource, offSwitchResourceRequest,
|
||||
node, schedulerKey, reservedContainer, schedulingMode,
|
||||
currentResoureLimits);
|
||||
allocation.requestNodeType = requestType;
|
||||
|
||||
|
||||
// When a returned allocation is LOCALITY_SKIPPED, since we're in
|
||||
// off-switch request now, we will skip this app w.r.t priorities
|
||||
if (allocation.state == AllocationState.LOCALITY_SKIPPED) {
|
||||
allocation.state = AllocationState.APP_SKIPPED;
|
||||
}
|
||||
allocation.requestLocalityType = requestLocalityType;
|
||||
|
||||
return allocation;
|
||||
}
|
||||
|
@ -671,33 +681,27 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
|
||||
private Container createContainer(FiCaSchedulerNode node, Resource capability,
|
||||
SchedulerRequestKey schedulerKey) {
|
||||
|
||||
NodeId nodeId = node.getRMNode().getNodeID();
|
||||
ContainerId containerId =
|
||||
BuilderUtils.newContainerId(application.getApplicationAttemptId(),
|
||||
application.getNewContainerId());
|
||||
|
||||
// Create the container
|
||||
return BuilderUtils.newContainer(containerId, nodeId,
|
||||
// Now set the containerId to null first, because it is possible the
|
||||
// container will be rejected because of concurrent resource allocation.
|
||||
// new containerId will be generated and assigned to the container
|
||||
// after confirmed.
|
||||
return BuilderUtils.newContainer(null, nodeId,
|
||||
node.getRMNode().getHttpAddress(), capability,
|
||||
schedulerKey.getPriority(), null,
|
||||
schedulerKey.getAllocationRequestId());
|
||||
}
|
||||
|
||||
|
||||
private ContainerAllocation handleNewContainerAllocation(
|
||||
ContainerAllocation allocationResult, FiCaSchedulerNode node,
|
||||
SchedulerRequestKey schedulerKey, RMContainer reservedContainer,
|
||||
Container container) {
|
||||
// Handling container allocation
|
||||
// Did we previously reserve containers at this 'priority'?
|
||||
if (reservedContainer != null) {
|
||||
application.unreserve(schedulerKey, node, reservedContainer);
|
||||
}
|
||||
|
||||
SchedulerRequestKey schedulerKey, Container container) {
|
||||
// Inform the application
|
||||
RMContainer allocatedContainer =
|
||||
application.allocate(allocationResult.containerNodeType, node,
|
||||
schedulerKey, lastResourceRequest, container);
|
||||
RMContainer allocatedContainer = application.allocate(node, schedulerKey,
|
||||
lastResourceRequest, container);
|
||||
|
||||
allocationResult.updatedContainer = allocatedContainer;
|
||||
|
||||
// Does the application need this resource?
|
||||
if (allocatedContainer == null) {
|
||||
|
@ -710,13 +714,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
ActivityDiagnosticConstant.FAIL_TO_ALLOCATE, ActivityState.REJECTED);
|
||||
return ret;
|
||||
}
|
||||
|
||||
// Inform the node
|
||||
node.allocateContainer(allocatedContainer);
|
||||
|
||||
// update locality statistics
|
||||
application.incNumAllocatedContainers(allocationResult.containerNodeType,
|
||||
allocationResult.requestNodeType);
|
||||
|
||||
return allocationResult;
|
||||
}
|
||||
|
@ -743,14 +740,18 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
|
||||
if (allocationResult.getAllocationState() == AllocationState.ALLOCATED) {
|
||||
// When allocating container
|
||||
allocationResult =
|
||||
handleNewContainerAllocation(allocationResult, node, schedulerKey,
|
||||
reservedContainer, container);
|
||||
allocationResult = handleNewContainerAllocation(allocationResult, node,
|
||||
schedulerKey, container);
|
||||
} else {
|
||||
// When reserving container
|
||||
application.reserve(schedulerKey, node, reservedContainer, container);
|
||||
RMContainer updatedContainer = reservedContainer;
|
||||
if (updatedContainer == null) {
|
||||
updatedContainer = new RMContainerImpl(container,
|
||||
application.getApplicationAttemptId(), node.getNodeID(),
|
||||
application.getAppSchedulingInfo().getUser(), rmContext);
|
||||
}
|
||||
allocationResult.updatedContainer = updatedContainer;
|
||||
}
|
||||
allocationResult.updatedContainer = container;
|
||||
|
||||
// Only reset opportunities when we FIRST allocate the container. (IAW, When
|
||||
// reservedContainer != null, it's not the first time)
|
||||
|
@ -788,16 +789,46 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
}
|
||||
|
||||
private ContainerAllocation allocate(Resource clusterResource,
|
||||
FiCaSchedulerNode node, SchedulingMode schedulingMode,
|
||||
PlacementSet<FiCaSchedulerNode> ps, SchedulingMode schedulingMode,
|
||||
ResourceLimits resourceLimits, SchedulerRequestKey schedulerKey,
|
||||
RMContainer reservedContainer) {
|
||||
ContainerAllocation result =
|
||||
preAllocation(clusterResource, node, schedulingMode, resourceLimits,
|
||||
schedulerKey, reservedContainer);
|
||||
// Do checks before determining which node to allocate
|
||||
// Directly return if this check fails.
|
||||
ContainerAllocation result;
|
||||
if (reservedContainer == null) {
|
||||
result = preCheckForPlacementSet(clusterResource, ps, schedulingMode,
|
||||
resourceLimits, schedulerKey);
|
||||
if (null != result) {
|
||||
return result;
|
||||
}
|
||||
} else {
|
||||
// pre-check when allocating reserved container
|
||||
if (application.getTotalRequiredResources(schedulerKey) == 0) {
|
||||
// Release
|
||||
return new ContainerAllocation(reservedContainer, null,
|
||||
AllocationState.QUEUE_SKIPPED);
|
||||
}
|
||||
}
|
||||
|
||||
if (AllocationState.ALLOCATED == result.state
|
||||
|| AllocationState.RESERVED == result.state) {
|
||||
result = doAllocation(result, node, schedulerKey, reservedContainer);
|
||||
SchedulingPlacementSet<FiCaSchedulerNode> schedulingPS =
|
||||
application.getAppSchedulingInfo().getSchedulingPlacementSet(
|
||||
schedulerKey);
|
||||
|
||||
result = ContainerAllocation.PRIORITY_SKIPPED;
|
||||
|
||||
Iterator<FiCaSchedulerNode> iter = schedulingPS.getPreferredNodeIterator(
|
||||
ps);
|
||||
while (iter.hasNext()) {
|
||||
FiCaSchedulerNode node = iter.next();
|
||||
|
||||
result = tryAllocateOnNode(clusterResource, node, schedulingMode,
|
||||
resourceLimits, schedulerKey, reservedContainer);
|
||||
|
||||
if (AllocationState.ALLOCATED == result.state
|
||||
|| AllocationState.RESERVED == result.state) {
|
||||
result = doAllocation(result, node, schedulerKey, reservedContainer);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
|
@ -805,17 +836,19 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
|
||||
@Override
|
||||
public CSAssignment assignContainers(Resource clusterResource,
|
||||
FiCaSchedulerNode node, SchedulingMode schedulingMode,
|
||||
PlacementSet<FiCaSchedulerNode> ps, SchedulingMode schedulingMode,
|
||||
ResourceLimits resourceLimits,
|
||||
RMContainer reservedContainer) {
|
||||
FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps);
|
||||
|
||||
if (reservedContainer == null) {
|
||||
// Check if application needs more resource, skip if it doesn't need more.
|
||||
if (!application.hasPendingResourceRequest(rc,
|
||||
node.getPartition(), clusterResource, schedulingMode)) {
|
||||
ps.getPartition(), clusterResource, schedulingMode)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Skip app_attempt=" + application.getApplicationAttemptId()
|
||||
+ ", because it doesn't need more resource, schedulingMode="
|
||||
+ schedulingMode.name() + " node-label=" + node.getPartition());
|
||||
+ schedulingMode.name() + " node-label=" + ps.getPartition());
|
||||
}
|
||||
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
|
||||
activitiesManager, node, application, application.getPriority(),
|
||||
|
@ -826,7 +859,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
// Schedule in priority order
|
||||
for (SchedulerRequestKey schedulerKey : application.getSchedulerKeys()) {
|
||||
ContainerAllocation result =
|
||||
allocate(clusterResource, node, schedulingMode, resourceLimits,
|
||||
allocate(clusterResource, ps, schedulingMode, resourceLimits,
|
||||
schedulerKey, null);
|
||||
|
||||
AllocationState allocationState = result.getAllocationState();
|
||||
|
@ -845,7 +878,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
|
|||
return CSAssignment.SKIP_ASSIGNMENT;
|
||||
} else {
|
||||
ContainerAllocation result =
|
||||
allocate(clusterResource, node, schedulingMode, resourceLimits,
|
||||
allocate(clusterResource, ps, schedulingMode, resourceLimits,
|
||||
reservedContainer.getReservedSchedulerKey(), reservedContainer);
|
||||
return getCSAssignmentFromAllocateResult(clusterResource, result,
|
||||
reservedContainer, node);
|
||||
|
|
|
@ -38,11 +38,13 @@ public class AssignmentInformation {
|
|||
}
|
||||
|
||||
public static class AssignmentDetails {
|
||||
public RMContainer rmContainer;
|
||||
public ContainerId containerId;
|
||||
public String queue;
|
||||
|
||||
public AssignmentDetails(ContainerId containerId, String queue) {
|
||||
this.containerId = containerId;
|
||||
public AssignmentDetails(RMContainer rmContainer, String queue) {
|
||||
this.containerId = rmContainer.getContainerId();
|
||||
this.rmContainer = rmContainer;
|
||||
this.queue = queue;
|
||||
}
|
||||
}
|
||||
|
@ -58,7 +60,7 @@ public class AssignmentInformation {
|
|||
for (Operation op : Operation.values()) {
|
||||
operationCounts.put(op, 0);
|
||||
operationResources.put(op, Resource.newInstance(0, 0));
|
||||
operationDetails.put(op, new ArrayList<AssignmentDetails>());
|
||||
operationDetails.put(op, new ArrayList<>());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -98,17 +100,17 @@ public class AssignmentInformation {
|
|||
return operationResources.get(Operation.RESERVATION);
|
||||
}
|
||||
|
||||
private void addAssignmentDetails(Operation op, ContainerId containerId,
|
||||
private void addAssignmentDetails(Operation op, RMContainer rmContainer,
|
||||
String queue) {
|
||||
operationDetails.get(op).add(new AssignmentDetails(containerId, queue));
|
||||
operationDetails.get(op).add(new AssignmentDetails(rmContainer, queue));
|
||||
}
|
||||
|
||||
public void addAllocationDetails(ContainerId containerId, String queue) {
|
||||
addAssignmentDetails(Operation.ALLOCATION, containerId, queue);
|
||||
public void addAllocationDetails(RMContainer rmContainer, String queue) {
|
||||
addAssignmentDetails(Operation.ALLOCATION, rmContainer, queue);
|
||||
}
|
||||
|
||||
public void addReservationDetails(ContainerId containerId, String queue) {
|
||||
addAssignmentDetails(Operation.RESERVATION, containerId, queue);
|
||||
public void addReservationDetails(RMContainer rmContainer, String queue) {
|
||||
addAssignmentDetails(Operation.RESERVATION, rmContainer, queue);
|
||||
}
|
||||
|
||||
public List<AssignmentDetails> getAllocationDetails() {
|
||||
|
@ -119,23 +121,31 @@ public class AssignmentInformation {
|
|||
return operationDetails.get(Operation.RESERVATION);
|
||||
}
|
||||
|
||||
private ContainerId getFirstContainerIdFromOperation(Operation op) {
|
||||
private RMContainer getFirstRMContainerFromOperation(Operation op) {
|
||||
if (null != operationDetails.get(op)) {
|
||||
List<AssignmentDetails> assignDetails =
|
||||
operationDetails.get(op);
|
||||
if (!assignDetails.isEmpty()) {
|
||||
return assignDetails.get(0).containerId;
|
||||
return assignDetails.get(0).rmContainer;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public ContainerId getFirstAllocatedOrReservedContainerId() {
|
||||
ContainerId containerId;
|
||||
containerId = getFirstContainerIdFromOperation(Operation.ALLOCATION);
|
||||
if (null != containerId) {
|
||||
return containerId;
|
||||
public RMContainer getFirstAllocatedOrReservedRMContainer() {
|
||||
RMContainer rmContainer;
|
||||
rmContainer = getFirstRMContainerFromOperation(Operation.ALLOCATION);
|
||||
if (null != rmContainer) {
|
||||
return rmContainer;
|
||||
}
|
||||
return getFirstContainerIdFromOperation(Operation.RESERVATION);
|
||||
return getFirstRMContainerFromOperation(Operation.RESERVATION);
|
||||
}
|
||||
}
|
||||
|
||||
public ContainerId getFirstAllocatedOrReservedContainerId() {
|
||||
RMContainer rmContainer = getFirstAllocatedOrReservedRMContainer();
|
||||
if (null != rmContainer) {
|
||||
return rmContainer.getContainerId();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,111 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common;
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Proposal to allocate/reserve a new container
|
||||
*/
|
||||
public class ContainerAllocationProposal<A extends SchedulerApplicationAttempt,
|
||||
N extends SchedulerNode> {
|
||||
// Container we allocated or reserved
|
||||
private SchedulerContainer<A, N> allocatedOrReservedContainer;
|
||||
|
||||
// Containers we need to release before allocating or reserving the
|
||||
// new container
|
||||
private List<SchedulerContainer<A, N>> toRelease = Collections.emptyList();
|
||||
|
||||
// When trying to allocate from a reserved container, set this, and this will
|
||||
// not be included by toRelease list
|
||||
private SchedulerContainer<A, N> allocateFromReservedContainer;
|
||||
|
||||
private boolean isIncreasedAllocation;
|
||||
|
||||
private NodeType allocationLocalityType;
|
||||
|
||||
private NodeType requestLocalityType;
|
||||
|
||||
private SchedulingMode schedulingMode;
|
||||
|
||||
private Resource allocatedResource; // newly allocated resource
|
||||
|
||||
public ContainerAllocationProposal(
|
||||
SchedulerContainer<A, N> allocatedOrReservedContainer,
|
||||
List<SchedulerContainer<A, N>> toRelease,
|
||||
SchedulerContainer<A, N> allocateFromReservedContainer,
|
||||
boolean isIncreasedAllocation, NodeType allocationLocalityType,
|
||||
NodeType requestLocalityType, SchedulingMode schedulingMode,
|
||||
Resource allocatedResource) {
|
||||
this.allocatedOrReservedContainer = allocatedOrReservedContainer;
|
||||
if (null != toRelease) {
|
||||
this.toRelease = toRelease;
|
||||
}
|
||||
this.allocateFromReservedContainer = allocateFromReservedContainer;
|
||||
this.isIncreasedAllocation = isIncreasedAllocation;
|
||||
this.allocationLocalityType = allocationLocalityType;
|
||||
this.requestLocalityType = requestLocalityType;
|
||||
this.schedulingMode = schedulingMode;
|
||||
this.allocatedResource = allocatedResource;
|
||||
}
|
||||
|
||||
public SchedulingMode getSchedulingMode() {
|
||||
return schedulingMode;
|
||||
}
|
||||
|
||||
public Resource getAllocatedOrReservedResource() {
|
||||
return allocatedResource;
|
||||
}
|
||||
|
||||
public NodeType getAllocationLocalityType() {
|
||||
return allocationLocalityType;
|
||||
}
|
||||
|
||||
public boolean isIncreasedAllocation() {
|
||||
return isIncreasedAllocation;
|
||||
}
|
||||
|
||||
public SchedulerContainer<A, N> getAllocateFromReservedContainer() {
|
||||
return allocateFromReservedContainer;
|
||||
}
|
||||
|
||||
public SchedulerContainer<A, N> getAllocatedOrReservedContainer() {
|
||||
return allocatedOrReservedContainer;
|
||||
}
|
||||
|
||||
public List<SchedulerContainer<A, N>> getToRelease() {
|
||||
return toRelease;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return allocatedOrReservedContainer.toString();
|
||||
}
|
||||
|
||||
public NodeType getRequestLocalityType() {
|
||||
return requestLocalityType;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,29 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common;
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
|
||||
/**
|
||||
* Scheduler should implement this interface if it wants to have multi-threading
|
||||
* plus global scheduling functionality
|
||||
*/
|
||||
public interface ResourceAllocationCommitter {
|
||||
void tryCommit(Resource cluster, ResourceCommitRequest proposal);
|
||||
}
|
|
@ -0,0 +1,164 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common;
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
public class ResourceCommitRequest<A extends SchedulerApplicationAttempt,
|
||||
N extends SchedulerNode> {
|
||||
// New containers to be allocated
|
||||
private List<ContainerAllocationProposal<A, N>> containersToAllocate =
|
||||
Collections.emptyList();
|
||||
|
||||
// New containers to be released
|
||||
private List<ContainerAllocationProposal<A, N>> containersToReserve =
|
||||
Collections.emptyList();
|
||||
|
||||
// We don't need these containers anymore
|
||||
private List<SchedulerContainer<A, N>> toReleaseContainers =
|
||||
Collections.emptyList();
|
||||
|
||||
private Resource totalAllocatedResource;
|
||||
private Resource totalReservedResource;
|
||||
private Resource totalReleasedResource;
|
||||
|
||||
public ResourceCommitRequest(
|
||||
List<ContainerAllocationProposal<A, N>> containersToAllocate,
|
||||
List<ContainerAllocationProposal<A, N>> containersToReserve,
|
||||
List<SchedulerContainer<A, N>> toReleaseContainers) {
|
||||
if (null != containersToAllocate) {
|
||||
this.containersToAllocate = containersToAllocate;
|
||||
}
|
||||
if (null != containersToReserve) {
|
||||
this.containersToReserve = containersToReserve;
|
||||
}
|
||||
if (null != toReleaseContainers) {
|
||||
this.toReleaseContainers = toReleaseContainers;
|
||||
}
|
||||
|
||||
totalAllocatedResource = Resources.createResource(0);
|
||||
totalReservedResource = Resources.createResource(0);
|
||||
|
||||
/*
|
||||
* For total-release resource, it has two parts:
|
||||
* 1) Unconditional release: for example, an app reserved a container,
|
||||
* but the app doesn't has any pending resource.
|
||||
* 2) Conditional release: for example, reservation continuous looking, or
|
||||
* Lazy preemption -- which we need to kill some resource to allocate
|
||||
* or reserve the new container.
|
||||
*
|
||||
* For the 2nd part, it is inside:
|
||||
* ContainerAllocationProposal#toRelease, which means we will kill/release
|
||||
* these containers to allocate/reserve the given container.
|
||||
*
|
||||
* So we need to account both of conditional/unconditional to-release
|
||||
* containers to the total release-able resource.
|
||||
*/
|
||||
totalReleasedResource = Resources.createResource(0);
|
||||
|
||||
for (ContainerAllocationProposal<A,N> c : this.containersToAllocate) {
|
||||
Resources.addTo(totalAllocatedResource,
|
||||
c.getAllocatedOrReservedResource());
|
||||
for (SchedulerContainer<A,N> r : c.getToRelease()) {
|
||||
Resources.addTo(totalReleasedResource,
|
||||
r.getRmContainer().getAllocatedOrReservedResource());
|
||||
}
|
||||
}
|
||||
|
||||
for (ContainerAllocationProposal<A,N> c : this.containersToReserve) {
|
||||
Resources.addTo(totalReservedResource,
|
||||
c.getAllocatedOrReservedResource());
|
||||
for (SchedulerContainer<A,N> r : c.getToRelease()) {
|
||||
Resources.addTo(totalReleasedResource,
|
||||
r.getRmContainer().getAllocatedOrReservedResource());
|
||||
}
|
||||
}
|
||||
|
||||
for (SchedulerContainer<A,N> r : this.toReleaseContainers) {
|
||||
Resources.addTo(totalReleasedResource,
|
||||
r.getRmContainer().getAllocatedOrReservedResource());
|
||||
}
|
||||
}
|
||||
|
||||
public List<ContainerAllocationProposal<A, N>> getContainersToAllocate() {
|
||||
return containersToAllocate;
|
||||
}
|
||||
|
||||
public List<ContainerAllocationProposal<A, N>> getContainersToReserve() {
|
||||
return containersToReserve;
|
||||
}
|
||||
|
||||
public List<SchedulerContainer<A, N>> getContainersToRelease() {
|
||||
return toReleaseContainers;
|
||||
}
|
||||
|
||||
public Resource getTotalAllocatedResource() {
|
||||
return totalAllocatedResource;
|
||||
}
|
||||
|
||||
public Resource getTotalReservedResource() {
|
||||
return totalReservedResource;
|
||||
}
|
||||
|
||||
public Resource getTotalReleasedResource() {
|
||||
return totalReleasedResource;
|
||||
}
|
||||
|
||||
/*
|
||||
* Util functions to make your life easier
|
||||
*/
|
||||
public boolean anythingAllocatedOrReserved() {
|
||||
return (!containersToAllocate.isEmpty()) || (!containersToReserve
|
||||
.isEmpty());
|
||||
}
|
||||
|
||||
public ContainerAllocationProposal<A, N> getFirstAllocatedOrReservedContainer() {
|
||||
ContainerAllocationProposal<A, N> c = null;
|
||||
if (!containersToAllocate.isEmpty()) {
|
||||
c = containersToAllocate.get(0);
|
||||
}
|
||||
if (c == null && !containersToReserve.isEmpty()) {
|
||||
c = containersToReserve.get(0);
|
||||
}
|
||||
|
||||
return c;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("New " + getClass().getName() + ":" + "\n");
|
||||
if (null != containersToAllocate && !containersToAllocate.isEmpty()) {
|
||||
sb.append("\t ALLOCATED=" + containersToAllocate.toString());
|
||||
}
|
||||
if (null != containersToReserve && !containersToReserve.isEmpty()) {
|
||||
sb.append("\t RESERVED=" + containersToReserve.toString());
|
||||
}
|
||||
if (null != toReleaseContainers && !toReleaseContainers.isEmpty()) {
|
||||
sb.append("\t RELEASED=" + toReleaseContainers.toString());
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common;
|
||||
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
|
||||
|
||||
/**
|
||||
* Contexts for a container inside scheduler
|
||||
*/
|
||||
public class SchedulerContainer<A extends SchedulerApplicationAttempt,
|
||||
N extends SchedulerNode> {
|
||||
private RMContainer rmContainer;
|
||||
private String nodePartition;
|
||||
private A schedulerApplicationAttempt;
|
||||
private N schedulerNode;
|
||||
private boolean allocated; // Allocated (True) or reserved (False)
|
||||
|
||||
public SchedulerContainer(A app, N node, RMContainer rmContainer,
|
||||
String nodePartition, boolean allocated) {
|
||||
this.schedulerApplicationAttempt = app;
|
||||
this.schedulerNode = node;
|
||||
this.rmContainer = rmContainer;
|
||||
this.nodePartition = nodePartition;
|
||||
this.allocated = allocated;
|
||||
}
|
||||
|
||||
public String getNodePartition() {
|
||||
return nodePartition;
|
||||
}
|
||||
|
||||
public RMContainer getRmContainer() {
|
||||
return rmContainer;
|
||||
}
|
||||
|
||||
public A getSchedulerApplicationAttempt() {
|
||||
return schedulerApplicationAttempt;
|
||||
}
|
||||
|
||||
public N getSchedulerNode() {
|
||||
return schedulerNode;
|
||||
}
|
||||
|
||||
public boolean isAllocated() {
|
||||
return allocated;
|
||||
}
|
||||
|
||||
public SchedulerRequestKey getSchedulerRequestKey() {
|
||||
if (rmContainer.getState() == RMContainerState.RESERVED) {
|
||||
return rmContainer.getReservedSchedulerKey();
|
||||
}
|
||||
return rmContainer.getAllocatedSchedulerKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "(Application=" + schedulerApplicationAttempt
|
||||
.getApplicationAttemptId() + "; Node=" + schedulerNode.getNodeID()
|
||||
+ "; Resource=" + rmContainer.getAllocatedOrReservedResource() + ")";
|
||||
}
|
||||
}
|
|
@ -18,14 +18,7 @@
|
|||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
|
@ -51,12 +44,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEven
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerFinishedEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
|
||||
|
@ -70,11 +64,25 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCap
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.AbstractContainerAllocator;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator.ContainerAllocator;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet;
|
||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
/**
|
||||
* Represents an application attempt from the viewpoint of the FIFO or Capacity
|
||||
|
@ -101,6 +109,9 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
*/
|
||||
private String appSkipNodeDiagnostics;
|
||||
|
||||
private Map<ContainerId, SchedContainerChangeRequest> toBeRemovedIncRequests =
|
||||
new ConcurrentHashMap<>();
|
||||
|
||||
public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId,
|
||||
String user, Queue queue, ActiveUsersManager activeUsersManager,
|
||||
RMContext rmContext) {
|
||||
|
@ -193,11 +204,11 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
}
|
||||
}
|
||||
|
||||
public RMContainer allocate(NodeType type, FiCaSchedulerNode node,
|
||||
public RMContainer allocate(FiCaSchedulerNode node,
|
||||
SchedulerRequestKey schedulerKey, ResourceRequest request,
|
||||
Container container) {
|
||||
try {
|
||||
writeLock.lock();
|
||||
readLock.lock();
|
||||
|
||||
if (isStopped) {
|
||||
return null;
|
||||
|
@ -216,41 +227,408 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
request.getNodeLabelExpression());
|
||||
((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
|
||||
|
||||
// FIXME, should set when confirmed
|
||||
updateAMContainerDiagnostics(AMState.ASSIGNED, null);
|
||||
|
||||
// Add it to allContainers list.
|
||||
newlyAllocatedContainers.add(rmContainer);
|
||||
|
||||
ContainerId containerId = container.getId();
|
||||
liveContainers.put(containerId, rmContainer);
|
||||
|
||||
// Update consumption and track allocations
|
||||
List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
|
||||
type, node, schedulerKey, request, container);
|
||||
|
||||
attemptResourceUsage.incUsed(node.getPartition(),
|
||||
container.getResource());
|
||||
|
||||
// Update resource requests related to "request" and store in RMContainer
|
||||
((RMContainerImpl) rmContainer).setResourceRequests(resourceRequestList);
|
||||
|
||||
// Inform the container
|
||||
rmContainer.handle(
|
||||
new RMContainerEvent(containerId, RMContainerEventType.START));
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("allocate: applicationAttemptId=" + containerId
|
||||
.getApplicationAttemptId() + " container=" + containerId + " host="
|
||||
+ container.getNodeId().getHost() + " type=" + type);
|
||||
}
|
||||
RMAuditLogger.logSuccess(getUser(), AuditConstants.ALLOC_CONTAINER,
|
||||
"SchedulerApp", getApplicationId(), containerId,
|
||||
container.getResource());
|
||||
|
||||
return rmContainer;
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
private boolean rmContainerInFinalState(RMContainer rmContainer) {
|
||||
if (null == rmContainer) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return rmContainer.completed();
|
||||
}
|
||||
|
||||
private boolean anyContainerInFinalState(
|
||||
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request) {
|
||||
for (SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> c : request
|
||||
.getContainersToRelease()) {
|
||||
if (rmContainerInFinalState(c.getRmContainer())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("To-release container=" + c.getRmContainer()
|
||||
+ " is in final state");
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
for (ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode> c : request
|
||||
.getContainersToAllocate()) {
|
||||
for (SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> r : c
|
||||
.getToRelease()) {
|
||||
if (rmContainerInFinalState(r.getRmContainer())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("To-release container=" + r.getRmContainer()
|
||||
+ ", for to a new allocated container, is in final state");
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
if (null != c.getAllocateFromReservedContainer()) {
|
||||
if (rmContainerInFinalState(
|
||||
c.getAllocateFromReservedContainer().getRmContainer())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Allocate from reserved container" + c
|
||||
.getAllocateFromReservedContainer().getRmContainer()
|
||||
+ " is in final state");
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for (ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode> c : request
|
||||
.getContainersToReserve()) {
|
||||
for (SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> r : c
|
||||
.getToRelease()) {
|
||||
if (rmContainerInFinalState(r.getRmContainer())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("To-release container=" + r.getRmContainer()
|
||||
+ ", for a reserved container, is in final state");
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
private SchedContainerChangeRequest getResourceChangeRequest(
|
||||
SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> schedulerContainer) {
|
||||
return appSchedulingInfo.getIncreaseRequest(
|
||||
schedulerContainer.getSchedulerNode().getNodeID(),
|
||||
schedulerContainer.getSchedulerRequestKey(),
|
||||
schedulerContainer.getRmContainer().getContainerId());
|
||||
}
|
||||
|
||||
private boolean checkIncreaseContainerAllocation(
|
||||
ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode> allocation,
|
||||
SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> schedulerContainer) {
|
||||
// When increase a container
|
||||
if (schedulerContainer.getRmContainer().getState()
|
||||
!= RMContainerState.RUNNING) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Trying to increase a container, but container="
|
||||
+ schedulerContainer.getRmContainer().getContainerId()
|
||||
+ " is not in running state.");
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
// Check if increase request is still valid
|
||||
SchedContainerChangeRequest increaseRequest = getResourceChangeRequest(
|
||||
schedulerContainer);
|
||||
|
||||
if (null == increaseRequest || !Resources.equals(
|
||||
increaseRequest.getDeltaCapacity(),
|
||||
allocation.getAllocatedOrReservedResource())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Increase request has been changed, reject this proposal");
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
if (allocation.getAllocateFromReservedContainer() != null) {
|
||||
// In addition, if allocation is from a reserved container, check
|
||||
// if the reserved container has enough reserved space
|
||||
if (!Resources.equals(
|
||||
allocation.getAllocateFromReservedContainer().getRmContainer()
|
||||
.getReservedResource(), increaseRequest.getDeltaCapacity())) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
private boolean commonCheckContainerAllocation(
|
||||
Resource cluster,
|
||||
ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode> allocation,
|
||||
SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> schedulerContainer) {
|
||||
// Make sure node is not reserved by anyone else
|
||||
RMContainer reservedContainerOnNode =
|
||||
schedulerContainer.getSchedulerNode().getReservedContainer();
|
||||
if (reservedContainerOnNode != null) {
|
||||
RMContainer fromReservedContainer =
|
||||
allocation.getAllocateFromReservedContainer().getRmContainer();
|
||||
|
||||
if (fromReservedContainer != reservedContainerOnNode) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(
|
||||
"Try to allocate from a non-existed reserved container");
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
// Do we have enough space on this node?
|
||||
Resource availableResource = Resources.clone(
|
||||
schedulerContainer.getSchedulerNode().getUnallocatedResource());
|
||||
|
||||
// If we have any to-release container in non-reserved state, they are
|
||||
// from lazy-preemption, add their consumption to available resource
|
||||
// of this node
|
||||
if (allocation.getToRelease() != null && !allocation.getToRelease()
|
||||
.isEmpty()) {
|
||||
for (SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode>
|
||||
releaseContainer : allocation.getToRelease()) {
|
||||
// Only consider non-reserved container (reserved container will
|
||||
// not affect available resource of node) on the same node
|
||||
if (releaseContainer.getRmContainer().getState()
|
||||
!= RMContainerState.RESERVED
|
||||
&& releaseContainer.getSchedulerNode() == schedulerContainer
|
||||
.getSchedulerNode()) {
|
||||
Resources.addTo(availableResource,
|
||||
releaseContainer.getRmContainer().getAllocatedResource());
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!Resources.fitsIn(rc, cluster,
|
||||
allocation.getAllocatedOrReservedResource(),
|
||||
availableResource)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Node doesn't have enough available resource, asked="
|
||||
+ allocation.getAllocatedOrReservedResource() + " available="
|
||||
+ availableResource);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
public boolean accept(Resource cluster,
|
||||
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request) {
|
||||
List<ResourceRequest> resourceRequests = null;
|
||||
boolean reReservation = false;
|
||||
|
||||
try {
|
||||
readLock.lock();
|
||||
|
||||
// First make sure no container in release list in final state
|
||||
if (anyContainerInFinalState(request)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
// TODO, make sure all scheduler nodes are existed
|
||||
// TODO, make sure all node labels are not changed
|
||||
|
||||
if (request.anythingAllocatedOrReserved()) {
|
||||
/*
|
||||
* 1) If this is a newly allocated container, check if the node is reserved
|
||||
* / not-reserved by any other application
|
||||
* 2) If this is a newly reserved container, check if the node is reserved or not
|
||||
*/
|
||||
// Assume we have only one container allocated or reserved
|
||||
ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode>
|
||||
allocation = request.getFirstAllocatedOrReservedContainer();
|
||||
SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode>
|
||||
schedulerContainer = allocation.getAllocatedOrReservedContainer();
|
||||
|
||||
if (schedulerContainer.isAllocated()) {
|
||||
if (!allocation.isIncreasedAllocation()) {
|
||||
// When allocate a new container
|
||||
resourceRequests =
|
||||
schedulerContainer.getRmContainer().getResourceRequests();
|
||||
|
||||
// Check pending resource request
|
||||
if (!appSchedulingInfo.checkAllocation(allocation.getAllocationLocalityType(),
|
||||
schedulerContainer.getSchedulerNode(),
|
||||
schedulerContainer.getSchedulerRequestKey())) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("No pending resource for: nodeType=" + allocation
|
||||
.getAllocationLocalityType() + ", node=" + schedulerContainer
|
||||
.getSchedulerNode() + ", requestKey=" + schedulerContainer
|
||||
.getSchedulerRequestKey() + ", application="
|
||||
+ getApplicationAttemptId());
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
} else {
|
||||
if (!checkIncreaseContainerAllocation(allocation,
|
||||
schedulerContainer)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
// Common part of check container allocation regardless if it is a
|
||||
// increase container or regular container
|
||||
commonCheckContainerAllocation(cluster, allocation,
|
||||
schedulerContainer);
|
||||
} else {
|
||||
// Container reserved first time will be NEW, after the container
|
||||
// accepted & confirmed, it will become RESERVED state
|
||||
if (schedulerContainer.getRmContainer().getState()
|
||||
== RMContainerState.RESERVED) {
|
||||
// Set reReservation == true
|
||||
reReservation = true;
|
||||
} else {
|
||||
// When reserve a resource (state == NEW is for new container,
|
||||
// state == RUNNING is for increase container).
|
||||
// Just check if the node is not already reserved by someone
|
||||
if (schedulerContainer.getSchedulerNode().getReservedContainer()
|
||||
!= null) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Try to reserve a container, but the node is "
|
||||
+ "already reserved by another container="
|
||||
+ schedulerContainer.getSchedulerNode()
|
||||
.getReservedContainer().getContainerId());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
|
||||
// Skip check parent if this is a re-reservation container
|
||||
boolean accepted = true;
|
||||
if (!reReservation) {
|
||||
// Check parent if anything allocated or reserved
|
||||
if (request.anythingAllocatedOrReserved()) {
|
||||
accepted = getCSLeafQueue().accept(cluster, request);
|
||||
}
|
||||
}
|
||||
|
||||
// When rejected, recover resource requests for this app
|
||||
if (!accepted && resourceRequests != null) {
|
||||
recoverResourceRequestsForContainer(resourceRequests);
|
||||
}
|
||||
|
||||
return accepted;
|
||||
}
|
||||
|
||||
public void apply(Resource cluster,
|
||||
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request) {
|
||||
boolean reReservation = false;
|
||||
|
||||
try {
|
||||
writeLock.lock();
|
||||
|
||||
// If we allocated something
|
||||
if (request.anythingAllocatedOrReserved()) {
|
||||
ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode>
|
||||
allocation = request.getFirstAllocatedOrReservedContainer();
|
||||
SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode>
|
||||
schedulerContainer = allocation.getAllocatedOrReservedContainer();
|
||||
RMContainer rmContainer = schedulerContainer.getRmContainer();
|
||||
|
||||
reReservation =
|
||||
(!schedulerContainer.isAllocated()) && (rmContainer.getState()
|
||||
== RMContainerState.RESERVED);
|
||||
|
||||
// Generate new containerId if it is not an allocation for increasing
|
||||
// Or re-reservation
|
||||
if (!allocation.isIncreasedAllocation()) {
|
||||
if (rmContainer.getContainer().getId() == null) {
|
||||
rmContainer.setContainerId(BuilderUtils
|
||||
.newContainerId(getApplicationAttemptId(),
|
||||
getNewContainerId()));
|
||||
}
|
||||
}
|
||||
ContainerId containerId = rmContainer.getContainerId();
|
||||
|
||||
if (schedulerContainer.isAllocated()) {
|
||||
// This allocation is from a reserved container
|
||||
// Unreserve it first
|
||||
if (allocation.getAllocateFromReservedContainer() != null) {
|
||||
RMContainer reservedContainer =
|
||||
allocation.getAllocateFromReservedContainer().getRmContainer();
|
||||
// Handling container allocation
|
||||
// Did we previously reserve containers at this 'priority'?
|
||||
unreserve(schedulerContainer.getSchedulerRequestKey(),
|
||||
schedulerContainer.getSchedulerNode(), reservedContainer);
|
||||
}
|
||||
|
||||
// Update this application for the allocated container
|
||||
if (!allocation.isIncreasedAllocation()) {
|
||||
// Allocate a new container
|
||||
newlyAllocatedContainers.add(rmContainer);
|
||||
liveContainers.put(containerId, rmContainer);
|
||||
|
||||
// Deduct pending resource requests
|
||||
List<ResourceRequest> requests = appSchedulingInfo.allocate(
|
||||
allocation.getAllocationLocalityType(), schedulerContainer.getSchedulerNode(),
|
||||
schedulerContainer.getSchedulerRequestKey(),
|
||||
schedulerContainer.getRmContainer().getContainer());
|
||||
((RMContainerImpl) rmContainer).setResourceRequests(requests);
|
||||
|
||||
attemptResourceUsage.incUsed(schedulerContainer.getNodePartition(),
|
||||
allocation.getAllocatedOrReservedResource());
|
||||
|
||||
rmContainer.handle(
|
||||
new RMContainerEvent(containerId, RMContainerEventType.START));
|
||||
|
||||
// Inform the node
|
||||
schedulerContainer.getSchedulerNode().allocateContainer(
|
||||
rmContainer);
|
||||
|
||||
// update locality statistics,
|
||||
incNumAllocatedContainers(allocation.getAllocationLocalityType(),
|
||||
allocation.getRequestLocalityType());
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("allocate: applicationAttemptId=" + containerId
|
||||
.getApplicationAttemptId() + " container=" + containerId
|
||||
+ " host=" + rmContainer.getAllocatedNode().getHost()
|
||||
+ " type=" + allocation.getAllocationLocalityType());
|
||||
}
|
||||
RMAuditLogger.logSuccess(getUser(), AuditConstants.ALLOC_CONTAINER,
|
||||
"SchedulerApp", getApplicationId(), containerId,
|
||||
allocation.getAllocatedOrReservedResource());
|
||||
} else{
|
||||
SchedContainerChangeRequest increaseRequest =
|
||||
getResourceChangeRequest(schedulerContainer);
|
||||
|
||||
// allocate resource for an increase request
|
||||
// Notify node
|
||||
schedulerContainer.getSchedulerNode().increaseContainer(
|
||||
increaseRequest.getContainerId(),
|
||||
increaseRequest.getDeltaCapacity());
|
||||
|
||||
// OK, we can allocate this increase request
|
||||
// Notify application
|
||||
increaseContainer(increaseRequest);
|
||||
}
|
||||
} else {
|
||||
if (!allocation.isIncreasedAllocation()) {
|
||||
// If the rmContainer's state is already updated to RESERVED, this is
|
||||
// a reReservation
|
||||
reserve(schedulerContainer.getSchedulerRequestKey(),
|
||||
schedulerContainer.getSchedulerNode(),
|
||||
schedulerContainer.getRmContainer(),
|
||||
schedulerContainer.getRmContainer().getContainer(),
|
||||
reReservation);
|
||||
} else{
|
||||
SchedContainerChangeRequest increaseRequest =
|
||||
getResourceChangeRequest(schedulerContainer);
|
||||
|
||||
reserveIncreasedContainer(
|
||||
schedulerContainer.getSchedulerRequestKey(),
|
||||
schedulerContainer.getSchedulerNode(),
|
||||
increaseRequest.getRMContainer(),
|
||||
increaseRequest.getDeltaCapacity());
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
|
||||
// Don't bother CS leaf queue if it is a re-reservation
|
||||
if (!reReservation) {
|
||||
getCSLeafQueue().apply(cluster, request);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean unreserve(SchedulerRequestKey schedulerKey,
|
||||
|
@ -347,9 +725,9 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
* of the resources that will be allocated to and preempted from this
|
||||
* application.
|
||||
*
|
||||
* @param resourceCalculator
|
||||
* @param clusterResource
|
||||
* @param minimumAllocation
|
||||
* @param resourceCalculator resourceCalculator
|
||||
* @param clusterResource clusterResource
|
||||
* @param minimumAllocation minimumAllocation
|
||||
* @return an allocation
|
||||
*/
|
||||
public Allocation getAllocation(ResourceCalculator resourceCalculator,
|
||||
|
@ -386,45 +764,40 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
public NodeId getNodeIdToUnreserve(
|
||||
SchedulerRequestKey schedulerKey, Resource resourceNeedUnreserve,
|
||||
ResourceCalculator rc, Resource clusterResource) {
|
||||
try {
|
||||
writeLock.lock();
|
||||
// first go around make this algorithm simple and just grab first
|
||||
// reservation that has enough resources
|
||||
Map<NodeId, RMContainer> reservedContainers = this.reservedContainers.get(
|
||||
schedulerKey);
|
||||
// first go around make this algorithm simple and just grab first
|
||||
// reservation that has enough resources
|
||||
Map<NodeId, RMContainer> reservedContainers = this.reservedContainers.get(
|
||||
schedulerKey);
|
||||
|
||||
if ((reservedContainers != null) && (!reservedContainers.isEmpty())) {
|
||||
for (Map.Entry<NodeId, RMContainer> entry : reservedContainers
|
||||
.entrySet()) {
|
||||
NodeId nodeId = entry.getKey();
|
||||
RMContainer reservedContainer = entry.getValue();
|
||||
if (reservedContainer.hasIncreaseReservation()) {
|
||||
// Currently, only regular container allocation supports continuous
|
||||
// reservation looking, we don't support canceling increase request
|
||||
// reservation when allocating regular container.
|
||||
continue;
|
||||
}
|
||||
|
||||
Resource reservedResource = reservedContainer.getReservedResource();
|
||||
|
||||
// make sure we unreserve one with at least the same amount of
|
||||
// resources, otherwise could affect capacity limits
|
||||
if (Resources.fitsIn(rc, clusterResource, resourceNeedUnreserve,
|
||||
reservedResource)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(
|
||||
"unreserving node with reservation size: " + reservedResource
|
||||
+ " in order to allocate container with size: "
|
||||
+ resourceNeedUnreserve);
|
||||
}
|
||||
return nodeId;
|
||||
if ((reservedContainers != null) && (!reservedContainers.isEmpty())) {
|
||||
for (Map.Entry<NodeId, RMContainer> entry : reservedContainers
|
||||
.entrySet()) {
|
||||
NodeId nodeId = entry.getKey();
|
||||
RMContainer reservedContainer = entry.getValue();
|
||||
if (reservedContainer.hasIncreaseReservation()) {
|
||||
// Currently, only regular container allocation supports continuous
|
||||
// reservation looking, we don't support canceling increase request
|
||||
// reservation when allocating regular container.
|
||||
continue;
|
||||
}
|
||||
|
||||
Resource reservedResource = reservedContainer.getReservedResource();
|
||||
|
||||
// make sure we unreserve one with at least the same amount of
|
||||
// resources, otherwise could affect capacity limits
|
||||
if (Resources.fitsIn(rc, clusterResource, resourceNeedUnreserve,
|
||||
reservedResource)) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(
|
||||
"unreserving node with reservation size: " + reservedResource
|
||||
+ " in order to allocate container with size: "
|
||||
+ resourceNeedUnreserve);
|
||||
}
|
||||
return nodeId;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public void setHeadroomProvider(
|
||||
|
@ -482,10 +855,11 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
return false;
|
||||
}
|
||||
|
||||
public void reserve(SchedulerRequestKey schedulerKey,
|
||||
FiCaSchedulerNode node, RMContainer rmContainer, Container container) {
|
||||
public void reserve(SchedulerRequestKey schedulerKey, FiCaSchedulerNode node,
|
||||
RMContainer rmContainer, Container container, boolean reReservation) {
|
||||
// Update reserved metrics if this is the first reservation
|
||||
if (rmContainer == null) {
|
||||
// rmContainer will be moved to reserved in the super.reserve
|
||||
if (!reReservation) {
|
||||
queue.getMetrics().reserveResource(
|
||||
getUser(), container.getResource());
|
||||
}
|
||||
|
@ -501,35 +875,39 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
public RMContainer findNodeToUnreserve(Resource clusterResource,
|
||||
FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
|
||||
Resource minimumUnreservedResource) {
|
||||
// need to unreserve some other container first
|
||||
NodeId idToUnreserve =
|
||||
getNodeIdToUnreserve(schedulerKey, minimumUnreservedResource,
|
||||
rc, clusterResource);
|
||||
if (idToUnreserve == null) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("checked to see if could unreserve for app but nothing "
|
||||
+ "reserved that matches for this app");
|
||||
try {
|
||||
readLock.lock();
|
||||
// need to unreserve some other container first
|
||||
NodeId idToUnreserve = getNodeIdToUnreserve(schedulerKey,
|
||||
minimumUnreservedResource, rc, clusterResource);
|
||||
if (idToUnreserve == null) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("checked to see if could unreserve for app but nothing "
|
||||
+ "reserved that matches for this app");
|
||||
}
|
||||
return null;
|
||||
}
|
||||
FiCaSchedulerNode nodeToUnreserve =
|
||||
((CapacityScheduler) scheduler).getNode(idToUnreserve);
|
||||
if (nodeToUnreserve == null) {
|
||||
LOG.error("node to unreserve doesn't exist, nodeid: " + idToUnreserve);
|
||||
return null;
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("unreserving for app: " + getApplicationId() + " on nodeId: "
|
||||
+ idToUnreserve
|
||||
+ " in order to replace reserved application and place it on node: "
|
||||
+ node.getNodeID() + " needing: " + minimumUnreservedResource);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
FiCaSchedulerNode nodeToUnreserve =
|
||||
((CapacityScheduler) scheduler).getNode(idToUnreserve);
|
||||
if (nodeToUnreserve == null) {
|
||||
LOG.error("node to unreserve doesn't exist, nodeid: " + idToUnreserve);
|
||||
return null;
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("unreserving for app: " + getApplicationId()
|
||||
+ " on nodeId: " + idToUnreserve
|
||||
+ " in order to replace reserved application and place it on node: "
|
||||
+ node.getNodeID() + " needing: " + minimumUnreservedResource);
|
||||
}
|
||||
|
||||
// headroom
|
||||
Resources.addTo(getHeadroom(), nodeToUnreserve
|
||||
.getReservedContainer().getReservedResource());
|
||||
// headroom
|
||||
Resources.addTo(getHeadroom(),
|
||||
nodeToUnreserve.getReservedContainer().getReservedResource());
|
||||
|
||||
return nodeToUnreserve.getReservedContainer();
|
||||
return nodeToUnreserve.getReservedContainer();
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
public LeafQueue getCSLeafQueue() {
|
||||
|
@ -537,7 +915,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
}
|
||||
|
||||
public CSAssignment assignContainers(Resource clusterResource,
|
||||
FiCaSchedulerNode node, ResourceLimits currentResourceLimits,
|
||||
PlacementSet<FiCaSchedulerNode> ps, ResourceLimits currentResourceLimits,
|
||||
SchedulingMode schedulingMode, RMContainer reservedContainer) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("pre-assignContainers for application "
|
||||
|
@ -545,13 +923,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
showRequests();
|
||||
}
|
||||
|
||||
try {
|
||||
writeLock.lock();
|
||||
return containerAllocator.assignContainers(clusterResource, node,
|
||||
schedulingMode, currentResourceLimits, reservedContainer);
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
return containerAllocator.assignContainers(clusterResource, ps,
|
||||
schedulingMode, currentResourceLimits, reservedContainer);
|
||||
}
|
||||
|
||||
public void nodePartitionUpdated(RMContainer rmContainer, String oldPartition,
|
||||
|
@ -626,13 +999,18 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
/**
|
||||
* Set the message temporarily if the reason is known for why scheduling did
|
||||
* not happen for a given node, if not message will be over written
|
||||
* @param message
|
||||
* @param message Message of app skip diagnostics
|
||||
*/
|
||||
public void updateAppSkipNodeDiagnostics(String message) {
|
||||
this.appSkipNodeDiagnostics = message;
|
||||
}
|
||||
|
||||
public void updateNodeInfoForAMDiagnostics(FiCaSchedulerNode node) {
|
||||
// FIXME, update AM diagnostics when global scheduling is enabled
|
||||
if (null == node) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (isWaitingForAMContainer()) {
|
||||
StringBuilder diagnosticMessageBldr = new StringBuilder();
|
||||
if (appSkipNodeDiagnostics != null) {
|
||||
|
@ -653,6 +1031,13 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public SchedulingPlacementSet<FiCaSchedulerNode> getSchedulingPlacementSet(
|
||||
SchedulerRequestKey schedulerRequestKey) {
|
||||
return super.getSchedulingPlacementSet(schedulerRequestKey);
|
||||
}
|
||||
|
||||
/**
|
||||
* Recalculates the per-app, percent of queue metric, specific to the
|
||||
* Capacity Scheduler.
|
||||
|
@ -690,4 +1075,29 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
|
|||
public ReentrantReadWriteLock.WriteLock getWriteLock() {
|
||||
return this.writeLock;
|
||||
}
|
||||
|
||||
public void addToBeRemovedIncreaseRequest(
|
||||
SchedContainerChangeRequest request) {
|
||||
toBeRemovedIncRequests.put(request.getContainerId(), request);
|
||||
}
|
||||
|
||||
public void removedToBeRemovedIncreaseRequests() {
|
||||
// Remove invalid in request requests
|
||||
if (!toBeRemovedIncRequests.isEmpty()) {
|
||||
try {
|
||||
writeLock.lock();
|
||||
Iterator<Map.Entry<ContainerId, SchedContainerChangeRequest>> iter =
|
||||
toBeRemovedIncRequests.entrySet().iterator();
|
||||
while (iter.hasNext()) {
|
||||
SchedContainerChangeRequest req = iter.next().getValue();
|
||||
appSchedulingInfo.removeIncreaseRequest(req.getNodeId(),
|
||||
req.getRMContainer().getAllocatedSchedulerKey(),
|
||||
req.getContainerId());
|
||||
iter.remove();
|
||||
}
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,110 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
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.ResourceRequest;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class FifoAppAttempt extends FiCaSchedulerApp {
|
||||
private static final Log LOG = LogFactory.getLog(FifoAppAttempt.class);
|
||||
|
||||
FifoAppAttempt(ApplicationAttemptId appAttemptId, String user,
|
||||
Queue queue, ActiveUsersManager activeUsersManager,
|
||||
RMContext rmContext) {
|
||||
super(appAttemptId, user, queue, activeUsersManager, rmContext);
|
||||
}
|
||||
|
||||
public RMContainer allocate(NodeType type, FiCaSchedulerNode node,
|
||||
SchedulerRequestKey schedulerKey, ResourceRequest request,
|
||||
Container container) {
|
||||
try {
|
||||
writeLock.lock();
|
||||
|
||||
if (isStopped) {
|
||||
return null;
|
||||
}
|
||||
|
||||
// Required sanity check - AM can call 'allocate' to update resource
|
||||
// request without locking the scheduler, hence we need to check
|
||||
if (getTotalRequiredResources(schedulerKey) <= 0) {
|
||||
return null;
|
||||
}
|
||||
|
||||
// Create RMContainer
|
||||
RMContainer rmContainer = new RMContainerImpl(container,
|
||||
this.getApplicationAttemptId(), node.getNodeID(),
|
||||
appSchedulingInfo.getUser(), this.rmContext,
|
||||
request.getNodeLabelExpression());
|
||||
((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
|
||||
|
||||
updateAMContainerDiagnostics(AMState.ASSIGNED, null);
|
||||
|
||||
// Add it to allContainers list.
|
||||
newlyAllocatedContainers.add(rmContainer);
|
||||
|
||||
ContainerId containerId = container.getId();
|
||||
liveContainers.put(containerId, rmContainer);
|
||||
|
||||
// Update consumption and track allocations
|
||||
List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
|
||||
type, node, schedulerKey, request, container);
|
||||
|
||||
attemptResourceUsage.incUsed(node.getPartition(),
|
||||
container.getResource());
|
||||
|
||||
// Update resource requests related to "request" and store in RMContainer
|
||||
((RMContainerImpl) rmContainer).setResourceRequests(resourceRequestList);
|
||||
|
||||
// Inform the container
|
||||
rmContainer.handle(
|
||||
new RMContainerEvent(containerId, RMContainerEventType.START));
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("allocate: applicationAttemptId=" + containerId
|
||||
.getApplicationAttemptId() + " container=" + containerId + " host="
|
||||
+ container.getNodeId().getHost() + " type=" + type);
|
||||
}
|
||||
RMAuditLogger.logSuccess(getUser(),
|
||||
RMAuditLogger.AuditConstants.ALLOC_CONTAINER, "SchedulerApp",
|
||||
getApplicationId(), containerId, container.getResource());
|
||||
|
||||
return rmContainer;
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -79,7 +79,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicat
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
|
||||
|
@ -103,7 +102,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
@Evolving
|
||||
@SuppressWarnings("unchecked")
|
||||
public class FifoScheduler extends
|
||||
AbstractYarnScheduler<FiCaSchedulerApp, FiCaSchedulerNode> implements
|
||||
AbstractYarnScheduler<FifoAppAttempt, FiCaSchedulerNode> implements
|
||||
Configurable {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(FifoScheduler.class);
|
||||
|
@ -239,7 +238,7 @@ public class FifoScheduler extends
|
|||
validateConf(conf);
|
||||
//Use ConcurrentSkipListMap because applications need to be ordered
|
||||
this.applications =
|
||||
new ConcurrentSkipListMap<ApplicationId, SchedulerApplication<FiCaSchedulerApp>>();
|
||||
new ConcurrentSkipListMap<>();
|
||||
this.minimumAllocation =
|
||||
Resources.createResource(conf.getInt(
|
||||
YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
|
||||
|
@ -328,7 +327,7 @@ public class FifoScheduler extends
|
|||
List<String> blacklistAdditions, List<String> blacklistRemovals,
|
||||
List<UpdateContainerRequest> increaseRequests,
|
||||
List<UpdateContainerRequest> decreaseRequests) {
|
||||
FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
|
||||
FifoAppAttempt application = getApplicationAttempt(applicationAttemptId);
|
||||
if (application == null) {
|
||||
LOG.error("Calling allocate on removed " +
|
||||
"or non existant application " + applicationAttemptId);
|
||||
|
@ -384,8 +383,8 @@ public class FifoScheduler extends
|
|||
@VisibleForTesting
|
||||
public synchronized void addApplication(ApplicationId applicationId,
|
||||
String queue, String user, boolean isAppRecovering) {
|
||||
SchedulerApplication<FiCaSchedulerApp> application =
|
||||
new SchedulerApplication<FiCaSchedulerApp>(DEFAULT_QUEUE, user);
|
||||
SchedulerApplication<FifoAppAttempt> application =
|
||||
new SchedulerApplication<>(DEFAULT_QUEUE, user);
|
||||
applications.put(applicationId, application);
|
||||
metrics.submitApp(user);
|
||||
LOG.info("Accepted application " + applicationId + " from user: " + user
|
||||
|
@ -405,12 +404,12 @@ public class FifoScheduler extends
|
|||
addApplicationAttempt(ApplicationAttemptId appAttemptId,
|
||||
boolean transferStateFromPreviousAttempt,
|
||||
boolean isAttemptRecovering) {
|
||||
SchedulerApplication<FiCaSchedulerApp> application =
|
||||
SchedulerApplication<FifoAppAttempt> application =
|
||||
applications.get(appAttemptId.getApplicationId());
|
||||
String user = application.getUser();
|
||||
// TODO: Fix store
|
||||
FiCaSchedulerApp schedulerApp =
|
||||
new FiCaSchedulerApp(appAttemptId, user, DEFAULT_QUEUE,
|
||||
FifoAppAttempt schedulerApp =
|
||||
new FifoAppAttempt(appAttemptId, user, DEFAULT_QUEUE,
|
||||
activeUsersManager, this.rmContext);
|
||||
|
||||
if (transferStateFromPreviousAttempt) {
|
||||
|
@ -436,7 +435,7 @@ public class FifoScheduler extends
|
|||
|
||||
private synchronized void doneApplication(ApplicationId applicationId,
|
||||
RMAppState finalState) {
|
||||
SchedulerApplication<FiCaSchedulerApp> application =
|
||||
SchedulerApplication<FifoAppAttempt> application =
|
||||
applications.get(applicationId);
|
||||
if (application == null){
|
||||
LOG.warn("Couldn't find application " + applicationId);
|
||||
|
@ -454,8 +453,8 @@ public class FifoScheduler extends
|
|||
ApplicationAttemptId applicationAttemptId,
|
||||
RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers)
|
||||
throws IOException {
|
||||
FiCaSchedulerApp attempt = getApplicationAttempt(applicationAttemptId);
|
||||
SchedulerApplication<FiCaSchedulerApp> application =
|
||||
FifoAppAttempt attempt = getApplicationAttempt(applicationAttemptId);
|
||||
SchedulerApplication<FifoAppAttempt> application =
|
||||
applications.get(applicationAttemptId.getApplicationId());
|
||||
if (application == null || attempt == null) {
|
||||
throw new IOException("Unknown application " + applicationAttemptId +
|
||||
|
@ -492,9 +491,9 @@ public class FifoScheduler extends
|
|||
" #applications=" + applications.size());
|
||||
|
||||
// Try to assign containers to applications in fifo order
|
||||
for (Map.Entry<ApplicationId, SchedulerApplication<FiCaSchedulerApp>> e : applications
|
||||
for (Map.Entry<ApplicationId, SchedulerApplication<FifoAppAttempt>> e : applications
|
||||
.entrySet()) {
|
||||
FiCaSchedulerApp application = e.getValue().getCurrentAppAttempt();
|
||||
FifoAppAttempt application = e.getValue().getCurrentAppAttempt();
|
||||
if (application == null) {
|
||||
continue;
|
||||
}
|
||||
|
@ -536,9 +535,9 @@ public class FifoScheduler extends
|
|||
|
||||
// Update the applications' headroom to correctly take into
|
||||
// account the containers assigned in this update.
|
||||
for (SchedulerApplication<FiCaSchedulerApp> application : applications.values()) {
|
||||
FiCaSchedulerApp attempt =
|
||||
(FiCaSchedulerApp) application.getCurrentAppAttempt();
|
||||
for (SchedulerApplication<FifoAppAttempt> application : applications.values()) {
|
||||
FifoAppAttempt attempt =
|
||||
(FifoAppAttempt) application.getCurrentAppAttempt();
|
||||
if (attempt == null) {
|
||||
continue;
|
||||
}
|
||||
|
@ -546,7 +545,7 @@ public class FifoScheduler extends
|
|||
}
|
||||
}
|
||||
|
||||
private int getMaxAllocatableContainers(FiCaSchedulerApp application,
|
||||
private int getMaxAllocatableContainers(FifoAppAttempt application,
|
||||
SchedulerRequestKey schedulerKey, FiCaSchedulerNode node, NodeType type) {
|
||||
int maxContainers = 0;
|
||||
|
||||
|
@ -585,7 +584,7 @@ public class FifoScheduler extends
|
|||
|
||||
|
||||
private int assignContainersOnNode(FiCaSchedulerNode node,
|
||||
FiCaSchedulerApp application, SchedulerRequestKey schedulerKey
|
||||
FifoAppAttempt application, SchedulerRequestKey schedulerKey
|
||||
) {
|
||||
// Data-local
|
||||
int nodeLocalContainers =
|
||||
|
@ -612,7 +611,7 @@ public class FifoScheduler extends
|
|||
}
|
||||
|
||||
private int assignNodeLocalContainers(FiCaSchedulerNode node,
|
||||
FiCaSchedulerApp application, SchedulerRequestKey schedulerKey) {
|
||||
FifoAppAttempt application, SchedulerRequestKey schedulerKey) {
|
||||
int assignedContainers = 0;
|
||||
ResourceRequest request =
|
||||
application.getResourceRequest(schedulerKey, node.getNodeName());
|
||||
|
@ -638,7 +637,7 @@ public class FifoScheduler extends
|
|||
}
|
||||
|
||||
private int assignRackLocalContainers(FiCaSchedulerNode node,
|
||||
FiCaSchedulerApp application, SchedulerRequestKey schedulerKey) {
|
||||
FifoAppAttempt application, SchedulerRequestKey schedulerKey) {
|
||||
int assignedContainers = 0;
|
||||
ResourceRequest request =
|
||||
application.getResourceRequest(schedulerKey, node.getRMNode()
|
||||
|
@ -664,7 +663,7 @@ public class FifoScheduler extends
|
|||
}
|
||||
|
||||
private int assignOffSwitchContainers(FiCaSchedulerNode node,
|
||||
FiCaSchedulerApp application, SchedulerRequestKey schedulerKey) {
|
||||
FifoAppAttempt application, SchedulerRequestKey schedulerKey) {
|
||||
int assignedContainers = 0;
|
||||
ResourceRequest request =
|
||||
application.getResourceRequest(schedulerKey, ResourceRequest.ANY);
|
||||
|
@ -676,7 +675,7 @@ public class FifoScheduler extends
|
|||
return assignedContainers;
|
||||
}
|
||||
|
||||
private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application,
|
||||
private int assignContainer(FiCaSchedulerNode node, FifoAppAttempt application,
|
||||
SchedulerRequestKey schedulerKey, int assignableContainers,
|
||||
ResourceRequest request, NodeType type) {
|
||||
LOG.debug("assignContainers:" +
|
||||
|
@ -710,8 +709,8 @@ public class FifoScheduler extends
|
|||
// Allocate!
|
||||
|
||||
// Inform the application
|
||||
RMContainer rmContainer =
|
||||
application.allocate(type, node, schedulerKey, request, container);
|
||||
RMContainer rmContainer = application.allocate(type, node, schedulerKey,
|
||||
request, container);
|
||||
|
||||
// Inform the node
|
||||
node.allocateContainer(rmContainer);
|
||||
|
@ -836,7 +835,7 @@ public class FifoScheduler extends
|
|||
|
||||
// Get the application for the finished container
|
||||
Container container = rmContainer.getContainer();
|
||||
FiCaSchedulerApp application =
|
||||
FifoAppAttempt application =
|
||||
getCurrentAttemptForContainer(container.getId());
|
||||
ApplicationId appId =
|
||||
container.getId().getApplicationAttemptId().getApplicationId();
|
||||
|
@ -916,7 +915,7 @@ public class FifoScheduler extends
|
|||
|
||||
@Override
|
||||
public RMContainer getRMContainer(ContainerId containerId) {
|
||||
FiCaSchedulerApp attempt = getCurrentAttemptForContainer(containerId);
|
||||
FifoAppAttempt attempt = getCurrentAttemptForContainer(containerId);
|
||||
return (attempt == null) ? null : attempt.getRMContainer(containerId);
|
||||
}
|
||||
|
||||
|
@ -937,7 +936,7 @@ public class FifoScheduler extends
|
|||
if (queueName.equals(DEFAULT_QUEUE.getQueueName())) {
|
||||
List<ApplicationAttemptId> attempts =
|
||||
new ArrayList<ApplicationAttemptId>(applications.size());
|
||||
for (SchedulerApplication<FiCaSchedulerApp> app : applications.values()) {
|
||||
for (SchedulerApplication<FifoAppAttempt> app : applications.values()) {
|
||||
attempts.add(app.getCurrentAppAttempt().getApplicationAttemptId());
|
||||
}
|
||||
return attempts;
|
||||
|
|
|
@ -0,0 +1,65 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* PlacementSet is the central place that decide the order of node to fit
|
||||
* asks by application.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Also, PlacementSet can cache results (for example, ordered list) for
|
||||
* better performance.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* PlacementSet can depend on one or more other PlacementSets.
|
||||
* </p>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
public interface PlacementSet<N extends SchedulerNode> {
|
||||
/**
|
||||
* Get all nodes for this PlacementSet
|
||||
* @return all nodes for this PlacementSet
|
||||
*/
|
||||
Map<NodeId, N> getAllNodes();
|
||||
|
||||
/**
|
||||
* Version of the PlacementSet, can help other PlacementSet with dependencies
|
||||
* deciding if update is required
|
||||
* @return version
|
||||
*/
|
||||
long getVersion();
|
||||
|
||||
/**
|
||||
* Partition of the PlacementSet.
|
||||
* @return node partition
|
||||
*/
|
||||
String getPartition();
|
||||
}
|
|
@ -0,0 +1,36 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
|
||||
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||
|
||||
public class PlacementSetUtils {
|
||||
/*
|
||||
* If the {@link PlacementSet} only has one entry, return it. otherwise
|
||||
* return null
|
||||
*/
|
||||
public static <N extends SchedulerNode> N getSingleNode(PlacementSet<N> ps) {
|
||||
N node = null;
|
||||
if (1 == ps.getAllNodes().size()) {
|
||||
node = ps.getAllNodes().values().iterator().next();
|
||||
}
|
||||
|
||||
return node;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,43 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||
|
||||
/**
|
||||
* Result of ResourceRequest update
|
||||
*/
|
||||
public class ResourceRequestUpdateResult {
|
||||
private final ResourceRequest lastAnyResourceRequest;
|
||||
private final ResourceRequest newResourceRequest;
|
||||
|
||||
public ResourceRequestUpdateResult(ResourceRequest lastAnyResourceRequest,
|
||||
ResourceRequest newResourceRequest) {
|
||||
this.lastAnyResourceRequest = lastAnyResourceRequest;
|
||||
this.newResourceRequest = newResourceRequest;
|
||||
}
|
||||
|
||||
public ResourceRequest getLastAnyResourceRequest() {
|
||||
return lastAnyResourceRequest;
|
||||
}
|
||||
|
||||
public ResourceRequest getNewResourceRequest() {
|
||||
return newResourceRequest;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
|
||||
|
||||
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* In addition to {@link PlacementSet}, this also maintains
|
||||
* pending ResourceRequests:
|
||||
* - When new ResourceRequest(s) added to scheduler, or,
|
||||
* - Or new container allocated, scheduler can notify corresponding
|
||||
* PlacementSet.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* Different set of resource requests (E.g., resource requests with the
|
||||
* same schedulerKey) can have one instance of PlacementSet, each PlacementSet
|
||||
* can have different ways to order nodes depends on requests.
|
||||
* </p>
|
||||
*/
|
||||
public interface SchedulingPlacementSet<N extends SchedulerNode>
|
||||
extends PlacementSet<N> {
|
||||
/**
|
||||
* Get iterator of preferred node depends on requirement and/or availability
|
||||
* @param clusterPlacementSet input cluster PlacementSet
|
||||
* @return iterator of preferred node
|
||||
*/
|
||||
Iterator<N> getPreferredNodeIterator(PlacementSet<N> clusterPlacementSet);
|
||||
|
||||
/**
|
||||
* Replace existing ResourceRequest by the new requests
|
||||
*
|
||||
* @param requests new ResourceRequests
|
||||
* @param recoverPreemptedRequestForAContainer if we're recovering resource
|
||||
* requests for preempted container
|
||||
* @return true if total pending resource changed
|
||||
*/
|
||||
ResourceRequestUpdateResult updateResourceRequests(
|
||||
List<ResourceRequest> requests,
|
||||
boolean recoverPreemptedRequestForAContainer);
|
||||
|
||||
/**
|
||||
* Get pending ResourceRequests by given schedulerRequestKey
|
||||
* @return Map of resourceName to ResourceRequest
|
||||
*/
|
||||
Map<String, ResourceRequest> getResourceRequests();
|
||||
|
||||
/**
|
||||
* Get ResourceRequest by given schedulerKey and resourceName
|
||||
* @param resourceName resourceName
|
||||
* @param schedulerRequestKey schedulerRequestKey
|
||||
* @return ResourceRequest
|
||||
*/
|
||||
ResourceRequest getResourceRequest(String resourceName,
|
||||
SchedulerRequestKey schedulerRequestKey);
|
||||
|
||||
/**
|
||||
* Notify container allocated.
|
||||
* @param type Type of the allocation
|
||||
* @param node Which node this container allocated on
|
||||
* @param request resource request
|
||||
* @return list of ResourceRequests deducted
|
||||
*/
|
||||
List<ResourceRequest> allocate(NodeType type, SchedulerNode node,
|
||||
ResourceRequest request);
|
||||
}
|
|
@ -0,0 +1,70 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.api.records.NodeLabel;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A simple PlacementSet which keeps an unordered map
|
||||
*/
|
||||
public class SimplePlacementSet<N extends SchedulerNode>
|
||||
implements PlacementSet<N> {
|
||||
|
||||
private Map<NodeId, N> map;
|
||||
private String partition;
|
||||
|
||||
public SimplePlacementSet(N node) {
|
||||
if (null != node) {
|
||||
// Only one node in the initial PlacementSet
|
||||
this.map = ImmutableMap.of(node.getNodeID(), node);
|
||||
this.partition = node.getPartition();
|
||||
} else {
|
||||
this.map = Collections.emptyMap();
|
||||
this.partition = NodeLabel.DEFAULT_NODE_LABEL_PARTITION;
|
||||
}
|
||||
}
|
||||
|
||||
public SimplePlacementSet(Map<NodeId, N> map, String partition) {
|
||||
this.map = map;
|
||||
this.partition = partition;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<NodeId, N> getAllNodes() {
|
||||
return map;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getVersion() {
|
||||
return 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPartition() {
|
||||
return partition;
|
||||
}
|
||||
}
|
|
@ -19,6 +19,8 @@
|
|||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||
|
@ -35,7 +37,7 @@ public abstract class AbstractComparatorOrderingPolicy<S extends SchedulableEnti
|
|||
|
||||
private static final Log LOG = LogFactory.getLog(OrderingPolicy.class);
|
||||
|
||||
protected TreeSet<S> schedulableEntities;
|
||||
protected ConcurrentSkipListSet<S> schedulableEntities;
|
||||
protected Comparator<SchedulableEntity> comparator;
|
||||
protected Map<String, S> entitiesToReorder = new HashMap<String, S>();
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
|
@ -61,7 +62,7 @@ public class FairOrderingPolicy<S extends SchedulableEntity> extends AbstractCom
|
|||
comparators
|
||||
);
|
||||
this.comparator = fairComparator;
|
||||
this.schedulableEntities = new TreeSet<S>(comparator);
|
||||
this.schedulableEntities = new ConcurrentSkipListSet<S>(comparator);
|
||||
}
|
||||
|
||||
private double getMagnitude(SchedulableEntity r) {
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||
|
||||
/**
|
||||
|
@ -32,7 +34,7 @@ public class FifoOrderingPolicy<S extends SchedulableEntity> extends AbstractCom
|
|||
comparators.add(new PriorityComparator());
|
||||
comparators.add(new FifoComparator());
|
||||
this.comparator = new CompoundComparator(comparators);
|
||||
this.schedulableEntities = new TreeSet<S>(comparator);
|
||||
this.schedulableEntities = new ConcurrentSkipListSet<S>(comparator);
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy;
|
|||
import java.util.*;
|
||||
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
|
||||
/**
|
||||
* This ordering policy is used for pending applications only.
|
||||
|
@ -46,7 +47,7 @@ public class FifoOrderingPolicyForPendingApps<S extends SchedulableEntity>
|
|||
comparators.add(new PriorityComparator());
|
||||
comparators.add(new FifoComparator());
|
||||
this.comparator = new CompoundComparator(comparators);
|
||||
this.schedulableEntities = new TreeSet<S>(comparator);
|
||||
this.schedulableEntities = new ConcurrentSkipListSet<S>(comparator);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
|
|||
import java.security.PrivilegedAction;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -167,6 +168,28 @@ public class MockRM extends ResourceManager {
|
|||
}
|
||||
}
|
||||
|
||||
private void waitForState(ApplicationId appId, EnumSet<RMAppState> finalStates)
|
||||
throws InterruptedException {
|
||||
RMApp app = getRMContext().getRMApps().get(appId);
|
||||
Assert.assertNotNull("app shouldn't be null", app);
|
||||
final int timeoutMsecs = 80 * SECOND;
|
||||
int timeWaiting = 0;
|
||||
while (!finalStates.contains(app.getState())) {
|
||||
if (timeWaiting >= timeoutMsecs) {
|
||||
break;
|
||||
}
|
||||
|
||||
LOG.info("App : " + appId + " State is : " + app.getState() +
|
||||
" Waiting for state : " + finalStates);
|
||||
Thread.sleep(WAIT_MS_PER_LOOP);
|
||||
timeWaiting += WAIT_MS_PER_LOOP;
|
||||
}
|
||||
|
||||
LOG.info("App State is : " + app.getState());
|
||||
Assert.assertTrue("App State is not correct (timeout).",
|
||||
finalStates.contains(app.getState()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait until an application has reached a specified state.
|
||||
* The timeout is 80 seconds.
|
||||
|
@ -254,7 +277,7 @@ public class MockRM extends ResourceManager {
|
|||
RMAppAttemptState finalState, int timeoutMsecs)
|
||||
throws InterruptedException {
|
||||
int timeWaiting = 0;
|
||||
while (!finalState.equals(attempt.getAppAttemptState())) {
|
||||
while (finalState != attempt.getAppAttemptState()) {
|
||||
if (timeWaiting >= timeoutMsecs) {
|
||||
break;
|
||||
}
|
||||
|
@ -267,7 +290,7 @@ public class MockRM extends ResourceManager {
|
|||
|
||||
LOG.info("Attempt State is : " + attempt.getAppAttemptState());
|
||||
Assert.assertEquals("Attempt state is not correct (timeout).", finalState,
|
||||
attempt.getState());
|
||||
attempt.getState());
|
||||
}
|
||||
|
||||
public void waitForContainerToComplete(RMAppAttempt attempt,
|
||||
|
@ -966,6 +989,26 @@ public class MockRM extends ResourceManager {
|
|||
rm.getResourceScheduler()).getApplicationAttempt(attemptId));
|
||||
}
|
||||
|
||||
public static MockAM launchAMWhenAsyncSchedulingEnabled(RMApp app, MockRM rm)
|
||||
throws Exception {
|
||||
int i = 0;
|
||||
while (app.getCurrentAppAttempt() == null) {
|
||||
if (i < 100) {
|
||||
i++;
|
||||
}
|
||||
Thread.sleep(50);
|
||||
}
|
||||
|
||||
RMAppAttempt attempt = app.getCurrentAppAttempt();
|
||||
|
||||
rm.waitForState(attempt.getAppAttemptId(),
|
||||
RMAppAttemptState.ALLOCATED);
|
||||
MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());
|
||||
rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.LAUNCHED);
|
||||
|
||||
return am;
|
||||
}
|
||||
|
||||
/**
|
||||
* NOTE: nm.nodeHeartbeat is explicitly invoked,
|
||||
* don't invoke it before calling launchAM
|
||||
|
|
|
@ -1091,7 +1091,7 @@ public class TestClientRMService {
|
|||
rmContext.getScheduler().getSchedulerAppInfo(attemptId)
|
||||
.getLiveContainers()).thenReturn(rmContainers);
|
||||
ContainerStatus cs = mock(ContainerStatus.class);
|
||||
when(containerimpl.getFinishedStatus()).thenReturn(cs);
|
||||
when(containerimpl.completed()).thenReturn(false);
|
||||
when(containerimpl.getDiagnosticsInfo()).thenReturn("N/A");
|
||||
when(containerimpl.getContainerExitStatus()).thenReturn(0);
|
||||
when(containerimpl.getContainerState()).thenReturn(ContainerState.COMPLETE);
|
||||
|
|
|
@ -238,8 +238,10 @@ public class TestSchedulerHealth {
|
|||
SchedulerHealth sh =
|
||||
((CapacityScheduler) resourceManager.getResourceScheduler())
|
||||
.getSchedulerHealth();
|
||||
Assert.assertEquals(2, sh.getAllocationCount().longValue());
|
||||
Assert.assertEquals(Resource.newInstance(3 * 1024, 2),
|
||||
// Now SchedulerHealth records last container allocated, aggregated
|
||||
// allocation account will not be changed
|
||||
Assert.assertEquals(1, sh.getAllocationCount().longValue());
|
||||
Assert.assertEquals(Resource.newInstance(1 * 1024, 1),
|
||||
sh.getResourcesAllocated());
|
||||
Assert.assertEquals(2, sh.getAggregateAllocationCount().longValue());
|
||||
Assert.assertEquals("host_0:1234", sh.getLastAllocationDetails()
|
||||
|
|
|
@ -134,6 +134,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSc
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimplePlacementSet;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FairOrderingPolicy;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
|
||||
|
@ -3453,7 +3454,7 @@ public class TestCapacityScheduler {
|
|||
scheduler.handle(new NodeRemovedSchedulerEvent(
|
||||
rm.getRMContext().getRMNodes().get(nm2.getNodeId())));
|
||||
// schedulerNode is removed, try allocate a container
|
||||
scheduler.allocateContainersToNode(node);
|
||||
scheduler.allocateContainersToNode(new SimplePlacementSet<>(node), true);
|
||||
|
||||
AppAttemptRemovedSchedulerEvent appRemovedEvent1 =
|
||||
new AppAttemptRemovedSchedulerEvent(
|
||||
|
@ -3699,4 +3700,57 @@ public class TestCapacityScheduler {
|
|||
cs.handle(addAttemptEvent1);
|
||||
return appAttemptId1;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAppAttemptLocalityStatistics() throws Exception {
|
||||
Configuration conf =
|
||||
TestUtils.getConfigurationWithMultipleQueues(new Configuration(false));
|
||||
conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
|
||||
|
||||
final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
|
||||
mgr.init(conf);
|
||||
|
||||
MockRM rm = new MockRM(conf) {
|
||||
protected RMNodeLabelsManager createNodeLabelManager() {
|
||||
return mgr;
|
||||
}
|
||||
};
|
||||
|
||||
rm.start();
|
||||
MockNM nm1 =
|
||||
new MockNM("h1:1234", 200 * GB, rm.getResourceTrackerService());
|
||||
nm1.registerNode();
|
||||
|
||||
// Launch app1 in queue=a1
|
||||
RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a");
|
||||
|
||||
// Got one offswitch request and offswitch allocation
|
||||
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
|
||||
|
||||
// am1 asks for 1 GB resource on h1/default-rack/offswitch
|
||||
am1.allocate(Arrays.asList(ResourceRequest
|
||||
.newInstance(Priority.newInstance(1), "*",
|
||||
Resources.createResource(1 * GB), 2), ResourceRequest
|
||||
.newInstance(Priority.newInstance(1), "/default-rack",
|
||||
Resources.createResource(1 * GB), 2), ResourceRequest
|
||||
.newInstance(Priority.newInstance(1), "h1",
|
||||
Resources.createResource(1 * GB), 1)), null);
|
||||
|
||||
CapacityScheduler cs = (CapacityScheduler) rm.getRMContext().getScheduler();
|
||||
|
||||
// Got one nodelocal request and nodelocal allocation
|
||||
cs.nodeUpdate(rm.getRMContext().getRMNodes().get(nm1.getNodeId()));
|
||||
|
||||
// Got one nodelocal request and racklocal allocation
|
||||
cs.nodeUpdate(rm.getRMContext().getRMNodes().get(nm1.getNodeId()));
|
||||
|
||||
RMAppAttemptMetrics attemptMetrics = rm.getRMContext().getRMApps().get(
|
||||
app1.getApplicationId()).getCurrentAppAttempt()
|
||||
.getRMAppAttemptMetrics();
|
||||
|
||||
// We should get one node-local allocation, one rack-local allocation
|
||||
// And one off-switch allocation
|
||||
Assert.assertArrayEquals(new int[][] { { 1, 0, 0 }, { 0, 1, 0 }, { 0, 0, 1 } },
|
||||
attemptMetrics.getLocalityStatistics());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,143 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
|
||||
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class TestCapacitySchedulerAsyncScheduling {
|
||||
private final int GB = 1024;
|
||||
|
||||
private YarnConfiguration conf;
|
||||
|
||||
RMNodeLabelsManager mgr;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
conf = new YarnConfiguration();
|
||||
conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
|
||||
ResourceScheduler.class);
|
||||
conf.setBoolean(
|
||||
CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_ENABLE, true);
|
||||
mgr = new NullRMNodeLabelsManager();
|
||||
mgr.init(conf);
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
public void testSingleThreadAsyncContainerAllocation() throws Exception {
|
||||
testAsyncContainerAllocation(1);
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
public void testTwoThreadsAsyncContainerAllocation() throws Exception {
|
||||
testAsyncContainerAllocation(2);
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
public void testThreeThreadsAsyncContainerAllocation() throws Exception {
|
||||
testAsyncContainerAllocation(3);
|
||||
}
|
||||
|
||||
public void testAsyncContainerAllocation(int numThreads) throws Exception {
|
||||
conf.setInt(
|
||||
CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_THREAD,
|
||||
numThreads);
|
||||
conf.setInt(CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_PREFIX
|
||||
+ ".scheduling-interval-ms", 100);
|
||||
|
||||
final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
|
||||
mgr.init(conf);
|
||||
|
||||
// inject node label manager
|
||||
MockRM rm = new MockRM(TestUtils.getConfigurationWithMultipleQueues(conf)) {
|
||||
@Override
|
||||
public RMNodeLabelsManager createNodeLabelManager() {
|
||||
return mgr;
|
||||
}
|
||||
};
|
||||
|
||||
rm.getRMContext().setNodeLabelManager(mgr);
|
||||
rm.start();
|
||||
|
||||
List<MockNM> nms = new ArrayList<>();
|
||||
// Add 10 nodes to the cluster, in the cluster we have 200 GB resource
|
||||
for (int i = 0; i < 10; i++) {
|
||||
nms.add(rm.registerNode("h-" + i + ":1234", 20 * GB));
|
||||
}
|
||||
|
||||
List<MockAM> ams = new ArrayList<>();
|
||||
// Add 3 applications to the cluster, one app in one queue
|
||||
// the i-th app ask (20 * i) containers. So in total we will have
|
||||
// 123G container allocated
|
||||
int totalAsked = 3 * GB; // 3 AMs
|
||||
|
||||
for (int i = 0; i < 3; i++) {
|
||||
RMApp rmApp = rm.submitApp(1024, "app", "user", null, false,
|
||||
Character.toString((char) (i % 34 + 97)), 1, null, null, false);
|
||||
MockAM am = MockRM.launchAMWhenAsyncSchedulingEnabled(rmApp, rm);
|
||||
am.registerAppAttempt();
|
||||
ams.add(am);
|
||||
}
|
||||
|
||||
for (int i = 0; i < 3; i++) {
|
||||
ams.get(i).allocate("*", 1024, 20 * (i + 1), new ArrayList<>());
|
||||
totalAsked += 20 * (i + 1) * GB;
|
||||
}
|
||||
|
||||
// Wait for at most 15000 ms
|
||||
int waitTime = 15000; // ms
|
||||
while (waitTime > 0) {
|
||||
if (rm.getResourceScheduler().getRootQueueMetrics().getAllocatedMB()
|
||||
== totalAsked) {
|
||||
break;
|
||||
}
|
||||
Thread.sleep(50);
|
||||
waitTime -= 50;
|
||||
}
|
||||
|
||||
Assert.assertEquals(
|
||||
rm.getResourceScheduler().getRootQueueMetrics().getAllocatedMB(),
|
||||
totalAsked);
|
||||
|
||||
// Wait for another 2 sec to make sure we will not allocate more than
|
||||
// required
|
||||
waitTime = 2000; // ms
|
||||
while (waitTime > 0) {
|
||||
Assert.assertEquals(
|
||||
rm.getResourceScheduler().getRootQueueMetrics().getAllocatedMB(),
|
||||
totalAsked);
|
||||
waitTime -= 50;
|
||||
Thread.sleep(50);
|
||||
}
|
||||
|
||||
rm.close();
|
||||
}
|
||||
}
|
|
@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||
|
@ -110,15 +111,16 @@ public class TestChildQueueOrder {
|
|||
return application;
|
||||
}
|
||||
|
||||
private void stubQueueAllocation(final CSQueue queue,
|
||||
final Resource clusterResource, final FiCaSchedulerNode node,
|
||||
private void stubQueueAllocation(final CSQueue queue,
|
||||
final Resource clusterResource, final FiCaSchedulerNode node,
|
||||
final int allocation) {
|
||||
stubQueueAllocation(queue, clusterResource, node, allocation,
|
||||
stubQueueAllocation(queue, clusterResource, node, allocation,
|
||||
NodeType.NODE_LOCAL);
|
||||
}
|
||||
|
||||
private void stubQueueAllocation(final CSQueue queue,
|
||||
final Resource clusterResource, final FiCaSchedulerNode node,
|
||||
@SuppressWarnings("unchecked")
|
||||
private void stubQueueAllocation(final CSQueue queue,
|
||||
final Resource clusterResource, final FiCaSchedulerNode node,
|
||||
final int allocation, final NodeType type) {
|
||||
|
||||
// Simulate the queue allocation
|
||||
|
@ -145,7 +147,7 @@ public class TestChildQueueOrder {
|
|||
if (allocation > 0) {
|
||||
doReturn(new CSAssignment(Resources.none(), type)).
|
||||
when(queue)
|
||||
.assignContainers(eq(clusterResource), eq(node),
|
||||
.assignContainers(eq(clusterResource), any(PlacementSet.class),
|
||||
any(ResourceLimits.class), any(SchedulingMode.class));
|
||||
|
||||
// Mock the node's resource availability
|
||||
|
@ -157,7 +159,7 @@ public class TestChildQueueOrder {
|
|||
return new CSAssignment(allocatedResource, type);
|
||||
}
|
||||
}).
|
||||
when(queue).assignContainers(eq(clusterResource), eq(node),
|
||||
when(queue).assignContainers(eq(clusterResource), any(PlacementSet.class),
|
||||
any(ResourceLimits.class), any(SchedulingMode.class));
|
||||
doNothing().when(node).releaseContainer(any(Container.class));
|
||||
}
|
||||
|
@ -214,6 +216,7 @@ public class TestChildQueueOrder {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testSortedQueues() throws Exception {
|
||||
// Setup queue configs
|
||||
setupSortedQueues(csConf);
|
||||
|
@ -418,10 +421,10 @@ public class TestChildQueueOrder {
|
|||
clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
InOrder allocationOrder = inOrder(d,b);
|
||||
allocationOrder.verify(d).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), any(ResourceLimits.class),
|
||||
any(PlacementSet.class), any(ResourceLimits.class),
|
||||
any(SchedulingMode.class));
|
||||
allocationOrder.verify(b).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), any(ResourceLimits.class),
|
||||
any(PlacementSet.class), any(ResourceLimits.class),
|
||||
any(SchedulingMode.class));
|
||||
verifyQueueMetrics(a, 3*GB, clusterResource);
|
||||
verifyQueueMetrics(b, 2*GB, clusterResource);
|
||||
|
|
|
@ -81,7 +81,7 @@ public class TestContainerAllocation {
|
|||
mgr.init(conf);
|
||||
}
|
||||
|
||||
@Test(timeout = 3000000)
|
||||
@Test(timeout = 60000)
|
||||
public void testExcessReservationThanNodeManagerCapacity() throws Exception {
|
||||
@SuppressWarnings("resource")
|
||||
MockRM rm = new MockRM(conf);
|
||||
|
@ -598,4 +598,47 @@ public class TestContainerAllocation {
|
|||
|
||||
rm1.close();
|
||||
}
|
||||
|
||||
@Test(timeout = 60000)
|
||||
public void testAssignMultipleOffswitchContainers() throws Exception {
|
||||
MockRM rm1 = new MockRM();
|
||||
|
||||
rm1.getRMContext().setNodeLabelManager(mgr);
|
||||
rm1.start();
|
||||
MockNM nm1 = rm1.registerNode("h1:1234", 80 * GB);
|
||||
|
||||
// launch an app to queue, AM container should be launched in nm1
|
||||
RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "default");
|
||||
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
|
||||
|
||||
am1.allocate("*", 1 * GB, 5, new ArrayList<ContainerId>());
|
||||
|
||||
CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
|
||||
RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
|
||||
|
||||
// Do node heartbeats once
|
||||
cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
|
||||
|
||||
FiCaSchedulerApp schedulerApp1 =
|
||||
cs.getApplicationAttempt(am1.getApplicationAttemptId());
|
||||
|
||||
// App1 will get one container allocated (plus AM container
|
||||
Assert.assertEquals(2, schedulerApp1.getLiveContainers().size());
|
||||
|
||||
// Set assign multiple off-switch containers to 3
|
||||
CapacitySchedulerConfiguration newCSConf = new CapacitySchedulerConfiguration();
|
||||
newCSConf.setInt(
|
||||
CapacitySchedulerConfiguration.OFFSWITCH_PER_HEARTBEAT_LIMIT, 3);
|
||||
|
||||
cs.reinitialize(newCSConf, rm1.getRMContext());
|
||||
|
||||
// Do node heartbeats once
|
||||
cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
|
||||
|
||||
// App1 will get 3 new container allocated (plus 2 previously allocated
|
||||
// container)
|
||||
Assert.assertEquals(5, schedulerApp1.getLiveContainers().size());
|
||||
|
||||
rm1.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -59,9 +59,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica
|
|||
.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimplePlacementSet;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestContainerResizing {
|
||||
|
@ -97,13 +100,14 @@ public class TestContainerResizing {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized void allocateContainersToNode(FiCaSchedulerNode node) {
|
||||
public CSAssignment allocateContainersToNode(
|
||||
PlacementSet<FiCaSchedulerNode> ps, boolean withNodeHeartbeat) {
|
||||
try {
|
||||
Thread.sleep(1000);
|
||||
} catch(InterruptedException e) {
|
||||
LOG.debug("Thread interrupted.");
|
||||
}
|
||||
super.allocateContainersToNode(node);
|
||||
return super.allocateContainersToNode(ps, withNodeHeartbeat);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -452,7 +456,7 @@ public class TestContainerResizing {
|
|||
ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
|
||||
sentRMContainerLaunched(rm1, containerId1);
|
||||
|
||||
// am1 asks to change its AM container from 1GB to 3GB
|
||||
// am1 asks to change its AM container from 1GB to 7GB
|
||||
am1.sendContainerResizingRequest(Arrays.asList(
|
||||
UpdateContainerRequest
|
||||
.newInstance(0, containerId1,
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -49,8 +49,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsMana
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.SchedulerContainer;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet;
|
||||
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
||||
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
|
@ -123,6 +127,27 @@ public class TestParentQueue {
|
|||
return application;
|
||||
}
|
||||
|
||||
private void applyAllocationToQueue(Resource clusterResource,
|
||||
int allocatedMem,
|
||||
CSQueue queue) {
|
||||
// Call accept & apply for queue
|
||||
ResourceCommitRequest request = mock(ResourceCommitRequest.class);
|
||||
when(request.anythingAllocatedOrReserved()).thenReturn(true);
|
||||
ContainerAllocationProposal allocation = mock(
|
||||
ContainerAllocationProposal.class);
|
||||
when(request.getTotalReleasedResource()).thenReturn(Resources.none());
|
||||
when(request.getFirstAllocatedOrReservedContainer()).thenReturn(allocation);
|
||||
SchedulerContainer scontainer = mock(SchedulerContainer.class);
|
||||
when(allocation.getAllocatedOrReservedContainer()).thenReturn(scontainer);
|
||||
when(allocation.getAllocatedOrReservedResource()).thenReturn(
|
||||
Resources.createResource(allocatedMem));
|
||||
when(scontainer.getNodePartition()).thenReturn("");
|
||||
|
||||
if (queue.accept(clusterResource, request)) {
|
||||
queue.apply(clusterResource, request);
|
||||
}
|
||||
}
|
||||
|
||||
private void stubQueueAllocation(final CSQueue queue,
|
||||
final Resource clusterResource, final FiCaSchedulerNode node,
|
||||
final int allocation) {
|
||||
|
@ -157,7 +182,7 @@ public class TestParentQueue {
|
|||
// Next call - nothing
|
||||
if (allocation > 0) {
|
||||
doReturn(new CSAssignment(Resources.none(), type)).when(queue)
|
||||
.assignContainers(eq(clusterResource), eq(node),
|
||||
.assignContainers(eq(clusterResource), any(PlacementSet.class),
|
||||
any(ResourceLimits.class), any(SchedulingMode.class));
|
||||
|
||||
// Mock the node's resource availability
|
||||
|
@ -168,7 +193,7 @@ public class TestParentQueue {
|
|||
|
||||
return new CSAssignment(allocatedResource, type);
|
||||
}
|
||||
}).when(queue).assignContainers(eq(clusterResource), eq(node),
|
||||
}).when(queue).assignContainers(eq(clusterResource), any(PlacementSet.class),
|
||||
any(ResourceLimits.class), any(SchedulingMode.class));
|
||||
}
|
||||
|
||||
|
@ -205,8 +230,8 @@ public class TestParentQueue {
|
|||
setupSingleLevelQueues(csConf);
|
||||
|
||||
Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
|
||||
CSQueue root =
|
||||
CapacityScheduler.parseQueue(csContext, csConf, null,
|
||||
CSQueue root =
|
||||
CapacityScheduler.parseQueue(csContext, csConf, null,
|
||||
CapacitySchedulerConfiguration.ROOT, queues, queues,
|
||||
TestUtils.spyHook);
|
||||
|
||||
|
@ -245,13 +270,18 @@ public class TestParentQueue {
|
|||
// Now, A should get the scheduling opportunity since A=0G/6G, B=1G/14G
|
||||
stubQueueAllocation(a, clusterResource, node_1, 2*GB);
|
||||
stubQueueAllocation(b, clusterResource, node_1, 1*GB);
|
||||
root.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
root.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
InOrder allocationOrder = inOrder(a, b);
|
||||
allocationOrder.verify(a).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
allocationOrder.verify(a).assignContainers(eq(clusterResource),
|
||||
any(PlacementSet.class), anyResourceLimits(),
|
||||
any(SchedulingMode.class));
|
||||
root.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
allocationOrder.verify(b).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
verifyQueueMetrics(a, 2*GB, clusterResource);
|
||||
verifyQueueMetrics(b, 2*GB, clusterResource);
|
||||
|
||||
|
@ -261,11 +291,13 @@ public class TestParentQueue {
|
|||
stubQueueAllocation(b, clusterResource, node_0, 2*GB);
|
||||
root.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
root.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
allocationOrder = inOrder(b, a);
|
||||
allocationOrder.verify(b).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
allocationOrder.verify(a).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
verifyQueueMetrics(a, 3*GB, clusterResource);
|
||||
verifyQueueMetrics(b, 4*GB, clusterResource);
|
||||
|
||||
|
@ -277,9 +309,9 @@ public class TestParentQueue {
|
|||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
allocationOrder = inOrder(b, a);
|
||||
allocationOrder.verify(b).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
allocationOrder.verify(a).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
verifyQueueMetrics(a, 3*GB, clusterResource);
|
||||
verifyQueueMetrics(b, 8*GB, clusterResource);
|
||||
|
||||
|
@ -287,13 +319,17 @@ public class TestParentQueue {
|
|||
// since A has 3/6G while B has 8/14G
|
||||
stubQueueAllocation(a, clusterResource, node_1, 1*GB);
|
||||
stubQueueAllocation(b, clusterResource, node_1, 1*GB);
|
||||
root.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
root.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
root.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
allocationOrder = inOrder(a, b);
|
||||
allocationOrder.verify(b).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
allocationOrder.verify(a).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
verifyQueueMetrics(a, 4*GB, clusterResource);
|
||||
verifyQueueMetrics(b, 9*GB, clusterResource);
|
||||
}
|
||||
|
@ -430,9 +466,9 @@ public class TestParentQueue {
|
|||
setupMultiLevelQueues(csConf);
|
||||
|
||||
Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
|
||||
CSQueue root =
|
||||
CapacityScheduler.parseQueue(csContext, csConf, null,
|
||||
CapacitySchedulerConfiguration.ROOT, queues, queues,
|
||||
CSQueue root =
|
||||
CapacityScheduler.parseQueue(csContext, csConf, null,
|
||||
CapacitySchedulerConfiguration.ROOT, queues, queues,
|
||||
TestUtils.spyHook);
|
||||
|
||||
// Setup some nodes
|
||||
|
@ -496,6 +532,8 @@ public class TestParentQueue {
|
|||
stubQueueAllocation(c, clusterResource, node_1, 0*GB);
|
||||
root.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
applyAllocationToQueue(clusterResource, 4*GB,
|
||||
b);
|
||||
verifyQueueMetrics(a, 0*GB, clusterResource);
|
||||
verifyQueueMetrics(b, 4*GB, clusterResource);
|
||||
verifyQueueMetrics(c, 1*GB, clusterResource);
|
||||
|
@ -506,15 +544,27 @@ public class TestParentQueue {
|
|||
stubQueueAllocation(a1, clusterResource, node_0, 1*GB);
|
||||
stubQueueAllocation(b3, clusterResource, node_0, 2*GB);
|
||||
stubQueueAllocation(c, clusterResource, node_0, 2*GB);
|
||||
|
||||
root.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
InOrder allocationOrder = inOrder(a, c, b);
|
||||
allocationOrder.verify(a).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
applyAllocationToQueue(clusterResource, 1*GB, a);
|
||||
|
||||
root.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
allocationOrder.verify(c).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
allocationOrder.verify(b).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
applyAllocationToQueue(clusterResource, 2*GB, root);
|
||||
|
||||
root.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
allocationOrder.verify(b).assignContainers(eq(clusterResource),
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
applyAllocationToQueue(clusterResource, 2*GB, b);
|
||||
verifyQueueMetrics(a, 1*GB, clusterResource);
|
||||
verifyQueueMetrics(b, 6*GB, clusterResource);
|
||||
verifyQueueMetrics(c, 3*GB, clusterResource);
|
||||
|
@ -533,17 +583,28 @@ public class TestParentQueue {
|
|||
stubQueueAllocation(b3, clusterResource, node_2, 1*GB);
|
||||
stubQueueAllocation(b1, clusterResource, node_2, 1*GB);
|
||||
stubQueueAllocation(c, clusterResource, node_2, 1*GB);
|
||||
root.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
root.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
allocationOrder = inOrder(a, a2, a1, b, c);
|
||||
allocationOrder.verify(a).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
allocationOrder.verify(a2).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
applyAllocationToQueue(clusterResource, 2*GB, a);
|
||||
|
||||
root.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
allocationOrder.verify(b).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
applyAllocationToQueue(clusterResource, 2*GB, b);
|
||||
|
||||
root.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
allocationOrder.verify(c).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
verifyQueueMetrics(a, 3*GB, clusterResource);
|
||||
verifyQueueMetrics(b, 8*GB, clusterResource);
|
||||
verifyQueueMetrics(c, 4*GB, clusterResource);
|
||||
|
@ -614,27 +675,25 @@ public class TestParentQueue {
|
|||
public void testOffSwitchScheduling() throws Exception {
|
||||
// Setup queue configs
|
||||
setupSingleLevelQueues(csConf);
|
||||
csConf.setOffSwitchPerHeartbeatLimit(2);
|
||||
|
||||
|
||||
Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
|
||||
CSQueue root =
|
||||
CapacityScheduler.parseQueue(csContext, csConf, null,
|
||||
CapacitySchedulerConfiguration.ROOT, queues, queues,
|
||||
CSQueue root =
|
||||
CapacityScheduler.parseQueue(csContext, csConf, null,
|
||||
CapacitySchedulerConfiguration.ROOT, queues, queues,
|
||||
TestUtils.spyHook);
|
||||
|
||||
// Setup some nodes
|
||||
final int memoryPerNode = 10;
|
||||
final int coresPerNode = 16;
|
||||
final int numNodes = 2;
|
||||
|
||||
FiCaSchedulerNode node_0 =
|
||||
|
||||
FiCaSchedulerNode node_0 =
|
||||
TestUtils.getMockNode("host_0", DEFAULT_RACK, 0, memoryPerNode*GB);
|
||||
FiCaSchedulerNode node_1 =
|
||||
FiCaSchedulerNode node_1 =
|
||||
TestUtils.getMockNode("host_1", DEFAULT_RACK, 0, memoryPerNode*GB);
|
||||
|
||||
final Resource clusterResource =
|
||||
Resources.createResource(numNodes * (memoryPerNode*GB),
|
||||
|
||||
final Resource clusterResource =
|
||||
Resources.createResource(numNodes * (memoryPerNode*GB),
|
||||
numNodes * coresPerNode);
|
||||
when(csContext.getNumClusterNodes()).thenReturn(numNodes);
|
||||
|
||||
|
@ -644,50 +703,46 @@ public class TestParentQueue {
|
|||
a.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
|
||||
b.getQueueResourceUsage().incPending(Resources.createResource(1 * GB));
|
||||
queues.get(CapacitySchedulerConfiguration.ROOT).getQueueResourceUsage()
|
||||
.incPending(Resources.createResource(1 * GB));
|
||||
|
||||
// Simulate returning 2 containers on node_0 before offswitch limit
|
||||
stubQueueAllocation(a, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
|
||||
stubQueueAllocation(b, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
|
||||
.incPending(Resources.createResource(1 * GB));
|
||||
|
||||
root.assignContainers(clusterResource, node_0,
|
||||
// Simulate B returning a container on node_0
|
||||
stubQueueAllocation(a, clusterResource, node_0, 0*GB, NodeType.OFF_SWITCH);
|
||||
stubQueueAllocation(b, clusterResource, node_0, 1*GB, NodeType.OFF_SWITCH);
|
||||
root.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
InOrder allocationOrder = inOrder(a, b);
|
||||
allocationOrder.verify(a, times(1)).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
allocationOrder.verify(b, times(1)).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
verifyQueueMetrics(a, 1*GB, clusterResource);
|
||||
verifyQueueMetrics(a, 0*GB, clusterResource);
|
||||
verifyQueueMetrics(b, 1*GB, clusterResource);
|
||||
|
||||
|
||||
// Now, A should get the scheduling opportunity since A=0G/6G, B=1G/14G
|
||||
// also, B gets a scheduling opportunity since A allocates RACK_LOCAL
|
||||
stubQueueAllocation(a, clusterResource, node_1, 2*GB, NodeType.RACK_LOCAL);
|
||||
stubQueueAllocation(b, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH);
|
||||
root.assignContainers(clusterResource, node_1,
|
||||
root.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
allocationOrder = inOrder(a, b);
|
||||
allocationOrder.verify(a, times(1)).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
allocationOrder.verify(b, times(1)).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
verifyQueueMetrics(a, 3*GB, clusterResource);
|
||||
InOrder allocationOrder = inOrder(a);
|
||||
allocationOrder.verify(a).assignContainers(eq(clusterResource),
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
root.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
allocationOrder = inOrder(b);
|
||||
allocationOrder.verify(b).assignContainers(eq(clusterResource),
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
verifyQueueMetrics(a, 2*GB, clusterResource);
|
||||
verifyQueueMetrics(b, 2*GB, clusterResource);
|
||||
|
||||
// Now, B should get the scheduling opportunity
|
||||
// since A has 2/6G while B has 2/14G,
|
||||
// A also gets an opportunity because offswitchlimit not reached
|
||||
|
||||
// Now, B should get the scheduling opportunity
|
||||
// since A has 2/6G while B has 2/14G,
|
||||
// However, since B returns off-switch, A won't get an opportunity
|
||||
stubQueueAllocation(a, clusterResource, node_0, 1*GB, NodeType.NODE_LOCAL);
|
||||
stubQueueAllocation(b, clusterResource, node_0, 2*GB, NodeType.OFF_SWITCH);
|
||||
|
||||
root.assignContainers(clusterResource, node_0,
|
||||
root.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
allocationOrder = inOrder(b, a);
|
||||
allocationOrder.verify(b, times(1)).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
allocationOrder.verify(a, times(1)).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
verifyQueueMetrics(a, 4*GB, clusterResource);
|
||||
allocationOrder.verify(b).assignContainers(eq(clusterResource),
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
allocationOrder.verify(a).assignContainers(eq(clusterResource),
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
verifyQueueMetrics(a, 2*GB, clusterResource);
|
||||
verifyQueueMetrics(b, 4*GB, clusterResource);
|
||||
|
||||
}
|
||||
|
@ -743,11 +798,13 @@ public class TestParentQueue {
|
|||
stubQueueAllocation(b3, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH);
|
||||
root.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
root.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
InOrder allocationOrder = inOrder(b2, b3);
|
||||
allocationOrder.verify(b2).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
allocationOrder.verify(b3).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
verifyQueueMetrics(b2, 1*GB, clusterResource);
|
||||
verifyQueueMetrics(b3, 2*GB, clusterResource);
|
||||
|
||||
|
@ -760,9 +817,9 @@ public class TestParentQueue {
|
|||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
allocationOrder = inOrder(b3, b2);
|
||||
allocationOrder.verify(b3).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
allocationOrder.verify(b2).assignContainers(eq(clusterResource),
|
||||
any(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
any(PlacementSet.class), anyResourceLimits(), any(SchedulingMode.class));
|
||||
verifyQueueMetrics(b2, 1*GB, clusterResource);
|
||||
verifyQueueMetrics(b3, 3*GB, clusterResource);
|
||||
|
||||
|
|
|
@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Mockito.doNothing;
|
||||
import static org.mockito.Mockito.doReturn;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.spy;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
@ -30,6 +32,7 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
|
@ -59,6 +62,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicat
|
|||
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerRequestKey;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
|
||||
|
@ -190,6 +194,15 @@ public class TestReservations {
|
|||
}
|
||||
|
||||
static LeafQueue stubLeafQueue(LeafQueue queue) {
|
||||
ParentQueue parent = (ParentQueue) queue.getParent();
|
||||
|
||||
if (parent != null) {
|
||||
// Stub out parent queue's accept and apply.
|
||||
doReturn(true).when(parent).accept(any(Resource.class),
|
||||
any(ResourceCommitRequest.class));
|
||||
doNothing().when(parent).apply(any(Resource.class),
|
||||
any(ResourceCommitRequest.class));
|
||||
}
|
||||
return queue;
|
||||
}
|
||||
|
||||
|
@ -239,6 +252,12 @@ public class TestReservations {
|
|||
FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, DEFAULT_RACK, 0,
|
||||
8 * GB);
|
||||
|
||||
Map<ApplicationAttemptId, FiCaSchedulerApp> apps = ImmutableMap.of(
|
||||
app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(),
|
||||
app_1);
|
||||
Map<NodeId, FiCaSchedulerNode> nodes = ImmutableMap.of(node_0.getNodeID(),
|
||||
node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2);
|
||||
|
||||
when(csContext.getNode(node_0.getNodeID())).thenReturn(node_0);
|
||||
when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1);
|
||||
when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2);
|
||||
|
@ -268,8 +287,10 @@ public class TestReservations {
|
|||
|
||||
// Start testing...
|
||||
// Only AM
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(2 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -280,8 +301,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||
|
||||
// Only 1 map - simulating reduce
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(5 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(5 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -292,8 +315,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||
|
||||
// Only 1 map to other node - simulating reduce
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(8 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -308,8 +333,10 @@ public class TestReservations {
|
|||
toSchedulerKey(priorityReduce)));
|
||||
|
||||
// try to assign reducer (5G on node 0 and should reserve)
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(13 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(5 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -325,8 +352,10 @@ public class TestReservations {
|
|||
toSchedulerKey(priorityReduce)));
|
||||
|
||||
// assign reducer to node 2
|
||||
a.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(18 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(5 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -343,8 +372,10 @@ public class TestReservations {
|
|||
|
||||
// node_1 heartbeat and unreserves from node_0 in order to allocate
|
||||
// on node_1
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(18 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(18 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -412,6 +443,12 @@ public class TestReservations {
|
|||
when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1);
|
||||
when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2);
|
||||
|
||||
Map<ApplicationAttemptId, FiCaSchedulerApp> apps = ImmutableMap.of(
|
||||
app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(),
|
||||
app_1);
|
||||
Map<NodeId, FiCaSchedulerNode> nodes = ImmutableMap.of(node_0.getNodeID(),
|
||||
node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2);
|
||||
|
||||
cs.getNodeTracker().addNode(node_0);
|
||||
cs.getNodeTracker().addNode(node_1);
|
||||
cs.getNodeTracker().addNode(node_2);
|
||||
|
@ -434,8 +471,10 @@ public class TestReservations {
|
|||
|
||||
// Start testing...
|
||||
// Only AM
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(2 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, app_1.getCurrentConsumption().getMemorySize());
|
||||
|
@ -446,8 +485,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(4 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(2 * GB, app_1.getCurrentConsumption().getMemorySize());
|
||||
|
@ -467,8 +508,10 @@ public class TestReservations {
|
|||
priorityMap, recordFactory)));
|
||||
|
||||
// add a reservation for app_0
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(12 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(2 * GB, app_1.getCurrentConsumption().getMemorySize());
|
||||
|
@ -481,8 +524,10 @@ public class TestReservations {
|
|||
|
||||
// next assignment is beyond user limit for user_0 but it should assign to
|
||||
// app_1 for user_1
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(14 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(4 * GB, app_1.getCurrentConsumption().getMemorySize());
|
||||
|
@ -544,6 +589,12 @@ public class TestReservations {
|
|||
FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, DEFAULT_RACK, 0,
|
||||
8 * GB);
|
||||
|
||||
Map<ApplicationAttemptId, FiCaSchedulerApp> apps = ImmutableMap.of(
|
||||
app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(),
|
||||
app_1);
|
||||
Map<NodeId, FiCaSchedulerNode> nodes = ImmutableMap.of(node_0.getNodeID(),
|
||||
node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2);
|
||||
|
||||
when(csContext.getNode(node_0.getNodeID())).thenReturn(node_0);
|
||||
when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1);
|
||||
when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2);
|
||||
|
@ -569,8 +620,10 @@ public class TestReservations {
|
|||
|
||||
// Start testing...
|
||||
// Only AM
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(2 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -581,8 +634,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||
|
||||
// Only 1 map - simulating reduce
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(5 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(5 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -593,8 +648,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||
|
||||
// Only 1 map to other node - simulating reduce
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(8 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -609,8 +666,10 @@ public class TestReservations {
|
|||
toSchedulerKey(priorityReduce)));
|
||||
|
||||
// try to assign reducer (5G on node 0 and should reserve)
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(13 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(5 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -626,8 +685,10 @@ public class TestReservations {
|
|||
toSchedulerKey(priorityReduce)));
|
||||
|
||||
// assign reducer to node 2
|
||||
a.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(18 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(5 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -644,8 +705,10 @@ public class TestReservations {
|
|||
|
||||
// node_1 heartbeat and won't unreserve from node_0, potentially stuck
|
||||
// if AM doesn't handle
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(18 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(5 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -706,6 +769,12 @@ public class TestReservations {
|
|||
FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0,
|
||||
8 * GB);
|
||||
|
||||
Map<ApplicationAttemptId, FiCaSchedulerApp> apps = ImmutableMap.of(
|
||||
app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(),
|
||||
app_1);
|
||||
Map<NodeId, FiCaSchedulerNode> nodes = ImmutableMap.of(node_0.getNodeID(),
|
||||
node_0, node_1.getNodeID(), node_1);
|
||||
|
||||
cs.getNodeTracker().addNode(node_0);
|
||||
cs.getNodeTracker().addNode(node_1);
|
||||
|
||||
|
@ -733,8 +802,10 @@ public class TestReservations {
|
|||
|
||||
// Start testing...
|
||||
// Only AM
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(2 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -744,8 +815,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||
|
||||
// Only 1 map - simulating reduce
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(5 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(5 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -755,8 +828,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||
|
||||
// Only 1 map to other node - simulating reduce
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(8 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -770,8 +845,10 @@ public class TestReservations {
|
|||
toSchedulerKey(priorityReduce)));
|
||||
|
||||
// try to assign reducer (5G on node 0 and should reserve)
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(13 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(5 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -786,8 +863,10 @@ public class TestReservations {
|
|||
toSchedulerKey(priorityReduce)));
|
||||
|
||||
// could allocate but told need to unreserve first
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(13 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -983,6 +1062,12 @@ public class TestReservations {
|
|||
when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1);
|
||||
when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2);
|
||||
|
||||
Map<ApplicationAttemptId, FiCaSchedulerApp> apps = ImmutableMap.of(
|
||||
app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(),
|
||||
app_1);
|
||||
Map<NodeId, FiCaSchedulerNode> nodes = ImmutableMap.of(node_0.getNodeID(),
|
||||
node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2);
|
||||
|
||||
final int numNodes = 2;
|
||||
Resource clusterResource = Resources.createResource(numNodes * (8 * GB));
|
||||
when(csContext.getNumClusterNodes()).thenReturn(numNodes);
|
||||
|
@ -1004,8 +1089,10 @@ public class TestReservations {
|
|||
|
||||
// Start testing...
|
||||
// Only AM
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(2 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1015,8 +1102,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||
|
||||
// Only 1 map - simulating reduce
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(5 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(5 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1026,8 +1115,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||
|
||||
// Only 1 map to other node - simulating reduce
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(8 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1040,8 +1131,10 @@ public class TestReservations {
|
|||
// now add in reservations and make sure it continues if config set
|
||||
// allocate to queue so that the potential new capacity is greater then
|
||||
// absoluteMaxCapacity
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(13 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(5 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1153,6 +1246,12 @@ public class TestReservations {
|
|||
FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, DEFAULT_RACK, 0,
|
||||
8 * GB);
|
||||
|
||||
Map<ApplicationAttemptId, FiCaSchedulerApp> apps = ImmutableMap.of(
|
||||
app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(),
|
||||
app_1);
|
||||
Map<NodeId, FiCaSchedulerNode> nodes = ImmutableMap.of(node_0.getNodeID(),
|
||||
node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2);
|
||||
|
||||
when(csContext.getNode(node_0.getNodeID())).thenReturn(node_0);
|
||||
when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1);
|
||||
when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2);
|
||||
|
@ -1178,8 +1277,10 @@ public class TestReservations {
|
|||
|
||||
// Start testing...
|
||||
// Only AM
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(2 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1189,8 +1290,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||
|
||||
// Only 1 map - simulating reduce
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(5 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(5 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1200,8 +1303,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_1.getAllocatedResource().getMemorySize());
|
||||
|
||||
// Only 1 map to other node - simulating reduce
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(8 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1214,8 +1319,10 @@ public class TestReservations {
|
|||
// now add in reservations and make sure it continues if config set
|
||||
// allocate to queue so that the potential new capacity is greater then
|
||||
// absoluteMaxCapacity
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(13 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(5 * GB, app_0.getCurrentReservation().getMemorySize());
|
||||
|
@ -1301,6 +1408,12 @@ public class TestReservations {
|
|||
FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, DEFAULT_RACK, 0,
|
||||
8 * GB);
|
||||
|
||||
Map<ApplicationAttemptId, FiCaSchedulerApp> apps = ImmutableMap.of(
|
||||
app_0.getApplicationAttemptId(), app_0, app_1.getApplicationAttemptId(),
|
||||
app_1);
|
||||
Map<NodeId, FiCaSchedulerNode> nodes = ImmutableMap.of(node_0.getNodeID(),
|
||||
node_0, node_1.getNodeID(), node_1, node_2.getNodeID(), node_2);
|
||||
|
||||
when(csContext.getNode(node_0.getNodeID())).thenReturn(node_0);
|
||||
when(csContext.getNode(node_1.getNodeID())).thenReturn(node_1);
|
||||
when(csContext.getNode(node_2.getNodeID())).thenReturn(node_2);
|
||||
|
@ -1330,8 +1443,10 @@ public class TestReservations {
|
|||
|
||||
// Start testing...
|
||||
// Only AM
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(2 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(2 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1342,8 +1457,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||
|
||||
// Only 1 map - simulating reduce
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(5 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(5 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1354,8 +1471,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||
|
||||
// Only 1 map to other node - simulating reduce
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_1,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(8 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1370,8 +1489,10 @@ public class TestReservations {
|
|||
// used (8G) + required (5G). It will not reserved since it has to unreserve
|
||||
// some resource. Even with continous reservation looking, we don't allow
|
||||
// unreserve resource to reserve container.
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(Resources.createResource(10 * GB)), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(Resources.createResource(10 * GB)),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(8 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1386,8 +1507,10 @@ public class TestReservations {
|
|||
// try to assign reducer (5G on node 0), but tell it's resource limits <
|
||||
// used (8G) + required (5G). It will not reserved since it has to unreserve
|
||||
// some resource. Unfortunately, there's nothing to unreserve.
|
||||
a.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(Resources.createResource(10 * GB)), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(Resources.createResource(10 * GB)),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(8 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(8 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1400,8 +1523,10 @@ public class TestReservations {
|
|||
assertEquals(0 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||
|
||||
// let it assign 5G to node_2
|
||||
a.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(13 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(0 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1413,8 +1538,10 @@ public class TestReservations {
|
|||
assertEquals(5 * GB, node_2.getAllocatedResource().getMemorySize());
|
||||
|
||||
// reserve 8G node_0
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_0,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(21 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(8 * GB, a.getMetrics().getReservedMB());
|
||||
|
@ -1428,8 +1555,10 @@ public class TestReservations {
|
|||
// try to assign (8G on node 2). No room to allocate,
|
||||
// continued to try due to having reservation above,
|
||||
// but hits queue limits so can't reserve anymore.
|
||||
a.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
||||
TestUtils.applyResourceCommitRequest(clusterResource,
|
||||
a.assignContainers(clusterResource, node_2,
|
||||
new ResourceLimits(clusterResource),
|
||||
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY), nodes, apps);
|
||||
assertEquals(21 * GB, a.getUsedResources().getMemorySize());
|
||||
assertEquals(13 * GB, app_0.getCurrentConsumption().getMemorySize());
|
||||
assertEquals(8 * GB, a.getMetrics().getReservedMB());
|
||||
|
|
|
@ -24,6 +24,8 @@ import static org.mockito.Mockito.mock;
|
|||
import static org.mockito.Mockito.spy;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -430,4 +432,28 @@ public class TestUtils {
|
|||
req.setAllocationRequestId(allocationRequestId);
|
||||
return SchedulerRequestKey.create(req);
|
||||
}
|
||||
|
||||
public static void applyResourceCommitRequest(Resource clusterResource,
|
||||
CSAssignment csAssignment,
|
||||
final Map<NodeId, FiCaSchedulerNode> nodes,
|
||||
final Map<ApplicationAttemptId, FiCaSchedulerApp> apps)
|
||||
throws IOException {
|
||||
CapacityScheduler cs = new CapacityScheduler() {
|
||||
@Override
|
||||
public FiCaSchedulerNode getNode(NodeId nodeId) {
|
||||
return nodes.get(nodeId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FiCaSchedulerApp getApplicationAttempt(
|
||||
ApplicationAttemptId applicationAttemptId) {
|
||||
return apps.get(applicationAttemptId);
|
||||
}
|
||||
};
|
||||
|
||||
cs.setResourceCalculator(new DefaultResourceCalculator());
|
||||
|
||||
cs.submitResourceCommitRequest(clusterResource,
|
||||
csAssignment);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -103,10 +103,6 @@ public class TestRMWebServicesSchedulerActivities
|
|||
verifyStateOfAllocations(allocations.getJSONObject(i),
|
||||
"finalAllocationState", "ALLOCATED");
|
||||
verifyQueueOrder(allocations.getJSONObject(i), "root-a-b-b2-b3-b1");
|
||||
} else {
|
||||
verifyStateOfAllocations(allocations.getJSONObject(i),
|
||||
"finalAllocationState", "SKIPPED");
|
||||
verifyQueueOrder(allocations.getJSONObject(i), "root-a-b");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -409,9 +405,9 @@ public class TestRMWebServicesSchedulerActivities
|
|||
verifyStateOfAllocations(allocations, "finalAllocationState",
|
||||
"ALLOCATED");
|
||||
|
||||
verifyNumberOfNodes(allocations, 6);
|
||||
verifyNumberOfNodes(allocations, 5);
|
||||
|
||||
verifyQueueOrder(json.getJSONObject("allocations"), "root-a-b-b1");
|
||||
verifyQueueOrder(json.getJSONObject("allocations"), "root-b-b1");
|
||||
}
|
||||
finally {
|
||||
rm.stop();
|
||||
|
|
Loading…
Reference in New Issue