YARN-6599. Support anti-affinity constraint via AppPlacementAllocator. (Wangda Tan via asuresh)

This commit is contained in:
Arun Suresh 2018-01-18 14:10:30 -08:00
parent 8779a35742
commit 38af237969
55 changed files with 2501 additions and 436 deletions

View File

@ -111,6 +111,7 @@
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.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.client.api.TimelineV2Client;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
@ -1751,6 +1752,7 @@ public MyFifoScheduler(RMContext rmContext) {
super();
try {
Configuration conf = new Configuration();
init(conf);
reinitialize(conf, rmContext);
} catch (IOException ie) {
LOG.info("add application failed with ", ie);
@ -1769,8 +1771,8 @@ public MyFifoScheduler(RMContext rmContext) {
@Override
public synchronized Allocation allocate(
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
List<ContainerId> release, List<String> blacklistAdditions,
List<String> blacklistRemovals,
List<SchedulingRequest> schedulingRequests, List<ContainerId> release,
List<String> blacklistAdditions, List<String> blacklistRemovals,
ContainerUpdates updateRequests) {
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
for (ResourceRequest req : ask) {
@ -1785,7 +1787,7 @@ public synchronized Allocation allocate(
lastBlacklistAdditions = blacklistAdditions;
lastBlacklistRemovals = blacklistRemovals;
Allocation allocation = super.allocate(
applicationAttemptId, askCopy, release, blacklistAdditions,
applicationAttemptId, askCopy, schedulingRequests, release, blacklistAdditions,
blacklistRemovals, updateRequests);
if (forceResourceLimit != null) {
// Test wants to force the non-default resource limit
@ -1805,6 +1807,7 @@ public ExcessReduceContainerAllocateScheduler(RMContext rmContext) {
super();
try {
Configuration conf = new Configuration();
init(conf);
reinitialize(conf, rmContext);
} catch (IOException ie) {
LOG.info("add application failed with ", ie);
@ -1815,8 +1818,8 @@ public ExcessReduceContainerAllocateScheduler(RMContext rmContext) {
@Override
public synchronized Allocation allocate(
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
List<ContainerId> release, List<String> blacklistAdditions,
List<String> blacklistRemovals,
List<SchedulingRequest> schedulingRequests, List<ContainerId> release,
List<String> blacklistAdditions, List<String> blacklistRemovals,
ContainerUpdates updateRequests) {
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
for (ResourceRequest req : ask) {
@ -1827,7 +1830,7 @@ public synchronized Allocation allocate(
}
SecurityUtil.setTokenServiceUseIp(false);
Allocation normalAlloc = super.allocate(
applicationAttemptId, askCopy, release,
applicationAttemptId, askCopy, schedulingRequests, release,
blacklistAdditions, blacklistRemovals, updateRequests);
List<Container> containers = normalAlloc.getContainers();
if(containers.size() > 0) {

View File

@ -35,6 +35,7 @@
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
@ -42,9 +43,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
@ -100,16 +99,17 @@ public void setConf(Configuration conf) {
@Override
public Allocation allocate(ApplicationAttemptId attemptId,
List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
List<String> strings, List<String> strings2,
ContainerUpdates updateRequests) {
List<ResourceRequest> resourceRequests,
List<SchedulingRequest> schedulingRequests, List<ContainerId> containerIds,
List<String> strings, List<String> strings2, ContainerUpdates updateRequests) {
if (metricsON) {
final Timer.Context context = schedulerMetrics.getSchedulerAllocateTimer()
.time();
Allocation allocation = null;
try {
allocation = super
.allocate(attemptId, resourceRequests, containerIds, strings,
.allocate(attemptId, resourceRequests, schedulingRequests,
containerIds, strings,
strings2, updateRequests);
return allocation;
} finally {
@ -123,7 +123,8 @@ public Allocation allocate(ApplicationAttemptId attemptId,
}
}
} else {
return super.allocate(attemptId, resourceRequests, containerIds, strings,
return super.allocate(attemptId, resourceRequests, schedulingRequests,
containerIds, strings,
strings2, updateRequests);
}
}

View File

@ -29,6 +29,7 @@
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
@ -39,8 +40,6 @@
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.fair.FSLeafQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
import org.apache.hadoop.yarn.sls.SLSRunner;
import org.apache.hadoop.yarn.sls.conf.SLSConfiguration;
@ -94,7 +93,8 @@ public void setConf(Configuration conf) {
@Override
public Allocation allocate(ApplicationAttemptId attemptId,
List<ResourceRequest> resourceRequests, List<ContainerId> containerIds,
List<ResourceRequest> resourceRequests,
List<SchedulingRequest> schedulingRequests, List<ContainerId> containerIds,
List<String> blacklistAdditions, List<String> blacklistRemovals,
ContainerUpdates updateRequests) {
if (metricsON) {
@ -102,7 +102,8 @@ public Allocation allocate(ApplicationAttemptId attemptId,
.time();
Allocation allocation = null;
try {
allocation = super.allocate(attemptId, resourceRequests, containerIds,
allocation = super.allocate(attemptId, resourceRequests,
schedulingRequests, containerIds,
blacklistAdditions, blacklistRemovals, updateRequests);
return allocation;
} finally {
@ -116,7 +117,8 @@ public Allocation allocate(ApplicationAttemptId attemptId,
}
}
} else {
return super.allocate(attemptId, resourceRequests, containerIds,
return super.allocate(attemptId, resourceRequests, schedulingRequests,
containerIds,
blacklistAdditions, blacklistRemovals, updateRequests);
}
}

View File

@ -650,4 +650,12 @@
<Method name="equals" />
<Bug pattern="EQ_OVERRIDING_EQUALS_NOT_SYMMETRIC" />
</Match>
<!-- Null pointer exception needs to be ignored here as Findbugs doesn't properly detect code logic -->
<Match>
<Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SingleConstraintAppPlacementAllocator" />
<Method name="validateAndSetSchedulingRequest" />
<Bug pattern="NP_NULL_ON_SOME_PATH" />
</Match>
</FindBugsFilter>

View File

@ -20,8 +20,12 @@
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.resource.PlacementConstraint.AbstractConstraint;
import org.apache.hadoop.yarn.api.resource.PlacementConstraint.And;
import org.apache.hadoop.yarn.api.resource.PlacementConstraint.DelayedOr;
@ -47,6 +51,14 @@ private PlacementConstraints() {
public static final String NODE = PlacementConstraint.NODE_SCOPE;
public static final String RACK = PlacementConstraint.RACK_SCOPE;
public static final String NODE_PARTITION = "yarn_node_partition/";
private static final String APPLICATION_LABEL_PREFIX =
"yarn_application_label/";
@InterfaceAudience.Private
public static final String APPLICATION_LABEL_INTRA_APPLICATION =
APPLICATION_LABEL_PREFIX + "%intra_app%";
/**
* Creates a constraint that requires allocations to be placed on nodes that
@ -186,6 +198,20 @@ public static TargetExpression nodeAttribute(String attributeKey,
attributeValues);
}
/**
* Constructs a target expression on a node partition. It is satisfied if
* the specified node partition has one of the specified nodePartitions
*
* @param nodePartitions the set of values that the attribute should take
* values from
* @return the resulting expression on the node attribute
*/
public static TargetExpression nodePartition(
String... nodePartitions) {
return new TargetExpression(TargetType.NODE_ATTRIBUTE, NODE_PARTITION,
nodePartitions);
}
/**
* Constructs a target expression on an allocation tag. It is satisfied if
* the there are allocations with one of the given tags.
@ -198,6 +224,22 @@ public static TargetExpression allocationTag(String... allocationTags) {
return new TargetExpression(TargetType.ALLOCATION_TAG, null,
allocationTags);
}
/**
* Constructs a target expression on an allocation tag. It is satisfied if
* the there are allocations with one of the given tags. Comparing to
* {@link PlacementTargets#allocationTag(String...)}, this only check tags
* within the application.
*
* @param allocationTags the set of tags that the attribute should take
* values from
* @return the resulting expression on the allocation tags
*/
public static TargetExpression allocationTagToIntraApp(
String... allocationTags) {
return new TargetExpression(TargetType.ALLOCATION_TAG,
APPLICATION_LABEL_INTRA_APPLICATION, allocationTags);
}
}
// Creation of compound constraints.
@ -277,5 +319,4 @@ public static TimedPlacementConstraint timedOpportunitiesConstraint(
public static PlacementConstraint build(AbstractConstraint constraintExpr) {
return constraintExpr.build();
}
}

View File

@ -543,7 +543,7 @@ public static boolean isAclEnabled(Configuration conf) {
public static final String RM_PLACEMENT_CONSTRAINTS_ENABLED =
RM_PREFIX + "placement-constraints.enabled";
public static final boolean DEFAULT_RM_PLACEMENT_CONSTRAINTS_ENABLED = true;
public static final boolean DEFAULT_RM_PLACEMENT_CONSTRAINTS_ENABLED = false;
public static final String RM_PLACEMENT_CONSTRAINTS_RETRY_ATTEMPTS =
RM_PREFIX + "placement-constraints.retry-attempts";

View File

@ -0,0 +1,47 @@
/**
* 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.exceptions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* This exception is thrown when any issue inside scheduler to handle a new or
* updated {@link org.apache.hadoop.yarn.api.records.SchedulingRequest}/
* {@link org.apache.hadoop.yarn.api.records.ResourceRequest} add to the
* scheduler.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class SchedulerInvalidResoureRequestException extends YarnRuntimeException {
private static final long serialVersionUID = 10081123982L;
public SchedulerInvalidResoureRequestException(String message) {
super(message);
}
public SchedulerInvalidResoureRequestException(Throwable cause) {
super(cause);
}
public SchedulerInvalidResoureRequestException(String message,
Throwable cause) {
super(message, cause);
}
}

View File

@ -44,6 +44,7 @@
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.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.client.api.AMRMClient;
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
@ -545,6 +546,7 @@ public MyFifoScheduler(RMContext rmContext) {
super();
try {
Configuration conf = new Configuration();
init(conf);
reinitialize(conf, rmContext);
} catch (IOException ie) {
assert (false);
@ -563,8 +565,8 @@ public MyFifoScheduler(RMContext rmContext) {
@Override
public synchronized Allocation allocate(
ApplicationAttemptId applicationAttemptId, List<ResourceRequest> ask,
List<ContainerId> release, List<String> blacklistAdditions,
List<String> blacklistRemovals,
List<SchedulingRequest> schedulingRequests, List<ContainerId> release,
List<String> blacklistAdditions, List<String> blacklistRemovals,
ContainerUpdates updateRequests) {
List<ResourceRequest> askCopy = new ArrayList<ResourceRequest>();
for (ResourceRequest req : ask) {
@ -580,7 +582,8 @@ public synchronized Allocation allocate(
lastDecrease = updateRequests.getDecreaseRequests();
lastBlacklistAdditions = blacklistAdditions;
lastBlacklistRemovals = blacklistRemovals;
return super.allocate(applicationAttemptId, askCopy, release,
return super.allocate(applicationAttemptId, askCopy, schedulingRequests,
release,
blacklistAdditions, blacklistRemovals, updateRequests);
}
}

View File

@ -194,6 +194,7 @@ public List<SchedulingRequest> getSchedulingRequests() {
public void setSchedulingRequests(
List<SchedulingRequest> schedulingRequests) {
if (schedulingRequests == null) {
builder.clearSchedulingRequests();
return;
}
initSchedulingRequests();

View File

@ -22,6 +22,7 @@
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
/**
@ -45,6 +46,16 @@ public static SchedulerRequestKey create(ResourceRequest req) {
req.getAllocationRequestId(), null);
}
/**
* Factory method to generate a SchedulerRequestKey from a SchedulingRequest.
* @param req SchedulingRequest
* @return SchedulerRequestKey
*/
public static SchedulerRequestKey create(SchedulingRequest req) {
return new SchedulerRequestKey(req.getPriority(),
req.getAllocationRequestId(), null);
}
public static SchedulerRequestKey create(UpdateContainerRequest req,
SchedulerRequestKey schedulerRequestKey) {
return new SchedulerRequestKey(schedulerRequestKey.getPriority(),

View File

@ -53,6 +53,7 @@
import org.apache.hadoop.yarn.exceptions.InvalidContainerReleaseException;
import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException;
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@ -273,10 +274,14 @@ public void allocate(ApplicationAttemptId appAttemptId,
" state, ignore container allocate request.");
allocation = EMPTY_ALLOCATION;
} else {
allocation =
getScheduler().allocate(appAttemptId, ask, release,
blacklistAdditions, blacklistRemovals,
containerUpdateRequests);
try {
allocation = getScheduler().allocate(appAttemptId, ask,
request.getSchedulingRequests(), release,
blacklistAdditions, blacklistRemovals, containerUpdateRequests);
} catch (SchedulerInvalidResoureRequestException e) {
LOG.warn("Exceptions caught when scheduler handling requests");
throw new YarnException(e);
}
}
if (!blacklistAdditions.isEmpty() || !blacklistRemovals.isEmpty()) {

View File

@ -1113,8 +1113,7 @@ public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
Allocation amContainerAllocation =
appAttempt.scheduler.allocate(
appAttempt.applicationAttemptId,
appAttempt.amReqs,
EMPTY_CONTAINER_RELEASE_LIST,
appAttempt.amReqs, null, EMPTY_CONTAINER_RELEASE_LIST,
amBlacklist.getBlacklistAdditions(),
amBlacklist.getBlacklistRemovals(),
new ContainerUpdates());
@ -1140,7 +1139,7 @@ public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
// Acquire the AM container from the scheduler.
Allocation amContainerAllocation =
appAttempt.scheduler.allocate(appAttempt.applicationAttemptId,
EMPTY_CONTAINER_REQUEST_LIST, EMPTY_CONTAINER_RELEASE_LIST, null,
EMPTY_CONTAINER_REQUEST_LIST, null, EMPTY_CONTAINER_RELEASE_LIST, null,
null, new ContainerUpdates());
// There must be at least one container allocated, because a
// CONTAINER_ALLOCATED is emitted after an RMContainer is constructed,

View File

@ -53,6 +53,7 @@
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceOption;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.ResourceSizing;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
@ -1155,7 +1156,7 @@ public Resource getNormalizedResource(Resource requestedResource) {
*
* @param asks resource requests
*/
protected void normalizeRequests(List<ResourceRequest> asks) {
protected void normalizeResourceRequests(List<ResourceRequest> asks) {
for (ResourceRequest ask: asks) {
ask.setCapability(getNormalizedResource(ask.getCapability()));
}

View File

@ -41,6 +41,8 @@
import org.apache.hadoop.yarn.api.records.ExecutionType;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
@ -49,7 +51,9 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.AppPlacementAllocator;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.LocalityAppPlacementAllocator;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PendingAskUpdateResult;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SingleConstraintAppPlacementAllocator;
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
import org.apache.hadoop.yarn.util.resource.Resources;
/**
@ -91,11 +95,12 @@ public class AppSchedulingInfo {
public final ContainerUpdateContext updateContext;
public final Map<String, String> applicationSchedulingEnvs = new HashMap<>();
private final RMContext rmContext;
public AppSchedulingInfo(ApplicationAttemptId appAttemptId, String user,
Queue queue, AbstractUsersManager abstractUsersManager, long epoch,
ResourceUsage appResourceUsage,
Map<String, String> applicationSchedulingEnvs) {
Map<String, String> applicationSchedulingEnvs, RMContext rmContext) {
this.applicationAttemptId = appAttemptId;
this.applicationId = appAttemptId.getApplicationId();
this.queue = queue;
@ -105,6 +110,7 @@ public AppSchedulingInfo(ApplicationAttemptId appAttemptId, String user,
epoch << ResourceManager.EPOCH_BIT_SHIFT);
this.appResourceUsage = appResourceUsage;
this.applicationSchedulingEnvs.putAll(applicationSchedulingEnvs);
this.rmContext = rmContext;
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
updateContext = new ContainerUpdateContext(this);
@ -163,74 +169,153 @@ public ContainerUpdateContext getUpdateContext() {
* application, by asking for more resources and releasing resources acquired
* by the application.
*
* @param requests
* resources to be acquired
* @param resourceRequests resource requests to be allocated
* @param recoverPreemptedRequestForAContainer
* recover ResourceRequest on preemption
* recover ResourceRequest/SchedulingRequest on preemption
* @return true if any resource was updated, false otherwise
*/
public boolean updateResourceRequests(List<ResourceRequest> requests,
public boolean updateResourceRequests(List<ResourceRequest> resourceRequests,
boolean recoverPreemptedRequestForAContainer) {
if (null == requests || requests.isEmpty()) {
return false;
}
// Flag to track if any incoming requests update "ANY" requests
boolean offswitchResourcesUpdated = false;
boolean offswitchResourcesUpdated;
writeLock.lock();
try {
this.writeLock.lock();
// A map to group resource requests and dedup
Map<SchedulerRequestKey, Map<String, ResourceRequest>> dedupRequests =
new HashMap<>();
// Group resource request by schedulerRequestKey and resourceName
for (ResourceRequest request : requests) {
SchedulerRequestKey schedulerKey = SchedulerRequestKey.create(request);
if (!dedupRequests.containsKey(schedulerKey)) {
dedupRequests.put(schedulerKey, new HashMap<>());
}
dedupRequests.get(schedulerKey).put(request.getResourceName(), request);
}
// Update AppPlacementAllocator by dedup requests.
offswitchResourcesUpdated =
addRequestToAppPlacement(
recoverPreemptedRequestForAContainer, dedupRequests);
return offswitchResourcesUpdated;
// Update AppPlacementAllocator by requests
offswitchResourcesUpdated = internalAddResourceRequests(
recoverPreemptedRequestForAContainer, resourceRequests);
} finally {
this.writeLock.unlock();
writeLock.unlock();
}
return offswitchResourcesUpdated;
}
/**
* The ApplicationMaster is updating resource requirements for the
* application, by asking for more resources and releasing resources acquired
* by the application.
*
* @param dedupRequests (dedup) resource requests to be allocated
* @param recoverPreemptedRequestForAContainer
* recover ResourceRequest/SchedulingRequest on preemption
* @return true if any resource was updated, false otherwise
*/
public boolean updateResourceRequests(
Map<SchedulerRequestKey, Map<String, ResourceRequest>> dedupRequests,
boolean recoverPreemptedRequestForAContainer) {
// Flag to track if any incoming requests update "ANY" requests
boolean offswitchResourcesUpdated;
writeLock.lock();
try {
// Update AppPlacementAllocator by requests
offswitchResourcesUpdated = internalAddResourceRequests(
recoverPreemptedRequestForAContainer, dedupRequests);
} finally {
writeLock.unlock();
}
return offswitchResourcesUpdated;
}
/**
* The ApplicationMaster is updating resource requirements for the
* application, by asking for more resources and releasing resources acquired
* by the application.
*
* @param schedulingRequests resource requests to be allocated
* @param recoverPreemptedRequestForAContainer
* recover ResourceRequest/SchedulingRequest on preemption
* @return true if any resource was updated, false otherwise
*/
public boolean updateSchedulingRequests(
List<SchedulingRequest> schedulingRequests,
boolean recoverPreemptedRequestForAContainer) {
// Flag to track if any incoming requests update "ANY" requests
boolean offswitchResourcesUpdated;
writeLock.lock();
try {
// Update AppPlacementAllocator by requests
offswitchResourcesUpdated = addSchedulingRequests(
recoverPreemptedRequestForAContainer, schedulingRequests);
} finally {
writeLock.unlock();
}
return offswitchResourcesUpdated;
}
public void removeAppPlacement(SchedulerRequestKey schedulerRequestKey) {
schedulerKeyToAppPlacementAllocator.remove(schedulerRequestKey);
}
boolean addRequestToAppPlacement(
private boolean addSchedulingRequests(
boolean recoverPreemptedRequestForAContainer,
List<SchedulingRequest> schedulingRequests) {
// Do we need to update pending resource for app/queue, etc.?
boolean requireUpdatePendingResource = false;
for (SchedulingRequest request : schedulingRequests) {
SchedulerRequestKey schedulerRequestKey = SchedulerRequestKey.create(
request);
AppPlacementAllocator appPlacementAllocator =
getAndAddAppPlacementAllocatorIfNotExist(schedulerRequestKey,
SingleConstraintAppPlacementAllocator.class.getCanonicalName());
// Update AppPlacementAllocator
PendingAskUpdateResult pendingAmountChanges =
appPlacementAllocator.updatePendingAsk(schedulerRequestKey,
request, recoverPreemptedRequestForAContainer);
if (null != pendingAmountChanges) {
updatePendingResources(pendingAmountChanges, schedulerRequestKey,
queue.getMetrics());
requireUpdatePendingResource = true;
}
}
return requireUpdatePendingResource;
}
/**
* Get and insert AppPlacementAllocator if it doesn't exist, this should be
* protected by write lock.
* @param schedulerRequestKey schedulerRequestKey
* @param placementTypeClass placementTypeClass
* @return AppPlacementAllocator
*/
private AppPlacementAllocator<SchedulerNode> getAndAddAppPlacementAllocatorIfNotExist(
SchedulerRequestKey schedulerRequestKey, String placementTypeClass) {
AppPlacementAllocator<SchedulerNode> appPlacementAllocator;
if ((appPlacementAllocator = schedulerKeyToAppPlacementAllocator.get(
schedulerRequestKey)) == null) {
appPlacementAllocator =
ApplicationPlacementAllocatorFactory.getAppPlacementAllocator(
placementTypeClass, this, schedulerRequestKey, rmContext);
schedulerKeyToAppPlacementAllocator.put(schedulerRequestKey,
appPlacementAllocator);
}
return appPlacementAllocator;
}
private boolean internalAddResourceRequests(
boolean recoverPreemptedRequestForAContainer,
Map<SchedulerRequestKey, Map<String, ResourceRequest>> dedupRequests) {
boolean offswitchResourcesUpdated = false;
for (Map.Entry<SchedulerRequestKey, Map<String, ResourceRequest>> entry :
dedupRequests.entrySet()) {
SchedulerRequestKey schedulerRequestKey = entry.getKey();
if (!schedulerKeyToAppPlacementAllocator
.containsKey(schedulerRequestKey)) {
AppPlacementAllocator<SchedulerNode> placementAllocatorInstance = ApplicationPlacementFactory
.getAppPlacementAllocator(applicationSchedulingEnvs
.get(ApplicationSchedulingConfig.ENV_APPLICATION_PLACEMENT_TYPE_CLASS));
placementAllocatorInstance.setAppSchedulingInfo(this);
schedulerKeyToAppPlacementAllocator.put(schedulerRequestKey,
placementAllocatorInstance);
}
AppPlacementAllocator<SchedulerNode> appPlacementAllocator =
getAndAddAppPlacementAllocatorIfNotExist(schedulerRequestKey,
applicationSchedulingEnvs.get(
ApplicationSchedulingConfig.ENV_APPLICATION_PLACEMENT_TYPE_CLASS));
// Update AppPlacementAllocator
PendingAskUpdateResult pendingAmountChanges = schedulerKeyToAppPlacementAllocator
.get(schedulerRequestKey).updatePendingAsk(entry.getValue().values(),
PendingAskUpdateResult pendingAmountChanges =
appPlacementAllocator.updatePendingAsk(entry.getValue().values(),
recoverPreemptedRequestForAContainer);
if (null != pendingAmountChanges) {
@ -242,6 +327,29 @@ boolean addRequestToAppPlacement(
return offswitchResourcesUpdated;
}
private boolean internalAddResourceRequests(boolean recoverPreemptedRequestForAContainer,
List<ResourceRequest> resourceRequests) {
if (null == resourceRequests || resourceRequests.isEmpty()) {
return false;
}
// A map to group resource requests and dedup
Map<SchedulerRequestKey, Map<String, ResourceRequest>> dedupRequests =
new HashMap<>();
// Group resource request by schedulerRequestKey and resourceName
for (ResourceRequest request : resourceRequests) {
SchedulerRequestKey schedulerKey = SchedulerRequestKey.create(request);
if (!dedupRequests.containsKey(schedulerKey)) {
dedupRequests.put(schedulerKey, new HashMap<>());
}
dedupRequests.get(schedulerKey).put(request.getResourceName(), request);
}
return internalAddResourceRequests(recoverPreemptedRequestForAContainer,
dedupRequests);
}
private void updatePendingResources(PendingAskUpdateResult updateResult,
SchedulerRequestKey schedulerKey, QueueMetrics metrics) {
@ -629,13 +737,22 @@ public boolean canDelayTo(
}
}
public boolean acceptNodePartition(SchedulerRequestKey schedulerKey,
String nodePartition, SchedulingMode schedulingMode) {
/**
* Pre-check node to see if it satisfy the given schedulerKey and
* scheduler mode
*
* @param schedulerKey schedulerKey
* @param schedulerNode schedulerNode
* @param schedulingMode schedulingMode
* @return can use the node or not.
*/
public boolean precheckNode(SchedulerRequestKey schedulerKey,
SchedulerNode schedulerNode, SchedulingMode schedulingMode) {
try {
this.readLock.lock();
AppPlacementAllocator ap =
schedulerKeyToAppPlacementAllocator.get(schedulerKey);
return (ap != null) && ap.acceptNodePartition(nodePartition,
return (ap != null) && ap.precheckNode(schedulerNode,
schedulingMode);
} finally {
this.readLock.unlock();

View File

@ -21,15 +21,17 @@
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ApplicationSchedulingConfig;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.AppPlacementAllocator;
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
/**
* Factory class to build various application placement policies.
*/
@Public
@Unstable
public class ApplicationPlacementFactory {
public class ApplicationPlacementAllocatorFactory {
/**
* Get AppPlacementAllocator related to the placement type requested.
@ -39,7 +41,8 @@ public class ApplicationPlacementFactory {
* @return Specific AppPlacementAllocator instance based on type
*/
public static AppPlacementAllocator<SchedulerNode> getAppPlacementAllocator(
String appPlacementAllocatorName) {
String appPlacementAllocatorName, AppSchedulingInfo appSchedulingInfo,
SchedulerRequestKey schedulerRequestKey, RMContext rmContext) {
Class<?> policyClass;
try {
if (appPlacementAllocatorName == null) {
@ -58,6 +61,8 @@ public static AppPlacementAllocator<SchedulerNode> getAppPlacementAllocator(
@SuppressWarnings("unchecked")
AppPlacementAllocator<SchedulerNode> placementAllocatorInstance = (AppPlacementAllocator<SchedulerNode>) ReflectionUtils
.newInstance(policyClass, null);
placementAllocatorInstance.initialize(appSchedulingInfo,
schedulerRequestKey, rmContext);
return placementAllocatorInstance;
}
}

View File

@ -146,7 +146,7 @@ public synchronized boolean checkAndAddToOutstandingIncreases(
createResourceRequests(rmContainer, schedulerNode,
schedulerKey, resToIncrease);
updateResReqs.put(schedulerKey, resMap);
appSchedulingInfo.addRequestToAppPlacement(false, updateResReqs);
appSchedulingInfo.updateResourceRequests(updateResReqs, false);
}
return true;
}
@ -290,7 +290,7 @@ public ContainerId matchContainerToOutstandingIncreaseReq(
(rmContainer, node, schedulerKey,
rmContainer.getContainer().getResource());
reqsToUpdate.put(schedulerKey, resMap);
appSchedulingInfo.addRequestToAppPlacement(true, reqsToUpdate);
appSchedulingInfo.updateResourceRequests(reqsToUpdate, true);
return UNDEFINED;
}
return retVal;

View File

@ -56,6 +56,7 @@
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceInformation;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
import org.apache.hadoop.yarn.server.api.ContainerType;
@ -231,7 +232,7 @@ public SchedulerApplicationAttempt(ApplicationAttemptId applicationAttemptId,
this.appSchedulingInfo = new AppSchedulingInfo(applicationAttemptId, user,
queue, abstractUsersManager, rmContext.getEpoch(), attemptResourceUsage,
applicationSchedulingEnvs);
applicationSchedulingEnvs, rmContext);
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
readLock = lock.readLock();
writeLock = lock.writeLock();
@ -451,6 +452,23 @@ public boolean updateResourceRequests(
writeLock.unlock();
}
}
public boolean updateSchedulingRequests(
List<SchedulingRequest> requests) {
if (requests == null) {
return false;
}
try {
writeLock.lock();
if (!isStopped) {
return appSchedulingInfo.updateSchedulingRequests(requests, false);
}
return false;
} finally {
writeLock.unlock();
}
}
public void recoverResourceRequestsForContainer(
ContainerRequest containerRequest) {

View File

@ -42,6 +42,7 @@
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.exceptions.YarnException;
@ -132,18 +133,18 @@ public QueueInfo getQueueInfo(String queueName, boolean includeChildQueues,
*
* @param appAttemptId
* @param ask
* @param schedulingRequests
* @param release
* @param blacklistAdditions
* @param blacklistRemovals
* @param updateRequests
* @return the {@link Allocation} for the application
* @param blacklistAdditions
* @param blacklistRemovals
* @param updateRequests @return the {@link Allocation} for the application
*/
@Public
@Stable
Allocation allocate(ApplicationAttemptId appAttemptId,
List<ResourceRequest> ask, List<ContainerId> release,
List<String> blacklistAdditions, List<String> blacklistRemovals,
ContainerUpdates updateRequests);
List<ResourceRequest> ask, List<SchedulingRequest> schedulingRequests,
List<ContainerId> release, List<String> blacklistAdditions,
List<String> blacklistRemovals, ContainerUpdates updateRequests);
/**
* Get node resource usage report.

View File

@ -60,8 +60,11 @@
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceOption;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.ResourceSizing;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
@ -1058,12 +1061,29 @@ private void doneApplicationAttempt(
}
}
/**
* Normalize a list of SchedulingRequest
*
* @param asks scheduling request
*/
private void normalizeSchedulingRequests(List<SchedulingRequest> asks) {
if (asks == null) {
return;
}
for (SchedulingRequest ask: asks) {
ResourceSizing sizing = ask.getResourceSizing();
if (sizing != null && sizing.getResources() != null) {
sizing.setResources(getNormalizedResource(sizing.getResources()));
}
}
}
@Override
@Lock(Lock.NoLock.class)
public Allocation allocate(ApplicationAttemptId applicationAttemptId,
List<ResourceRequest> ask, List<ContainerId> release,
List<String> blacklistAdditions, List<String> blacklistRemovals,
ContainerUpdates updateRequests) {
List<ResourceRequest> ask, List<SchedulingRequest> schedulingRequests,
List<ContainerId> release, List<String> blacklistAdditions,
List<String> blacklistRemovals, ContainerUpdates updateRequests) {
FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
if (application == null) {
LOG.error("Calling allocate on removed or non existent application " +
@ -1071,6 +1091,18 @@ public Allocation allocate(ApplicationAttemptId applicationAttemptId,
return EMPTY_ALLOCATION;
}
if ((!getConfiguration().getBoolean(
CapacitySchedulerConfiguration.SCHEDULING_REQUEST_ALLOWED,
CapacitySchedulerConfiguration.DEFAULT_SCHEDULING_REQUEST_ALLOWED))
&& schedulingRequests != null && (!schedulingRequests.isEmpty())) {
throw new SchedulerInvalidResoureRequestException(
"Application attempt:" + applicationAttemptId
+ " is using SchedulingRequest, which is disabled. Please update "
+ CapacitySchedulerConfiguration.SCHEDULING_REQUEST_ALLOWED
+ " to true in capacity-scheduler.xml in order to use this "
+ "feature.");
}
// The allocate may be the leftover from previous attempt, and it will
// impact current attempt, such as confuse the request and allocation for
// current attempt's AM container.
@ -1091,7 +1123,10 @@ public Allocation allocate(ApplicationAttemptId applicationAttemptId,
LeafQueue updateDemandForQueue = null;
// Sanity check for new allocation requests
normalizeRequests(ask);
normalizeResourceRequests(ask);
// Normalize scheduling requests
normalizeSchedulingRequests(schedulingRequests);
Allocation allocation;
@ -1104,7 +1139,8 @@ public Allocation allocate(ApplicationAttemptId applicationAttemptId,
}
// Process resource requests
if (!ask.isEmpty()) {
if (!ask.isEmpty() || (schedulingRequests != null && !schedulingRequests
.isEmpty())) {
if (LOG.isDebugEnabled()) {
LOG.debug(
"allocate: pre-update " + applicationAttemptId + " ask size ="
@ -1113,7 +1149,8 @@ public Allocation allocate(ApplicationAttemptId applicationAttemptId,
}
// Update application requests
if (application.updateResourceRequests(ask)) {
if (application.updateResourceRequests(ask) || application
.updateSchedulingRequests(schedulingRequests)) {
updateDemandForQueue = (LeafQueue) application.getQueue();
}
@ -2580,10 +2617,9 @@ public boolean attemptAllocationOnNode(SchedulerApplicationAttempt appAttempt,
// Validate placement constraint is satisfied before
// committing the request.
try {
if (!PlacementConstraintsUtil.canSatisfyConstraints(
if (!PlacementConstraintsUtil.canSatisfySingleConstraint(
appAttempt.getApplicationId(),
schedulingRequest.getAllocationTags(),
schedulerNode,
schedulingRequest.getAllocationTags(), schedulerNode,
rmContext.getPlacementConstraintManager(),
rmContext.getAllocationTagsManager())) {
LOG.debug("Failed to allocate container for application "

View File

@ -77,6 +77,11 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
@Private
public static final String PREFIX = "yarn.scheduler.capacity.";
@Private
public static final String SCHEDULING_REQUEST_ALLOWED =
PREFIX + "scheduling-request.allowed";
public static final boolean DEFAULT_SCHEDULING_REQUEST_ALLOWED = false;
@Private
public static final String DOT = ".";

View File

@ -143,8 +143,7 @@ private ContainerAllocation preCheckForNodeCandidateSet(
// Is the nodePartition of pending request matches the node's partition
// If not match, jump to next priority.
if (!appInfo.acceptNodePartition(schedulerKey, node.getPartition(),
schedulingMode)) {
if (!appInfo.precheckNode(schedulerKey, node, schedulingMode)) {
ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation(
activitiesManager, node, application, priority,
ActivityDiagnosticConstant.

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import java.util.List;
@ -43,12 +44,23 @@
*/
public class ContainerRequest {
private List<ResourceRequest> requests;
private SchedulingRequest schedulingRequest;
public ContainerRequest(List<ResourceRequest> requests) {
this.requests = requests;
schedulingRequest = null;
}
public ContainerRequest(SchedulingRequest schedulingRequest) {
this.schedulingRequest = schedulingRequest;
this.requests = null;
}
public List<ResourceRequest> getResourceRequests() {
return requests;
}
public SchedulingRequest getSchedulingRequest() {
return schedulingRequest;
}
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceSizing;
import org.apache.hadoop.yarn.util.resource.Resources;
/**
@ -31,6 +32,11 @@ public class PendingAsk {
private final int count;
public final static PendingAsk ZERO = new PendingAsk(Resources.none(), 0);
public PendingAsk(ResourceSizing sizing) {
this.perAllocationResource = sizing.getResources();
this.count = sizing.getNumAllocations();
}
public PendingAsk(Resource res, int num) {
this.perAllocationResource = res;
this.count = num;

View File

@ -542,6 +542,12 @@ public void apply(Resource cluster, ResourceCommitRequest<FiCaSchedulerApp,
schedulerContainer.getRmContainer().getContainer());
((RMContainerImpl) rmContainer).setContainerRequest(
containerRequest);
// If this is from a SchedulingRequest, set allocation tags.
if (containerRequest.getSchedulingRequest() != null) {
((RMContainerImpl) rmContainer).setAllocationTags(
containerRequest.getSchedulingRequest().getAllocationTags());
}
}
attemptResourceUsage.incUsed(schedulerContainer.getNodePartition(),

View File

@ -29,6 +29,7 @@
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.log4j.Logger;
@ -287,21 +288,15 @@ public void addTempContainer(NodeId nodeId, ApplicationId applicationId,
* {@link SchedulingRequest#getAllocationTags()}
* application_id will be added to allocationTags.
*/
@SuppressWarnings("unchecked")
public void addContainer(NodeId nodeId, ContainerId containerId,
Set<String> allocationTags) {
// Do nothing for empty allocation tags.
if (allocationTags == null || allocationTags.isEmpty()) {
return;
}
ApplicationId applicationId =
containerId.getApplicationAttemptId().getApplicationId();
String applicationIdTag =
AllocationTagsNamespaces.APP_ID + applicationId.toString();
boolean useSet = false;
if (allocationTags != null && !allocationTags.isEmpty()) {
// Copy before edit it.
allocationTags = new HashSet<>(allocationTags);
allocationTags.add(applicationIdTag);
useSet = true;
}
writeLock.lock();
try {
TypeToCountedTags perAppTagsMapping = perAppNodeMappings
@ -311,19 +306,12 @@ public void addContainer(NodeId nodeId, ContainerId containerId,
// Covering test-cases where context is mocked
String nodeRack = (rmContext.getRMNodes() != null
&& rmContext.getRMNodes().get(nodeId) != null)
? rmContext.getRMNodes().get(nodeId).getRackName()
: "default-rack";
if (useSet) {
perAppTagsMapping.addTags(nodeId, allocationTags);
perAppRackTagsMapping.addTags(nodeRack, allocationTags);
globalNodeMapping.addTags(nodeId, allocationTags);
globalRackMapping.addTags(nodeRack, allocationTags);
} else {
perAppTagsMapping.addTag(nodeId, applicationIdTag);
perAppRackTagsMapping.addTag(nodeRack, applicationIdTag);
globalNodeMapping.addTag(nodeId, applicationIdTag);
globalRackMapping.addTag(nodeRack, applicationIdTag);
}
? rmContext.getRMNodes().get(nodeId).getRackName() :
"default-rack";
perAppTagsMapping.addTags(nodeId, allocationTags);
perAppRackTagsMapping.addTags(nodeRack, allocationTags);
globalNodeMapping.addTags(nodeId, allocationTags);
globalRackMapping.addTags(nodeRack, allocationTags);
if (LOG.isDebugEnabled()) {
LOG.debug("Added container=" + containerId + " with tags=["
@ -341,20 +329,15 @@ public void addContainer(NodeId nodeId, ContainerId containerId,
* @param containerId containerId.
* @param allocationTags allocation tags for given container
*/
@SuppressWarnings("unchecked")
public void removeContainer(NodeId nodeId,
ContainerId containerId, Set<String> allocationTags) {
// Do nothing for empty allocation tags.
if (allocationTags == null || allocationTags.isEmpty()) {
return;
}
ApplicationId applicationId =
containerId.getApplicationAttemptId().getApplicationId();
String applicationIdTag =
AllocationTagsNamespaces.APP_ID + applicationId.toString();
boolean useSet = false;
if (allocationTags != null && !allocationTags.isEmpty()) {
// Copy before edit it.
allocationTags = new HashSet<>(allocationTags);
allocationTags.add(applicationIdTag);
useSet = true;
}
writeLock.lock();
try {
@ -368,19 +351,12 @@ public void removeContainer(NodeId nodeId,
// Covering test-cases where context is mocked
String nodeRack = (rmContext.getRMNodes() != null
&& rmContext.getRMNodes().get(nodeId) != null)
? rmContext.getRMNodes().get(nodeId).getRackName()
: "default-rack";
if (useSet) {
perAppTagsMapping.removeTags(nodeId, allocationTags);
perAppRackTagsMapping.removeTags(nodeRack, allocationTags);
globalNodeMapping.removeTags(nodeId, allocationTags);
globalRackMapping.removeTags(nodeRack, allocationTags);
} else {
perAppTagsMapping.removeTag(nodeId, applicationIdTag);
perAppRackTagsMapping.removeTag(nodeRack, applicationIdTag);
globalNodeMapping.removeTag(nodeId, applicationIdTag);
globalRackMapping.removeTag(nodeRack, applicationIdTag);
}
? rmContext.getRMNodes().get(nodeId).getRackName() :
"default-rack";
perAppTagsMapping.removeTags(nodeId, allocationTags);
perAppRackTagsMapping.removeTags(nodeRack, allocationTags);
globalNodeMapping.removeTags(nodeId, allocationTags);
globalRackMapping.removeTags(nodeRack, allocationTags);
if (perAppTagsMapping.isEmpty()) {
perAppNodeMappings.remove(applicationId);
@ -602,6 +578,7 @@ public long getNodeCardinalityByOp(NodeId nodeId, ApplicationId applicationId,
* @throws InvalidAllocationTagsQueryException when illegal query
* parameter specified
*/
@SuppressWarnings("unchecked")
public long getRackCardinalityByOp(String rack, ApplicationId applicationId,
Set<String> tags, LongBinaryOperator op)
throws InvalidAllocationTagsQueryException {

View File

@ -1,31 +0,0 @@
/*
* *
* 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.constraint;
/**
* Predefined namespaces for tags
*
* Same as namespace of resource types. Namespaces of placement tags are start
* with alphabets and ended with "/"
*/
public class AllocationTagsNamespaces {
public static final String APP_ID = "yarn_app_id/";
}

View File

@ -20,6 +20,8 @@
import java.util.Iterator;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -30,9 +32,12 @@
import org.apache.hadoop.yarn.api.resource.PlacementConstraint.TargetExpression.TargetType;
import org.apache.hadoop.yarn.api.resource.PlacementConstraintTransformations.SingleConstraintTransformer;
import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.algorithm.DefaultPlacementAlgorithm;
import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE_PARTITION;
/**
* This class contains various static methods used by the Placement Algorithms
* to simplify constrained placement.
@ -41,16 +46,20 @@
@Public
@Unstable
public final class PlacementConstraintsUtil {
private static final Log LOG =
LogFactory.getLog(PlacementConstraintsUtil.class);
// Suppresses default constructor, ensuring non-instantiability.
private PlacementConstraintsUtil() {
}
/**
* Returns true if **single** application constraint with associated
* Returns true if **single** placement constraint with associated
* allocationTags and scope is satisfied by a specific scheduler Node.
*
* @param appId the application id
* @param targetApplicationId the application id, which could be override by
* target application id specified inside allocation
* tags.
* @param sc the placement constraint
* @param te the target expression
* @param node the scheduler node
@ -59,32 +68,123 @@ private PlacementConstraintsUtil() {
* @throws InvalidAllocationTagsQueryException
*/
private static boolean canSatisfySingleConstraintExpression(
ApplicationId appId, SingleConstraint sc, TargetExpression te,
SchedulerNode node, AllocationTagsManager tm)
ApplicationId targetApplicationId, SingleConstraint sc,
TargetExpression te, SchedulerNode node, AllocationTagsManager tm)
throws InvalidAllocationTagsQueryException {
long minScopeCardinality = 0;
long maxScopeCardinality = 0;
// Optimizations to only check cardinality if necessary.
int desiredMinCardinality = sc.getMinCardinality();
int desiredMaxCardinality = sc.getMaxCardinality();
boolean checkMinCardinality = desiredMinCardinality > 0;
boolean checkMaxCardinality = desiredMaxCardinality < Integer.MAX_VALUE;
if (sc.getScope().equals(PlacementConstraints.NODE)) {
minScopeCardinality = tm.getNodeCardinalityByOp(node.getNodeID(), appId,
te.getTargetValues(), Long::max);
maxScopeCardinality = tm.getNodeCardinalityByOp(node.getNodeID(), appId,
te.getTargetValues(), Long::min);
if (checkMinCardinality) {
minScopeCardinality = tm.getNodeCardinalityByOp(node.getNodeID(),
targetApplicationId, te.getTargetValues(), Long::max);
}
if (checkMaxCardinality) {
maxScopeCardinality = tm.getNodeCardinalityByOp(node.getNodeID(),
targetApplicationId, te.getTargetValues(), Long::min);
}
} else if (sc.getScope().equals(PlacementConstraints.RACK)) {
minScopeCardinality = tm.getRackCardinalityByOp(node.getRackName(), appId,
te.getTargetValues(), Long::max);
maxScopeCardinality = tm.getRackCardinalityByOp(node.getRackName(), appId,
te.getTargetValues(), Long::min);
if (checkMinCardinality) {
minScopeCardinality = tm.getRackCardinalityByOp(node.getRackName(),
targetApplicationId, te.getTargetValues(), Long::max);
}
if (checkMaxCardinality) {
maxScopeCardinality = tm.getRackCardinalityByOp(node.getRackName(),
targetApplicationId, te.getTargetValues(), Long::min);
}
}
// Make sure Anti-affinity satisfies hard upper limit
maxScopeCardinality = sc.getMaxCardinality() == 0 ? maxScopeCardinality - 1
maxScopeCardinality = desiredMaxCardinality == 0 ? maxScopeCardinality - 1
: maxScopeCardinality;
return (minScopeCardinality >= sc.getMinCardinality()
&& maxScopeCardinality < sc.getMaxCardinality());
return (desiredMinCardinality <= 0
|| minScopeCardinality >= desiredMinCardinality) && (
desiredMaxCardinality == Integer.MAX_VALUE
|| maxScopeCardinality < desiredMaxCardinality);
}
private static boolean canSatisfyNodePartitionConstraintExpresssion(
TargetExpression targetExpression, SchedulerNode schedulerNode) {
Set<String> values = targetExpression.getTargetValues();
if (values == null || values.isEmpty()) {
return schedulerNode.getPartition().equals(
RMNodeLabelsManager.NO_LABEL);
} else{
String nodePartition = values.iterator().next();
if (!nodePartition.equals(schedulerNode.getPartition())) {
return false;
}
}
return true;
}
private static boolean canSatisfySingleConstraint(ApplicationId applicationId,
SingleConstraint singleConstraint, SchedulerNode schedulerNode,
AllocationTagsManager tagsManager)
throws InvalidAllocationTagsQueryException {
// Iterate through TargetExpressions
Iterator<TargetExpression> expIt =
singleConstraint.getTargetExpressions().iterator();
while (expIt.hasNext()) {
TargetExpression currentExp = expIt.next();
// Supporting AllocationTag Expressions for now
if (currentExp.getTargetType().equals(TargetType.ALLOCATION_TAG)) {
// Check if conditions are met
if (!canSatisfySingleConstraintExpression(applicationId,
singleConstraint, currentExp, schedulerNode, tagsManager)) {
return false;
}
} else if (currentExp.getTargetType().equals(TargetType.NODE_ATTRIBUTE)
&& currentExp.getTargetKey().equals(NODE_PARTITION)) {
// This is a node partition expression, check it.
canSatisfyNodePartitionConstraintExpresssion(currentExp, schedulerNode);
}
}
// return true if all targetExpressions are satisfied
return true;
}
/**
* Returns true if all application constraints with associated allocationTags
* Returns true if all placement constraints are **currently** satisfied by a
* specific scheduler Node..
*
* To do so the method retrieves and goes through all application constraint
* expressions and checks if the specific allocation is between the allowed
* min-max cardinality values under the constraint scope (Node/Rack/etc).
*
* @param applicationId applicationId,
* @param placementConstraint placement constraint.
* @param node the scheduler node
* @param tagsManager the allocation tags store
* @return true if all application constraints are satisfied by node
* @throws InvalidAllocationTagsQueryException
*/
public static boolean canSatisfySingleConstraint(ApplicationId applicationId,
PlacementConstraint placementConstraint, SchedulerNode node,
AllocationTagsManager tagsManager)
throws InvalidAllocationTagsQueryException {
if (placementConstraint == null) {
return true;
}
// Transform to SimpleConstraint
SingleConstraintTransformer singleTransformer =
new SingleConstraintTransformer(placementConstraint);
placementConstraint = singleTransformer.transform();
AbstractConstraint sConstraintExpr = placementConstraint.getConstraintExpr();
SingleConstraint single = (SingleConstraint) sConstraintExpr;
return canSatisfySingleConstraint(applicationId, single, node, tagsManager);
}
/**
* Returns true if all placement constraints with associated allocationTags
* are **currently** satisfied by a specific scheduler Node.
* To do so the method retrieves and goes through all application constraint
* expressions and checks if the specific allocation is between the allowed
@ -98,41 +198,12 @@ private static boolean canSatisfySingleConstraintExpression(
* @return true if all application constraints are satisfied by node
* @throws InvalidAllocationTagsQueryException
*/
public static boolean canSatisfyConstraints(ApplicationId appId,
public static boolean canSatisfySingleConstraint(ApplicationId appId,
Set<String> allocationTags, SchedulerNode node,
PlacementConstraintManager pcm, AllocationTagsManager tagsManager)
throws InvalidAllocationTagsQueryException {
PlacementConstraint constraint = pcm.getConstraint(appId, allocationTags);
if (constraint == null) {
return true;
}
// Transform to SimpleConstraint
SingleConstraintTransformer singleTransformer =
new SingleConstraintTransformer(constraint);
constraint = singleTransformer.transform();
AbstractConstraint sConstraintExpr = constraint.getConstraintExpr();
SingleConstraint single = (SingleConstraint) sConstraintExpr;
// Iterate through TargetExpressions
Iterator<TargetExpression> expIt = single.getTargetExpressions().iterator();
while (expIt.hasNext()) {
TargetExpression currentExp = expIt.next();
// Supporting AllocationTag Expressions for now
if (currentExp.getTargetType().equals(TargetType.ALLOCATION_TAG)) {
// If source and tag allocation tags are the same, we do not enforce
// constraints with minimum cardinality.
if (currentExp.getTargetValues().equals(allocationTags)
&& single.getMinCardinality() > 0) {
return true;
}
// Check if conditions are met
if (!canSatisfySingleConstraintExpression(appId, single, currentExp,
node, tagsManager)) {
return false;
}
}
}
// return true if all targetExpressions are satisfied
return true;
return canSatisfySingleConstraint(appId, constraint, node, tagsManager);
}
}

View File

@ -67,7 +67,7 @@ public boolean attemptPlacementOnNode(ApplicationId appId,
throws InvalidAllocationTagsQueryException {
int numAllocs = schedulingRequest.getResourceSizing().getNumAllocations();
if (numAllocs > 0) {
if (PlacementConstraintsUtil.canSatisfyConstraints(appId,
if (PlacementConstraintsUtil.canSatisfySingleConstraint(appId,
schedulingRequest.getAllocationTags(), schedulerNode,
constraintManager, tagsManager)) {
return true;

View File

@ -188,12 +188,18 @@ private void processPlacementConstraints(ApplicationId applicationId,
@Override
public void allocate(ApplicationAttemptId appAttemptId,
AllocateRequest request, AllocateResponse response) throws YarnException {
// Copy the scheduling request since we will clear it later after sending
// to dispatcher
List<SchedulingRequest> schedulingRequests =
request.getSchedulingRequests();
new ArrayList<>(request.getSchedulingRequests());
dispatchRequestsForPlacement(appAttemptId, schedulingRequests);
reDispatchRetryableRequests(appAttemptId);
schedulePlacedRequests(appAttemptId);
// Remove SchedulingRequest from AllocateRequest to avoid SchedulingRequest
// added to scheduler.
request.setSchedulingRequests(Collections.emptyList());
nextAMSProcessor.allocate(appAttemptId, request, response);
handleRejectedRequests(appAttemptId, response);

View File

@ -40,6 +40,7 @@
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceOption;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@ -830,9 +831,9 @@ public void killContainer(RMContainer container) {
@Override
public Allocation allocate(ApplicationAttemptId appAttemptId,
List<ResourceRequest> ask, List<ContainerId> release,
List<String> blacklistAdditions, List<String> blacklistRemovals,
ContainerUpdates updateRequests) {
List<ResourceRequest> ask, List<SchedulingRequest> schedulingRequests,
List<ContainerId> release, List<String> blacklistAdditions,
List<String> blacklistRemovals, ContainerUpdates updateRequests) {
// Make sure this application exists
FSAppAttempt application = getSchedulerApp(appAttemptId);
@ -857,7 +858,9 @@ public Allocation allocate(ApplicationAttemptId appAttemptId,
handleContainerUpdates(application, updateRequests);
// Sanity check
normalizeRequests(ask);
normalizeResourceRequests(ask);
// TODO, normalize SchedulingRequest
// Record container allocation start time
application.recordContainerRequestTime(getClock().getTime());
@ -879,6 +882,7 @@ public Allocation allocate(ApplicationAttemptId appAttemptId,
// Update application requests
application.updateResourceRequests(ask);
// TODO, handle SchedulingRequest
application.showRequests();
}
} finally {

View File

@ -40,6 +40,7 @@
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.factories.RecordFactory;
@ -320,8 +321,8 @@ public synchronized void setRMContext(RMContext rmContext) {
@Override
public Allocation allocate(ApplicationAttemptId applicationAttemptId,
List<ResourceRequest> ask, List<ContainerId> release,
List<String> blacklistAdditions, List<String> blacklistRemovals,
List<ResourceRequest> ask, List<SchedulingRequest> schedulingRequests,
List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals,
ContainerUpdates updateRequests) {
FifoAppAttempt application = getApplicationAttempt(applicationAttemptId);
if (application == null) {
@ -342,7 +343,7 @@ public Allocation allocate(ApplicationAttemptId applicationAttemptId,
}
// Sanity check
normalizeRequests(ask);
normalizeResourceRequests(ask);
// Release containers
releaseContainers(release, application);

View File

@ -19,6 +19,8 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
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.SchedulerNode;
@ -29,7 +31,6 @@
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
/**
@ -50,13 +51,18 @@
* requests.
* </p>
*/
public interface AppPlacementAllocator<N extends SchedulerNode> {
public abstract class AppPlacementAllocator<N extends SchedulerNode> {
protected AppSchedulingInfo appSchedulingInfo;
protected SchedulerRequestKey schedulerRequestKey;
protected RMContext rmContext;
/**
* Get iterator of preferred node depends on requirement and/or availability
* @param candidateNodeSet input CandidateNodeSet
* @return iterator of preferred node
*/
Iterator<N> getPreferredNodeIterator(CandidateNodeSet<N> candidateNodeSet);
public abstract Iterator<N> getPreferredNodeIterator(
CandidateNodeSet<N> candidateNodeSet);
/**
* Replace existing pending asks by the new requests
@ -66,15 +72,29 @@ public interface AppPlacementAllocator<N extends SchedulerNode> {
* requests for preempted container
* @return true if total pending resource changed
*/
PendingAskUpdateResult updatePendingAsk(
public abstract PendingAskUpdateResult updatePendingAsk(
Collection<ResourceRequest> requests,
boolean recoverPreemptedRequestForAContainer);
/**
* Replace existing pending asks by the new SchedulingRequest
*
* @param schedulerRequestKey scheduler request key
* @param schedulingRequest new asks
* @param recoverPreemptedRequestForAContainer if we're recovering resource
* requests for preempted container
* @return true if total pending resource changed
*/
public abstract PendingAskUpdateResult updatePendingAsk(
SchedulerRequestKey schedulerRequestKey,
SchedulingRequest schedulingRequest,
boolean recoverPreemptedRequestForAContainer);
/**
* Get pending ResourceRequests by given schedulerRequestKey
* @return Map of resourceName to ResourceRequest
*/
Map<String, ResourceRequest> getResourceRequests();
public abstract Map<String, ResourceRequest> getResourceRequests();
/**
* Get pending ask for given resourceName. If there's no such pendingAsk,
@ -83,7 +103,7 @@ PendingAskUpdateResult updatePendingAsk(
* @param resourceName resourceName
* @return PendingAsk
*/
PendingAsk getPendingAsk(String resourceName);
public abstract PendingAsk getPendingAsk(String resourceName);
/**
* Get #pending-allocations for given resourceName. If there's no such
@ -92,7 +112,7 @@ PendingAskUpdateResult updatePendingAsk(
* @param resourceName resourceName
* @return #pending-allocations
*/
int getOutstandingAsksCount(String resourceName);
public abstract int getOutstandingAsksCount(String resourceName);
/**
* Notify container allocated.
@ -103,7 +123,7 @@ PendingAskUpdateResult updatePendingAsk(
* the container. This will be used by scheduler to recover requests.
* Please refer to {@link ContainerRequest} for more details.
*/
ContainerRequest allocate(SchedulerRequestKey schedulerKey,
public abstract ContainerRequest allocate(SchedulerRequestKey schedulerKey,
NodeType type, SchedulerNode node);
/**
@ -112,7 +132,7 @@ ContainerRequest allocate(SchedulerRequestKey schedulerKey,
* @param node which node we will allocate on
* @return true if we has pending requirement
*/
boolean canAllocate(NodeType type, SchedulerNode node);
public abstract boolean canAllocate(NodeType type, SchedulerNode node);
/**
* Can delay to give locality?
@ -123,16 +143,16 @@ ContainerRequest allocate(SchedulerRequestKey schedulerKey,
* @param resourceName resourceName
* @return can/cannot
*/
boolean canDelayTo(String resourceName);
public abstract boolean canDelayTo(String resourceName);
/**
* Does this {@link AppPlacementAllocator} accept resources on nodePartition?
* Does this {@link AppPlacementAllocator} accept resources on given node?
*
* @param nodePartition nodePartition
* @param schedulerNode schedulerNode
* @param schedulingMode schedulingMode
* @return accepted/not
*/
boolean acceptNodePartition(String nodePartition,
public abstract boolean precheckNode(SchedulerNode schedulerNode,
SchedulingMode schedulingMode);
/**
@ -142,7 +162,7 @@ boolean acceptNodePartition(String nodePartition,
*
* @return primary requested node partition
*/
String getPrimaryRequestedNodePartition();
public abstract String getPrimaryRequestedNodePartition();
/**
* @return number of unique location asks with #pending greater than 0,
@ -152,18 +172,24 @@ boolean acceptNodePartition(String nodePartition,
* and should belong to specific delay scheduling policy impl.
* See YARN-7457 for more details.
*/
int getUniqueLocationAsks();
public abstract int getUniqueLocationAsks();
/**
* Print human-readable requests to LOG debug.
*/
void showRequests();
public abstract void showRequests();
/**
* Set app scheduling info.
* Initialize this allocator, this will be called by Factory automatically
*
* @param appSchedulingInfo
* app info object.
* @param appSchedulingInfo appSchedulingInfo
* @param schedulerRequestKey schedulerRequestKey
* @param rmContext rmContext
*/
void setAppSchedulingInfo(AppSchedulingInfo appSchedulingInfo);
public void initialize(AppSchedulingInfo appSchedulingInfo,
SchedulerRequestKey schedulerRequestKey, RMContext rmContext) {
this.appSchedulingInfo = appSchedulingInfo;
this.rmContext = rmContext;
this.schedulerRequestKey = schedulerRequestKey;
}
}

View File

@ -22,8 +22,9 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
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.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
@ -46,26 +47,18 @@
* containers.
*/
public class LocalityAppPlacementAllocator <N extends SchedulerNode>
implements AppPlacementAllocator<N> {
extends AppPlacementAllocator<N> {
private static final Log LOG =
LogFactory.getLog(LocalityAppPlacementAllocator.class);
private final Map<String, ResourceRequest> resourceRequestMap =
new ConcurrentHashMap<>();
private AppSchedulingInfo appSchedulingInfo;
private volatile String primaryRequestedPartition =
RMNodeLabelsManager.NO_LABEL;
private final ReentrantReadWriteLock.ReadLock readLock;
private final ReentrantReadWriteLock.WriteLock writeLock;
public LocalityAppPlacementAllocator(AppSchedulingInfo info) {
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
readLock = lock.readLock();
writeLock = lock.writeLock();
this.appSchedulingInfo = info;
}
public LocalityAppPlacementAllocator() {
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
readLock = lock.readLock();
@ -181,6 +174,19 @@ public PendingAskUpdateResult updatePendingAsk(
}
}
@Override
public PendingAskUpdateResult updatePendingAsk(
SchedulerRequestKey schedulerRequestKey,
SchedulingRequest schedulingRequest,
boolean recoverPreemptedRequestForAContainer)
throws SchedulerInvalidResoureRequestException {
throw new SchedulerInvalidResoureRequestException(this.getClass().getName()
+ " not be able to handle SchedulingRequest, there exists a "
+ "ResourceRequest with the same scheduler key=" + schedulerRequestKey
+ ", please send SchedulingRequest with a different allocationId and "
+ "priority");
}
@Override
public Map<String, ResourceRequest> getResourceRequests() {
return resourceRequestMap;
@ -362,13 +368,13 @@ public boolean canDelayTo(String resourceName) {
}
@Override
public boolean acceptNodePartition(String nodePartition,
public boolean precheckNode(SchedulerNode schedulerNode,
SchedulingMode schedulingMode) {
// We will only look at node label = nodeLabelToLookAt according to
// schedulingMode and partition of node.
String nodePartitionToLookAt;
if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
nodePartitionToLookAt = nodePartition;
nodePartitionToLookAt = schedulerNode.getPartition();
} else {
nodePartitionToLookAt = RMNodeLabelsManager.NO_LABEL;
}
@ -425,9 +431,4 @@ public ContainerRequest allocate(SchedulerRequestKey schedulerKey,
writeLock.unlock();
}
}
@Override
public void setAppSchedulingInfo(AppSchedulingInfo appSchedulingInfo) {
this.appSchedulingInfo = appSchedulingInfo;
}
}

View File

@ -0,0 +1,531 @@
/**
* 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.annotations.VisibleForTesting;
import org.apache.commons.collections.IteratorUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ExecutionType;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.ResourceSizing;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.api.records.impl.pb.SchedulingRequestPBImpl;
import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
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.SchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.InvalidAllocationTagsQueryException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.PlacementConstraintsUtil;
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.APPLICATION_LABEL_INTRA_APPLICATION;
import static org.apache.hadoop.yarn.api.resource.PlacementConstraints.NODE_PARTITION;
/**
* This is a simple implementation to do affinity or anti-affinity for
* inter/intra apps.
*/
public class SingleConstraintAppPlacementAllocator<N extends SchedulerNode>
extends AppPlacementAllocator<N> {
private static final Log LOG =
LogFactory.getLog(SingleConstraintAppPlacementAllocator.class);
private ReentrantReadWriteLock.ReadLock readLock;
private ReentrantReadWriteLock.WriteLock writeLock;
private SchedulingRequest schedulingRequest = null;
private String targetNodePartition;
private Set<String> targetAllocationTags;
private AllocationTagsManager allocationTagsManager;
public SingleConstraintAppPlacementAllocator() {
ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
readLock = lock.readLock();
writeLock = lock.writeLock();
}
@Override
@SuppressWarnings("unchecked")
public Iterator<N> getPreferredNodeIterator(
CandidateNodeSet<N> candidateNodeSet) {
// Now only handle the case that single node in the candidateNodeSet
// TODO, Add support to multi-hosts inside candidateNodeSet which is passed
// in.
N singleNode = CandidateNodeSetUtils.getSingleNode(candidateNodeSet);
if (null != singleNode) {
return IteratorUtils.singletonIterator(singleNode);
}
return IteratorUtils.emptyIterator();
}
@Override
public PendingAskUpdateResult updatePendingAsk(
Collection<ResourceRequest> requests,
boolean recoverPreemptedRequestForAContainer) {
if (requests != null && !requests.isEmpty()) {
throw new SchedulerInvalidResoureRequestException(
this.getClass().getName()
+ " not be able to handle ResourceRequest, there exists a "
+ "SchedulingRequest with the same scheduler key="
+ SchedulerRequestKey.create(requests.iterator().next())
+ ", please send ResourceRequest with a different allocationId and "
+ "priority");
}
// Do nothing
return null;
}
private PendingAskUpdateResult internalUpdatePendingAsk(
SchedulingRequest newSchedulingRequest, boolean recoverContainer) {
// When it is a recover container, there must exists an schedulingRequest.
if (recoverContainer && schedulingRequest == null) {
throw new SchedulerInvalidResoureRequestException("Trying to recover a "
+ "container request=" + newSchedulingRequest.toString() + ", however"
+ "there's no existing scheduling request, this should not happen.");
}
if (schedulingRequest != null) {
// If we have an old scheduling request, we will make sure that no changes
// made except sizing.
// To avoid unnecessary copy of the data structure, we do this by
// replacing numAllocations with old numAllocations in the
// newSchedulingRequest#getResourceSizing, and compare the two objects.
ResourceSizing sizing = newSchedulingRequest.getResourceSizing();
int existingNumAllocations =
schedulingRequest.getResourceSizing().getNumAllocations();
// When it is a recovered container request, just set
// #newAllocations = #existingAllocations + 1;
int newNumAllocations;
if (recoverContainer) {
newNumAllocations = existingNumAllocations + 1;
} else {
newNumAllocations = sizing.getNumAllocations();
}
sizing.setNumAllocations(existingNumAllocations);
// Compare two objects
if (!schedulingRequest.equals(newSchedulingRequest)) {
// Rollback #numAllocations
sizing.setNumAllocations(newNumAllocations);
throw new SchedulerInvalidResoureRequestException(
"Invalid updated SchedulingRequest added to scheduler, "
+ " we only allows changing numAllocations for the updated "
+ "SchedulingRequest. Old=" + schedulingRequest.toString()
+ " new=" + newSchedulingRequest.toString()
+ ", if any fields need to be updated, please cancel the "
+ "old request (by setting numAllocations to 0) and send a "
+ "SchedulingRequest with different combination of "
+ "priority/allocationId");
} else {
if (newNumAllocations == existingNumAllocations) {
// No update on pending asks, return null.
return null;
}
}
// Rollback #numAllocations
sizing.setNumAllocations(newNumAllocations);
// Basic sanity check
if (newNumAllocations < 0) {
throw new SchedulerInvalidResoureRequestException(
"numAllocation in ResourceSizing field must be >= 0, "
+ "updating schedulingRequest failed.");
}
PendingAskUpdateResult updateResult = new PendingAskUpdateResult(
new PendingAsk(schedulingRequest.getResourceSizing()),
new PendingAsk(newSchedulingRequest.getResourceSizing()),
targetNodePartition, targetNodePartition);
// Ok, now everything is same except numAllocation, update numAllocation.
this.schedulingRequest.getResourceSizing().setNumAllocations(
newNumAllocations);
LOG.info(
"Update numAllocation from old=" + existingNumAllocations + " to new="
+ newNumAllocations);
return updateResult;
}
// For a new schedulingRequest, we need to validate if we support its asks.
// This will update internal partitions, etc. after the SchedulingRequest is
// valid.
validateAndSetSchedulingRequest(newSchedulingRequest);
return new PendingAskUpdateResult(null,
new PendingAsk(newSchedulingRequest.getResourceSizing()), null,
targetNodePartition);
}
@Override
public PendingAskUpdateResult updatePendingAsk(
SchedulerRequestKey schedulerRequestKey,
SchedulingRequest newSchedulingRequest,
boolean recoverPreemptedRequestForAContainer) {
writeLock.lock();
try {
return internalUpdatePendingAsk(newSchedulingRequest,
recoverPreemptedRequestForAContainer);
} finally {
writeLock.unlock();
}
}
private String throwExceptionWithMetaInfo(String message) {
StringBuilder sb = new StringBuilder();
sb.append("AppId=").append(appSchedulingInfo.getApplicationId()).append(
" Key=").append(this.schedulerRequestKey).append(". Exception message:")
.append(message);
throw new SchedulerInvalidResoureRequestException(sb.toString());
}
private void validateAndSetSchedulingRequest(SchedulingRequest newSchedulingRequest)
throws SchedulerInvalidResoureRequestException {
// Check sizing exists
if (newSchedulingRequest.getResourceSizing() == null
|| newSchedulingRequest.getResourceSizing().getResources() == null) {
throwExceptionWithMetaInfo(
"No ResourceSizing found in the scheduling request, please double "
+ "check");
}
// Check execution type == GUARANTEED
if (newSchedulingRequest.getExecutionType() != null
&& newSchedulingRequest.getExecutionType().getExecutionType()
!= ExecutionType.GUARANTEED) {
throwExceptionWithMetaInfo(
"Only GUARANTEED execution type is supported.");
}
PlacementConstraint constraint =
newSchedulingRequest.getPlacementConstraint();
// We only accept SingleConstraint
PlacementConstraint.AbstractConstraint ac = constraint.getConstraintExpr();
if (!(ac instanceof PlacementConstraint.SingleConstraint)) {
throwExceptionWithMetaInfo(
"Only accepts " + PlacementConstraint.SingleConstraint.class.getName()
+ " as constraint-expression. Rejecting the new added "
+ "constraint-expression.class=" + ac.getClass().getName());
}
PlacementConstraint.SingleConstraint singleConstraint =
(PlacementConstraint.SingleConstraint) ac;
// Make sure it is an anti-affinity request (actually this implementation
// should be able to support both affinity / anti-affinity without much
// effort. Considering potential test effort required. Limit to
// anti-affinity to intra-app and scope is node.
if (!singleConstraint.getScope().equals(PlacementConstraints.NODE)) {
throwExceptionWithMetaInfo(
"Only support scope=" + PlacementConstraints.NODE
+ "now. PlacementConstraint=" + singleConstraint);
}
if (singleConstraint.getMinCardinality() != 0
|| singleConstraint.getMaxCardinality() != 1) {
throwExceptionWithMetaInfo(
"Only support anti-affinity, which is: minCardinality=0, "
+ "maxCardinality=1");
}
Set<PlacementConstraint.TargetExpression> targetExpressionSet =
singleConstraint.getTargetExpressions();
if (targetExpressionSet == null || targetExpressionSet.isEmpty()) {
throwExceptionWithMetaInfo(
"TargetExpression should not be null or empty");
}
// Set node partition
String nodePartition = null;
// Target allocation tags
Set<String> targetAllocationTags = null;
for (PlacementConstraint.TargetExpression targetExpression : targetExpressionSet) {
// Handle node partition
if (targetExpression.getTargetType().equals(
PlacementConstraint.TargetExpression.TargetType.NODE_ATTRIBUTE)) {
// For node attribute target, we only support Partition now. And once
// YARN-3409 is merged, we will support node attribute.
if (!targetExpression.getTargetKey().equals(NODE_PARTITION)) {
throwExceptionWithMetaInfo("When TargetType="
+ PlacementConstraint.TargetExpression.TargetType.NODE_ATTRIBUTE
+ " only " + NODE_PARTITION + " is accepted as TargetKey.");
}
if (nodePartition != null) {
// This means we have duplicated node partition entry inside placement
// constraint, which might be set by mistake.
throwExceptionWithMetaInfo(
"Only one node partition targetExpression is allowed");
}
Set<String> values = targetExpression.getTargetValues();
if (values == null || values.isEmpty()) {
nodePartition = RMNodeLabelsManager.NO_LABEL;
continue;
}
if (values.size() > 1) {
throwExceptionWithMetaInfo("Inside one targetExpression, we only "
+ "support affinity to at most one node partition now");
}
nodePartition = values.iterator().next();
} else if (targetExpression.getTargetType().equals(
PlacementConstraint.TargetExpression.TargetType.ALLOCATION_TAG)) {
// Handle allocation tags
if (targetAllocationTags != null) {
// This means we have duplicated AllocationTag expressions entries
// inside placement constraint, which might be set by mistake.
throwExceptionWithMetaInfo(
"Only one AllocationTag targetExpression is allowed");
}
if (targetExpression.getTargetValues() == null || targetExpression
.getTargetValues().isEmpty()) {
throwExceptionWithMetaInfo("Failed to find allocation tags from "
+ "TargetExpressions or couldn't find self-app target.");
}
targetAllocationTags = new HashSet<>(
targetExpression.getTargetValues());
if (targetExpression.getTargetKey() == null || !targetExpression
.getTargetKey().equals(APPLICATION_LABEL_INTRA_APPLICATION)) {
throwExceptionWithMetaInfo(
"As of now, the only accepted target key for targetKey of "
+ "allocation_tag target expression is: ["
+ APPLICATION_LABEL_INTRA_APPLICATION
+ "]. Please make changes to placement constraints "
+ "accordingly.");
}
}
}
if (targetAllocationTags == null) {
// That means we don't have ALLOCATION_TAG specified
throwExceptionWithMetaInfo(
"Couldn't find target expression with type == ALLOCATION_TAG, it is "
+ "required to include one and only one target expression with "
+ "type == ALLOCATION_TAG");
}
if (nodePartition == null) {
nodePartition = RMNodeLabelsManager.NO_LABEL;
}
// Validation is done. set local results:
this.targetNodePartition = nodePartition;
this.targetAllocationTags = targetAllocationTags;
this.schedulingRequest = new SchedulingRequestPBImpl(
((SchedulingRequestPBImpl) newSchedulingRequest).getProto());
LOG.info("Successfully added SchedulingRequest to app=" + appSchedulingInfo
.getApplicationAttemptId() + " targetAllocationTags=[" + StringUtils
.join(",", targetAllocationTags) + "]. nodePartition="
+ targetNodePartition);
}
@Override
@SuppressWarnings("unchecked")
public Map<String, ResourceRequest> getResourceRequests() {
return Collections.EMPTY_MAP;
}
@Override
public PendingAsk getPendingAsk(String resourceName) {
readLock.lock();
try {
if (resourceName.equals("*") && schedulingRequest != null) {
return new PendingAsk(schedulingRequest.getResourceSizing());
}
return PendingAsk.ZERO;
} finally {
readLock.unlock();
}
}
@Override
public int getOutstandingAsksCount(String resourceName) {
readLock.lock();
try {
if (resourceName.equals("*") && schedulingRequest != null) {
return schedulingRequest.getResourceSizing().getNumAllocations();
}
return 0;
} finally {
readLock.unlock();
}
}
private void decreasePendingNumAllocation() {
// Deduct pending #allocations by 1
ResourceSizing sizing = schedulingRequest.getResourceSizing();
sizing.setNumAllocations(sizing.getNumAllocations() - 1);
}
@Override
public ContainerRequest allocate(SchedulerRequestKey schedulerKey,
NodeType type, SchedulerNode node) {
writeLock.lock();
try {
// Per container scheduling request, it is just a copy of existing
// scheduling request with #allocations=1
SchedulingRequest containerSchedulingRequest = new SchedulingRequestPBImpl(
((SchedulingRequestPBImpl) schedulingRequest).getProto());
containerSchedulingRequest.getResourceSizing().setNumAllocations(1);
// Deduct sizing
decreasePendingNumAllocation();
return new ContainerRequest(containerSchedulingRequest);
} finally {
writeLock.unlock();
}
}
private boolean checkCardinalityAndPending(SchedulerNode node) {
// Do we still have pending resource?
if (schedulingRequest.getResourceSizing().getNumAllocations() <= 0) {
return false;
}
// node type will be ignored.
try {
return PlacementConstraintsUtil.canSatisfySingleConstraint(
appSchedulingInfo.getApplicationId(),
this.schedulingRequest.getPlacementConstraint(), node,
allocationTagsManager);
} catch (InvalidAllocationTagsQueryException e) {
LOG.warn("Failed to query node cardinality:", e);
return false;
}
}
@Override
public boolean canAllocate(NodeType type, SchedulerNode node) {
try {
readLock.lock();
return checkCardinalityAndPending(node);
} finally {
readLock.unlock();
}
}
@Override
public boolean canDelayTo(String resourceName) {
return true;
}
@Override
public boolean precheckNode(SchedulerNode schedulerNode,
SchedulingMode schedulingMode) {
// We will only look at node label = nodeLabelToLookAt according to
// schedulingMode and partition of node.
String nodePartitionToLookAt;
if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) {
nodePartitionToLookAt = schedulerNode.getPartition();
} else{
nodePartitionToLookAt = RMNodeLabelsManager.NO_LABEL;
}
readLock.lock();
try {
// Check node partition as well as cardinality/pending resources.
return this.targetNodePartition.equals(nodePartitionToLookAt)
&& checkCardinalityAndPending(schedulerNode);
} finally {
readLock.unlock();
}
}
@Override
public String getPrimaryRequestedNodePartition() {
return targetNodePartition;
}
@Override
public int getUniqueLocationAsks() {
return 1;
}
@Override
public void showRequests() {
try {
readLock.lock();
if (schedulingRequest != null) {
LOG.info(schedulingRequest.toString());
}
} finally {
readLock.unlock();
}
}
@VisibleForTesting
SchedulingRequest getSchedulingRequest() {
return schedulingRequest;
}
@VisibleForTesting
String getTargetNodePartition() {
return targetNodePartition;
}
@VisibleForTesting
Set<String> getTargetAllocationTags() {
return targetAllocationTags;
}
@Override
public void initialize(AppSchedulingInfo appSchedulingInfo,
SchedulerRequestKey schedulerRequestKey, RMContext rmContext) {
super.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
this.allocationTagsManager = rmContext.getAllocationTagsManager();
}
}

View File

@ -331,8 +331,7 @@ public synchronized List<Container> getResources() throws IOException {
// Get resources from the ResourceManager
Allocation allocation = resourceManager.getResourceScheduler().allocate(
applicationAttemptId, new ArrayList<ResourceRequest>(ask),
new ArrayList<ContainerId>(), null, null,
applicationAttemptId, new ArrayList<ResourceRequest>(ask), null, new ArrayList<ContainerId>(), null, null,
new ContainerUpdates());
if (LOG.isInfoEnabled()) {
@ -431,7 +430,7 @@ private void updateResourceRequests(Map<String, ResourceRequest> requests,
if (type == NodeType.NODE_LOCAL) {
for (String host : task.getHosts()) {
if(LOG.isDebugEnabled()) {
LOG.debug("updatePendingAsk:" + " application=" + applicationId
LOG.debug("updateResourceDemands:" + " application=" + applicationId
+ " type=" + type + " host=" + host
+ " request=" + ((requests == null) ? "null" : requests.get(host)));
}
@ -442,7 +441,7 @@ private void updateResourceRequests(Map<String, ResourceRequest> requests,
if (type == NodeType.NODE_LOCAL || type == NodeType.RACK_LOCAL) {
for (String rack : task.getRacks()) {
if(LOG.isDebugEnabled()) {
LOG.debug("updatePendingAsk:" + " application=" + applicationId
LOG.debug("updateResourceDemands:" + " application=" + applicationId
+ " type=" + type + " rack=" + rack
+ " request=" + ((requests == null) ? "null" : requests.get(rack)));
}
@ -453,7 +452,7 @@ private void updateResourceRequests(Map<String, ResourceRequest> requests,
updateResourceRequest(requests.get(ResourceRequest.ANY));
if(LOG.isDebugEnabled()) {
LOG.debug("updatePendingAsk:" + " application=" + applicationId
LOG.debug("updateResourceDemands:" + " application=" + applicationId
+ " #asks=" + ask.size());
}
}

View File

@ -21,6 +21,7 @@
import java.lang.reflect.UndeclaredThrowableException;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -37,14 +38,17 @@
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.ExecutionType;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
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.api.records.ExecutionTypeRequest;
import org.apache.hadoop.yarn.api.records.ResourceSizing;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@ -281,6 +285,53 @@ public AllocateResponse allocate(
}
return allocate(req);
}
public AllocateResponse allocate(List<ResourceRequest> resourceRequest,
List<SchedulingRequest> newSchedulingRequests, List<ContainerId> releases)
throws Exception {
final AllocateRequest req =
AllocateRequest.newInstance(0, 0F, resourceRequest,
releases, null);
if (newSchedulingRequests != null) {
addSchedulingRequest(newSchedulingRequests);
}
if (!schedulingRequests.isEmpty()) {
req.setSchedulingRequests(schedulingRequests);
schedulingRequests.clear();
}
return allocate(req);
}
public AllocateResponse allocateIntraAppAntiAffinity(
ResourceSizing resourceSizing, Priority priority, long allocationId,
Set<String> allocationTags, String... targetTags) throws Exception {
return this.allocate(null,
Arrays.asList(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(allocationId).priority(priority)
.allocationTags(allocationTags).placementConstraintExpression(
PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp(targetTags)).build())
.resourceSizing(resourceSizing).build()), null);
}
public AllocateResponse allocateIntraAppAntiAffinity(
String nodePartition, ResourceSizing resourceSizing, Priority priority,
long allocationId, String... tags) throws Exception {
return this.allocate(null,
Arrays.asList(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(allocationId).priority(priority)
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp(tags),
PlacementConstraints.PlacementTargets
.nodePartition(nodePartition)).build())
.resourceSizing(resourceSizing).build()), null);
}
public AllocateResponse sendContainerResizingRequest(
List<UpdateContainerRequest> updateRequests) throws Exception {

View File

@ -474,7 +474,7 @@ private void testAppAttemptScheduledState() {
assertEquals(expectedState, applicationAttempt.getAppAttemptState());
verify(scheduler, times(expectedAllocateCount)).allocate(
any(ApplicationAttemptId.class), any(List.class), any(List.class),
any(ApplicationAttemptId.class), any(List.class), eq(null), any(List.class),
any(List.class), any(List.class), any(ContainerUpdates.class));
assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
@ -495,7 +495,7 @@ private void testAppAttemptAllocatedState(Container amContainer) {
// Check events
verify(applicationMasterLauncher).handle(any(AMLauncherEvent.class));
verify(scheduler, times(2)).allocate(any(ApplicationAttemptId.class),
any(List.class), any(List.class), any(List.class), any(List.class),
any(List.class), any(List.class), any(List.class), any(List.class), any(List.class),
any(ContainerUpdates.class));
verify(nmTokenManager).clearNodeSetForAttempt(
applicationAttempt.getAppAttemptId());
@ -643,7 +643,7 @@ private Container allocateApplicationAttempt() {
when(allocation.getContainers()).
thenReturn(Collections.singletonList(container));
when(scheduler.allocate(any(ApplicationAttemptId.class), any(List.class),
any(List.class), any(List.class), any(List.class),
any(List.class), any(List.class), any(List.class), any(List.class),
any(ContainerUpdates.class))).
thenReturn(allocation);
RMContainer rmContainer = mock(RMContainerImpl.class);
@ -1161,7 +1161,7 @@ public void testLaunchedFailWhileAHSEnabled() {
when(allocation.getContainers()).
thenReturn(Collections.singletonList(amContainer));
when(scheduler.allocate(any(ApplicationAttemptId.class), any(List.class),
any(List.class), any(List.class), any(List.class),
any(List.class), any(List.class), any(List.class), any(List.class),
any(ContainerUpdates.class)))
.thenReturn(allocation);
RMContainer rmContainer = mock(RMContainerImpl.class);
@ -1636,7 +1636,7 @@ scheduler, masterService, submissionContext, new Configuration(),
public void testScheduleTransitionReplaceAMContainerRequestWithDefaults() {
YarnScheduler mockScheduler = mock(YarnScheduler.class);
when(mockScheduler.allocate(any(ApplicationAttemptId.class),
any(List.class), any(List.class), any(List.class), any(List.class),
any(List.class), any(List.class), any(List.class), any(List.class), any(List.class),
any(ContainerUpdates.class)))
.thenAnswer(new Answer<Allocation>() {

View File

@ -33,6 +33,7 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import com.google.common.collect.ImmutableSet;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -420,9 +421,10 @@ public void testContainerTransitionNotifyPlacementTagsManager()
when(rmContext.getYarnConfiguration()).thenReturn(conf);
/* First container: ALLOCATED -> KILLED */
RMContainer rmContainer = new RMContainerImpl(container,
RMContainerImpl rmContainer = new RMContainerImpl(container,
SchedulerRequestKey.extractFrom(container), appAttemptId,
nodeId, "user", rmContext);
rmContainer.setAllocationTags(ImmutableSet.of("mapper"));
Assert.assertEquals(0,
tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
@ -448,6 +450,7 @@ public void testContainerTransitionNotifyPlacementTagsManager()
Assert.assertEquals(0,
tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));
rmContainer.setAllocationTags(ImmutableSet.of("mapper"));
rmContainer.handle(new RMContainerEvent(containerId,
RMContainerEventType.START));
@ -468,6 +471,7 @@ public void testContainerTransitionNotifyPlacementTagsManager()
rmContainer = new RMContainerImpl(container,
SchedulerRequestKey.extractFrom(container), appAttemptId,
nodeId, "user", rmContext);
rmContainer.setAllocationTags(ImmutableSet.of("mapper"));
Assert.assertEquals(0,
tagsManager.getNodeCardinalityByOp(nodeId, appId, null, Long::max));

View File

@ -46,7 +46,7 @@ public void testBacklistChanged() {
doReturn("test").when(queue).getQueueName();
AppSchedulingInfo appSchedulingInfo = new AppSchedulingInfo(appAttemptId,
"test", queue, null, 0, new ResourceUsage(),
new HashMap<String, String>());
new HashMap<String, String>(), null);
appSchedulingInfo.updatePlacesBlacklistedByApp(new ArrayList<String>(),
new ArrayList<String>());
@ -118,7 +118,7 @@ public void testSchedulerKeyAccounting() {
doReturn(mock(QueueMetrics.class)).when(queue).getMetrics();
AppSchedulingInfo info = new AppSchedulingInfo(
appAttemptId, "test", queue, mock(ActiveUsersManager.class), 0,
new ResourceUsage(), new HashMap<String, String>());
new ResourceUsage(), new HashMap<>(), null);
Assert.assertEquals(0, info.getSchedulerKeys().size());
Priority pri1 = Priority.newInstance(1);

View File

@ -0,0 +1,79 @@
/**
* 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 com.google.common.collect.Sets;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.junit.Assert;
import java.util.Set;
public class CapacitySchedulerTestBase {
protected final int GB = 1024;
protected static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
protected static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
protected static final String A1 = A + ".a1";
protected static final String A2 = A + ".a2";
protected static final String B1 = B + ".b1";
protected static final String B2 = B + ".b2";
protected static final String B3 = B + ".b3";
protected static float A_CAPACITY = 10.5f;
protected static float B_CAPACITY = 89.5f;
protected static final String P1 = CapacitySchedulerConfiguration.ROOT + ".p1";
protected static final String P2 = CapacitySchedulerConfiguration.ROOT + ".p2";
protected static final String X1 = P1 + ".x1";
protected static final String X2 = P1 + ".x2";
protected static final String Y1 = P2 + ".y1";
protected static final String Y2 = P2 + ".y2";
protected static float A1_CAPACITY = 30;
protected static float A2_CAPACITY = 70;
protected static float B1_CAPACITY = 79.2f;
protected static float B2_CAPACITY = 0.8f;
protected static float B3_CAPACITY = 20;
@SuppressWarnings("unchecked")
protected <E> Set<E> toSet(E... elements) {
Set<E> set = Sets.newHashSet(elements);
return set;
}
protected void checkPendingResource(MockRM rm, String queueName, int memory,
String label) {
CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
CSQueue queue = cs.getQueue(queueName);
Assert.assertEquals(
memory,
queue.getQueueResourceUsage()
.getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label)
.getMemorySize());
}
protected void checkPendingResourceGreaterThanZero(MockRM rm, String queueName,
String label) {
CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
CSQueue queue = cs.getQueue(queueName);
Assert.assertTrue(queue.getQueueResourceUsage()
.getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label)
.getMemorySize() > 0);
}
}

View File

@ -103,7 +103,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager;
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.placement.UserGroupMappingPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
@ -167,33 +166,10 @@
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
public class TestCapacityScheduler {
public class TestCapacityScheduler extends CapacitySchedulerTestBase {
private static final Log LOG = LogFactory.getLog(TestCapacityScheduler.class);
private final int GB = 1024;
private final static ContainerUpdates NULL_UPDATE_REQUESTS =
new ContainerUpdates();
private static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
private static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
private static final String A1 = A + ".a1";
private static final String A2 = A + ".a2";
private static final String B1 = B + ".b1";
private static final String B2 = B + ".b2";
private static final String B3 = B + ".b3";
private static float A_CAPACITY = 10.5f;
private static float B_CAPACITY = 89.5f;
private static final String P1 = CapacitySchedulerConfiguration.ROOT + ".p1";
private static final String P2 = CapacitySchedulerConfiguration.ROOT + ".p2";
private static final String X1 = P1 + ".x1";
private static final String X2 = P1 + ".x2";
private static final String Y1 = P2 + ".y1";
private static final String Y2 = P2 + ".y2";
private static float A1_CAPACITY = 30;
private static float A2_CAPACITY = 70;
private static float B1_CAPACITY = 79.2f;
private static float B2_CAPACITY = 0.8f;
private static float B3_CAPACITY = 20;
private ResourceManager resourceManager = null;
private RMContext mockContext;
@ -1116,12 +1092,12 @@ public void testBlackListNodes() throws Exception {
cs.handle(addAttemptEvent);
// Verify the blacklist can be updated independent of requesting containers
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), null,
Collections.<ContainerId>emptyList(),
Collections.singletonList(host), null, NULL_UPDATE_REQUESTS);
Assert.assertTrue(cs.getApplicationAttempt(appAttemptId)
.isPlaceBlacklisted(host));
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
cs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(), null,
Collections.<ContainerId>emptyList(), null,
Collections.singletonList(host), NULL_UPDATE_REQUESTS);
Assert.assertFalse(cs.getApplicationAttempt(appAttemptId)
@ -1217,8 +1193,7 @@ public void testAllocateReorder() throws Exception {
//This will allocate for app1
cs.allocate(appAttemptId1,
Collections.<ResourceRequest>singletonList(r1),
Collections.<ContainerId>emptyList(),
Collections.<ResourceRequest>singletonList(r1), null, Collections.<ContainerId>emptyList(),
null, null, NULL_UPDATE_REQUESTS);
//And this will result in container assignment for app1
@ -1234,8 +1209,7 @@ public void testAllocateReorder() throws Exception {
//Now, allocate for app2 (this would be the first/AM allocation)
ResourceRequest r2 = TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1, true, priority, recordFactory);
cs.allocate(appAttemptId2,
Collections.<ResourceRequest>singletonList(r2),
Collections.<ContainerId>emptyList(),
Collections.<ResourceRequest>singletonList(r2), null, Collections.<ContainerId>emptyList(),
null, null, NULL_UPDATE_REQUESTS);
//In this case we do not perform container assignment because we want to
@ -3481,12 +3455,6 @@ public void testParentQueueMaxCapsAreRespected() throws Exception {
+ "queue-a's max capacity will be violated if container allocated");
}
@SuppressWarnings("unchecked")
private <E> Set<E> toSet(E... elements) {
Set<E> set = Sets.newHashSet(elements);
return set;
}
@Test
public void testQueueHierarchyPendingResourceUpdate() throws Exception {
Configuration conf =
@ -3618,26 +3586,6 @@ protected RMNodeLabelsManager createNodeLabelManager() {
checkPendingResource(rm, "root", 0 * GB, "x");
}
private void checkPendingResource(MockRM rm, String queueName, int memory,
String label) {
CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
CSQueue queue = cs.getQueue(queueName);
Assert.assertEquals(
memory,
queue.getQueueResourceUsage()
.getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label)
.getMemorySize());
}
private void checkPendingResourceGreaterThanZero(MockRM rm, String queueName,
String label) {
CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
CSQueue queue = cs.getQueue(queueName);
Assert.assertTrue(queue.getQueueResourceUsage()
.getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label)
.getMemorySize() > 0);
}
// Test verifies AM Used resource for LeafQueue when AM ResourceRequest is
// lesser than minimumAllocation
@Test(timeout = 30000)
@ -3707,7 +3655,7 @@ public void testApplicationHeadRoom() throws Exception {
Allocation allocate =
cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
Collections.<ContainerId> emptyList(), null, null,
null, Collections.<ContainerId> emptyList(), null, null,
NULL_UPDATE_REQUESTS);
Assert.assertNotNull(attempt);
@ -3724,7 +3672,7 @@ Collections.<ContainerId> emptyList(), null, null,
allocate =
cs.allocate(appAttemptId, Collections.<ResourceRequest> emptyList(),
Collections.<ContainerId> emptyList(), null, null,
null, Collections.<ContainerId> emptyList(), null, null,
NULL_UPDATE_REQUESTS);
// All resources should be sent as headroom
@ -4250,8 +4198,7 @@ public void testCSReservationWithRootUnblocked() throws Exception {
y1Req = TestUtils.createResourceRequest(
ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
cs.allocate(appAttemptId3,
Collections.<ResourceRequest>singletonList(y1Req),
Collections.<ContainerId>emptyList(),
Collections.<ResourceRequest>singletonList(y1Req), null, Collections.<ContainerId>emptyList(),
null, null, NULL_UPDATE_REQUESTS);
CapacityScheduler.schedule(cs);
}
@ -4264,8 +4211,7 @@ public void testCSReservationWithRootUnblocked() throws Exception {
x1Req = TestUtils.createResourceRequest(
ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
cs.allocate(appAttemptId1,
Collections.<ResourceRequest>singletonList(x1Req),
Collections.<ContainerId>emptyList(),
Collections.<ResourceRequest>singletonList(x1Req), null, Collections.<ContainerId>emptyList(),
null, null, NULL_UPDATE_REQUESTS);
CapacityScheduler.schedule(cs);
}
@ -4277,8 +4223,7 @@ public void testCSReservationWithRootUnblocked() throws Exception {
x2Req = TestUtils.createResourceRequest(
ResourceRequest.ANY, 2 * GB, 1, true, priority, recordFactory);
cs.allocate(appAttemptId2,
Collections.<ResourceRequest>singletonList(x2Req),
Collections.<ContainerId>emptyList(),
Collections.<ResourceRequest>singletonList(x2Req), null, Collections.<ContainerId>emptyList(),
null, null, NULL_UPDATE_REQUESTS);
CapacityScheduler.schedule(cs);
assertEquals("X2 Used Resource should be 0", 0,
@ -4289,8 +4234,7 @@ public void testCSReservationWithRootUnblocked() throws Exception {
x1Req = TestUtils.createResourceRequest(
ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
cs.allocate(appAttemptId1,
Collections.<ResourceRequest>singletonList(x1Req),
Collections.<ContainerId>emptyList(),
Collections.<ResourceRequest>singletonList(x1Req), null, Collections.<ContainerId>emptyList(),
null, null, NULL_UPDATE_REQUESTS);
CapacityScheduler.schedule(cs);
assertEquals("X1 Used Resource should be 7 GB", 7 * GB,
@ -4303,8 +4247,7 @@ public void testCSReservationWithRootUnblocked() throws Exception {
y1Req = TestUtils.createResourceRequest(
ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
cs.allocate(appAttemptId3,
Collections.<ResourceRequest>singletonList(y1Req),
Collections.<ContainerId>emptyList(),
Collections.<ResourceRequest>singletonList(y1Req), null, Collections.<ContainerId>emptyList(),
null, null, NULL_UPDATE_REQUESTS);
CapacityScheduler.schedule(cs);
}
@ -4363,7 +4306,7 @@ public void testCSQueueBlocked() throws Exception {
ResourceRequest.ANY, 2 * GB, 1, true, priority, recordFactory);
//This will allocate for app1
cs.allocate(appAttemptId1, Collections.<ResourceRequest>singletonList(r1),
Collections.<ContainerId>emptyList(),
null, Collections.<ContainerId>emptyList(),
null, null, NULL_UPDATE_REQUESTS).getContainers().size();
CapacityScheduler.schedule(cs);
ResourceRequest r2 = null;
@ -4371,8 +4314,7 @@ public void testCSQueueBlocked() throws Exception {
r2 = TestUtils.createResourceRequest(
ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
cs.allocate(appAttemptId2,
Collections.<ResourceRequest>singletonList(r2),
Collections.<ContainerId>emptyList(),
Collections.<ResourceRequest>singletonList(r2), null, Collections.<ContainerId>emptyList(),
null, null, NULL_UPDATE_REQUESTS);
CapacityScheduler.schedule(cs);
}
@ -4385,12 +4327,12 @@ public void testCSQueueBlocked() throws Exception {
r2 = TestUtils.createResourceRequest(
ResourceRequest.ANY, 1 * GB, 1, true, priority, recordFactory);
cs.allocate(appAttemptId1, Collections.<ResourceRequest>singletonList(r1),
Collections.<ContainerId>emptyList(),
null, Collections.<ContainerId>emptyList(),
null, null, NULL_UPDATE_REQUESTS).getContainers().size();
CapacityScheduler.schedule(cs);
cs.allocate(appAttemptId2, Collections.<ResourceRequest>singletonList(r2),
Collections.<ContainerId>emptyList(), null, null, NULL_UPDATE_REQUESTS);
null, Collections.<ContainerId>emptyList(), null, null, NULL_UPDATE_REQUESTS);
CapacityScheduler.schedule(cs);
//Check blocked Resource
assertEquals("A Used Resource should be 2 GB", 2 * GB,

View File

@ -106,7 +106,7 @@ public void testAsyncContainerAllocation(int numThreads) throws Exception {
CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_THREAD,
numThreads);
conf.setInt(CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_PREFIX
+ ".scheduling-interval-ms", 100);
+ ".scheduling-interval-ms", 0);
final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
mgr.init(conf);

View File

@ -170,7 +170,7 @@ public void testReinitializeStoppedAutoCreatedLeafQueue() throws Exception {
1 * GB, 1, true, priority, recordFactory);
cs.allocate(appAttemptId, Collections.<ResourceRequest>singletonList(r1),
Collections.<ContainerId>emptyList(), Collections.singletonList(host),
null, Collections.<ContainerId>emptyList(), Collections.singletonList(host),
null, NULL_UPDATE_REQUESTS);
//And this will result in container assignment for app1

View File

@ -0,0 +1,260 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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 com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.ResourceSizing;
import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
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.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.Test;
import java.util.Arrays;
public class TestCapacitySchedulerSchedulingRequestUpdate
extends CapacitySchedulerTestBase {
@Test
public void testBasicPendingResourceUpdate() throws Exception {
Configuration conf = TestUtils.getConfigurationWithQueueLabels(
new Configuration(false));
conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
mgr.init(conf);
mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y"));
mgr.addLabelsToNode(
ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
MockRM rm = new MockRM(conf) {
protected RMNodeLabelsManager createNodeLabelManager() {
return mgr;
}
};
rm.start();
MockNM nm1 = // label = x
new MockNM("h1:1234", 200 * GB, rm.getResourceTrackerService());
nm1.registerNode();
MockNM nm2 = // label = ""
new MockNM("h2:1234", 200 * GB, rm.getResourceTrackerService());
nm2.registerNode();
// Launch app1 in queue=a1
RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1");
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
// Launch app2 in queue=b1
RMApp app2 = rm.submitApp(8 * GB, "app", "user", null, "b1");
MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
// am1 asks for 8 * 1GB container for no label
am1.allocateIntraAppAntiAffinity(
ResourceSizing.newInstance(8, Resource.newInstance(1 * GB, 1)),
Priority.newInstance(1), 0, ImmutableSet.of("mapper", "reducer"),
"mapper", "reducer");
checkPendingResource(rm, "a1", 8 * GB, null);
checkPendingResource(rm, "a", 8 * GB, null);
checkPendingResource(rm, "root", 8 * GB, null);
// am2 asks for 8 * 1GB container for no label
am2.allocate(Arrays.asList(ResourceRequest
.newInstance(Priority.newInstance(1), "*",
Resources.createResource(1 * GB), 8)), null);
checkPendingResource(rm, "a1", 8 * GB, null);
checkPendingResource(rm, "a", 8 * GB, null);
checkPendingResource(rm, "b1", 8 * GB, null);
checkPendingResource(rm, "b", 8 * GB, null);
// root = a + b
checkPendingResource(rm, "root", 16 * GB, null);
// am2 asks for 8 * 1GB container in another priority for no label
am2.allocate(Arrays.asList(ResourceRequest
.newInstance(Priority.newInstance(2), "*",
Resources.createResource(1 * GB), 8)), null);
checkPendingResource(rm, "a1", 8 * GB, null);
checkPendingResource(rm, "a", 8 * GB, null);
checkPendingResource(rm, "b1", 16 * GB, null);
checkPendingResource(rm, "b", 16 * GB, null);
// root = a + b
checkPendingResource(rm, "root", 24 * GB, null);
// am1 asks 4 GB resource instead of 8 * GB for priority=1
// am1 asks for 8 * 1GB container for no label
am1.allocateIntraAppAntiAffinity(
ResourceSizing.newInstance(4, Resource.newInstance(1 * GB, 1)),
Priority.newInstance(1), 0, ImmutableSet.of("mapper", "reducer"),
"mapper", "reducer");
checkPendingResource(rm, "a1", 4 * GB, null);
checkPendingResource(rm, "a", 4 * GB, null);
checkPendingResource(rm, "b1", 16 * GB, null);
checkPendingResource(rm, "b", 16 * GB, null);
// root = a + b
checkPendingResource(rm, "root", 20 * GB, null);
// am1 asks 8 * GB resource which label=x
am1.allocate(Arrays.asList(ResourceRequest
.newInstance(Priority.newInstance(2), "*",
Resources.createResource(8 * GB), 1, true, "x")), null);
checkPendingResource(rm, "a1", 4 * GB, null);
checkPendingResource(rm, "a", 4 * GB, null);
checkPendingResource(rm, "a1", 8 * GB, "x");
checkPendingResource(rm, "a", 8 * GB, "x");
checkPendingResource(rm, "b1", 16 * GB, null);
checkPendingResource(rm, "b", 16 * GB, null);
// root = a + b
checkPendingResource(rm, "root", 20 * GB, null);
checkPendingResource(rm, "root", 8 * GB, "x");
// complete am1/am2, pending resource should be 0 now
AppAttemptRemovedSchedulerEvent appRemovedEvent =
new AppAttemptRemovedSchedulerEvent(am2.getApplicationAttemptId(),
RMAppAttemptState.FINISHED, false);
rm.getResourceScheduler().handle(appRemovedEvent);
appRemovedEvent = new AppAttemptRemovedSchedulerEvent(
am1.getApplicationAttemptId(), RMAppAttemptState.FINISHED, false);
rm.getResourceScheduler().handle(appRemovedEvent);
checkPendingResource(rm, "a1", 0 * GB, null);
checkPendingResource(rm, "a", 0 * GB, null);
checkPendingResource(rm, "a1", 0 * GB, "x");
checkPendingResource(rm, "a", 0 * GB, "x");
checkPendingResource(rm, "b1", 0 * GB, null);
checkPendingResource(rm, "b", 0 * GB, null);
checkPendingResource(rm, "root", 0 * GB, null);
checkPendingResource(rm, "root", 0 * GB, "x");
}
@Test
public void testNodePartitionPendingResourceUpdate() throws Exception {
Configuration conf = TestUtils.getConfigurationWithQueueLabels(
new Configuration(false));
conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
mgr.init(conf);
mgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x", "y"));
mgr.addLabelsToNode(
ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
MockRM rm = new MockRM(conf) {
protected RMNodeLabelsManager createNodeLabelManager() {
return mgr;
}
};
rm.start();
MockNM nm1 = // label = x
new MockNM("h1:1234", 200 * GB, rm.getResourceTrackerService());
nm1.registerNode();
MockNM nm2 = // label = ""
new MockNM("h2:1234", 200 * GB, rm.getResourceTrackerService());
nm2.registerNode();
// Launch app1 in queue=a1
RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1");
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
// Launch app2 in queue=b1
RMApp app2 = rm.submitApp(8 * GB, "app", "user", null, "b1");
MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
// am1 asks for 8 * 1GB container for "x"
am1.allocateIntraAppAntiAffinity("x",
ResourceSizing.newInstance(8, Resource.newInstance(1 * GB, 1)),
Priority.newInstance(1), 0, "mapper", "reducer");
checkPendingResource(rm, "a1", 8 * GB, "x");
checkPendingResource(rm, "a", 8 * GB, "x");
checkPendingResource(rm, "root", 8 * GB, "x");
// am2 asks for 8 * 1GB container for "x"
am2.allocateIntraAppAntiAffinity("x",
ResourceSizing.newInstance(8, Resource.newInstance(1 * GB, 1)),
Priority.newInstance(1), 0, "mapper", "reducer");
checkPendingResource(rm, "a1", 8 * GB, "x");
checkPendingResource(rm, "a", 8 * GB, "x");
checkPendingResource(rm, "b1", 8 * GB, "x");
checkPendingResource(rm, "b", 8 * GB, "x");
// root = a + b
checkPendingResource(rm, "root", 16 * GB, "x");
// am1 asks for 6 * 1GB container for "x" in another priority
am1.allocateIntraAppAntiAffinity("x",
ResourceSizing.newInstance(6, Resource.newInstance(1 * GB, 1)),
Priority.newInstance(2), 0, "mapper", "reducer");
checkPendingResource(rm, "a1", 14 * GB, "x");
checkPendingResource(rm, "a", 14 * GB, "x");
checkPendingResource(rm, "b1", 8 * GB, "x");
checkPendingResource(rm, "b", 8 * GB, "x");
// root = a + b
checkPendingResource(rm, "root", 22 * GB, "x");
// am1 asks for 4 * 1GB container for "x" in priority=1, which should
// override 8 * 1GB
am1.allocateIntraAppAntiAffinity("x",
ResourceSizing.newInstance(4, Resource.newInstance(1 * GB, 1)),
Priority.newInstance(1), 0, "mapper", "reducer");
checkPendingResource(rm, "a1", 10 * GB, "x");
checkPendingResource(rm, "a", 10 * GB, "x");
checkPendingResource(rm, "b1", 8 * GB, "x");
checkPendingResource(rm, "b", 8 * GB, "x");
// root = a + b
checkPendingResource(rm, "root", 18 * GB, "x");
// complete am1/am2, pending resource should be 0 now
AppAttemptRemovedSchedulerEvent appRemovedEvent =
new AppAttemptRemovedSchedulerEvent(am2.getApplicationAttemptId(),
RMAppAttemptState.FINISHED, false);
rm.getResourceScheduler().handle(appRemovedEvent);
appRemovedEvent = new AppAttemptRemovedSchedulerEvent(
am1.getApplicationAttemptId(), RMAppAttemptState.FINISHED, false);
rm.getResourceScheduler().handle(appRemovedEvent);
checkPendingResource(rm, "a1", 0 * GB, null);
checkPendingResource(rm, "a", 0 * GB, null);
checkPendingResource(rm, "a1", 0 * GB, "x");
checkPendingResource(rm, "a", 0 * GB, "x");
checkPendingResource(rm, "b1", 0 * GB, null);
checkPendingResource(rm, "b", 0 * GB, null);
checkPendingResource(rm, "root", 0 * GB, null);
checkPendingResource(rm, "root", 0 * GB, "x");
}
}

View File

@ -132,7 +132,7 @@ public void testContainerIsRemovedFromAllocationExpirer()
Assert.assertEquals(RMContainerState.RUNNING,
rm1.getResourceScheduler().getRMContainer(containerId2).getState());
// Verify container size is 3G
Assert.assertEquals(
Assert.assertEquals(
3 * GB, rm1.getResourceScheduler().getRMContainer(containerId2)
.getAllocatedResource().getMemorySize());
// Verify total resource usage

View File

@ -0,0 +1,277 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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 com.google.common.collect.ImmutableSet;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceSizing;
import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
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.ResourceManager;
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.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class TestSchedulingRequestContainerAllocation {
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);
mgr = new NullRMNodeLabelsManager();
mgr.init(conf);
}
@Test
public void testIntraAppAntiAffinity() throws Exception {
Configuration csConf = TestUtils.getConfigurationWithMultipleQueues(
new Configuration());
csConf.setBoolean(CapacitySchedulerConfiguration.SCHEDULING_REQUEST_ALLOWED,
true);
// inject node label manager
MockRM rm1 = new MockRM(csConf) {
@Override
public RMNodeLabelsManager createNodeLabelManager() {
return mgr;
}
};
rm1.getRMContext().setNodeLabelManager(mgr);
rm1.start();
// 4 NMs.
MockNM[] nms = new MockNM[4];
RMNode[] rmNodes = new RMNode[4];
for (int i = 0; i < 4; i++) {
nms[i] = rm1.registerNode("192.168.0." + i + ":1234", 10 * GB);
rmNodes[i] = rm1.getRMContext().getRMNodes().get(nms[i].getNodeId());
}
// app1 -> c
RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "c");
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nms[0]);
// app1 asks for 10 anti-affinity containers for the same app. It should
// only get 4 containers allocated because we only have 4 nodes.
am1.allocateIntraAppAntiAffinity(
ResourceSizing.newInstance(10, Resource.newInstance(1024, 1)),
Priority.newInstance(1), 1L, ImmutableSet.of("mapper"), "mapper");
CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
for (int i = 0; i < 3; i++) {
for (int j = 0; j < 4; j++) {
cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
}
}
// App1 should get 5 containers allocated (1 AM + 1 node each).
FiCaSchedulerApp schedulerApp = cs.getApplicationAttempt(
am1.getApplicationAttemptId());
Assert.assertEquals(5, schedulerApp.getLiveContainers().size());
// Similarly, app1 asks 10 anti-affinity containers at different priority,
// it should be satisfied as well.
// app1 asks for 10 anti-affinity containers for the same app. It should
// only get 4 containers allocated because we only have 4 nodes.
am1.allocateIntraAppAntiAffinity(
ResourceSizing.newInstance(10, Resource.newInstance(2048, 1)),
Priority.newInstance(2), 1L, ImmutableSet.of("reducer"), "reducer");
for (int i = 0; i < 3; i++) {
for (int j = 0; j < 4; j++) {
cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
}
}
// App1 should get 9 containers allocated (1 AM + 8 containers).
Assert.assertEquals(9, schedulerApp.getLiveContainers().size());
// Test anti-affinity to both of "mapper/reducer", we should only get no
// container allocated
am1.allocateIntraAppAntiAffinity(
ResourceSizing.newInstance(10, Resource.newInstance(2048, 1)),
Priority.newInstance(3), 1L, ImmutableSet.of("reducer2"), "mapper");
for (int i = 0; i < 3; i++) {
for (int j = 0; j < 4; j++) {
cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
}
}
// App1 should get 10 containers allocated (1 AM + 9 containers).
Assert.assertEquals(9, schedulerApp.getLiveContainers().size());
rm1.close();
}
@Test
public void testIntraAppAntiAffinityWithMultipleTags() throws Exception {
Configuration csConf = TestUtils.getConfigurationWithMultipleQueues(
new Configuration());
csConf.setBoolean(CapacitySchedulerConfiguration.SCHEDULING_REQUEST_ALLOWED,
true);
// inject node label manager
MockRM rm1 = new MockRM(csConf) {
@Override
public RMNodeLabelsManager createNodeLabelManager() {
return mgr;
}
};
rm1.getRMContext().setNodeLabelManager(mgr);
rm1.start();
// 4 NMs.
MockNM[] nms = new MockNM[4];
RMNode[] rmNodes = new RMNode[4];
for (int i = 0; i < 4; i++) {
nms[i] = rm1.registerNode("192.168.0." + i + ":1234", 10 * GB);
rmNodes[i] = rm1.getRMContext().getRMNodes().get(nms[i].getNodeId());
}
// app1 -> c
RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "c");
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nms[0]);
// app1 asks for 2 anti-affinity containers for the same app.
am1.allocateIntraAppAntiAffinity(
ResourceSizing.newInstance(2, Resource.newInstance(1024, 1)),
Priority.newInstance(1), 1L, ImmutableSet.of("tag_1_1", "tag_1_2"),
"tag_1_1", "tag_1_2");
CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
for (int i = 0; i < 3; i++) {
for (int j = 0; j < 4; j++) {
cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
}
}
// App1 should get 3 containers allocated (1 AM + 2 task).
FiCaSchedulerApp schedulerApp = cs.getApplicationAttempt(
am1.getApplicationAttemptId());
Assert.assertEquals(3, schedulerApp.getLiveContainers().size());
// app1 asks for 1 anti-affinity containers for the same app. anti-affinity
// to tag_1_1/tag_1_2. With allocation_tag = tag_2_1/tag_2_2
am1.allocateIntraAppAntiAffinity(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)),
Priority.newInstance(2), 1L, ImmutableSet.of("tag_2_1", "tag_2_2"),
"tag_1_1", "tag_1_2");
for (int i = 0; i < 3; i++) {
for (int j = 0; j < 4; j++) {
cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
}
}
// App1 should get 4 containers allocated (1 AM + 2 task (first request) +
// 1 task (2nd request).
Assert.assertEquals(4, schedulerApp.getLiveContainers().size());
// app1 asks for 10 anti-affinity containers for the same app. anti-affinity
// to tag_1_1/tag_1_2/tag_2_1/tag_2_2. With allocation_tag = tag_3
am1.allocateIntraAppAntiAffinity(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)),
Priority.newInstance(3), 1L, ImmutableSet.of("tag_3"),
"tag_1_1", "tag_1_2", "tag_2_1", "tag_2_2");
for (int i = 0; i < 3; i++) {
for (int j = 0; j < 4; j++) {
cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
}
}
// App1 should get 1 more containers allocated
// 1 AM + 2 task (first request) + 1 task (2nd request) +
// 1 task (3rd request)
Assert.assertEquals(5, schedulerApp.getLiveContainers().size());
rm1.close();
}
@Test
public void testSchedulingRequestDisabledByDefault() throws Exception {
Configuration csConf = TestUtils.getConfigurationWithMultipleQueues(
new Configuration());
// inject node label manager
MockRM rm1 = new MockRM(csConf) {
@Override
public RMNodeLabelsManager createNodeLabelManager() {
return mgr;
}
};
rm1.getRMContext().setNodeLabelManager(mgr);
rm1.start();
// 4 NMs.
MockNM[] nms = new MockNM[4];
RMNode[] rmNodes = new RMNode[4];
for (int i = 0; i < 4; i++) {
nms[i] = rm1.registerNode("192.168.0." + i + ":1234", 10 * GB);
rmNodes[i] = rm1.getRMContext().getRMNodes().get(nms[i].getNodeId());
}
// app1 -> c
RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "c");
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nms[0]);
// app1 asks for 2 anti-affinity containers for the same app.
boolean caughtException = false;
try {
// Since feature is disabled by default, we should expect exception.
am1.allocateIntraAppAntiAffinity(
ResourceSizing.newInstance(2, Resource.newInstance(1024, 1)),
Priority.newInstance(1), 1L, ImmutableSet.of("tag_1_1", "tag_1_2"),
"tag_1_1", "tag_1_2");
} catch (Exception e) {
caughtException = true;
}
Assert.assertTrue(caughtException);
rm1.close();
}
}

View File

@ -0,0 +1,139 @@
/**
* 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 com.google.common.collect.ImmutableSet;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceSizing;
import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
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.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class TestSchedulingRequestContainerAllocationAsync {
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);
mgr = new NullRMNodeLabelsManager();
mgr.init(conf);
}
private void testIntraAppAntiAffinityAsync(int numThreads) throws Exception {
Configuration csConf = TestUtils.getConfigurationWithMultipleQueues(
new Configuration());
csConf.setBoolean(CapacitySchedulerConfiguration.SCHEDULING_REQUEST_ALLOWED,
true);
csConf.setInt(
CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_MAXIMUM_THREAD,
numThreads);
csConf.setInt(CapacitySchedulerConfiguration.SCHEDULE_ASYNCHRONOUSLY_PREFIX
+ ".scheduling-interval-ms", 0);
// inject node label manager
MockRM rm1 = new MockRM(csConf) {
@Override
public RMNodeLabelsManager createNodeLabelManager() {
return mgr;
}
};
rm1.getRMContext().setNodeLabelManager(mgr);
rm1.start();
// 200 NMs.
int nNMs = 200;
MockNM[] nms = new MockNM[nNMs];
RMNode[] rmNodes = new RMNode[nNMs];
for (int i = 0; i < nNMs; i++) {
nms[i] = rm1.registerNode("127.0.0." + i + ":1234", 10 * GB);
rmNodes[i] = rm1.getRMContext().getRMNodes().get(nms[i].getNodeId());
}
// app1 -> c
RMApp app1 = rm1.submitApp(1 * GB, "app", "user", null, "c");
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nms[0]);
// app1 asks for 10 anti-affinity containers for the same app. It should
// only get 4 containers allocated because we only have 4 nodes.
am1.allocateIntraAppAntiAffinity(
ResourceSizing.newInstance(1000, Resource.newInstance(1024, 1)),
Priority.newInstance(1), 1L, ImmutableSet.of("mapper"), "mapper");
CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
for (int i = 0; i < 3; i++) {
for (int j = 0; j < nNMs; j++) {
cs.handle(new NodeUpdateSchedulerEvent(rmNodes[j]));
}
}
// App1 should get #NM + 1 containers allocated (1 node each + 1 AM).
FiCaSchedulerApp schedulerApp = cs.getApplicationAttempt(
am1.getApplicationAttemptId());
Assert.assertEquals(nNMs + 1, schedulerApp.getLiveContainers().size());
rm1.close();
}
@Test(timeout = 300000)
public void testSingleThreadAsyncContainerAllocation() throws Exception {
testIntraAppAntiAffinityAsync(1);
}
@Test(timeout = 300000)
public void testTwoThreadsAsyncContainerAllocation() throws Exception {
testIntraAppAntiAffinityAsync(2);
}
@Test(timeout = 300000)
public void testThreeThreadsAsyncContainerAllocation() throws Exception {
testIntraAppAntiAffinityAsync(3);
}
@Test(timeout = 300000)
public void testFourThreadsAsyncContainerAllocation() throws Exception {
testIntraAppAntiAffinityAsync(4);
}
@Test(timeout = 300000)
public void testFiveThreadsAsyncContainerAllocation() throws Exception {
testIntraAppAntiAffinityAsync(5);
}
}

View File

@ -275,6 +275,8 @@ public static <E> Set<E> toSet(E... elements) {
public static Configuration getConfigurationWithQueueLabels(Configuration config) {
CapacitySchedulerConfiguration conf =
new CapacitySchedulerConfiguration(config);
conf.setBoolean(CapacitySchedulerConfiguration.SCHEDULING_REQUEST_ALLOWED,
true);
// Define top-level queues
conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"});

View File

@ -20,10 +20,10 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint;
import java.util.List;
import com.google.common.collect.ImmutableSet;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@ -33,7 +33,7 @@
import org.junit.Before;
import org.junit.Test;
import com.google.common.collect.ImmutableSet;
import java.util.List;
/**
* Test functionality of AllocationTagsManager.
@ -54,7 +54,6 @@ public void setup() {
rmContext = rm.getRMContext();
}
@Test
public void testAllocationTagsManagerSimpleCases()
throws InvalidAllocationTagsQueryException {
@ -141,30 +140,31 @@ public void testAllocationTagsManagerSimpleCases()
// Get Node Cardinality of app1 on node2, with tag "<applicationId>", op=max
// (Expect this returns #containers from app1 on node2)
Assert
.assertEquals(2,
atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
TestUtils.getMockApplicationId(1),
ImmutableSet.of(AllocationTagsNamespaces.APP_ID
+ TestUtils.getMockApplicationId(1).toString()),
Long::max));
Assert.assertEquals(2,
atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
TestUtils.getMockApplicationId(1), null, Long::max));
// Get Node Cardinality of app1 on node2, with empty tag set, op=max
Assert.assertEquals(2,
atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
TestUtils.getMockApplicationId(1), null, Long::max));
// Get Cardinality of app1 on node2, with empty tag set, op=max
Assert.assertEquals(2,
atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::max));
// Get Node Cardinality of all apps on node2, with empty tag set, op=sum
Assert.assertEquals(7, atm.getNodeCardinalityByOp(
Assert.assertEquals(4, atm.getNodeCardinalityByOp(
NodeId.fromString("host2:123"), null, ImmutableSet.of(), Long::sum));
// Get Node Cardinality of app_1 on node2, with empty tag set, op=sum
Assert.assertEquals(5,
Assert.assertEquals(3,
atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::sum));
// Get Node Cardinality of app_1 on node2, with empty tag set, op=sum
Assert.assertEquals(2,
Assert.assertEquals(1,
atm.getNodeCardinalityByOp(NodeId.fromString("host2:123"),
TestUtils.getMockApplicationId(2), ImmutableSet.of(), Long::sum));
@ -296,7 +296,7 @@ public void testAllocationTagsManagerRackMapping()
Assert.assertEquals(3, atm.getRackCardinality("rack0", null, "reducer"));
// Get Rack Cardinality of app_1 on rack0, with empty tag set, op=max
Assert.assertEquals(2, atm.getRackCardinalityByOp("rack0",
Assert.assertEquals(1, atm.getRackCardinalityByOp("rack0",
TestUtils.getMockApplicationId(1), ImmutableSet.of(), Long::max));
// Get Rack Cardinality of app_1 on rack0, with empty tag set, op=min

View File

@ -117,9 +117,9 @@ public void testNodeAffinityAssignment()
RMNode currentNode = nodeIterator.next();
FiCaSchedulerNode schedulerNode = TestUtils.getMockNode(
currentNode.getHostName(), currentNode.getRackName(), 123, 4 * GB);
Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag1, schedulerNode, pcm, tm));
Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag2, schedulerNode, pcm, tm));
}
/**
@ -145,14 +145,14 @@ public void testNodeAffinityAssignment()
tm.addContainer(n0_r1.getNodeID(), hbase_m, ImmutableSet.of("hbase-m"));
// 'spark' placement on Node0 should now SUCCEED
Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag1, schedulerNode0, pcm, tm));
// FAIL on the rest of the nodes
Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag1, schedulerNode1, pcm, tm));
Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag1, schedulerNode2, pcm, tm));
Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag1, schedulerNode3, pcm, tm));
}
@ -187,15 +187,15 @@ public void testRackAffinityAssignment()
FiCaSchedulerNode schedulerNode3 = TestUtils
.getMockNode(n3_r2.getHostName(), n3_r2.getRackName(), 123, 4 * GB);
// 'zk' placement on Rack1 should now SUCCEED
Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag2, schedulerNode0, pcm, tm));
Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag2, schedulerNode1, pcm, tm));
// FAIL on the rest of the RACKs
Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag2, schedulerNode2, pcm, tm));
Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag2, schedulerNode3, pcm, tm));
}
@ -230,14 +230,14 @@ public void testNodeAntiAffinityAssignment()
tm.addContainer(n0_r1.getNodeID(), hbase_m, ImmutableSet.of("hbase-m"));
// 'spark' placement on Node0 should now FAIL
Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag1, schedulerNode0, pcm, tm));
// SUCCEED on the rest of the nodes
Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag1, schedulerNode1, pcm, tm));
Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag1, schedulerNode2, pcm, tm));
Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag1, schedulerNode3, pcm, tm));
}
@ -273,15 +273,15 @@ public void testRackAntiAffinityAssignment()
.getMockNode(n3_r2.getHostName(), n3_r2.getRackName(), 123, 4 * GB);
// 'zk' placement on Rack1 should FAIL
Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag2, schedulerNode0, pcm, tm));
Assert.assertFalse(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertFalse(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag2, schedulerNode1, pcm, tm));
// SUCCEED on the rest of the RACKs
Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag2, schedulerNode2, pcm, tm));
Assert.assertTrue(PlacementConstraintsUtil.canSatisfyConstraints(appId1,
Assert.assertTrue(PlacementConstraintsUtil.canSatisfySingleConstraint(appId1,
sourceTag2, schedulerNode3, pcm, tm));
}
}

View File

@ -192,7 +192,7 @@ protected ApplicationAttemptId createSchedulingRequest(
resourceManager.getRMContext().getRMApps()
.put(id.getApplicationId(), rmApp);
scheduler.allocate(id, ask, new ArrayList<ContainerId>(),
scheduler.allocate(id, ask, null, new ArrayList<ContainerId>(),
null, null, NULL_UPDATE_REQUESTS);
scheduler.update();
return id;
@ -222,7 +222,7 @@ protected ApplicationAttemptId createSchedulingRequest(String queueId,
resourceManager.getRMContext().getRMApps()
.put(id.getApplicationId(), rmApp);
scheduler.allocate(id, ask, new ArrayList<ContainerId>(),
scheduler.allocate(id, ask, null, new ArrayList<ContainerId>(),
null, null, NULL_UPDATE_REQUESTS);
return id;
}
@ -245,7 +245,7 @@ protected void createSchedulingRequestExistingApplication(
ResourceRequest request, ApplicationAttemptId attId) {
List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
ask.add(request);
scheduler.allocate(attId, ask, new ArrayList<ContainerId>(),
scheduler.allocate(attId, ask, null, new ArrayList<ContainerId>(),
null, null, NULL_UPDATE_REQUESTS);
scheduler.update();
}

View File

@ -125,7 +125,7 @@ public void testBasic() throws InterruptedException {
List<ResourceRequest> ask = new ArrayList<>();
ask.add(createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true));
scheduler.allocate(
appAttemptId, ask, new ArrayList<ContainerId>(),
appAttemptId, ask, null, new ArrayList<ContainerId>(),
null, null, NULL_UPDATE_REQUESTS);
FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
@ -163,8 +163,7 @@ public void testSortedNodes() throws Exception {
ResourceRequest request =
createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
ask.add(request);
scheduler.allocate(appAttemptId, ask,
new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS);
scheduler.allocate(appAttemptId, ask, null, new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS);
triggerSchedulingAttempt();
FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
@ -175,8 +174,7 @@ public void testSortedNodes() throws Exception {
createResourceRequest(1024, 1, ResourceRequest.ANY, 2, 1, true);
ask.clear();
ask.add(request);
scheduler.allocate(appAttemptId, ask,
new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS);
scheduler.allocate(appAttemptId, ask, null, new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS);
triggerSchedulingAttempt();
checkAppConsumption(app, Resources.createResource(2048,2));
@ -373,7 +371,7 @@ public void testFairSchedulerContinuousSchedulingInitTime() throws Exception {
true);
ask1.add(request1);
ask1.add(request2);
scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null, null,
scheduler.allocate(id11, ask1, null, new ArrayList<ContainerId>(), null, null,
NULL_UPDATE_REQUESTS);
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);

View File

@ -1280,7 +1280,7 @@ public void testRackLocalAppReservationThreshold() throws Exception {
List<ResourceRequest> asks = new ArrayList<ResourceRequest>();
asks.add(createResourceRequest(2048, node2.getRackName(), 1, 1, false));
scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
scheduler.allocate(attemptId, asks, null, new ArrayList<ContainerId>(), null,
null, NULL_UPDATE_REQUESTS);
ApplicationAttemptId attId = createSchedulingRequest(2048, "queue1", "user1", 1);
@ -2125,7 +2125,7 @@ public void testQueueDemandCalculation() throws Exception {
ResourceRequest request1 = createResourceRequest(minReqSize * 2,
ResourceRequest.ANY, 1, 1, true);
ask1.add(request1);
scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(),
scheduler.allocate(id11, ask1, null, new ArrayList<ContainerId>(),
null, null, NULL_UPDATE_REQUESTS);
// Second ask, queue2 requests 1 large.
@ -2141,7 +2141,7 @@ public void testQueueDemandCalculation() throws Exception {
ResourceRequest.ANY, 1, 1, false);
ask2.add(request2);
ask2.add(request3);
scheduler.allocate(id21, ask2, new ArrayList<ContainerId>(),
scheduler.allocate(id21, ask2, null, new ArrayList<ContainerId>(),
null, null, NULL_UPDATE_REQUESTS);
// Third ask, queue2 requests 2 small (minReqSize).
@ -2157,7 +2157,7 @@ public void testQueueDemandCalculation() throws Exception {
ResourceRequest.ANY, 2, 2, true);
ask3.add(request4);
ask3.add(request5);
scheduler.allocate(id22, ask3, new ArrayList<ContainerId>(),
scheduler.allocate(id22, ask3, null, new ArrayList<ContainerId>(),
null, null, NULL_UPDATE_REQUESTS);
scheduler.update();
@ -2683,7 +2683,7 @@ public void testReservationWithMultiplePriorities() throws IOException {
// Complete the first container so we can trigger allocation for app2
ContainerId containerId =
app1.getLiveContainers().iterator().next().getContainerId();
scheduler.allocate(app1.getApplicationAttemptId(), new ArrayList<>(),
scheduler.allocate(app1.getApplicationAttemptId(), new ArrayList<>(), null,
Arrays.asList(containerId), null, null, NULL_UPDATE_REQUESTS);
// Trigger allocation for app2
@ -2769,7 +2769,7 @@ public void testMultipleNodesSingleRackRequest() throws Exception {
asks.add(createResourceRequest(1024, node3.getRackName(), 1, 1, true));
asks.add(createResourceRequest(1024, ResourceRequest.ANY, 1, 2, true));
scheduler.allocate(attemptId, asks, new ArrayList<ContainerId>(), null,
scheduler.allocate(attemptId, asks, null, new ArrayList<ContainerId>(), null,
null, NULL_UPDATE_REQUESTS);
// node 1 checks in
@ -3216,7 +3216,7 @@ public void testCancelStrictLocality() throws IOException {
createResourceRequest(1024, node1.getHostName(), 1, 0, true),
createResourceRequest(1024, "rack1", 1, 0, true),
createResourceRequest(1024, ResourceRequest.ANY, 1, 1, true));
scheduler.allocate(attId1, update, new ArrayList<ContainerId>(),
scheduler.allocate(attId1, update, null, new ArrayList<ContainerId>(),
null, null, NULL_UPDATE_REQUESTS);
// then node2 should get the container
@ -4432,7 +4432,7 @@ public void testSchedulingOnRemovedNode() throws Exception {
createResourceRequest(1024, 8, ResourceRequest.ANY, 1, 1, true);
ask1.add(request1);
scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(), null,
scheduler.allocate(id11, ask1, null, new ArrayList<ContainerId>(), null,
null, NULL_UPDATE_REQUESTS);
String hostName = "127.0.0.1";
@ -4508,11 +4508,11 @@ public void testBlacklistNodes() throws Exception {
// Verify the blacklist can be updated independent of requesting containers
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
Collections.<ContainerId>emptyList(),
null, Collections.<ContainerId>emptyList(),
Collections.singletonList(host), null, NULL_UPDATE_REQUESTS);
assertTrue(app.isPlaceBlacklisted(host));
scheduler.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
Collections.<ContainerId>emptyList(), null,
null, Collections.<ContainerId>emptyList(), null,
Collections.singletonList(host), NULL_UPDATE_REQUESTS);
assertFalse(scheduler.getSchedulerApp(appAttemptId)
.isPlaceBlacklisted(host));
@ -4521,8 +4521,7 @@ public void testBlacklistNodes() throws Exception {
createResourceRequest(GB, node.getHostName(), 1, 0, true));
// Verify a container does not actually get placed on the blacklisted host
scheduler.allocate(appAttemptId, update,
Collections.<ContainerId>emptyList(),
scheduler.allocate(appAttemptId, update, null, Collections.<ContainerId>emptyList(),
Collections.singletonList(host), null, NULL_UPDATE_REQUESTS);
assertTrue(app.isPlaceBlacklisted(host));
scheduler.update();
@ -4531,8 +4530,7 @@ public void testBlacklistNodes() throws Exception {
.getLiveContainers().size());
// Verify a container gets placed on the empty blacklist
scheduler.allocate(appAttemptId, update,
Collections.<ContainerId>emptyList(), null,
scheduler.allocate(appAttemptId, update, null, Collections.<ContainerId>emptyList(), null,
Collections.singletonList(host), NULL_UPDATE_REQUESTS);
assertFalse(app.isPlaceBlacklisted(host));
createSchedulingRequest(GB, "root.default", "user", 1);
@ -5391,8 +5389,8 @@ public void testCompletedContainerOnRemovedNode() throws IOException {
ask1.add(request3);
// Perform allocation
scheduler.allocate(appAttemptId, ask1, new ArrayList<ContainerId>(), null,
null, NULL_UPDATE_REQUESTS);
scheduler.allocate(appAttemptId, ask1, null, new ArrayList<ContainerId>(),
null, null, NULL_UPDATE_REQUESTS);
scheduler.update();
scheduler.handle(new NodeUpdateSchedulerEvent(node));

View File

@ -281,7 +281,7 @@ public void testNodeLocalAssignment() throws Exception {
ask.add(nodeLocal);
ask.add(rackLocal);
ask.add(any);
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(),
scheduler.allocate(appAttemptId, ask, null, new ArrayList<ContainerId>(),
null, null, NULL_UPDATE_REQUESTS);
NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0);
@ -378,7 +378,7 @@ public void testUpdateResourceOnNode() throws Exception {
ask.add(nodeLocal);
ask.add(rackLocal);
ask.add(any);
scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(),
scheduler.allocate(appAttemptId, ask, null, new ArrayList<ContainerId>(),
null, null, NULL_UPDATE_REQUESTS);
// Before the node update event, there are one local request
@ -954,7 +954,7 @@ public void testBlackListNodes() throws Exception {
ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1,
RMNodeLabelsManager.NO_LABEL));
fs.allocate(appAttemptId1, ask1, emptyId,
fs.allocate(appAttemptId1, ask1, null, emptyId,
Collections.singletonList(host_1_0), null, NULL_UPDATE_REQUESTS);
// Trigger container assignment
@ -963,7 +963,7 @@ public void testBlackListNodes() throws Exception {
// Get the allocation for the application and verify no allocation on
// blacklist node
Allocation allocation1 =
fs.allocate(appAttemptId1, emptyAsk, emptyId,
fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
null, null, NULL_UPDATE_REQUESTS);
Assert.assertEquals("allocation1", 0, allocation1.getContainers().size());
@ -971,7 +971,7 @@ public void testBlackListNodes() throws Exception {
// verify host_1_1 can get allocated as not in blacklist
fs.handle(new NodeUpdateSchedulerEvent(n4));
Allocation allocation2 =
fs.allocate(appAttemptId1, emptyAsk, emptyId,
fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
null, null, NULL_UPDATE_REQUESTS);
Assert.assertEquals("allocation2", 1, allocation2.getContainers().size());
List<Container> containerList = allocation2.getContainers();
@ -986,33 +986,33 @@ public void testBlackListNodes() throws Exception {
// be assigned
ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
fs.allocate(appAttemptId1, ask2, emptyId,
fs.allocate(appAttemptId1, ask2, null, emptyId,
Collections.singletonList("rack0"), null, NULL_UPDATE_REQUESTS);
// verify n1 is not qualified to be allocated
fs.handle(new NodeUpdateSchedulerEvent(n1));
Allocation allocation3 =
fs.allocate(appAttemptId1, emptyAsk, emptyId,
fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
null, null, NULL_UPDATE_REQUESTS);
Assert.assertEquals("allocation3", 0, allocation3.getContainers().size());
// verify n2 is not qualified to be allocated
fs.handle(new NodeUpdateSchedulerEvent(n2));
Allocation allocation4 =
fs.allocate(appAttemptId1, emptyAsk, emptyId,
fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
null, null, NULL_UPDATE_REQUESTS);
Assert.assertEquals("allocation4", 0, allocation4.getContainers().size());
// verify n3 is not qualified to be allocated
fs.handle(new NodeUpdateSchedulerEvent(n3));
Allocation allocation5 =
fs.allocate(appAttemptId1, emptyAsk, emptyId,
fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
null, null, NULL_UPDATE_REQUESTS);
Assert.assertEquals("allocation5", 0, allocation5.getContainers().size());
fs.handle(new NodeUpdateSchedulerEvent(n4));
Allocation allocation6 =
fs.allocate(appAttemptId1, emptyAsk, emptyId,
fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
null, null, NULL_UPDATE_REQUESTS);
Assert.assertEquals("allocation6", 1, allocation6.getContainers().size());
@ -1072,14 +1072,14 @@ public void testHeadroom() throws Exception {
List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
ask1.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
ResourceRequest.ANY, BuilderUtils.newResource(GB, 1), 1));
fs.allocate(appAttemptId1, ask1, emptyId,
fs.allocate(appAttemptId1, ask1, null, emptyId,
null, null, NULL_UPDATE_REQUESTS);
// Ask for a 2 GB container for app 2
List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
ask2.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0),
ResourceRequest.ANY, BuilderUtils.newResource(2 * GB, 1), 1));
fs.allocate(appAttemptId2, ask2, emptyId,
fs.allocate(appAttemptId2, ask2, null, emptyId,
null, null, NULL_UPDATE_REQUESTS);
// Trigger container assignment
@ -1087,13 +1087,13 @@ public void testHeadroom() throws Exception {
// Get the allocation for the applications and verify headroom
Allocation allocation1 =
fs.allocate(appAttemptId1, emptyAsk, emptyId,
fs.allocate(appAttemptId1, emptyAsk, null, emptyId,
null, null, NULL_UPDATE_REQUESTS);
Assert.assertEquals("Allocation headroom", 1 * GB, allocation1
.getResourceLimit().getMemorySize());
Allocation allocation2 =
fs.allocate(appAttemptId2, emptyAsk, emptyId,
fs.allocate(appAttemptId2, emptyAsk, null, emptyId,
null, null, NULL_UPDATE_REQUESTS);
Assert.assertEquals("Allocation headroom", 1 * GB, allocation2
.getResourceLimit().getMemorySize());

View File

@ -0,0 +1,403 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.ImmutableSet;
import org.apache.hadoop.yarn.api.records.ExecutionType;
import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceSizing;
import org.apache.hadoop.yarn.api.records.SchedulingRequest;
import org.apache.hadoop.yarn.api.resource.PlacementConstraints;
import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
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.SchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.InvalidAllocationTagsQueryException;
import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
import java.util.function.LongBinaryOperator;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
/**
* Test behaviors of single constraint app placement allocator.
*/
public class TestSingleConstraintAppPlacementAllocator {
private AppSchedulingInfo appSchedulingInfo;
private AllocationTagsManager spyAllocationTagsManager;
private RMContext rmContext;
private SchedulerRequestKey schedulerRequestKey;
private SingleConstraintAppPlacementAllocator allocator;
@Before
public void setup() throws Exception {
// stub app scheduling info.
appSchedulingInfo = mock(AppSchedulingInfo.class);
when(appSchedulingInfo.getApplicationId()).thenReturn(
TestUtils.getMockApplicationId(1));
when(appSchedulingInfo.getApplicationAttemptId()).thenReturn(
TestUtils.getMockApplicationAttemptId(1, 1));
// stub RMContext
rmContext = TestUtils.getMockRMContext();
// Create allocation tags manager
AllocationTagsManager allocationTagsManager = new AllocationTagsManager(
rmContext);
spyAllocationTagsManager = spy(allocationTagsManager);
schedulerRequestKey = new SchedulerRequestKey(Priority.newInstance(1), 2L,
TestUtils.getMockContainerId(1, 1));
rmContext.setAllocationTagsManager(spyAllocationTagsManager);
// Create allocator
allocator = new SingleConstraintAppPlacementAllocator();
allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
}
private void assertValidSchedulingRequest(
SchedulingRequest schedulingRequest) {
// Create allocator to avoid fields polluted by previous runs
allocator = new SingleConstraintAppPlacementAllocator();
allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
}
private void assertInvalidSchedulingRequest(
SchedulingRequest schedulingRequest, boolean recreateAllocator) {
try {
// Create allocator
if (recreateAllocator) {
allocator = new SingleConstraintAppPlacementAllocator();
allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
}
allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
} catch (SchedulerInvalidResoureRequestException e) {
// Expected
return;
}
Assert.fail(
"Expect failure for schedulingRequest=" + schedulingRequest.toString());
}
@Test
public void testSchedulingRequestValidation() {
// Valid
assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper", "reducer"),
PlacementConstraints.PlacementTargets.nodePartition(""))
.build()).resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build());
Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
allocator.getTargetAllocationTags());
Assert.assertEquals("", allocator.getTargetNodePartition());
// Valid (with partition)
assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper", "reducer"),
PlacementConstraints.PlacementTargets.nodePartition("x"))
.build()).resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build());
Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
allocator.getTargetAllocationTags());
Assert.assertEquals("x", allocator.getTargetNodePartition());
// Valid (without specifying node partition)
assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper", "reducer")).build())
.resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build());
Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
allocator.getTargetAllocationTags());
Assert.assertEquals("", allocator.getTargetNodePartition());
// Valid (with application Id target)
assertValidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper", "reducer")).build())
.resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build());
// Allocation tags should not include application Id
Assert.assertEquals(ImmutableSet.of("mapper", "reducer"),
allocator.getTargetAllocationTags());
Assert.assertEquals("", allocator.getTargetNodePartition());
// Invalid (without sizing)
assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper", "reducer")).build())
.build(), true);
// Invalid (without target tags)
assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1).build())
.build(), true);
// Invalid (with multiple allocation tags expression specified)
assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper"),
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("reducer"),
PlacementConstraints.PlacementTargets.nodePartition(""))
.build()).resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build(), true);
// Invalid (with multiple node partition target expression specified)
assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper"),
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp(""),
PlacementConstraints.PlacementTargets.nodePartition("x"))
.build()).resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build(), true);
// Invalid (not anti-affinity cardinality)
assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 1, 2,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper"),
PlacementConstraints.PlacementTargets.nodePartition(""))
.build()).resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build(), true);
// Invalid (not anti-affinity cardinality)
assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 2,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper"),
PlacementConstraints.PlacementTargets.nodePartition(""))
.build()).resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build(), true);
// Invalid (not NODE scope)
assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.RACK, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper", "reducer"),
PlacementConstraints.PlacementTargets.nodePartition(""))
.build()).resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build(), true);
// Invalid (not GUARANTEED)
assertInvalidSchedulingRequest(SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.OPPORTUNISTIC))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper", "reducer"),
PlacementConstraints.PlacementTargets.nodePartition(""))
.build()).resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build(), true);
}
@Test
public void testSchedulingRequestUpdate() {
SchedulingRequest schedulingRequest =
SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper", "reducer"),
PlacementConstraints.PlacementTargets.nodePartition(""))
.build()).resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build();
allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
// Update allocator with exactly same scheduling request, should succeeded.
allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
// Update allocator with scheduling request different at #allocations,
// should succeeded.
schedulingRequest.getResourceSizing().setNumAllocations(10);
allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
// Update allocator with scheduling request different at resource,
// should failed.
schedulingRequest.getResourceSizing().setResources(
Resource.newInstance(2048, 1));
assertInvalidSchedulingRequest(schedulingRequest, false);
// Update allocator with a different placement target (allocator tag),
// should failed
schedulingRequest = SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper"),
PlacementConstraints.PlacementTargets.nodePartition(""))
.build()).resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build();
assertInvalidSchedulingRequest(schedulingRequest, false);
// Update allocator with recover == true
int existingNumAllocations =
allocator.getSchedulingRequest().getResourceSizing()
.getNumAllocations();
schedulingRequest = SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper", "reducer"),
PlacementConstraints.PlacementTargets.nodePartition(""))
.build()).resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build();
allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, true);
Assert.assertEquals(existingNumAllocations + 1,
allocator.getSchedulingRequest().getResourceSizing()
.getNumAllocations());
}
@Test
public void testFunctionality() throws InvalidAllocationTagsQueryException {
SchedulingRequest schedulingRequest =
SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper", "reducer"),
PlacementConstraints.PlacementTargets.nodePartition(""))
.build()).resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build();
allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
allocator.canAllocate(NodeType.NODE_LOCAL,
TestUtils.getMockNode("host1", "/rack1", 123, 1024));
verify(spyAllocationTagsManager, Mockito.times(1)).getNodeCardinalityByOp(
eq(NodeId.fromString("host1:123")), eq(TestUtils.getMockApplicationId(1)),
eq(ImmutableSet.of("mapper", "reducer")),
any(LongBinaryOperator.class));
allocator = new SingleConstraintAppPlacementAllocator();
allocator.initialize(appSchedulingInfo, schedulerRequestKey, rmContext);
// Valid (with partition)
schedulingRequest = SchedulingRequest.newBuilder().executionType(
ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED))
.allocationRequestId(10L).priority(Priority.newInstance(1))
.placementConstraintExpression(PlacementConstraints
.targetCardinality(PlacementConstraints.NODE, 0, 1,
PlacementConstraints.PlacementTargets
.allocationTagToIntraApp("mapper", "reducer"),
PlacementConstraints.PlacementTargets.nodePartition("x"))
.build()).resourceSizing(
ResourceSizing.newInstance(1, Resource.newInstance(1024, 1)))
.build();
allocator.updatePendingAsk(schedulerRequestKey, schedulingRequest, false);
allocator.canAllocate(NodeType.NODE_LOCAL,
TestUtils.getMockNode("host1", "/rack1", 123, 1024));
verify(spyAllocationTagsManager, Mockito.atLeast(1)).getNodeCardinalityByOp(
eq(NodeId.fromString("host1:123")),
eq(TestUtils.getMockApplicationId(1)), eq(ImmutableSet
.of("mapper", "reducer")), any(LongBinaryOperator.class));
SchedulerNode node1 = mock(SchedulerNode.class);
when(node1.getPartition()).thenReturn("x");
when(node1.getNodeID()).thenReturn(NodeId.fromString("host1:123"));
Assert.assertTrue(allocator
.precheckNode(node1, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY));
SchedulerNode node2 = mock(SchedulerNode.class);
when(node1.getPartition()).thenReturn("");
when(node1.getNodeID()).thenReturn(NodeId.fromString("host2:123"));
Assert.assertFalse(allocator
.precheckNode(node2, SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY));
}
}