Skip to content

Commit

Permalink
YARN-3026. Move application-specific container allocation logic from …
Browse files Browse the repository at this point in the history
…LeafQueue to FiCaSchedulerApp. Contributed by Wangda Tan
  • Loading branch information
jian-he committed Jul 24, 2015
1 parent fc42fa8 commit 83fe34a
Show file tree
Hide file tree
Showing 16 changed files with 1,048 additions and 1,046 deletions.
3 changes: 3 additions & 0 deletions hadoop-yarn-project/CHANGES.txt
Expand Up @@ -345,6 +345,9 @@ Release 2.8.0 - UNRELEASED
YARN-3844. Make hadoop-yarn-project Native code -Wall-clean (Alan Burlison
via Colin P. McCabe)

YARN-3026. Move application-specific container allocation logic from
LeafQueue to FiCaSchedulerApp. (Wangda Tan via jianhe)

OPTIMIZATIONS

YARN-3339. TestDockerContainerExecutor should pull a single image and not
Expand Down
Expand Up @@ -292,7 +292,8 @@ void setNMTokenSecretManager(NMTokenSecretManagerInRM nmTokenSecretManager) {
activeServiceContext.setNMTokenSecretManager(nmTokenSecretManager);
}

void setScheduler(ResourceScheduler scheduler) {
@VisibleForTesting
public void setScheduler(ResourceScheduler scheduler) {
activeServiceContext.setScheduler(scheduler);
}

Expand Down
Expand Up @@ -26,27 +26,40 @@
* that, it's not "extra") resource you can get.
*/
public class ResourceLimits {
volatile Resource limit;
private volatile Resource limit;

// This is special limit that goes with the RESERVE_CONT_LOOK_ALL_NODES
// config. This limit indicates how much we need to unreserve to allocate
// another container.
private volatile Resource amountNeededUnreserve;

// How much resource you can use for next allocation, if this isn't enough for
// next container allocation, you may need to consider unreserve some
// containers.
private volatile Resource headroom;

public ResourceLimits(Resource limit) {
this.amountNeededUnreserve = Resources.none();
this.limit = limit;
this(limit, Resources.none());
}

public ResourceLimits(Resource limit, Resource amountNeededUnreserve) {
this.amountNeededUnreserve = amountNeededUnreserve;
this.headroom = limit;
this.limit = limit;
}

public Resource getLimit() {
return limit;
}

public Resource getHeadroom() {
return headroom;
}

public void setHeadroom(Resource headroom) {
this.headroom = headroom;
}

public Resource getAmountNeededUnreserve() {
return amountNeededUnreserve;
}
Expand Down
Expand Up @@ -65,7 +65,7 @@ public abstract class AbstractCSQueue implements CSQueue {
volatile int numContainers;

final Resource minimumAllocation;
Resource maximumAllocation;
volatile Resource maximumAllocation;
QueueState state;
final CSQueueMetrics metrics;
protected final PrivilegedEntity queueEntity;
Expand All @@ -77,7 +77,7 @@ public abstract class AbstractCSQueue implements CSQueue {

Map<AccessType, AccessControlList> acls =
new HashMap<AccessType, AccessControlList>();
boolean reservationsContinueLooking;
volatile boolean reservationsContinueLooking;
private boolean preemptionDisabled;

// Track resource usage-by-label like used-resource/pending-resource, etc.
Expand Down Expand Up @@ -333,7 +333,7 @@ public QueueStatistics getQueueStatistics() {
}

@Private
public synchronized Resource getMaximumAllocation() {
public Resource getMaximumAllocation() {
return maximumAllocation;
}

Expand Down Expand Up @@ -448,13 +448,8 @@ private Resource getCurrentLimitResource(String nodePartition,
}

synchronized boolean canAssignToThisQueue(Resource clusterResource,
String nodePartition, ResourceLimits currentResourceLimits,
Resource nowRequired, Resource resourceCouldBeUnreserved,
String nodePartition, ResourceLimits currentResourceLimits, Resource resourceCouldBeUnreserved,
SchedulingMode schedulingMode) {
// New total resource = used + required
Resource newTotalResource =
Resources.add(queueUsage.getUsed(nodePartition), nowRequired);

// Get current limited resource:
// - When doing RESPECT_PARTITION_EXCLUSIVITY allocation, we will respect
// queues' max capacity.
Expand All @@ -470,8 +465,14 @@ synchronized boolean canAssignToThisQueue(Resource clusterResource,
getCurrentLimitResource(nodePartition, clusterResource,
currentResourceLimits, schedulingMode);

if (Resources.greaterThan(resourceCalculator, clusterResource,
newTotalResource, currentLimitResource)) {
Resource nowTotalUsed = queueUsage.getUsed(nodePartition);

// Set headroom for currentResourceLimits
currentResourceLimits.setHeadroom(Resources.subtract(currentLimitResource,
nowTotalUsed));

if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource,
nowTotalUsed, currentLimitResource)) {

// if reservation continous looking enabled, check to see if could we
// potentially use this node instead of a reserved node if the application
Expand All @@ -483,7 +484,7 @@ synchronized boolean canAssignToThisQueue(Resource clusterResource,
resourceCouldBeUnreserved, Resources.none())) {
// resource-without-reserved = used - reserved
Resource newTotalWithoutReservedResource =
Resources.subtract(newTotalResource, resourceCouldBeUnreserved);
Resources.subtract(nowTotalUsed, resourceCouldBeUnreserved);

// when total-used-without-reserved-resource < currentLimit, we still
// have chance to allocate on this node by unreserving some containers
Expand All @@ -498,8 +499,6 @@ synchronized boolean canAssignToThisQueue(Resource clusterResource,
+ newTotalWithoutReservedResource + ", maxLimitCapacity: "
+ currentLimitResource);
}
currentResourceLimits.setAmountNeededUnreserve(Resources.subtract(newTotalResource,
currentLimitResource));
return true;
}
}
Expand Down
Expand Up @@ -31,8 +31,8 @@ public class CSAssignment {

final private Resource resource;
private NodeType type;
private final RMContainer excessReservation;
private final FiCaSchedulerApp application;
private RMContainer excessReservation;
private FiCaSchedulerApp application;
private final boolean skipped;
private boolean fulfilledReservation;
private final AssignmentInformation assignmentInformation;
Expand Down Expand Up @@ -80,10 +80,18 @@ public FiCaSchedulerApp getApplication() {
return application;
}

public void setApplication(FiCaSchedulerApp application) {
this.application = application;
}

public RMContainer getExcessReservation() {
return excessReservation;
}

public void setExcessReservation(RMContainer rmContainer) {
excessReservation = rmContainer;
}

public boolean getSkipped() {
return skipped;
}
Expand Down
Expand Up @@ -25,22 +25,16 @@ public class CapacityHeadroomProvider {
LeafQueue.User user;
LeafQueue queue;
FiCaSchedulerApp application;
Resource required;
LeafQueue.QueueResourceLimitsInfo queueResourceLimitsInfo;

public CapacityHeadroomProvider(
LeafQueue.User user,
LeafQueue queue,
FiCaSchedulerApp application,
Resource required,
LeafQueue.QueueResourceLimitsInfo queueResourceLimitsInfo) {

public CapacityHeadroomProvider(LeafQueue.User user, LeafQueue queue,
FiCaSchedulerApp application,
LeafQueue.QueueResourceLimitsInfo queueResourceLimitsInfo) {

this.user = user;
this.queue = queue;
this.application = application;
this.required = required;
this.queueResourceLimitsInfo = queueResourceLimitsInfo;

}

public Resource getHeadroom() {
Expand All @@ -52,7 +46,7 @@ public Resource getHeadroom() {
clusterResource = queueResourceLimitsInfo.getClusterResource();
}
Resource headroom = queue.getHeadroom(user, queueCurrentLimit,
clusterResource, application, required);
clusterResource, application);

// Corner case to deal with applications being slightly over-limit
if (headroom.getMemory() < 0) {
Expand Down
Expand Up @@ -1178,16 +1178,6 @@ private synchronized void allocateContainersToNode(FiCaSchedulerNode node) {
updateSchedulerHealth(lastNodeUpdateTime, node, tmp);
schedulerHealth.updateSchedulerFulfilledReservationCounts(1);
}

RMContainer excessReservation = assignment.getExcessReservation();
if (excessReservation != null) {
Container container = excessReservation.getContainer();
queue.completedContainer(clusterResource, assignment.getApplication(),
node, excessReservation, SchedulerUtils
.createAbnormalContainerStatus(container.getId(),
SchedulerUtils.UNRESERVED_CONTAINER),
RMContainerEventType.RELEASED, null, true);
}
}

// Try to schedule more if there are no reservations to fulfill
Expand Down Expand Up @@ -1241,10 +1231,6 @@ private synchronized void allocateContainersToNode(FiCaSchedulerNode node) {
RMNodeLabelsManager.NO_LABEL, clusterResource)),
SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY);
updateSchedulerHealth(lastNodeUpdateTime, node, assignment);
if (Resources.greaterThan(calculator, clusterResource,
assignment.getResource(), Resources.none())) {
return;
}
}
} else {
LOG.info("Skipping scheduling since node "
Expand Down

0 comments on commit 83fe34a

Please sign in to comment.