YARN-5716. Add global scheduler interface definition and update CapacityScheduler to use it. Contributed by Wangda Tan

This commit is contained in:
Jian He 2016-11-07 10:14:39 -08:00
parent acd509dc57
commit de3b4aac56
49 changed files with 4246 additions and 1368 deletions

View File

@ -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>

View File

@ -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();
}

View File

@ -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,8 +487,13 @@ 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
@ -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);
}
}
}

View File

@ -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;
}
};
}
}

View File

@ -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();
public void resetSchedulingOpportunities(SchedulerRequestKey schedulerKey,
long currentTimeMs) {
lastScheduledContainer.put(schedulerKey, currentTimeMs);
schedulingOpportunities.setCount(schedulerKey, 0);
} finally {
writeLock.unlock();
}
}
@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
*/

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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);
}

View File

@ -32,7 +32,9 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.lang.StringUtils;
@ -112,7 +114,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.Alloca
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.AssignmentInformation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceAllocationCommitter;
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.event.AppAddedSchedulerEvent;
@ -128,6 +134,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourc
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.event.SchedulerEventType;
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.SimplePlacementSet;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.utils.Lock;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
@ -142,7 +151,8 @@ import com.google.common.base.Preconditions;
@SuppressWarnings("unchecked")
public class CapacityScheduler extends
AbstractYarnScheduler<FiCaSchedulerApp, FiCaSchedulerNode> implements
PreemptableResourceScheduler, CapacitySchedulerContext, Configurable {
PreemptableResourceScheduler, CapacitySchedulerContext, Configurable,
ResourceAllocationCommitter {
private static final Log LOG = LogFactory.getLog(CapacityScheduler.class);
private YarnAuthorizationProvider authorizer;
@ -155,6 +165,8 @@ public class CapacityScheduler extends
private volatile boolean isLazyPreemptionEnabled = false;
private int offswitchPerHeartbeatLimit;
static final Comparator<CSQueue> nonPartitionedQueueComparator =
new Comparator<CSQueue>() {
@Override
@ -229,7 +241,8 @@ public class CapacityScheduler extends
private boolean usePortForNodeName;
private boolean scheduleAsynchronously;
private AsyncScheduleThread asyncSchedulerThread;
private List<AsyncScheduleThread> asyncSchedulerThreads;
private ResourceCommitterService resourceCommitterService;
private RMNodeLabelsManager labelManager;
/**
@ -269,6 +282,11 @@ public class CapacityScheduler extends
return calculator;
}
@VisibleForTesting
public void setResourceCalculator(ResourceCalculator rc) {
this.calculator = rc;
}
@Override
public Comparator<CSQueue> getNonPartitionedQueueComparator() {
return nonPartitionedQueueComparator;
@ -294,7 +312,8 @@ public class CapacityScheduler extends
this.rmContext = rmContext;
}
private void initScheduler(Configuration configuration) throws
@VisibleForTesting
void initScheduler(Configuration configuration) throws
IOException {
try {
writeLock.lock();
@ -315,9 +334,23 @@ public class CapacityScheduler extends
scheduleAsynchronously = this.conf.getScheduleAynschronously();
asyncScheduleInterval = this.conf.getLong(ASYNC_SCHEDULER_INTERVAL,
DEFAULT_ASYNC_SCHEDULER_INTERVAL);
// number of threads for async scheduling
int maxAsyncSchedulingThreads = this.conf.getInt(
CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_THREAD,
1);
maxAsyncSchedulingThreads = Math.max(maxAsyncSchedulingThreads, 1);
if (scheduleAsynchronously) {
asyncSchedulerThread = new AsyncScheduleThread(this);
asyncSchedulerThreads = new ArrayList<>();
for (int i = 0; i < maxAsyncSchedulingThreads; i++) {
asyncSchedulerThreads.add(new AsyncScheduleThread(this));
}
resourceCommitterService = new ResourceCommitterService(this);
}
// Setup how many containers we can allocate for each round
offswitchPerHeartbeatLimit = this.conf.getOffSwitchPerHeartbeatLimit();
LOG.info("Initialized CapacityScheduler with " + "calculator="
+ getResourceCalculator().getClass() + ", " + "minimumAllocation=<"
@ -335,9 +368,13 @@ public class CapacityScheduler extends
writeLock.lock();
activitiesManager.start();
if (scheduleAsynchronously) {
Preconditions.checkNotNull(asyncSchedulerThread,
"asyncSchedulerThread is null");
asyncSchedulerThread.start();
Preconditions.checkNotNull(asyncSchedulerThreads,
"asyncSchedulerThreads is null");
for (Thread t : asyncSchedulerThreads) {
t.start();
}
resourceCommitterService.start();
}
} finally {
writeLock.unlock();
@ -361,9 +398,13 @@ public class CapacityScheduler extends
public void serviceStop() throws Exception {
try {
writeLock.lock();
if (scheduleAsynchronously && asyncSchedulerThread != null) {
asyncSchedulerThread.interrupt();
asyncSchedulerThread.join(THREAD_JOIN_TIMEOUT_MS);
if (scheduleAsynchronously && asyncSchedulerThreads != null) {
for (Thread t : asyncSchedulerThreads) {
t.interrupt();
t.join(THREAD_JOIN_TIMEOUT_MS);
}
resourceCommitterService.interrupt();
resourceCommitterService.join(THREAD_JOIN_TIMEOUT_MS);
}
} finally {
writeLock.unlock();
@ -393,6 +434,9 @@ public class CapacityScheduler extends
// update lazy preemption
this.isLazyPreemptionEnabled = this.conf.getLazyPreemptionEnabled();
// Setup how many containers we can allocate for each round
offswitchPerHeartbeatLimit = this.conf.getOffSwitchPerHeartbeatLimit();
} finally {
writeLock.unlock();
}
@ -413,15 +457,17 @@ public class CapacityScheduler extends
int current = 0;
Collection<FiCaSchedulerNode> nodes = cs.nodeTracker.getAllNodes();
int start = random.nextInt(nodes.size());
for (FiCaSchedulerNode node : nodes) {
if (current++ >= start) {
cs.allocateContainersToNode(node);
cs.allocateContainersToNode(node.getNodeID(), false);
}
}
// Now, just get everyone to be safe
for (FiCaSchedulerNode node : nodes) {
cs.allocateContainersToNode(node);
cs.allocateContainersToNode(node.getNodeID(), false);
}
try {
Thread.sleep(cs.getAsyncScheduleInterval());
} catch (InterruptedException e) {}
@ -440,14 +486,21 @@ public class CapacityScheduler extends
@Override
public void run() {
while (true) {
if (!runSchedules.get()) {
try {
if (!runSchedules.get() || Thread.currentThread().isInterrupted()) {
Thread.sleep(100);
} catch (InterruptedException ie) {}
} else {
// Don't run schedule if we have some pending backlogs already
if (cs.getAsyncSchedulingPendingBacklogs() > 100) {
Thread.sleep(1);
} else{
schedule(cs);
}
}
} catch (InterruptedException ie) {
// Do nothing
}
}
}
public void beginSchedule() {
@ -460,6 +513,46 @@ public class CapacityScheduler extends
}
static class ResourceCommitterService extends Thread {
private final CapacityScheduler cs;
private BlockingQueue<ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode>>
backlogs = new LinkedBlockingQueue<>();
public ResourceCommitterService(CapacityScheduler cs) {
this.cs = cs;
setDaemon(true);
}
@Override
public void run() {
while (!Thread.currentThread().isInterrupted()) {
try {
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request =
backlogs.take();
try {
cs.writeLock.lock();
cs.tryCommit(cs.getClusterResource(), request);
} finally {
cs.writeLock.unlock();
}
} catch (InterruptedException e) {
LOG.error(e);
}
}
}
public void addNewCommitRequest(
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> proposal) {
backlogs.add(proposal);
}
public int getPendingBacklogs() {
return backlogs.size();
}
}
static class QueueHook {
public CSQueue hook(CSQueue queue) {
return queue;
@ -960,6 +1053,9 @@ public class CapacityScheduler extends
private LeafQueue updateIncreaseRequests(
List<UpdateContainerRequest> increaseRequests, FiCaSchedulerApp app) {
// When application has some pending to-be-removed resource requests,
app.removedToBeRemovedIncreaseRequests();
if (null == increaseRequests || increaseRequests.isEmpty()) {
return null;
}
@ -1094,22 +1190,35 @@ public class CapacityScheduler extends
}
@Override
protected synchronized void nodeUpdate(RMNode nm) {
protected synchronized void nodeUpdate(RMNode rmNode) {
try {
readLock.lock();
setLastNodeUpdateTime(Time.now());
super.nodeUpdate(rmNode);
} finally {
readLock.unlock();
}
// Try to do scheduling
if (!scheduleAsynchronously) {
try {
writeLock.lock();
setLastNodeUpdateTime(Time.now());
super.nodeUpdate(nm);
if (!scheduleAsynchronously) {
ActivitiesLogger.NODE.startNodeUpdateRecording(activitiesManager,
nm.getNodeID());
allocateContainersToNode(getNode(nm.getNodeID()));
rmNode.getNodeID());
// reset allocation and reservation stats before we start doing any
// work
updateSchedulerHealth(lastNodeUpdateTime, rmNode.getNodeID(),
CSAssignment.NULL_ASSIGNMENT);
allocateContainersToNode(rmNode.getNodeID(), true);
ActivitiesLogger.NODE.finishNodeUpdateRecording(activitiesManager,
nm.getNodeID());
}
rmNode.getNodeID());
} finally {
writeLock.unlock();
}
}
}
/**
* Process resource update on a node.
@ -1174,10 +1283,8 @@ public class CapacityScheduler extends
node.updateLabels(newLabels);
}
private void updateSchedulerHealth(long now, FiCaSchedulerNode node,
private void updateSchedulerHealth(long now, NodeId nodeId,
CSAssignment assignment) {
NodeId nodeId = node.getNodeID();
List<AssignmentInformation.AssignmentDetails> allocations =
assignment.getAssignmentInformation().getAllocationDetails();
List<AssignmentInformation.AssignmentDetails> reservations =
@ -1205,59 +1312,104 @@ public class CapacityScheduler extends
.getAssignmentInformation().getReserved());
}
@VisibleForTesting
public void allocateContainersToNode(FiCaSchedulerNode node) {
try {
writeLock.lock();
if (rmContext.isWorkPreservingRecoveryEnabled() && !rmContext
.isSchedulerReadyForAllocatingContainers()) {
return;
private boolean canAllocateMore(CSAssignment assignment, int offswitchCount) {
if (null != assignment && Resources.greaterThan(getResourceCalculator(),
getClusterResource(), assignment.getResource(), Resources.none())
&& offswitchCount < offswitchPerHeartbeatLimit) {
// And it should not be a reserved container
if (assignment.getAssignmentInformation().getNumReservations() == 0) {
return true;
}
}
if (!nodeTracker.exists(node.getNodeID())) {
LOG.info("Skipping scheduling as the node " + node.getNodeID()
+ " has been removed");
return;
return false;
}
// reset allocation and reservation stats before we start doing any work
updateSchedulerHealth(lastNodeUpdateTime, node,
new CSAssignment(Resources.none(), NodeType.NODE_LOCAL));
/**
* We need to make sure when doing allocation, Node should be existed
* And we will construct a {@link PlacementSet} before proceeding
*/
private void allocateContainersToNode(NodeId nodeId,
boolean withNodeHeartbeat) {
FiCaSchedulerNode node = getNode(nodeId);
if (null != node) {
int offswitchCount = 0;
PlacementSet<FiCaSchedulerNode> ps = new SimplePlacementSet<>(node);
CSAssignment assignment = allocateContainersToNode(ps, withNodeHeartbeat);
// Only check if we can allocate more container on the same node when
// scheduling is triggered by node heartbeat
if (null != assignment && withNodeHeartbeat) {
if (assignment.getType() == NodeType.OFF_SWITCH) {
offswitchCount++;
}
while (canAllocateMore(assignment, offswitchCount)) {
// Try to see if it is possible to allocate multiple container for
// the same node heartbeat
assignment = allocateContainersToNode(ps, true);
if (null != assignment
&& assignment.getType() == NodeType.OFF_SWITCH) {
offswitchCount++;
}
}
if (offswitchCount >= offswitchPerHeartbeatLimit) {
if (LOG.isDebugEnabled()) {
LOG.debug("Assigned maximum number of off-switch containers: "
+ offswitchCount + ", assignments so far: " + assignment);
}
}
}
}
}
/*
* Logics of allocate container on a single node (Old behavior)
*/
private CSAssignment allocateContainerOnSingleNode(PlacementSet<FiCaSchedulerNode> ps,
FiCaSchedulerNode node, boolean withNodeHeartbeat) {
// Backward compatible way to make sure previous behavior which allocation
// driven by node heartbeat works.
if (getNode(node.getNodeID()) != node) {
LOG.error("Trying to schedule on a removed node, please double check.");
return null;
}
CSAssignment assignment;
// Assign new containers...
// 1. Check for reserved applications
// 2. Schedule if there are no reservations
RMContainer reservedContainer = node.getReservedContainer();
if (reservedContainer != null) {
FiCaSchedulerApp reservedApplication = getCurrentAttemptForContainer(
reservedContainer.getContainerId());
// Try to fulfill the reservation
LOG.info("Trying to fulfill reservation for application "
+ reservedApplication.getApplicationId() + " on node: " + node
.getNodeID());
LOG.info(
"Trying to fulfill reservation for application " + reservedApplication
.getApplicationId() + " on node: " + node.getNodeID());
LeafQueue queue = ((LeafQueue) reservedApplication.getQueue());
assignment = queue.assignContainers(getClusterResource(), node,
assignment = queue.assignContainers(getClusterResource(), ps,
// TODO, now we only consider limits for parent for non-labeled
// resources, should consider labeled resources as well.
new ResourceLimits(labelManager
.getResourceByLabel(RMNodeLabelsManager.NO_LABEL,
getClusterResource())),
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
if (assignment.isFulfilledReservation()) {
CSAssignment tmp = new CSAssignment(
reservedContainer.getReservedResource(), assignment.getType());
Resources.addTo(assignment.getAssignmentInformation().getAllocated(),
reservedContainer.getReservedResource());
tmp.getAssignmentInformation().addAllocationDetails(
reservedContainer.getContainerId(), queue.getQueuePath());
tmp.getAssignmentInformation().incrAllocations();
updateSchedulerHealth(lastNodeUpdateTime, node, tmp);
if (withNodeHeartbeat) {
// Only update SchedulerHealth in sync scheduling, existing
// Data structure of SchedulerHealth need to be updated for
// Async mode
updateSchedulerHealth(lastNodeUpdateTime, node.getNodeID(),
assignment);
}
schedulerHealth.updateSchedulerFulfilledReservationCounts(1);
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
@ -1271,69 +1423,135 @@ public class CapacityScheduler extends
queue.getParent().getQueueName(), queue.getQueueName(),
ActivityState.ACCEPTED, ActivityDiagnosticConstant.EMPTY);
ActivitiesLogger.NODE.finishAllocatedNodeAllocation(activitiesManager,
node, reservedContainer.getContainerId(),
AllocationState.SKIPPED);
}
node, reservedContainer.getContainerId(), AllocationState.SKIPPED);
}
// Try to schedule more if there are no reservations to fulfill
if (node.getReservedContainer() == null) {
assignment.setSchedulingMode(
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
submitResourceCommitRequest(getClusterResource(), assignment);
}
// Do not schedule if there are any reservations to fulfill on the node
if (node.getReservedContainer() != null) {
if (LOG.isDebugEnabled()) {
LOG.debug("Skipping scheduling since node " + node.getNodeID()
+ " is reserved by application " + node.getReservedContainer()
.getContainerId().getApplicationAttemptId());
}
return null;
}
// First check if we can schedule
// When this time look at one node only, try schedule if the node
// has any available or killable resource
if (calculator.computeAvailableContainers(Resources
.add(node.getUnallocatedResource(),
node.getTotalKillableResources()), minimumAllocation) > 0) {
.add(node.getUnallocatedResource(), node.getTotalKillableResources()),
minimumAllocation) <= 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("This node or this node partition doesn't have available or"
+ "killable resource");
}
return null;
}
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to schedule on node: " + node.getNodeName()
+ ", available: " + node.getUnallocatedResource());
LOG.debug(
"Trying to schedule on node: " + node.getNodeName() + ", available: "
+ node.getUnallocatedResource());
}
assignment = root.assignContainers(getClusterResource(), node,
return allocateOrReserveNewContainers(ps, withNodeHeartbeat);
}
private CSAssignment allocateOrReserveNewContainers(
PlacementSet<FiCaSchedulerNode> ps, boolean withNodeHeartbeat) {
CSAssignment assignment = root.assignContainers(getClusterResource(), ps,
new ResourceLimits(labelManager
.getResourceByLabel(node.getPartition(),
getClusterResource())),
.getResourceByLabel(ps.getPartition(), getClusterResource())),
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
assignment.setSchedulingMode(SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
submitResourceCommitRequest(getClusterResource(), assignment);
if (Resources.greaterThan(calculator, getClusterResource(),
assignment.getResource(), Resources.none())) {
updateSchedulerHealth(lastNodeUpdateTime, node, assignment);
return;
if (withNodeHeartbeat) {
updateSchedulerHealth(lastNodeUpdateTime,
PlacementSetUtils.getSingleNode(ps).getNodeID(), assignment);
}
return assignment;
}
// Only do non-exclusive allocation when node has node-labels.
if (StringUtils.equals(node.getPartition(),
RMNodeLabelsManager.NO_LABEL)) {
return;
if (StringUtils.equals(ps.getPartition(), RMNodeLabelsManager.NO_LABEL)) {
return null;
}
// Only do non-exclusive allocation when the node-label supports that
try {
if (rmContext.getNodeLabelManager().isExclusiveNodeLabel(
node.getPartition())) {
return;
ps.getPartition())) {
return null;
}
} catch (IOException e) {
LOG.warn(
"Exception when trying to get exclusivity of node label=" + node
LOG.warn("Exception when trying to get exclusivity of node label=" + ps
.getPartition(), e);
return;
return null;
}
// Try to use NON_EXCLUSIVE
assignment = root.assignContainers(getClusterResource(), node,
assignment = root.assignContainers(getClusterResource(), ps,
// TODO, now we only consider limits for parent for non-labeled
// resources, should consider labeled resources as well.
new ResourceLimits(labelManager
.getResourceByLabel(RMNodeLabelsManager.NO_LABEL,
getClusterResource())),
SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY);
updateSchedulerHealth(lastNodeUpdateTime, node, assignment);
assignment.setSchedulingMode(SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY);
submitResourceCommitRequest(getClusterResource(), assignment);
return assignment;
}
} else{
LOG.info("Skipping scheduling since node " + node.getNodeID()
+ " is reserved by application " + node.getReservedContainer()
.getContainerId().getApplicationAttemptId());
/*
* New behavior, allocate containers considering multiple nodes
*/
private CSAssignment allocateContainersOnMultiNodes(
PlacementSet<FiCaSchedulerNode> ps) {
// When this time look at multiple nodes, try schedule if the
// partition has any available resource or killable resource
if (root.getQueueCapacities().getUsedCapacity(ps.getPartition()) >= 1.0f
&& preemptionManager.getKillableResource(
CapacitySchedulerConfiguration.ROOT, ps.getPartition()) == Resources
.none()) {
if (LOG.isDebugEnabled()) {
LOG.debug("This node or this node partition doesn't have available or"
+ "killable resource");
}
} finally {
writeLock.unlock();
return null;
}
return allocateOrReserveNewContainers(ps, false);
}
@VisibleForTesting
CSAssignment allocateContainersToNode(PlacementSet<FiCaSchedulerNode> ps,
boolean withNodeHeartbeat) {
if (rmContext.isWorkPreservingRecoveryEnabled() && !rmContext
.isSchedulerReadyForAllocatingContainers()) {
return null;
}
// Backward compatible way to make sure previous behavior which allocation
// driven by node heartbeat works.
FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps);
// We have two different logics to handle allocation on single node / multi
// nodes.
if (null != node) {
return allocateContainerOnSingleNode(ps, node, withNodeHeartbeat);
} else {
return allocateContainersOnMultiNodes(ps);
}
}
@ -1515,7 +1733,9 @@ public class CapacityScheduler extends
+ clusterResource);
if (scheduleAsynchronously && getNumClusterNodes() == 1) {
asyncSchedulerThread.beginSchedule();
for (AsyncScheduleThread t : asyncSchedulerThreads) {
t.beginSchedule();
}
}
} finally {
writeLock.unlock();
@ -1561,7 +1781,9 @@ public class CapacityScheduler extends
int numNodes = nodeTracker.nodeCount();
if (scheduleAsynchronously && numNodes == 0) {
asyncSchedulerThread.suspendSchedule();
for (AsyncScheduleThread t : asyncSchedulerThreads) {
t.suspendSchedule();
}
}
LOG.info(
@ -2223,4 +2445,207 @@ public class CapacityScheduler extends
public ResourceUsage getClusterResourceUsage() {
return root.getQueueResourceUsage();
}
private SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> getSchedulerContainer(
RMContainer rmContainer, boolean allocated) {
if (null == rmContainer) {
return null;
}
FiCaSchedulerApp app = getApplicationAttempt(
rmContainer.getApplicationAttemptId());
if (null == app) { return null; }
NodeId nodeId;
// Get nodeId
if (rmContainer.getState() == RMContainerState.RESERVED) {
nodeId = rmContainer.getReservedNode();
} else {
nodeId = rmContainer.getNodeId();
}
FiCaSchedulerNode node = getNode(nodeId);
if (null == node) {
return null;
}
return new SchedulerContainer<>(app, node, rmContainer,
// TODO, node partition should come from CSAssignment to avoid partition
// get updated before submitting the commit
node.getPartition(), allocated);
}
private List<SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode>>
getSchedulerContainersToRelease(
CSAssignment csAssignment) {
List<SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode>> list = null;
if (csAssignment.getContainersToKill() != null && !csAssignment
.getContainersToKill().isEmpty()) {
list = new ArrayList<>();
for (RMContainer rmContainer : csAssignment.getContainersToKill()) {
list.add(getSchedulerContainer(rmContainer, false));
}
}
if (csAssignment.getExcessReservation() != null) {
if (null == list) {
list = new ArrayList<>();
}
list.add(
getSchedulerContainer(csAssignment.getExcessReservation(), false));
}
return list;
}
@VisibleForTesting
public void submitResourceCommitRequest(Resource cluster,
CSAssignment csAssignment) {
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request =
createResourceCommitRequest(csAssignment);
if (null == request) {
return;
}
if (scheduleAsynchronously) {
// Submit to a commit thread and commit it async-ly
resourceCommitterService.addNewCommitRequest(request);
} else{
// Otherwise do it sync-ly.
tryCommit(cluster, request);
}
}
@VisibleForTesting
public ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode>
createResourceCommitRequest(CSAssignment csAssignment) {
ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode> allocated =
null;
ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode> reserved =
null;
List<SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode>> released =
null;
if (Resources.greaterThan(calculator, getClusterResource(),
csAssignment.getResource(), Resources.none())) {
// Allocated something
List<AssignmentInformation.AssignmentDetails> allocations =
csAssignment.getAssignmentInformation().getAllocationDetails();
if (!allocations.isEmpty()) {
RMContainer rmContainer = allocations.get(0).rmContainer;
allocated = new ContainerAllocationProposal<>(
getSchedulerContainer(rmContainer, true),
getSchedulerContainersToRelease(csAssignment),
getSchedulerContainer(csAssignment.getFulfilledReservedContainer(),
false), csAssignment.isIncreasedAllocation(),
csAssignment.getType(), csAssignment.getRequestLocalityType(),
csAssignment.getSchedulingMode() != null ?
csAssignment.getSchedulingMode() :
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY,
csAssignment.getResource());
}
// Reserved something
List<AssignmentInformation.AssignmentDetails> reservation =
csAssignment.getAssignmentInformation().getReservationDetails();
if (!reservation.isEmpty()) {
RMContainer rmContainer = reservation.get(0).rmContainer;
reserved = new ContainerAllocationProposal<>(
getSchedulerContainer(rmContainer, false),
getSchedulerContainersToRelease(csAssignment),
getSchedulerContainer(csAssignment.getFulfilledReservedContainer(),
false), csAssignment.isIncreasedAllocation(),
csAssignment.getType(), csAssignment.getRequestLocalityType(),
csAssignment.getSchedulingMode() != null ?
csAssignment.getSchedulingMode() :
SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY,
csAssignment.getResource());
}
}
// When we don't need to allocate/reserve anything, we can feel free to
// kill all to-release containers in the request.
if (null == allocated && null == reserved) {
released = getSchedulerContainersToRelease(csAssignment);
}
if (null != allocated || null != reserved || (null != released && !released
.isEmpty())) {
List<ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode>>
allocationsList = null;
if (allocated != null) {
allocationsList = new ArrayList<>();
allocationsList.add(allocated);
}
List<ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode>>
reservationsList = null;
if (reserved != null) {
reservationsList = new ArrayList<>();
reservationsList.add(reserved);
}
return new ResourceCommitRequest<>(allocationsList, reservationsList,
released);
}
return null;
}
@Override
public void tryCommit(Resource cluster, ResourceCommitRequest r) {
ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode> request =
(ResourceCommitRequest<FiCaSchedulerApp, FiCaSchedulerNode>) r;
ApplicationAttemptId attemptId = null;
// We need to update unconfirmed allocated resource of application when
// any container allocated.
boolean updateUnconfirmedAllocatedResource =
request.getContainersToAllocate() != null && !request
.getContainersToAllocate().isEmpty();
// find the application to accept and apply the ResourceCommitRequest
if (request.anythingAllocatedOrReserved()) {
ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode> c =
request.getFirstAllocatedOrReservedContainer();
attemptId =
c.getAllocatedOrReservedContainer().getSchedulerApplicationAttempt()
.getApplicationAttemptId();
} else {
if (!request.getContainersToRelease().isEmpty()) {
attemptId = request.getContainersToRelease().get(0)
.getSchedulerApplicationAttempt().getApplicationAttemptId();
}
}
if (LOG.isDebugEnabled()) {
LOG.debug("Try to commit allocation proposal=" + request);
}
if (attemptId != null) {
FiCaSchedulerApp app = getApplicationAttempt(attemptId);
if (app != null) {
if (app.accept(cluster, request)) {
app.apply(cluster, request);
LOG.info("Allocation proposal accepted");
} else{
LOG.info("Failed to accept allocation proposal");
}
// Update unconfirmed allocated resource.
if (updateUnconfirmedAllocatedResource) {
app.decUnconfirmedRes(request.getTotalAllocatedResource());
}
}
}
}
public int getAsyncSchedulingPendingBacklogs() {
if (scheduleAsynchronously) {
return resourceCommitterService.getPendingBacklogs();
}
return 0;
}
}

View File

@ -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;

View File

@ -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;
@ -915,54 +920,6 @@ public class LeafQueue extends AbstractCSQueue {
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:
// For leaf queue, only under-utilized queue is allowed to preempt resources from other queues
@ -971,58 +928,70 @@ 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);
if (LOG.isDebugEnabled()) {
LOG.debug(
"assignContainers: node=" + node.getNodeName() + " #applications="
+ orderingPolicy.getNumSchedulableEntities());
private CSAssignment allocateFromReservedContainer(
Resource clusterResource, PlacementSet<FiCaSchedulerNode> ps,
ResourceLimits currentResourceLimits, SchedulingMode schedulingMode) {
FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps);
if (null == node) {
return null;
}
setPreemptionAllowed(currentResourceLimits, node.getPartition());
// Check for reserved resources
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;
}
}
return null;
}
@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(node.getPartition())) {
&& !accessibleToPartition(ps.getPartition())) {
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
getParent().getQueueName(), getQueueName(), ActivityState.REJECTED,
ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION + node
ActivityDiagnosticConstant.NOT_ABLE_TO_ACCESS_PARTITION + ps
.getPartition());
return CSAssignment.NULL_ASSIGNMENT;
}
// Check if this queue need more resource, simply skip allocation if this
// queue doesn't need more resources.
if (!hasPendingResourceRequest(node.getPartition(), clusterResource,
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=" + node
.getPartition());
+ schedulingMode.name() + " node-partition=" + ps.getPartition());
}
ActivitiesLogger.QUEUE.recordQueueActivity(activitiesManager, node,
getParent().getQueueName(), getQueueName(), ActivityState.SKIPPED,
@ -1039,7 +1008,7 @@ public class LeafQueue extends AbstractCSQueue {
node.getNodeID(), SystemClock.getInstance().getTime(), application);
// Check queue max-capacity limit
if (!super.canAssignToThisQueue(clusterResource, node.getPartition(),
if (!super.canAssignToThisQueue(clusterResource, ps.getPartition(),
currentResourceLimits, application.getCurrentReservation(),
schedulingMode)) {
ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue(
@ -1052,11 +1021,11 @@ public class LeafQueue extends AbstractCSQueue {
}
Resource userLimit = computeUserLimitAndSetHeadroom(application,
clusterResource, node.getPartition(), schedulingMode);
clusterResource, ps.getPartition(), schedulingMode);
// Check user limit
if (!canAssignToUser(clusterResource, application.getUser(), userLimit,
application, node.getPartition(), currentResourceLimits)) {
application, ps.getPartition(), currentResourceLimits)) {
application.updateAMContainerDiagnostics(AMState.ACTIVATED,
"User capacity has reached its maximum limit.");
ActivitiesLogger.APP.recordRejectedAppActivityFromLeafQueue(
@ -1066,8 +1035,8 @@ public class LeafQueue extends AbstractCSQueue {
}
// Try to schedule
CSAssignment assignment = application.assignContainers(clusterResource,
node, currentResourceLimits, schedulingMode, null);
assignment = application.assignContainers(clusterResource,
ps, currentResourceLimits, schedulingMode, null);
if (LOG.isDebugEnabled()) {
LOG.debug("post-assignContainers for application " + application
@ -1078,35 +1047,11 @@ public class LeafQueue extends AbstractCSQueue {
// 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) {
@ -1134,10 +1079,160 @@ public class LeafQueue extends AbstractCSQueue {
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) {

View File

@ -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,17 +429,16 @@ public class ParentQueue extends AbstractCSQueue {
@Override
public CSAssignment assignContainers(Resource clusterResource,
FiCaSchedulerNode node, ResourceLimits resourceLimits,
PlacementSet<FiCaSchedulerNode> ps, ResourceLimits resourceLimits,
SchedulingMode schedulingMode) {
int offswitchCount = 0;
try {
writeLock.lock();
FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps);
// if our queue cannot access this node, just return
if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY
&& !accessibleToPartition(node.getPartition())) {
&& !accessibleToPartition(ps.getPartition())) {
if (LOG.isDebugEnabled()) {
LOG.debug("Skip this queue=" + getQueuePath()
+ ", because it is not able to access partition=" + node
+ ", because it is not able to access partition=" + ps
.getPartition());
}
@ -463,12 +456,12 @@ public class ParentQueue extends AbstractCSQueue {
// Check if this queue need more resource, simply skip allocation if this
// queue doesn't need more resources.
if (!super.hasPendingResourceRequest(node.getPartition(), clusterResource,
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=" + node
+ schedulingMode.name() + " node-partition=" + ps
.getPartition());
}
@ -495,7 +488,7 @@ public class ParentQueue extends AbstractCSQueue {
// 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(),
if (!super.canAssignToThisQueue(clusterResource, ps.getPartition(),
resourceLimits, Resources
.createResource(getMetrics().getReservedMB(),
getMetrics().getReservedVirtualCores()), schedulingMode)) {
@ -513,8 +506,12 @@ public class ParentQueue extends AbstractCSQueue {
// Schedule
CSAssignment assignedToChild = assignContainersToChildQueues(
clusterResource, node, resourceLimits, schedulingMode);
clusterResource, ps, resourceLimits, schedulingMode);
assignment.setType(assignedToChild.getType());
assignment.setRequestLocalityType(
assignedToChild.getRequestLocalityType());
assignment.setExcessReservation(assignedToChild.getExcessReservation());
assignment.setContainersToKill(assignedToChild.getContainersToKill());
// Done if no child-queue assigned anything
if (Resources.greaterThan(resourceCalculator, clusterResource,
@ -524,7 +521,11 @@ public class ParentQueue extends AbstractCSQueue {
getParentName(), getQueueName(), ActivityState.ACCEPTED,
ActivityDiagnosticConstant.EMPTY);
if (node.getReservedContainer() == null) {
boolean isReserved =
assignedToChild.getAssignmentInformation().getReservationDetails()
!= null && !assignedToChild.getAssignmentInformation()
.getReservationDetails().isEmpty();
if (node != null && !isReserved) {
if (rootQueue) {
ActivitiesLogger.NODE.finishAllocatedNodeAllocation(
activitiesManager, node,
@ -542,10 +543,6 @@ public class ParentQueue extends AbstractCSQueue {
}
}
// 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());
@ -571,6 +568,13 @@ public class ParentQueue extends AbstractCSQueue {
+ getAbsoluteUsedCapacity() + " used=" + queueUsage.getUsed()
+ " cluster=" + clusterResource);
if (LOG.isDebugEnabled()) {
LOG.debug(
"ParentQ=" + getQueueName() + " assignedSoFarInThisIteration="
+ assignment.getResource() + " usedCapacity="
+ getUsedCapacity() + " absoluteUsedCapacity="
+ getAbsoluteUsedCapacity());
}
} else{
assignment.setSkippedType(assignedToChild.getSkippedType());
@ -585,46 +589,33 @@ public class ParentQueue extends AbstractCSQueue {
break;
}
if (LOG.isDebugEnabled()) {
LOG.debug(
"ParentQ=" + getQueueName() + " assignedSoFarInThisIteration="
+ assignment.getResource() + " usedCapacity="
+ getUsedCapacity() + " absoluteUsedCapacity="
+ getAbsoluteUsedCapacity());
}
if (assignment.getType() == NodeType.OFF_SWITCH) {
offswitchCount++;
}
// 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);
}
}
/*
* 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;
} finally {
writeLock.unlock();
}
}
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,
@ -663,27 +654,19 @@ 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() ==
@ -772,8 +742,8 @@ public class ParentQueue extends AbstractCSQueue {
}
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);
}
}
}

View File

@ -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);
}

View File

@ -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;
}

View File

@ -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);
}
}

View File

@ -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,9 +92,13 @@ 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
.getCSLeafQueue()
@ -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) {
} else{
// 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;
}
}
}
@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;
}
@ -319,15 +293,6 @@ public class IncreaseContainerAllocator extends AbstractContainerAllocator {
}
}
// 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()
== CSAssignment.SkippedType.NONE) {

View File

@ -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
@ -214,27 +213,38 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
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);
// Sanity checks before assigning to this node
result = checkIfNodeBlackListed(node, 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);
}
}
// 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 =
@ -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,14 +681,14 @@ 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());
@ -686,18 +696,12 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
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) {
@ -711,13 +715,6 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
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);
}
}
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);

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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);
}

View File

@ -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();
}
}

View File

@ -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() + ")";
}
}

View File

@ -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,8 +764,6 @@ 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(
@ -422,9 +798,6 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
}
}
return null;
} finally {
writeLock.unlock();
}
}
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,10 +875,11 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
public RMContainer findNodeToUnreserve(Resource clusterResource,
FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
Resource minimumUnreservedResource) {
try {
readLock.lock();
// need to unreserve some other container first
NodeId idToUnreserve =
getNodeIdToUnreserve(schedulerKey, minimumUnreservedResource,
rc, clusterResource);
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 "
@ -519,17 +894,20 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
return null;
}
if (LOG.isDebugEnabled()) {
LOG.debug("unreserving for app: " + getApplicationId()
+ " on nodeId: " + idToUnreserve
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());
Resources.addTo(getHeadroom(),
nodeToUnreserve.getReservedContainer().getReservedResource());
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,
return containerAllocator.assignContainers(clusterResource, ps,
schedulingMode, currentResourceLimits, reservedContainer);
} finally {
writeLock.unlock();
}
}
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();
}
}
}
}

View File

@ -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();
}
}
}

View File

@ -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;

View File

@ -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();
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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);
}

View File

@ -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;
}
}

View File

@ -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>();

View File

@ -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) {

View File

@ -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);
}

View File

@ -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

View File

@ -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;
}
@ -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

View File

@ -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);

View File

@ -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()

View File

@ -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());
}
}

View File

@ -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();
}
}

View File

@ -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;
@ -117,6 +118,7 @@ public class TestChildQueueOrder {
NodeType.NODE_LOCAL);
}
@SuppressWarnings("unchecked")
private void stubQueueAllocation(final CSQueue queue,
final Resource clusterResource, final FiCaSchedulerNode node,
final int allocation, final NodeType type) {
@ -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);

View File

@ -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();
}
}

View File

@ -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,

View File

@ -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));
}
@ -246,12 +271,17 @@ public class TestParentQueue {
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);
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));
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);
@ -259,13 +289,15 @@ public class TestParentQueue {
// since A has 2/6G while B has 2/14G
stubQueueAllocation(a, clusterResource, node_0, 1*GB);
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);
@ -288,12 +320,16 @@ public class TestParentQueue {
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);
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);
}
@ -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));
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(FiCaSchedulerNode.class), anyResourceLimits(), any(SchedulingMode.class));
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);
@ -534,16 +584,27 @@ public class TestParentQueue {
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);
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,8 +675,6 @@ 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 =
@ -646,18 +705,12 @@ public class TestParentQueue {
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);
// 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
@ -666,28 +719,30 @@ public class TestParentQueue {
stubQueueAllocation(b, clusterResource, node_1, 1*GB, NodeType.OFF_SWITCH);
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
// 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,
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);
}
@ -741,13 +796,15 @@ public class TestParentQueue {
// also, B3 gets a scheduling opportunity since B2 allocates RACK_LOCAL
stubQueueAllocation(b2, clusterResource, node_1, 1*GB, NodeType.RACK_LOCAL);
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);

View File

@ -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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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)
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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());
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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)
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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)
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_1,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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.
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(Resources.createResource(10 * GB)), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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.
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_2,
new ResourceLimits(Resources.createResource(10 * GB)), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_0,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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.
TestUtils.applyResourceCommitRequest(clusterResource,
a.assignContainers(clusterResource, node_2,
new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
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());

View File

@ -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);
}
}

View File

@ -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();