YARN-2496. Enhanced Capacity Scheduler to have basic support for allocating resources based on node-labels. Contributed by Wangda Tan.

YARN-2500. Ehnaced ResourceManager to support schedulers allocating resources based on node-labels. Contributed by Wangda Tan.

(cherry picked from commit f2ea555ac6)
This commit is contained in:
Vinod Kumar Vavilapalli 2014-10-15 18:33:06 -07:00
parent 08eeb3e5b6
commit e8e3a36213
57 changed files with 2870 additions and 799 deletions

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.sls.nodemanager;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
@ -159,6 +160,10 @@ public class NodeInfo {
return null;
}
@Override
public Set<String> getNodeLabels() {
return null;
}
}
public static RMNode newNodeInfo(String rackName, String hostName,

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode
import java.util.Collections;
import java.util.List;
import java.util.Set;
@Private
@Unstable
@ -147,4 +148,8 @@ public class RMNodeWrapper implements RMNode {
return node.getNodeManagerVersion();
}
@Override
public Set<String> getNodeLabels() {
return null;
}
}

View File

@ -138,6 +138,12 @@ Release 2.6.0 - UNRELEASED
YARN-2656. Made RM web services authentication filter support proxy user.
(Varun Vasudev and Zhijie Shen via zjshen)
YARN-2496. Enhanced Capacity Scheduler to have basic support for allocating
resources based on node-labels. (Wangda Tan via vinodkv)
YARN-2500. Ehnaced ResourceManager to support schedulers allocating resources
based on node-labels. (Wangda Tan via vinodkv)
IMPROVEMENTS
YARN-2242. Improve exception information on AM launch crashes. (Li Lu

View File

@ -188,6 +188,23 @@
</Or>
<Bug pattern="IS2_INCONSISTENT_SYNC" />
</Match>
<Match>
<Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AbstractCSQueue" />
<Or>
<Field name="absoluteCapacity" />
<Field name="absoluteMaxCapacity" />
<Field name="acls" />
<Field name="capacity" />
<Field name="maximumCapacity" />
<Field name="state" />
<Field name="labelManager" />
<Field name="defaultLabelExpression" />
<Field name="accessibleLabels" />
<Field name="absoluteNodeLabelCapacities" />
<Field name="reservationsContinueLooking" />
</Or>
<Bug pattern="IS2_INCONSISTENT_SYNC" />
</Match>
<!-- Inconsistent sync warning - scheduleAsynchronously is only initialized once and never changed -->
<Match>
<Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler" />

View File

@ -49,6 +49,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
import org.apache.hadoop.yarn.api.records.AMCommand;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NMToken;
@ -254,13 +255,13 @@ public class ApplicationMasterService extends AbstractService implements
if (hasApplicationMasterRegistered(applicationAttemptId)) {
String message =
"Application Master is already registered : "
+ applicationAttemptId.getApplicationId();
+ appID;
LOG.warn(message);
RMAuditLogger.logFailure(
this.rmContext.getRMApps()
.get(applicationAttemptId.getApplicationId()).getUser(),
.get(appID).getUser(),
AuditConstants.REGISTER_AM, "", "ApplicationMasterService", message,
applicationAttemptId.getApplicationId(), applicationAttemptId);
appID, applicationAttemptId);
throw new InvalidApplicationMasterRequestException(message);
}
@ -340,6 +341,7 @@ public class ApplicationMasterService extends AbstractService implements
ApplicationAttemptId applicationAttemptId =
authorizeRequest().getApplicationAttemptId();
ApplicationId appId = applicationAttemptId.getApplicationId();
AllocateResponseLock lock = responseMap.get(applicationAttemptId);
if (lock == null) {
@ -351,13 +353,13 @@ public class ApplicationMasterService extends AbstractService implements
if (!hasApplicationMasterRegistered(applicationAttemptId)) {
String message =
"Application Master is trying to unregister before registering for: "
+ applicationAttemptId.getApplicationId();
+ appId;
LOG.error(message);
RMAuditLogger.logFailure(
this.rmContext.getRMApps()
.get(applicationAttemptId.getApplicationId()).getUser(),
.get(appId).getUser(),
AuditConstants.UNREGISTER_AM, "", "ApplicationMasterService",
message, applicationAttemptId.getApplicationId(),
message, appId,
applicationAttemptId);
throw new ApplicationMasterNotRegisteredException(message);
}
@ -365,7 +367,7 @@ public class ApplicationMasterService extends AbstractService implements
this.amLivelinessMonitor.receivedPing(applicationAttemptId);
RMApp rmApp =
rmContext.getRMApps().get(applicationAttemptId.getApplicationId());
rmContext.getRMApps().get(appId);
if (rmApp.isAppFinalStateStored()) {
return FinishApplicationMasterResponse.newInstance(true);
@ -418,6 +420,7 @@ public class ApplicationMasterService extends AbstractService implements
ApplicationAttemptId appAttemptId =
amrmTokenIdentifier.getApplicationAttemptId();
ApplicationId applicationId = appAttemptId.getApplicationId();
this.amLivelinessMonitor.receivedPing(appAttemptId);
@ -432,14 +435,14 @@ public class ApplicationMasterService extends AbstractService implements
if (!hasApplicationMasterRegistered(appAttemptId)) {
String message =
"Application Master is not registered for known application: "
+ appAttemptId.getApplicationId()
+ applicationId
+ ". Let AM resync.";
LOG.info(message);
RMAuditLogger.logFailure(
this.rmContext.getRMApps().get(appAttemptId.getApplicationId())
this.rmContext.getRMApps().get(applicationId)
.getUser(), AuditConstants.REGISTER_AM, "",
"ApplicationMasterService", message,
appAttemptId.getApplicationId(),
applicationId,
appAttemptId);
return resync;
}
@ -481,11 +484,22 @@ public class ApplicationMasterService extends AbstractService implements
List<String> blacklistRemovals =
(blacklistRequest != null) ?
blacklistRequest.getBlacklistRemovals() : Collections.EMPTY_LIST;
RMApp app =
this.rmContext.getRMApps().get(applicationId);
// set label expression for Resource Requests
ApplicationSubmissionContext asc = app.getApplicationSubmissionContext();
for (ResourceRequest req : ask) {
if (null == req.getNodeLabelExpression()) {
req.setNodeLabelExpression(asc.getNodeLabelExpression());
}
}
// sanity check
try {
RMServerUtils.validateResourceRequests(ask,
rScheduler.getMaximumResourceCapability());
rScheduler.getMaximumResourceCapability(), app.getQueue(),
rScheduler);
} catch (InvalidResourceRequestException e) {
LOG.warn("Invalid resource ask by application " + appAttemptId, e);
throw e;
@ -498,8 +512,6 @@ public class ApplicationMasterService extends AbstractService implements
throw e;
}
RMApp app =
this.rmContext.getRMApps().get(appAttemptId.getApplicationId());
// In the case of work-preserving AM restart, it's possible for the
// AM to release containers from the earlier attempt.
if (!app.getApplicationSubmissionContext()
@ -582,7 +594,7 @@ public class ApplicationMasterService extends AbstractService implements
.toString(), amrmToken.getPassword(), amrmToken.getService()
.toString()));
LOG.info("The AMRMToken has been rolled-over. Send new AMRMToken back"
+ " to application: " + appAttemptId.getApplicationId());
+ " to application: " + applicationId);
}
/*

View File

@ -343,7 +343,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
long submitTime, String user)
throws YarnException {
ApplicationId applicationId = submissionContext.getApplicationId();
validateResourceRequest(submissionContext);
ResourceRequest amReq = validateAndCreateResourceRequest(submissionContext);
// Create RMApp
RMAppImpl application =
new RMAppImpl(applicationId, rmContext, this.conf,
@ -351,7 +351,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
submissionContext.getQueue(),
submissionContext, this.scheduler, this.masterService,
submitTime, submissionContext.getApplicationType(),
submissionContext.getApplicationTags());
submissionContext.getApplicationTags(), amReq);
// Concurrent app submissions with same applicationId will fail here
// Concurrent app submissions with different applicationIds will not
@ -373,7 +373,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
return application;
}
private void validateResourceRequest(
private ResourceRequest validateAndCreateResourceRequest(
ApplicationSubmissionContext submissionContext)
throws InvalidResourceRequestException {
// Validation of the ApplicationSubmissionContext needs to be completed
@ -383,18 +383,36 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
// Check whether AM resource requirements are within required limits
if (!submissionContext.getUnmanagedAM()) {
ResourceRequest amReq = BuilderUtils.newResourceRequest(
RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
submissionContext.getResource(), 1);
ResourceRequest amReq;
if (submissionContext.getAMContainerResourceRequest() != null) {
amReq = submissionContext.getAMContainerResourceRequest();
} else {
amReq =
BuilderUtils.newResourceRequest(
RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
submissionContext.getResource(), 1);
}
// set label expression for AM container
if (null == amReq.getNodeLabelExpression()) {
amReq.setNodeLabelExpression(submissionContext
.getNodeLabelExpression());
}
try {
SchedulerUtils.validateResourceRequest(amReq,
scheduler.getMaximumResourceCapability());
scheduler.getMaximumResourceCapability(),
submissionContext.getQueue(), scheduler);
} catch (InvalidResourceRequestException e) {
LOG.warn("RM app submission failed in validating AM resource request"
+ " for application " + submissionContext.getApplicationId(), e);
throw e;
}
return amReq;
}
return null;
}
private boolean isApplicationInFinalState(RMAppState rmAppState) {

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.conf.ConfigurationProvider;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@ -108,6 +109,10 @@ public interface RMContext {
boolean isWorkPreservingRecoveryEnabled();
RMNodeLabelsManager getNodeLabelManager();
public void setNodeLabelManager(RMNodeLabelsManager mgr);
long getEpoch();
ReservationSystem getReservationSystem();

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
@ -91,6 +92,7 @@ public class RMContextImpl implements RMContext {
private RMApplicationHistoryWriter rmApplicationHistoryWriter;
private SystemMetricsPublisher systemMetricsPublisher;
private ConfigurationProvider configurationProvider;
private RMNodeLabelsManager nodeLabelManager;
private long epoch;
private Clock systemClock = new SystemClock();
private long schedulerRecoveryStartTime = 0;
@ -406,6 +408,16 @@ public class RMContextImpl implements RMContext {
this.epoch = epoch;
}
@Override
public RMNodeLabelsManager getNodeLabelManager() {
return nodeLabelManager;
}
@Override
public void setNodeLabelManager(RMNodeLabelsManager mgr) {
nodeLabelManager = mgr;
}
public void setSchedulerRecoveryStartAndWaitTime(long waitTime) {
this.schedulerRecoveryStartTime = systemClock.getTime();
this.schedulerRecoveryWaitTime = waitTime;

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.resource.Resources;
@ -84,9 +85,11 @@ public class RMServerUtils {
* requested memory/vcore is non-negative and not greater than max
*/
public static void validateResourceRequests(List<ResourceRequest> ask,
Resource maximumResource) throws InvalidResourceRequestException {
Resource maximumResource, String queueName, YarnScheduler scheduler)
throws InvalidResourceRequestException {
for (ResourceRequest resReq : ask) {
SchedulerUtils.validateResourceRequest(resReq, maximumResource);
SchedulerUtils.validateResourceRequest(resReq, maximumResource,
queueName, scheduler);
}
}
@ -132,17 +135,25 @@ public class RMServerUtils {
}
}
public static UserGroupInformation verifyAccess(
AccessControlList acl, String method, final Log LOG)
throws IOException {
// by default, this method will use AdminService as module name
return verifyAccess(acl, method, "AdminService", LOG);
}
/**
* Utility method to verify if the current user has access based on the
* passed {@link AccessControlList}
* @param acl the {@link AccessControlList} to check against
* @param method the method name to be logged
* @param module, like AdminService or NodeLabelManager
* @param LOG the logger to use
* @return {@link UserGroupInformation} of the current user
* @throws IOException
*/
public static UserGroupInformation verifyAccess(
AccessControlList acl, String method, final Log LOG)
AccessControlList acl, String method, String module, final Log LOG)
throws IOException {
UserGroupInformation user;
try {
@ -159,7 +170,7 @@ public class RMServerUtils {
" to call '" + method + "'");
RMAuditLogger.logFailure(user.getShortUserName(), method,
acl.toString(), "AdminService",
acl.toString(), module,
RMAuditLogger.AuditConstants.UNAUTHORIZED_USER);
throw new AccessControlException("User " + user.getShortUserName() +

View File

@ -67,6 +67,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMaste
import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
@ -320,6 +321,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
return new AMLivelinessMonitor(this.rmDispatcher);
}
protected RMNodeLabelsManager createNodeLabelManager() {
return new RMNodeLabelsManager();
}
protected DelegationTokenRenewer createDelegationTokenRenewer() {
return new DelegationTokenRenewer();
}
@ -399,6 +404,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
AMLivelinessMonitor amFinishingMonitor = createAMLivelinessMonitor();
addService(amFinishingMonitor);
rmContext.setAMFinishingMonitor(amFinishingMonitor);
RMNodeLabelsManager nlm = createNodeLabelManager();
addService(nlm);
rmContext.setNodeLabelManager(nlm);
boolean isRecoveryEnabled = conf.getBoolean(
YarnConfiguration.RECOVERY_ENABLED,
@ -962,7 +971,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
* instance of {@link RMActiveServices} and initializes it.
* @throws Exception
*/
void createAndInitActiveServices() throws Exception {
protected void createAndInitActiveServices() throws Exception {
activeServices = new RMActiveServices();
activeServices.init(conf);
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.yarn.server.resourcemanager.reservation;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@ -126,14 +127,18 @@ public class CapacitySchedulerPlanFollower implements PlanFollower {
// create the default reservation queue if it doesnt exist
String defReservationQueue = planQueueName + PlanQueue.DEFAULT_QUEUE_SUFFIX;
if (scheduler.getQueue(defReservationQueue) == null) {
ReservationQueue defQueue =
new ReservationQueue(scheduler, defReservationQueue, planQueue);
try {
ReservationQueue defQueue =
new ReservationQueue(scheduler, defReservationQueue, planQueue);
scheduler.addQueue(defQueue);
} catch (SchedulerDynamicEditException e) {
LOG.warn(
"Exception while trying to create default reservation queue for plan: {}",
planQueueName, e);
} catch (IOException e) {
LOG.warn(
"Exception while trying to create default reservation queue for plan: {}",
planQueueName, e);
}
}
curReservationNames.add(defReservationQueue);
@ -186,14 +191,18 @@ public class CapacitySchedulerPlanFollower implements PlanFollower {
for (ReservationAllocation res : sortedAllocations) {
String currResId = res.getReservationId().toString();
if (curReservationNames.contains(currResId)) {
ReservationQueue resQueue =
new ReservationQueue(scheduler, currResId, planQueue);
try {
ReservationQueue resQueue =
new ReservationQueue(scheduler, currResId, planQueue);
scheduler.addQueue(resQueue);
} catch (SchedulerDynamicEditException e) {
LOG.warn(
"Exception while trying to activate reservation: {} for plan: {}",
currResId, planQueueName, e);
} catch (IOException e) {
LOG.warn(
"Exception while trying to activate reservation: {} for plan: {}",
currResId, planQueueName, e);
}
}
Resource capToAssign = res.getResourcesAtTime(now);

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeState;
import org.apache.hadoop.yarn.api.records.ReservationId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
@ -143,6 +144,7 @@ public class RMAppImpl implements RMApp, Recoverable {
private RMAppEvent eventCausingFinalSaving;
private RMAppState targetedFinalState;
private RMAppState recoveredFinalState;
private ResourceRequest amReq;
Object transitionTodo;
@ -342,7 +344,8 @@ public class RMAppImpl implements RMApp, Recoverable {
Configuration config, String name, String user, String queue,
ApplicationSubmissionContext submissionContext, YarnScheduler scheduler,
ApplicationMasterService masterService, long submitTime,
String applicationType, Set<String> applicationTags) {
String applicationType, Set<String> applicationTags,
ResourceRequest amReq) {
this.systemClock = new SystemClock();
@ -361,6 +364,7 @@ public class RMAppImpl implements RMApp, Recoverable {
this.startTime = this.systemClock.getTime();
this.applicationType = applicationType;
this.applicationTags = applicationTags;
this.amReq = amReq;
int globalMaxAppAttempts = conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
@ -732,7 +736,7 @@ public class RMAppImpl implements RMApp, Recoverable {
// previously failed attempts(which should not include Preempted,
// hardware error and NM resync) + 1) equal to the max-attempt
// limit.
maxAppAttempts == (getNumFailedAppAttempts() + 1));
maxAppAttempts == (getNumFailedAppAttempts() + 1), amReq);
attempts.put(appAttemptId, attempt);
currentAttempt = attempt;
}

View File

@ -93,7 +93,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
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.security.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils;
import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
import org.apache.hadoop.yarn.state.MultipleArcTransition;
@ -177,6 +176,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
private Object transitionTodo;
private RMAppAttemptMetrics attemptMetrics = null;
private ResourceRequest amReq = null;
private static final StateMachineFactory<RMAppAttemptImpl,
RMAppAttemptState,
@ -426,7 +426,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
RMContext rmContext, YarnScheduler scheduler,
ApplicationMasterService masterService,
ApplicationSubmissionContext submissionContext,
Configuration conf, boolean maybeLastAttempt) {
Configuration conf, boolean maybeLastAttempt, ResourceRequest amReq) {
this.conf = conf;
this.applicationAttemptId = appAttemptId;
this.rmContext = rmContext;
@ -442,8 +442,11 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
this.proxiedTrackingUrl = generateProxyUriWithScheme(null);
this.maybeLastAttempt = maybeLastAttempt;
this.stateMachine = stateMachineFactory.make(this);
this.attemptMetrics =
new RMAppAttemptMetrics(applicationAttemptId, rmContext);
this.amReq = amReq;
}
@Override
@ -885,24 +888,34 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
private static final List<ResourceRequest> EMPTY_CONTAINER_REQUEST_LIST =
new ArrayList<ResourceRequest>();
private static final class ScheduleTransition
@VisibleForTesting
public static final class ScheduleTransition
implements
MultipleArcTransition<RMAppAttemptImpl, RMAppAttemptEvent, RMAppAttemptState> {
@Override
public RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) {
if (!appAttempt.submissionContext.getUnmanagedAM()) {
// Request a container for the AM.
ResourceRequest request =
BuilderUtils.newResourceRequest(
AM_CONTAINER_PRIORITY, ResourceRequest.ANY, appAttempt
.getSubmissionContext().getResource(), 1);
ApplicationSubmissionContext subCtx = appAttempt.submissionContext;
if (!subCtx.getUnmanagedAM()) {
// Need reset #containers before create new attempt, because this request
// will be passed to scheduler, and scheduler will deduct the number after
// AM container allocated
// Currently, following fields are all hard code,
// TODO: change these fields when we want to support
// priority/resource-name/relax-locality specification for AM containers
// allocation.
appAttempt.amReq.setNumContainers(1);
appAttempt.amReq.setPriority(AM_CONTAINER_PRIORITY);
appAttempt.amReq.setResourceName(ResourceRequest.ANY);
appAttempt.amReq.setRelaxLocality(true);
// SchedulerUtils.validateResourceRequests is not necessary because
// AM resource has been checked when submission
Allocation amContainerAllocation = appAttempt.scheduler.allocate(
appAttempt.applicationAttemptId,
Collections.singletonList(request), EMPTY_CONTAINER_RELEASE_LIST, null, null);
Allocation amContainerAllocation =
appAttempt.scheduler.allocate(appAttempt.applicationAttemptId,
Collections.singletonList(appAttempt.amReq),
EMPTY_CONTAINER_RELEASE_LIST, null, null);
if (amContainerAllocation != null
&& amContainerAllocation.getContainers() != null) {
assert (amContainerAllocation.getContainers().size() == 0);

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmnode;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -135,4 +136,11 @@ public interface RMNode {
* @return containerUpdates accumulated across NM heartbeats.
*/
public List<UpdatedContainerInfo> pullContainerUpdates();
/**
* Get set of labels in this node
*
* @return labels in this node
*/
public Set<String> getNodeLabels();
}

View File

@ -855,4 +855,12 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
public Set<ContainerId> getLaunchedContainers() {
return this.launchedContainers;
}
@Override
public Set<String> getNodeLabels() {
if (context.getNodeLabelManager() == null) {
return null;
}
return context.getNodeLabelManager().getLabelsOnNode(nodeId);
}
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
import org.apache.hadoop.classification.InterfaceStability.Evolving;
@ -71,4 +72,22 @@ public interface Queue {
*/
public void recoverContainer(Resource clusterResource,
SchedulerApplicationAttempt schedulerAttempt, RMContainer rmContainer);
/**
* Get labels can be accessed of this queue
* labels={*}, means this queue can access any label
* labels={ }, means this queue cannot access any label except node without label
* labels={a, b, c} means this queue can access a or b or c
* @return labels
*/
public Set<String> getAccessibleNodeLabels();
/**
* Get default label expression of this queue. If label expression of
* ApplicationSubmissionContext and label expression of Resource Request not
* set, this will be used.
*
* @return default label expression
*/
public String getDefaultNodeLabelExpression();
}

View File

@ -17,23 +17,29 @@
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
import java.io.IOException;
import java.util.List;
import java.util.Set;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.QueueInfo;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.common.collect.Sets;
/**
* Utilities shared by schedulers.
*/
@ -190,7 +196,8 @@ public class SchedulerUtils {
* request
*/
public static void validateResourceRequest(ResourceRequest resReq,
Resource maximumResource) throws InvalidResourceRequestException {
Resource maximumResource, String queueName, YarnScheduler scheduler)
throws InvalidResourceRequestException {
if (resReq.getCapability().getMemory() < 0 ||
resReq.getCapability().getMemory() > maximumResource.getMemory()) {
throw new InvalidResourceRequestException("Invalid resource request"
@ -209,5 +216,116 @@ public class SchedulerUtils {
+ resReq.getCapability().getVirtualCores()
+ ", maxVirtualCores=" + maximumResource.getVirtualCores());
}
// Get queue from scheduler
QueueInfo queueInfo = null;
try {
queueInfo = scheduler.getQueueInfo(queueName, false, false);
} catch (IOException e) {
// it is possible queue cannot get when queue mapping is set, just ignore
// the queueInfo here, and move forward
}
// check labels in the resource request.
String labelExp = resReq.getNodeLabelExpression();
// if queue has default label expression, and RR doesn't have, use the
// default label expression of queue
if (labelExp == null && queueInfo != null) {
labelExp = queueInfo.getDefaultNodeLabelExpression();
resReq.setNodeLabelExpression(labelExp);
}
if (labelExp != null && !labelExp.trim().isEmpty() && queueInfo != null) {
if (!checkQueueLabelExpression(queueInfo.getAccessibleNodeLabels(),
labelExp)) {
throw new InvalidResourceRequestException("Invalid resource request"
+ ", queue="
+ queueInfo.getQueueName()
+ " doesn't have permission to access all labels "
+ "in resource request. labelExpression of resource request="
+ labelExp
+ ". Queue labels="
+ (queueInfo.getAccessibleNodeLabels() == null ? "" : StringUtils.join(queueInfo
.getAccessibleNodeLabels().iterator(), ',')));
}
}
}
public static boolean checkQueueAccessToNode(Set<String> queueLabels,
Set<String> nodeLabels) {
// if queue's label is *, it can access any node
if (queueLabels != null && queueLabels.contains(RMNodeLabelsManager.ANY)) {
return true;
}
// any queue can access to a node without label
if (nodeLabels == null || nodeLabels.isEmpty()) {
return true;
}
// a queue can access to a node only if it contains any label of the node
if (queueLabels != null
&& Sets.intersection(queueLabels, nodeLabels).size() > 0) {
return true;
}
// sorry, you cannot access
return false;
}
public static void checkIfLabelInClusterNodeLabels(RMNodeLabelsManager mgr,
Set<String> labels) throws IOException {
if (mgr == null) {
if (labels != null && !labels.isEmpty()) {
throw new IOException("NodeLabelManager is null, please check");
}
return;
}
if (labels != null) {
for (String label : labels) {
if (!label.equals(RMNodeLabelsManager.ANY)
&& !mgr.containsNodeLabel(label)) {
throw new IOException("NodeLabelManager doesn't include label = "
+ label + ", please check.");
}
}
}
}
public static boolean checkNodeLabelExpression(Set<String> nodeLabels,
String labelExpression) {
// empty label expression can only allocate on node with empty labels
if (labelExpression == null || labelExpression.trim().isEmpty()) {
if (!nodeLabels.isEmpty()) {
return false;
}
}
if (labelExpression != null) {
for (String str : labelExpression.split("&&")) {
if (!str.trim().isEmpty()
&& (nodeLabels == null || !nodeLabels.contains(str.trim()))) {
return false;
}
}
}
return true;
}
public static boolean checkQueueLabelExpression(Set<String> queueLabels,
String labelExpression) {
if (queueLabels != null && queueLabels.contains(RMNodeLabelsManager.ANY)) {
return true;
}
// if label expression is empty, we can allocate container on any node
if (labelExpression == null) {
return true;
}
for (String str : labelExpression.split("&&")) {
if (!str.trim().isEmpty()
&& (queueLabels == null || !queueLabels.contains(str.trim()))) {
return false;
}
}
return true;
}
}

View File

@ -0,0 +1,448 @@
/**
* 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 java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.yarn.api.records.QueueACL;
import org.apache.hadoop.yarn.api.records.QueueInfo;
import org.apache.hadoop.yarn.api.records.QueueState;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.common.collect.Sets;
public abstract class AbstractCSQueue implements CSQueue {
CSQueue parent;
final String queueName;
float capacity;
float maximumCapacity;
float absoluteCapacity;
float absoluteMaxCapacity;
float absoluteUsedCapacity = 0.0f;
float usedCapacity = 0.0f;
volatile int numContainers;
final Resource minimumAllocation;
final Resource maximumAllocation;
QueueState state;
final QueueMetrics metrics;
final ResourceCalculator resourceCalculator;
Set<String> accessibleLabels;
RMNodeLabelsManager labelManager;
String defaultLabelExpression;
Resource usedResources = Resources.createResource(0, 0);
QueueInfo queueInfo;
Map<String, Float> absoluteCapacityByNodeLabels;
Map<String, Float> capacitiyByNodeLabels;
Map<String, Resource> usedResourcesByNodeLabels = new HashMap<String, Resource>();
Map<String, Float> absoluteMaxCapacityByNodeLabels;
Map<String, Float> maxCapacityByNodeLabels;
Map<QueueACL, AccessControlList> acls =
new HashMap<QueueACL, AccessControlList>();
boolean reservationsContinueLooking;
private final RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);
public AbstractCSQueue(CapacitySchedulerContext cs,
String queueName, CSQueue parent, CSQueue old) throws IOException {
this.minimumAllocation = cs.getMinimumResourceCapability();
this.maximumAllocation = cs.getMaximumResourceCapability();
this.labelManager = cs.getRMContext().getNodeLabelManager();
this.parent = parent;
this.queueName = queueName;
this.resourceCalculator = cs.getResourceCalculator();
this.queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
// must be called after parent and queueName is set
this.metrics = old != null ? old.getMetrics() :
QueueMetrics.forQueue(getQueuePath(), parent,
cs.getConfiguration().getEnableUserMetrics(),
cs.getConf());
// get labels
this.accessibleLabels = cs.getConfiguration().getAccessibleNodeLabels(getQueuePath());
this.defaultLabelExpression = cs.getConfiguration()
.getDefaultNodeLabelExpression(getQueuePath());
this.queueInfo.setQueueName(queueName);
// inherit from parent if labels not set
if (this.accessibleLabels == null && parent != null) {
this.accessibleLabels = parent.getAccessibleNodeLabels();
SchedulerUtils.checkIfLabelInClusterNodeLabels(labelManager,
this.accessibleLabels);
}
// inherit from parent if labels not set
if (this.defaultLabelExpression == null && parent != null
&& this.accessibleLabels.containsAll(parent.getAccessibleNodeLabels())) {
this.defaultLabelExpression = parent.getDefaultNodeLabelExpression();
}
// set capacity by labels
capacitiyByNodeLabels =
cs.getConfiguration().getNodeLabelCapacities(getQueuePath(), accessibleLabels,
labelManager);
// set maximum capacity by labels
maxCapacityByNodeLabels =
cs.getConfiguration().getMaximumNodeLabelCapacities(getQueuePath(),
accessibleLabels, labelManager);
}
@Override
public synchronized float getCapacity() {
return capacity;
}
@Override
public synchronized float getAbsoluteCapacity() {
return absoluteCapacity;
}
@Override
public float getAbsoluteMaximumCapacity() {
return absoluteMaxCapacity;
}
@Override
public synchronized float getAbsoluteUsedCapacity() {
return absoluteUsedCapacity;
}
@Override
public float getMaximumCapacity() {
return maximumCapacity;
}
@Override
public synchronized float getUsedCapacity() {
return usedCapacity;
}
@Override
public synchronized Resource getUsedResources() {
return usedResources;
}
public synchronized int getNumContainers() {
return numContainers;
}
@Override
public synchronized QueueState getState() {
return state;
}
@Override
public QueueMetrics getMetrics() {
return metrics;
}
@Override
public String getQueueName() {
return queueName;
}
@Override
public synchronized CSQueue getParent() {
return parent;
}
@Override
public synchronized void setParent(CSQueue newParentQueue) {
this.parent = (ParentQueue)newParentQueue;
}
public Set<String> getAccessibleNodeLabels() {
return accessibleLabels;
}
@Override
public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
synchronized (this) {
if (acls.get(acl).isUserAllowed(user)) {
return true;
}
}
if (parent != null) {
return parent.hasAccess(acl, user);
}
return false;
}
@Override
public synchronized void setUsedCapacity(float usedCapacity) {
this.usedCapacity = usedCapacity;
}
@Override
public synchronized void setAbsoluteUsedCapacity(float absUsedCapacity) {
this.absoluteUsedCapacity = absUsedCapacity;
}
/**
* Set maximum capacity - used only for testing.
* @param maximumCapacity new max capacity
*/
synchronized void setMaxCapacity(float maximumCapacity) {
// Sanity check
CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
float absMaxCapacity =
CSQueueUtils.computeAbsoluteMaximumCapacity(maximumCapacity, parent);
CSQueueUtils.checkAbsoluteCapacity(getQueueName(), absoluteCapacity,
absMaxCapacity);
this.maximumCapacity = maximumCapacity;
this.absoluteMaxCapacity = absMaxCapacity;
}
@Override
public float getAbsActualCapacity() {
// for now, simply return actual capacity = guaranteed capacity for parent
// queue
return absoluteCapacity;
}
@Override
public String getDefaultNodeLabelExpression() {
return defaultLabelExpression;
}
synchronized void setupQueueConfigs(Resource clusterResource, float capacity,
float absoluteCapacity, float maximumCapacity, float absoluteMaxCapacity,
QueueState state, Map<QueueACL, AccessControlList> acls,
Set<String> labels, String defaultLabelExpression,
Map<String, Float> nodeLabelCapacities,
Map<String, Float> maximumNodeLabelCapacities,
boolean reservationContinueLooking)
throws IOException {
// Sanity check
CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
CSQueueUtils.checkAbsoluteCapacity(getQueueName(), absoluteCapacity,
absoluteMaxCapacity);
this.capacity = capacity;
this.absoluteCapacity = absoluteCapacity;
this.maximumCapacity = maximumCapacity;
this.absoluteMaxCapacity = absoluteMaxCapacity;
this.state = state;
this.acls = acls;
// set labels
this.accessibleLabels = labels;
// set label expression
this.defaultLabelExpression = defaultLabelExpression;
// copy node label capacity
this.capacitiyByNodeLabels = new HashMap<String, Float>(nodeLabelCapacities);
this.maxCapacityByNodeLabels =
new HashMap<String, Float>(maximumNodeLabelCapacities);
this.queueInfo.setAccessibleNodeLabels(this.accessibleLabels);
this.queueInfo.setCapacity(this.capacity);
this.queueInfo.setMaximumCapacity(this.maximumCapacity);
this.queueInfo.setQueueState(this.state);
this.queueInfo.setDefaultNodeLabelExpression(this.defaultLabelExpression);
// Update metrics
CSQueueUtils.updateQueueStatistics(
resourceCalculator, this, parent, clusterResource, minimumAllocation);
// Check if labels of this queue is a subset of parent queue, only do this
// when we not root
if (parent != null && parent.getParent() != null) {
if (parent.getAccessibleNodeLabels() != null
&& !parent.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
// if parent isn't "*", child shouldn't be "*" too
if (this.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
throw new IOException("Parent's accessible queue is not ANY(*), "
+ "but child's accessible queue is *");
} else {
Set<String> diff =
Sets.difference(this.getAccessibleNodeLabels(),
parent.getAccessibleNodeLabels());
if (!diff.isEmpty()) {
throw new IOException("Some labels of child queue is not a subset "
+ "of parent queue, these labels=["
+ StringUtils.join(diff, ",") + "]");
}
}
}
}
// calculate absolute capacity by each node label
this.absoluteCapacityByNodeLabels =
CSQueueUtils.computeAbsoluteCapacityByNodeLabels(
this.capacitiyByNodeLabels, parent);
// calculate maximum capacity by each node label
this.absoluteMaxCapacityByNodeLabels =
CSQueueUtils.computeAbsoluteMaxCapacityByNodeLabels(
maximumNodeLabelCapacities, parent);
// check absoluteMaximumNodeLabelCapacities is valid
CSQueueUtils.checkAbsoluteCapacitiesByLabel(getQueueName(),
absoluteCapacityByNodeLabels, absoluteCapacityByNodeLabels);
this.reservationsContinueLooking = reservationContinueLooking;
}
@Private
public Resource getMaximumAllocation() {
return maximumAllocation;
}
@Private
public Resource getMinimumAllocation() {
return minimumAllocation;
}
synchronized void allocateResource(Resource clusterResource,
Resource resource, Set<String> nodeLabels) {
Resources.addTo(usedResources, resource);
// Update usedResources by labels
if (nodeLabels == null || nodeLabels.isEmpty()) {
if (!usedResourcesByNodeLabels.containsKey(RMNodeLabelsManager.NO_LABEL)) {
usedResourcesByNodeLabels.put(RMNodeLabelsManager.NO_LABEL,
Resources.createResource(0));
}
Resources.addTo(usedResourcesByNodeLabels.get(RMNodeLabelsManager.NO_LABEL),
resource);
} else {
for (String label : Sets.intersection(accessibleLabels, nodeLabels)) {
if (!usedResourcesByNodeLabels.containsKey(label)) {
usedResourcesByNodeLabels.put(label, Resources.createResource(0));
}
Resources.addTo(usedResourcesByNodeLabels.get(label), resource);
}
}
++numContainers;
CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(),
clusterResource, minimumAllocation);
}
protected synchronized void releaseResource(Resource clusterResource,
Resource resource, Set<String> nodeLabels) {
// Update queue metrics
Resources.subtractFrom(usedResources, resource);
// Update usedResources by labels
if (null == nodeLabels || nodeLabels.isEmpty()) {
if (!usedResourcesByNodeLabels.containsKey(RMNodeLabelsManager.NO_LABEL)) {
usedResourcesByNodeLabels.put(RMNodeLabelsManager.NO_LABEL,
Resources.createResource(0));
}
Resources.subtractFrom(
usedResourcesByNodeLabels.get(RMNodeLabelsManager.NO_LABEL), resource);
} else {
for (String label : Sets.intersection(accessibleLabels, nodeLabels)) {
if (!usedResourcesByNodeLabels.containsKey(label)) {
usedResourcesByNodeLabels.put(label, Resources.createResource(0));
}
Resources.subtractFrom(usedResourcesByNodeLabels.get(label), resource);
}
}
CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(),
clusterResource, minimumAllocation);
--numContainers;
}
@Private
public float getCapacityByNodeLabel(String label) {
if (null == parent) {
return 1f;
}
if (StringUtils.equals(label, RMNodeLabelsManager.NO_LABEL)) {
return getCapacity();
}
if (!capacitiyByNodeLabels.containsKey(label)) {
return 0;
} else {
return capacitiyByNodeLabels.get(label);
}
}
@Private
public float getAbsoluteCapacityByNodeLabel(String label) {
if (null == parent) {
return 1;
}
if (StringUtils.equals(label, RMNodeLabelsManager.NO_LABEL)) {
return getAbsoluteCapacity();
}
if (!absoluteMaxCapacityByNodeLabels.containsKey(label)) {
return 0;
} else {
return absoluteMaxCapacityByNodeLabels.get(label);
}
}
@Private
public float getAbsoluteMaximumCapacityByNodeLabel(String label) {
if (StringUtils.equals(label, RMNodeLabelsManager.NO_LABEL)) {
return getAbsoluteMaximumCapacity();
}
return getAbsoluteCapacityByNodeLabel(label);
}
@Private
public boolean getReservationContinueLooking() {
return reservationsContinueLooking;
}
@Private
public Map<QueueACL, AccessControlList> getACLs() {
return acls;
}
}

View File

@ -72,9 +72,18 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
/**
* Get the configured <em>capacity</em> of the queue.
* @return queue capacity
* @return configured queue capacity
*/
public float getCapacity();
/**
* Get actual <em>capacity</em> of the queue, this may be different from
* configured capacity when mis-config take place, like add labels to the
* cluster
*
* @return actual queue capacity
*/
public float getAbsActualCapacity();
/**
* Get capacity of the parent of the queue as a function of the
@ -105,29 +114,32 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
*/
public float getAbsoluteUsedCapacity();
/**
* Get the current used capacity of the queue
* and it's children (if any).
* @return queue used capacity
*/
public float getUsedCapacity();
/**
* Set used capacity of the queue.
* @param usedCapacity used capacity of the queue
* @param usedCapacity
* used capacity of the queue
*/
public void setUsedCapacity(float usedCapacity);
/**
* Set absolute used capacity of the queue.
* @param absUsedCapacity absolute used capacity of the queue
* @param absUsedCapacity
* absolute used capacity of the queue
*/
public void setAbsoluteUsedCapacity(float absUsedCapacity);
/**
* Get the currently utilized resources in the cluster
* by the queue and children (if any).
* @return used resources by the queue and it's children
* Get the current used capacity of nodes without label(s) of the queue
* and it's children (if any).
* @return queue used capacity
*/
public float getUsedCapacity();
/**
* Get the currently utilized resources which allocated at nodes without any
* labels in the cluster by the queue and children (if any).
*
* @return used resources by the queue and it's children
*/
public Resource getUsedResources();
@ -259,4 +271,25 @@ extends org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue {
*/
public void attachContainer(Resource clusterResource,
FiCaSchedulerApp application, RMContainer container);
/**
* Get absolute capacity by label of this queue can use
* @param nodeLabel
* @return absolute capacity by label of this queue can use
*/
public float getAbsoluteCapacityByNodeLabel(String nodeLabel);
/**
* Get absolute max capacity by label of this queue can use
* @param nodeLabel
* @return absolute capacity by label of this queue can use
*/
public float getAbsoluteMaximumCapacityByNodeLabel(String nodeLabel);
/**
* Get capacity by node label
* @param nodeLabel
* @return capacity by node label
*/
public float getCapacityByNodeLabel(String nodeLabel);
}

View File

@ -17,9 +17,12 @@
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.utils.Lock;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@ -40,7 +43,7 @@ class CSQueueUtils {
}
}
public static void checkAbsoluteCapacities(String queueName,
public static void checkAbsoluteCapacity(String queueName,
float absCapacity, float absMaxCapacity) {
if (absMaxCapacity < (absCapacity - EPSILON)) {
throw new IllegalArgumentException("Illegal call to setMaxCapacity. "
@ -49,6 +52,23 @@ class CSQueueUtils {
+ ")");
}
}
public static void checkAbsoluteCapacitiesByLabel(String queueName,
Map<String, Float> absCapacities,
Map<String, Float> absMaximumCapacities) {
for (Entry<String, Float> entry : absCapacities.entrySet()) {
String label = entry.getKey();
float absCapacity = entry.getValue();
float absMaxCapacity = absMaximumCapacities.get(label);
if (absMaxCapacity < (absCapacity - EPSILON)) {
throw new IllegalArgumentException("Illegal call to setMaxCapacity. "
+ "Queue '" + queueName + "' has " + "an absolute capacity ("
+ absCapacity + ") greater than "
+ "its absolute maximumCapacity (" + absMaxCapacity + ") of label="
+ label);
}
}
}
public static float computeAbsoluteMaximumCapacity(
float maximumCapacity, CSQueue parent) {
@ -56,6 +76,39 @@ class CSQueueUtils {
(parent == null) ? 1.0f : parent.getAbsoluteMaximumCapacity();
return (parentAbsMaxCapacity * maximumCapacity);
}
public static Map<String, Float> computeAbsoluteCapacityByNodeLabels(
Map<String, Float> nodeLabelToCapacities, CSQueue parent) {
if (parent == null) {
return nodeLabelToCapacities;
}
Map<String, Float> absoluteCapacityByNodeLabels =
new HashMap<String, Float>();
for (Entry<String, Float> entry : nodeLabelToCapacities.entrySet()) {
String label = entry.getKey();
float capacity = entry.getValue();
absoluteCapacityByNodeLabels.put(label,
capacity * parent.getAbsoluteCapacityByNodeLabel(label));
}
return absoluteCapacityByNodeLabels;
}
public static Map<String, Float> computeAbsoluteMaxCapacityByNodeLabels(
Map<String, Float> maximumNodeLabelToCapacities, CSQueue parent) {
if (parent == null) {
return maximumNodeLabelToCapacities;
}
Map<String, Float> absoluteMaxCapacityByNodeLabels =
new HashMap<String, Float>();
for (Entry<String, Float> entry : maximumNodeLabelToCapacities.entrySet()) {
String label = entry.getKey();
float maxCapacity = entry.getValue();
absoluteMaxCapacityByNodeLabels.put(label,
maxCapacity * parent.getAbsoluteMaximumCapacityByNodeLabel(label));
}
return absoluteMaxCapacityByNodeLabels;
}
public static int computeMaxActiveApplications(
ResourceCalculator calculator,

View File

@ -20,7 +20,15 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import java.io.IOException;
import java.io.InputStream;
import java.util.*;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@ -53,8 +61,13 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.*;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@ -191,6 +204,7 @@ public class CapacityScheduler extends
private boolean scheduleAsynchronously;
private AsyncScheduleThread asyncSchedulerThread;
private RMNodeLabelsManager labelManager;
/**
* EXPERT
@ -275,6 +289,8 @@ public class CapacityScheduler extends
this.applications =
new ConcurrentHashMap<ApplicationId,
SchedulerApplication<FiCaSchedulerApp>>();
this.labelManager = rmContext.getNodeLabelManager();
initializeQueues(this.conf);
scheduleAsynchronously = this.conf.getScheduleAynschronously();
@ -446,7 +462,7 @@ public class CapacityScheduler extends
root =
parseQueue(this, conf, null, CapacitySchedulerConfiguration.ROOT,
queues, queues, noop);
labelManager.reinitializeQueueLabels(getQueueToLabels());
LOG.info("Initialized root queue " + root);
initializeQueueMappings();
}
@ -469,10 +485,19 @@ public class CapacityScheduler extends
// Re-configure queues
root.reinitialize(newRoot, clusterResource);
initializeQueueMappings();
// Re-calculate headroom for active applications
root.updateClusterResource(clusterResource);
labelManager.reinitializeQueueLabels(getQueueToLabels());
}
private Map<String, Set<String>> getQueueToLabels() {
Map<String, Set<String>> queueToLabels = new HashMap<String, Set<String>>();
for (CSQueue queue : queues.values()) {
queueToLabels.put(queue.getQueueName(), queue.getAccessibleNodeLabels());
}
return queueToLabels;
}
/**
@ -515,7 +540,7 @@ public class CapacityScheduler extends
@Lock(CapacityScheduler.class)
static CSQueue parseQueue(
CapacitySchedulerContext csContext,
CapacitySchedulerContext csContext,
CapacitySchedulerConfiguration conf,
CSQueue parent, String queueName, Map<String, CSQueue> queues,
Map<String, CSQueue> oldQueues,
@ -1094,11 +1119,18 @@ public class CapacityScheduler extends
}
private synchronized void addNode(RMNode nodeManager) {
// update this node to node label manager
if (labelManager != null) {
labelManager.activateNode(nodeManager.getNodeID(),
nodeManager.getTotalCapability());
}
this.nodes.put(nodeManager.getNodeID(), new FiCaSchedulerNode(nodeManager,
usePortForNodeName));
Resources.addTo(clusterResource, nodeManager.getTotalCapability());
root.updateClusterResource(clusterResource);
int numNodes = numNodeManagers.incrementAndGet();
LOG.info("Added node " + nodeManager.getNodeAddress() +
" clusterResource: " + clusterResource);
@ -1108,6 +1140,11 @@ public class CapacityScheduler extends
}
private synchronized void removeNode(RMNode nodeInfo) {
// update this node to node label manager
if (labelManager != null) {
labelManager.deactivateNode(nodeInfo.getNodeID());
}
FiCaSchedulerNode node = nodes.get(nodeInfo.getNodeID());
if (node == null) {
return;
@ -1141,6 +1178,7 @@ public class CapacityScheduler extends
}
this.nodes.remove(nodeInfo.getNodeID());
LOG.info("Removed node " + nodeInfo.getNodeAddress() +
" clusterResource: " + clusterResource);
}

View File

@ -18,7 +18,15 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import java.util.*;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.StringTokenizer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -31,10 +39,14 @@ import org.apache.hadoop.yarn.api.records.QueueACL;
import org.apache.hadoop.yarn.api.records.QueueState;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.common.collect.ImmutableSet;
public class CapacitySchedulerConfiguration extends Configuration {
private static final Log LOG =
@ -83,6 +95,12 @@ public class CapacitySchedulerConfiguration extends Configuration {
public static final String STATE = "state";
@Private
public static final String ACCESSIBLE_NODE_LABELS = "accessible-node-labels";
@Private
public static final String DEFAULT_NODE_LABEL_EXPRESSION =
"default-node-label-expression";
public static final String RESERVE_CONT_LOOK_ALL_NODES = PREFIX
+ "reservations-continue-look-all-nodes";
@ -268,6 +286,10 @@ public class CapacitySchedulerConfiguration extends Configuration {
return queueName;
}
private String getNodeLabelPrefix(String queue, String label) {
return getQueuePrefix(queue) + ACCESSIBLE_NODE_LABELS + DOT + label + DOT;
}
public int getMaximumSystemApplications() {
int maxApplications =
getInt(MAXIMUM_SYSTEM_APPLICATIONS, DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS);
@ -343,6 +365,15 @@ public class CapacitySchedulerConfiguration extends Configuration {
", maxCapacity=" + maxCapacity);
}
public void setCapacityByLabel(String queue, String label, float capacity) {
setFloat(getNodeLabelPrefix(queue, label) + CAPACITY, capacity);
}
public void setMaximumCapacityByLabel(String queue, String label,
float capacity) {
setFloat(getNodeLabelPrefix(queue, label) + MAXIMUM_CAPACITY, capacity);
}
public int getUserLimit(String queue) {
int userLimit = getInt(getQueuePrefix(queue) + USER_LIMIT,
DEFAULT_USER_LIMIT);
@ -372,6 +403,121 @@ public class CapacitySchedulerConfiguration extends Configuration {
QueueState.valueOf(state.toUpperCase()) : QueueState.RUNNING;
}
public void setAccessibleNodeLabels(String queue, Set<String> labels) {
if (labels == null) {
return;
}
String str = StringUtils.join(",", labels);
set(getQueuePrefix(queue) + ACCESSIBLE_NODE_LABELS, str);
}
public Set<String> getAccessibleNodeLabels(String queue) {
String accessibleLabelStr =
get(getQueuePrefix(queue) + ACCESSIBLE_NODE_LABELS);
// When accessible-label is null,
if (accessibleLabelStr == null) {
// Only return null when queue is not ROOT
if (!queue.equals(ROOT)) {
return null;
}
} else {
// print a warning when accessibleNodeLabel specified in config and queue
// is ROOT
if (queue.equals(ROOT)) {
LOG.warn("Accessible node labels for root queue will be ignored,"
+ " it will be automatically set to \"*\".");
}
}
// always return ANY for queue root
if (queue.equals(ROOT)) {
return ImmutableSet.of(RMNodeLabelsManager.ANY);
}
// In other cases, split the accessibleLabelStr by ","
Set<String> set = new HashSet<String>();
for (String str : accessibleLabelStr.split(",")) {
if (!str.trim().isEmpty()) {
set.add(str.trim());
}
}
// if labels contains "*", only keep ANY behind
if (set.contains(RMNodeLabelsManager.ANY)) {
set.clear();
set.add(RMNodeLabelsManager.ANY);
}
return Collections.unmodifiableSet(set);
}
public Map<String, Float> getNodeLabelCapacities(String queue,
Set<String> labels, RMNodeLabelsManager mgr) {
Map<String, Float> nodeLabelCapacities = new HashMap<String, Float>();
if (labels == null) {
return nodeLabelCapacities;
}
for (String label : labels.contains(CommonNodeLabelsManager.ANY) ? mgr
.getClusterNodeLabels() : labels) {
// capacity of all labels in each queue should be 1
if (org.apache.commons.lang.StringUtils.equals(ROOT, queue)) {
nodeLabelCapacities.put(label, 1.0f);
continue;
}
float capacity =
getFloat(getNodeLabelPrefix(queue, label) + CAPACITY, UNDEFINED);
if (capacity < MINIMUM_CAPACITY_VALUE
|| capacity > MAXIMUM_CAPACITY_VALUE) {
throw new IllegalArgumentException("Illegal " + "capacity of "
+ capacity + " for label=" + label + " in queue=" + queue);
}
if (LOG.isDebugEnabled()) {
LOG.debug("CSConf - getCapacityOfLabel: prefix="
+ getNodeLabelPrefix(queue, label) + ", capacity=" + capacity);
}
nodeLabelCapacities.put(label, capacity / 100f);
}
return nodeLabelCapacities;
}
public Map<String, Float> getMaximumNodeLabelCapacities(String queue,
Set<String> labels, RMNodeLabelsManager mgr) {
Map<String, Float> maximumNodeLabelCapacities = new HashMap<String, Float>();
if (labels == null) {
return maximumNodeLabelCapacities;
}
for (String label : labels.contains(CommonNodeLabelsManager.ANY) ? mgr
.getClusterNodeLabels() : labels) {
float maxCapacity =
getFloat(getNodeLabelPrefix(queue, label) + MAXIMUM_CAPACITY,
UNDEFINED);
maxCapacity = (maxCapacity == DEFAULT_MAXIMUM_CAPACITY_VALUE) ?
MAXIMUM_CAPACITY_VALUE : maxCapacity;
if (maxCapacity < MINIMUM_CAPACITY_VALUE
|| maxCapacity > MAXIMUM_CAPACITY_VALUE) {
throw new IllegalArgumentException("Illegal " + "capacity of "
+ maxCapacity + " for label=" + label + " in queue=" + queue);
}
LOG.debug("CSConf - getCapacityOfLabel: prefix="
+ getNodeLabelPrefix(queue, label) + ", capacity=" + maxCapacity);
maximumNodeLabelCapacities.put(label, maxCapacity / 100f);
}
return maximumNodeLabelCapacities;
}
public String getDefaultNodeLabelExpression(String queue) {
return get(getQueuePrefix(queue) + DEFAULT_NODE_LABEL_EXPRESSION);
}
public void setDefaultNodeLabelExpression(String queue, String exp) {
set(getQueuePrefix(queue) + DEFAULT_NODE_LABEL_EXPRESSION, exp);
}
/*
* Returns whether we should continue to look at all heart beating nodes even
* after the reservation limit was hit. The node heart beating in could

View File

@ -23,12 +23,14 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
@ -46,77 +48,42 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.common.collect.Sets;
@Private
@Evolving
public class ParentQueue implements CSQueue {
public class ParentQueue extends AbstractCSQueue {
private static final Log LOG = LogFactory.getLog(ParentQueue.class);
private CSQueue parent;
private final String queueName;
private float capacity;
private float maximumCapacity;
private float absoluteCapacity;
private float absoluteMaxCapacity;
private float absoluteUsedCapacity = 0.0f;
private float usedCapacity = 0.0f;
protected final Set<CSQueue> childQueues;
private final Comparator<CSQueue> queueComparator;
private Resource usedResources = Resources.createResource(0, 0);
protected final Set<CSQueue> childQueues;
private final boolean rootQueue;
private final Resource minimumAllocation;
private volatile int numApplications;
private volatile int numContainers;
private QueueState state;
private final QueueMetrics metrics;
private QueueInfo queueInfo;
private Map<QueueACL, AccessControlList> acls =
new HashMap<QueueACL, AccessControlList>();
final Comparator<CSQueue> queueComparator;
volatile int numApplications;
private final RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);
private final ResourceCalculator resourceCalculator;
private boolean reservationsContinueLooking;
public ParentQueue(CapacitySchedulerContext cs,
String queueName, CSQueue parent, CSQueue old) {
minimumAllocation = cs.getMinimumResourceCapability();
String queueName, CSQueue parent, CSQueue old) throws IOException {
super(cs, queueName, parent, old);
this.parent = parent;
this.queueName = queueName;
this.rootQueue = (parent == null);
this.resourceCalculator = cs.getResourceCalculator();
this.queueComparator = cs.getQueueComparator();
// must be called after parent and queueName is set
this.metrics = old != null ? old.getMetrics() :
QueueMetrics.forQueue(getQueuePath(), parent,
cs.getConfiguration().getEnableUserMetrics(),
cs.getConf());
this.rootQueue = (parent == null);
float rawCapacity = cs.getConfiguration().getCapacity(getQueuePath());
@ -141,17 +108,14 @@ public class ParentQueue implements CSQueue {
Map<QueueACL, AccessControlList> acls =
cs.getConfiguration().getAcls(getQueuePath());
this.queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
this.queueInfo.setQueueName(queueName);
this.queueInfo.setChildQueues(new ArrayList<QueueInfo>());
setupQueueConfigs(cs.getClusterResource(),
capacity, absoluteCapacity,
maximumCapacity, absoluteMaxCapacity, state, acls,
setupQueueConfigs(cs.getClusterResource(), capacity, absoluteCapacity,
maximumCapacity, absoluteMaxCapacity, state, acls, accessibleLabels,
defaultLabelExpression, capacitiyByNodeLabels, maxCapacityByNodeLabels,
cs.getConfiguration().getReservationContinueLook());
this.queueComparator = cs.getQueueComparator();
this.childQueues = new TreeSet<CSQueue>(queueComparator);
LOG.info("Initialized parent-queue " + queueName +
@ -159,41 +123,29 @@ public class ParentQueue implements CSQueue {
", fullname=" + getQueuePath());
}
protected synchronized void setupQueueConfigs(
Resource clusterResource,
float capacity, float absoluteCapacity,
float maximumCapacity, float absoluteMaxCapacity,
synchronized void setupQueueConfigs(Resource clusterResource, float capacity,
float absoluteCapacity, float maximumCapacity, float absoluteMaxCapacity,
QueueState state, Map<QueueACL, AccessControlList> acls,
boolean continueLooking
) {
// Sanity check
CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
CSQueueUtils.checkAbsoluteCapacities(getQueueName(), absoluteCapacity, absoluteMaxCapacity);
this.capacity = capacity;
this.absoluteCapacity = absoluteCapacity;
this.maximumCapacity = maximumCapacity;
this.absoluteMaxCapacity = absoluteMaxCapacity;
this.state = state;
this.acls = acls;
this.queueInfo.setCapacity(this.capacity);
this.queueInfo.setMaximumCapacity(this.maximumCapacity);
this.queueInfo.setQueueState(this.state);
this.reservationsContinueLooking = continueLooking;
StringBuilder aclsString = new StringBuilder();
Set<String> accessibleLabels, String defaultLabelExpression,
Map<String, Float> nodeLabelCapacities,
Map<String, Float> maximumCapacitiesByLabel,
boolean reservationContinueLooking) throws IOException {
super.setupQueueConfigs(clusterResource, capacity, absoluteCapacity,
maximumCapacity, absoluteMaxCapacity, state, acls, accessibleLabels,
defaultLabelExpression, nodeLabelCapacities, maximumCapacitiesByLabel,
reservationContinueLooking);
StringBuilder aclsString = new StringBuilder();
for (Map.Entry<QueueACL, AccessControlList> e : acls.entrySet()) {
aclsString.append(e.getKey() + ":" + e.getValue().getAclString());
}
// Update metrics
CSQueueUtils.updateQueueStatistics(
resourceCalculator, this, parent, clusterResource, minimumAllocation);
StringBuilder labelStrBuilder = new StringBuilder();
if (accessibleLabels != null) {
for (String s : accessibleLabels) {
labelStrBuilder.append(s);
labelStrBuilder.append(",");
}
}
LOG.info(queueName +
", capacity=" + capacity +
@ -201,13 +153,13 @@ public class ParentQueue implements CSQueue {
", maxCapacity=" + maximumCapacity +
", asboluteMaxCapacity=" + absoluteMaxCapacity +
", state=" + state +
", acls=" + aclsString +
", acls=" + aclsString +
", labels=" + labelStrBuilder.toString() + "\n" +
", reservationsContinueLooking=" + reservationsContinueLooking);
}
private static float PRECISION = 0.0005f; // 0.05% precision
void setChildQueues(Collection<CSQueue> childQueues) {
// Validate
float childCapacities = 0;
for (CSQueue queue : childQueues) {
@ -221,6 +173,21 @@ public class ParentQueue implements CSQueue {
" capacity of " + childCapacities +
" for children of queue " + queueName);
}
// check label capacities
for (String nodeLabel : labelManager.getClusterNodeLabels()) {
float capacityByLabel = getCapacityByNodeLabel(nodeLabel);
// check children's labels
float sum = 0;
for (CSQueue queue : childQueues) {
sum += queue.getCapacityByNodeLabel(nodeLabel);
}
if ((capacityByLabel > 0 && Math.abs(1.0f - sum) > PRECISION)
|| (capacityByLabel == 0) && (sum > 0)) {
throw new IllegalArgumentException("Illegal" + " capacity of "
+ sum + " for children of queue " + queueName
+ " for label=" + nodeLabel);
}
}
this.childQueues.clear();
this.childQueues.addAll(childQueues);
@ -228,21 +195,6 @@ public class ParentQueue implements CSQueue {
LOG.debug("setChildQueues: " + getChildQueuesToPrint());
}
}
@Override
public synchronized CSQueue getParent() {
return parent;
}
@Override
public synchronized void setParent(CSQueue newParentQueue) {
this.parent = (ParentQueue)newParentQueue;
}
@Override
public String getQueueName() {
return queueName;
}
@Override
public String getQueuePath() {
@ -250,65 +202,6 @@ public class ParentQueue implements CSQueue {
return parentPath + getQueueName();
}
@Override
public synchronized float getCapacity() {
return capacity;
}
@Override
public synchronized float getAbsoluteCapacity() {
return absoluteCapacity;
}
@Override
public float getAbsoluteMaximumCapacity() {
return absoluteMaxCapacity;
}
@Override
public synchronized float getAbsoluteUsedCapacity() {
return absoluteUsedCapacity;
}
@Override
public float getMaximumCapacity() {
return maximumCapacity;
}
@Override
public ActiveUsersManager getActiveUsersManager() {
// Should never be called since all applications are submitted to LeafQueues
return null;
}
@Override
public synchronized float getUsedCapacity() {
return usedCapacity;
}
@Override
public synchronized Resource getUsedResources() {
return usedResources;
}
@Override
public synchronized List<CSQueue> getChildQueues() {
return new ArrayList<CSQueue>(childQueues);
}
public synchronized int getNumContainers() {
return numContainers;
}
public synchronized int getNumApplications() {
return numApplications;
}
@Override
public synchronized QueueState getState() {
return state;
}
@Override
public synchronized QueueInfo getQueueInfo(
boolean includeChildQueues, boolean recursive) {
@ -391,6 +284,10 @@ public class ParentQueue implements CSQueue {
newlyParsedParentQueue.absoluteMaxCapacity,
newlyParsedParentQueue.state,
newlyParsedParentQueue.acls,
newlyParsedParentQueue.accessibleLabels,
newlyParsedParentQueue.defaultLabelExpression,
newlyParsedParentQueue.capacitiyByNodeLabels,
newlyParsedParentQueue.maxCapacityByNodeLabels,
newlyParsedParentQueue.reservationsContinueLooking);
// Re-configure existing child queues and add new ones
@ -434,21 +331,6 @@ public class ParentQueue implements CSQueue {
}
return queuesMap;
}
@Override
public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
synchronized (this) {
if (acls.get(acl).isUserAllowed(user)) {
return true;
}
}
if (parent != null) {
return parent.hasAccess(acl, user);
}
return false;
}
@Override
public void submitApplication(ApplicationId applicationId, String user,
@ -521,7 +403,7 @@ public class ParentQueue implements CSQueue {
}
}
public synchronized void removeApplication(ApplicationId applicationId,
private synchronized void removeApplication(ApplicationId applicationId,
String user) {
--numApplications;
@ -532,30 +414,6 @@ public class ParentQueue implements CSQueue {
" leaf-queue of parent: " + getQueueName() +
" #applications: " + getNumApplications());
}
@Override
public synchronized void setUsedCapacity(float usedCapacity) {
this.usedCapacity = usedCapacity;
}
@Override
public synchronized void setAbsoluteUsedCapacity(float absUsedCapacity) {
this.absoluteUsedCapacity = absUsedCapacity;
}
/**
* Set maximum capacity - used only for testing.
* @param maximumCapacity new max capacity
*/
synchronized void setMaxCapacity(float maximumCapacity) {
// Sanity check
CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
float absMaxCapacity = CSQueueUtils.computeAbsoluteMaximumCapacity(maximumCapacity, parent);
CSQueueUtils.checkAbsoluteCapacities(getQueueName(), absoluteCapacity, absMaxCapacity);
this.maximumCapacity = maximumCapacity;
this.absoluteMaxCapacity = absMaxCapacity;
}
@Override
public synchronized CSAssignment assignContainers(
@ -563,6 +421,12 @@ public class ParentQueue implements CSQueue {
CSAssignment assignment =
new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
// if our queue cannot access this node, just return
if (!SchedulerUtils.checkQueueAccessToNode(accessibleLabels,
labelManager.getLabelsOnNode(node.getNodeID()))) {
return assignment;
}
while (canAssign(clusterResource, node)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Trying to assign containers to child-queue of "
@ -570,8 +434,10 @@ public class ParentQueue implements CSQueue {
}
boolean localNeedToUnreserve = false;
Set<String> nodeLabels = labelManager.getLabelsOnNode(node.getNodeID());
// Are we over maximum-capacity for this queue?
if (!assignToQueue(clusterResource)) {
if (!canAssignToThisQueue(clusterResource, nodeLabels)) {
// check to see if we could if we unreserve first
localNeedToUnreserve = assignToQueueIfUnreserve(clusterResource);
if (!localNeedToUnreserve) {
@ -589,7 +455,8 @@ public class ParentQueue implements CSQueue {
resourceCalculator, clusterResource,
assignedToChild.getResource(), Resources.none())) {
// Track resource utilization for the parent-queue
allocateResource(clusterResource, assignedToChild.getResource());
super.allocateResource(clusterResource, assignedToChild.getResource(),
nodeLabels);
// Track resource utilization in this pass of the scheduler
Resources.addTo(assignment.getResource(), assignedToChild.getResource());
@ -628,22 +495,41 @@ public class ParentQueue implements CSQueue {
return assignment;
}
private synchronized boolean assignToQueue(Resource clusterResource) {
// Check how of the cluster's absolute capacity we are currently using...
float currentCapacity =
Resources.divide(
resourceCalculator, clusterResource,
usedResources, clusterResource);
if (currentCapacity >= absoluteMaxCapacity) {
LOG.info(getQueueName() +
" used=" + usedResources +
" current-capacity (" + currentCapacity + ") " +
" >= max-capacity (" + absoluteMaxCapacity + ")");
return false;
private synchronized boolean canAssignToThisQueue(Resource clusterResource,
Set<String> nodeLabels) {
Set<String> labelCanAccess =
new HashSet<String>(
accessibleLabels.contains(CommonNodeLabelsManager.ANY) ? nodeLabels
: Sets.intersection(accessibleLabels, nodeLabels));
if (nodeLabels.isEmpty()) {
// Any queue can always access any node without label
labelCanAccess.add(RMNodeLabelsManager.NO_LABEL);
}
return true;
boolean canAssign = true;
for (String label : labelCanAccess) {
if (!usedResourcesByNodeLabels.containsKey(label)) {
usedResourcesByNodeLabels.put(label, Resources.createResource(0));
}
float currentAbsoluteLabelUsedCapacity =
Resources.divide(resourceCalculator, clusterResource,
usedResourcesByNodeLabels.get(label),
labelManager.getResourceByLabel(label, clusterResource));
// if any of the label doesn't beyond limit, we can allocate on this node
if (currentAbsoluteLabelUsedCapacity >=
getAbsoluteMaximumCapacityByNodeLabel(label)) {
if (LOG.isDebugEnabled()) {
LOG.debug(getQueueName() + " used=" + usedResources
+ " current-capacity (" + usedResourcesByNodeLabels.get(label) + ") "
+ " >= max-capacity ("
+ labelManager.getResourceByLabel(label, clusterResource) + ")");
}
canAssign = false;
break;
}
}
return canAssign;
}
@ -685,7 +571,7 @@ public class ParentQueue implements CSQueue {
node.getAvailableResource(), minimumAllocation);
}
synchronized CSAssignment assignContainersToChildQueues(Resource cluster,
private synchronized CSAssignment assignContainersToChildQueues(Resource cluster,
FiCaSchedulerNode node, boolean needToUnreserve) {
CSAssignment assignment =
new CSAssignment(Resources.createResource(0, 0), NodeType.NODE_LOCAL);
@ -728,11 +614,16 @@ public class ParentQueue implements CSQueue {
String getChildQueuesToPrint() {
StringBuilder sb = new StringBuilder();
for (CSQueue q : childQueues) {
sb.append(q.getQueuePath() + "(" + q.getUsedCapacity() + "), ");
sb.append(q.getQueuePath() +
"usedCapacity=(" + q.getUsedCapacity() + "), " +
" label=("
+ StringUtils.join(q.getAccessibleNodeLabels().iterator(), ",")
+ ")");
}
return sb.toString();
}
void printChildQueues() {
private void printChildQueues() {
if (LOG.isDebugEnabled()) {
LOG.debug("printChildQueues - queue: " + getQueuePath()
+ " child-queues: " + getChildQueuesToPrint());
@ -749,8 +640,8 @@ public class ParentQueue implements CSQueue {
// Careful! Locking order is important!
// Book keeping
synchronized (this) {
releaseResource(clusterResource,
rmContainer.getContainer().getResource());
super.releaseResource(clusterResource, rmContainer.getContainer()
.getResource(), labelManager.getLabelsOnNode(node.getNodeID()));
LOG.info("completedContainer" +
" queue=" + getQueueName() +
@ -787,27 +678,6 @@ public class ParentQueue implements CSQueue {
}
}
@Private
boolean getReservationContinueLooking() {
return reservationsContinueLooking;
}
synchronized void allocateResource(Resource clusterResource,
Resource resource) {
Resources.addTo(usedResources, resource);
CSQueueUtils.updateQueueStatistics(
resourceCalculator, this, parent, clusterResource, minimumAllocation);
++numContainers;
}
synchronized void releaseResource(Resource clusterResource,
Resource resource) {
Resources.subtractFrom(usedResources, resource);
CSQueueUtils.updateQueueStatistics(
resourceCalculator, this, parent, clusterResource, minimumAllocation);
--numContainers;
}
@Override
public synchronized void updateClusterResource(Resource clusterResource) {
// Update all children
@ -821,10 +691,9 @@ public class ParentQueue implements CSQueue {
}
@Override
public QueueMetrics getMetrics() {
return metrics;
public synchronized List<CSQueue> getChildQueues() {
return new ArrayList<CSQueue>(childQueues);
}
@Override
public void recoverContainer(Resource clusterResource,
@ -834,12 +703,20 @@ public class ParentQueue implements CSQueue {
}
// Careful! Locking order is important!
synchronized (this) {
allocateResource(clusterResource,rmContainer.getContainer().getResource());
super.allocateResource(clusterResource, rmContainer.getContainer()
.getResource(), labelManager.getLabelsOnNode(rmContainer
.getContainer().getNodeId()));
}
if (parent != null) {
parent.recoverContainer(clusterResource, attempt, rmContainer);
}
}
@Override
public ActiveUsersManager getActiveUsersManager() {
// Should never be called since all applications are submitted to LeafQueues
return null;
}
@Override
public void collectSchedulerApplications(
@ -853,8 +730,9 @@ public class ParentQueue implements CSQueue {
public void attachContainer(Resource clusterResource,
FiCaSchedulerApp application, RMContainer rmContainer) {
if (application != null) {
allocateResource(clusterResource, rmContainer.getContainer()
.getResource());
super.allocateResource(clusterResource, rmContainer.getContainer()
.getResource(), labelManager.getLabelsOnNode(rmContainer
.getContainer().getNodeId()));
LOG.info("movedContainer" + " queueMoveIn=" + getQueueName()
+ " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
+ getAbsoluteUsedCapacity() + " used=" + usedResources + " cluster="
@ -870,7 +748,9 @@ public class ParentQueue implements CSQueue {
public void detachContainer(Resource clusterResource,
FiCaSchedulerApp application, RMContainer rmContainer) {
if (application != null) {
releaseResource(clusterResource, rmContainer.getContainer().getResource());
super.releaseResource(clusterResource,
rmContainer.getContainer().getResource(),
labelManager.getLabelsOnNode(rmContainer.getContainer().getNodeId()));
LOG.info("movedContainer" + " queueMoveOut=" + getQueueName()
+ " usedCapacity=" + getUsedCapacity() + " absoluteUsedCapacity="
+ getAbsoluteUsedCapacity() + " used=" + usedResources + " cluster="
@ -882,7 +762,14 @@ public class ParentQueue implements CSQueue {
}
}
public Map<QueueACL, AccessControlList> getACLs() {
return acls;
@Override
public float getAbsActualCapacity() {
// for now, simply return actual capacity = guaranteed capacity for parent
// queue
return absoluteCapacity;
}
public synchronized int getNumApplications() {
return numApplications;
}
}

View File

@ -19,6 +19,8 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import org.apache.hadoop.yarn.api.records.Resource;
@ -47,7 +49,7 @@ public class PlanQueue extends ParentQueue {
private boolean showReservationsAsQueues;
public PlanQueue(CapacitySchedulerContext cs, String queueName,
CSQueue parent, CSQueue old) {
CSQueue parent, CSQueue old) throws IOException {
super(cs, queueName, parent, old);
this.schedulerContext = cs;
@ -104,6 +106,10 @@ public class PlanQueue extends ParentQueue {
newlyParsedParentQueue.getMaximumCapacity(),
newlyParsedParentQueue.getAbsoluteMaximumCapacity(),
newlyParsedParentQueue.getState(), newlyParsedParentQueue.getACLs(),
newlyParsedParentQueue.accessibleLabels,
newlyParsedParentQueue.defaultLabelExpression,
newlyParsedParentQueue.capacitiyByNodeLabels,
newlyParsedParentQueue.maxCapacityByNodeLabels,
newlyParsedParentQueue.getReservationContinueLooking());
updateQuotas(newlyParsedParentQueue.userLimit,

View File

@ -42,7 +42,7 @@ public class ReservationQueue extends LeafQueue {
private int maxSystemApps;
public ReservationQueue(CapacitySchedulerContext cs, String queueName,
PlanQueue parent) {
PlanQueue parent) throws IOException {
super(cs, queueName, parent, null);
maxSystemApps = cs.getConfiguration().getMaximumSystemApplications();
// the following parameters are common to all reservation in the plan

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
@ -270,4 +271,16 @@ public abstract class FSQueue implements Queue, Schedulable {
return String.format("[%s, demand=%s, running=%s, share=%s, w=%s]",
getName(), getDemand(), getResourceUsage(), fairShare, getWeights());
}
@Override
public Set<String> getAccessibleNodeLabels() {
// TODO, add implementation for FS
return null;
}
@Override
public String getDefaultNodeLabelExpression() {
// TODO, add implementation for FS
return null;
}
}

View File

@ -25,6 +25,7 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentSkipListMap;
import org.apache.commons.logging.Log;
@ -187,6 +188,18 @@ public class FifoScheduler extends
updateAppHeadRoom(schedulerAttempt);
updateAvailableResourcesMetrics();
}
@Override
public Set<String> getAccessibleNodeLabels() {
// TODO add implementation for FIFO scheduler
return null;
}
@Override
public String getDefaultNodeLabelExpression() {
// TODO add implementation for FIFO scheduler
return null;
}
};
public FifoScheduler() {

View File

@ -147,6 +147,7 @@ public class Application {
return used;
}
@SuppressWarnings("deprecation")
public synchronized void submit() throws IOException, YarnException {
ApplicationSubmissionContext context = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
context.setApplicationId(this.applicationId);

View File

@ -135,34 +135,52 @@ public class MockAM {
public void addContainerToBeReleased(ContainerId containerId) {
releases.add(containerId);
}
public AllocateResponse allocate(
String host, int memory, int numContainers,
List<ContainerId> releases) throws Exception {
List<ResourceRequest> reqs = createReq(new String[]{host}, memory, 1, numContainers);
return allocate(host, memory, numContainers, releases, null);
}
public AllocateResponse allocate(
String host, int memory, int numContainers,
List<ContainerId> releases, String labelExpression) throws Exception {
List<ResourceRequest> reqs =
createReq(new String[] { host }, memory, 1, numContainers,
labelExpression);
return allocate(reqs, releases);
}
public List<ResourceRequest> createReq(String[] hosts, int memory, int priority,
int containers) throws Exception {
return createReq(hosts, memory, priority, containers, null);
}
public List<ResourceRequest> createReq(String[] hosts, int memory, int priority,
int containers) throws Exception {
int containers, String labelExpression) throws Exception {
List<ResourceRequest> reqs = new ArrayList<ResourceRequest>();
for (String host : hosts) {
ResourceRequest hostReq = createResourceReq(host, memory, priority,
containers);
containers, labelExpression);
reqs.add(hostReq);
ResourceRequest rackReq = createResourceReq("/default-rack", memory,
priority, containers);
priority, containers, labelExpression);
reqs.add(rackReq);
}
ResourceRequest offRackReq = createResourceReq(ResourceRequest.ANY, memory,
priority, containers);
priority, containers, labelExpression);
reqs.add(offRackReq);
return reqs;
}
public ResourceRequest createResourceReq(String resource, int memory, int priority,
int containers) throws Exception {
return createResourceReq(resource, memory, priority, containers, null);
}
public ResourceRequest createResourceReq(String resource, int memory, int priority,
int containers) throws Exception {
int containers, String labelExpression) throws Exception {
ResourceRequest req = Records.newRecord(ResourceRequest.class);
req.setResourceName(resource);
req.setNumContainers(containers);
@ -172,6 +190,9 @@ public class MockAM {
Resource capability = Records.newRecord(Resource.class);
capability.setMemory(memory);
req.setCapability(capability);
if (labelExpression != null) {
req.setNodeLabelExpression(labelExpression);
}
return req;
}

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -202,7 +203,11 @@ public class MockNodes {
public long getLastHealthReportTime() {
return lastHealthReportTime;
}
@Override
public Set<String> getNodeLabels() {
return null;
}
};
private static RMNode buildRMNode(int rack, final Resource perNode, NodeState state, String httpAddr) {

View File

@ -59,6 +59,8 @@ import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.DummyRMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
@ -110,6 +112,13 @@ public class MockRM extends ResourceManager {
Logger rootLogger = LogManager.getRootLogger();
rootLogger.setLevel(Level.DEBUG);
}
@Override
protected RMNodeLabelsManager createNodeLabelManager() {
RMNodeLabelsManager mgr = new DummyRMNodeLabelsManager();
mgr.init(getConfig());
return mgr;
}
public void waitForState(ApplicationId appId, RMAppState finalState)
throws Exception {
@ -183,27 +192,43 @@ public class MockRM extends ResourceManager {
return launchAndRegisterAM(app, this, nm);
}
public void waitForState(MockNM nm, ContainerId containerId,
public boolean waitForState(MockNM nm, ContainerId containerId,
RMContainerState containerState) throws Exception {
// default is wait for 30,000 ms
return waitForState(nm, containerId, containerState, 30 * 1000);
}
public boolean waitForState(MockNM nm, ContainerId containerId,
RMContainerState containerState, int timeoutMillisecs) throws Exception {
RMContainer container = getResourceScheduler().getRMContainer(containerId);
int timeoutSecs = 0;
while(container == null && timeoutSecs++ < 100) {
while(container == null && timeoutSecs++ < timeoutMillisecs / 100) {
nm.nodeHeartbeat(true);
container = getResourceScheduler().getRMContainer(containerId);
System.out.println("Waiting for container " + containerId + " to be allocated.");
Thread.sleep(100);
if (timeoutMillisecs <= timeoutSecs * 100) {
return false;
}
}
Assert.assertNotNull("Container shouldn't be null", container);
timeoutSecs = 0;
while (!containerState.equals(container.getState()) && timeoutSecs++ < 40) {
while (!containerState.equals(container.getState())
&& timeoutSecs++ < timeoutMillisecs / 100) {
System.out.println("Container : " + containerId + " State is : "
+ container.getState() + " Waiting for state : " + containerState);
nm.nodeHeartbeat(true);
Thread.sleep(300);
Thread.sleep(100);
if (timeoutMillisecs <= timeoutSecs * 100) {
return false;
}
}
System.out.println("Container State is : " + container.getState());
Assert.assertEquals("Container state is not correct (timedout)",
containerState, container.getState());
return true;
}
// get new application id
@ -310,6 +335,7 @@ public class MockRM extends ResourceManager {
isAppIdProvided, applicationId, 0, null);
}
@SuppressWarnings("deprecation")
public RMApp submitApp(int masterMemory,
LogAggregationContext logAggregationContext) throws Exception {
return submitApp(masterMemory, "", UserGroupInformation.getCurrentUser()
@ -319,6 +345,7 @@ public class MockRM extends ResourceManager {
false, null, 0, logAggregationContext);
}
@SuppressWarnings("deprecation")
public RMApp submitApp(int masterMemory, String name, String user,
Map<ApplicationAccessType, String> acls, boolean unmanaged, String queue,
int maxAppAttempts, Credentials ts, String appType,

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.conf.HAUtil;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
@ -34,9 +35,11 @@ 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.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@ -150,7 +153,7 @@ public class RMHATestBase extends ClientBaseWithFixes{
this.rmContext.getScheduler(),
this.rmContext.getApplicationMasterService(),
submitTime, submissionContext.getApplicationType(),
submissionContext.getApplicationTags());
submissionContext.getApplicationTags(), null);
this.rmContext.getRMApps().put(submissionContext.getApplicationId(),
application);
//Do not send RMAppEventType.START event

View File

@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
import static org.mockito.Matchers.isA;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyLong;
@ -37,7 +38,6 @@ import java.util.List;
import java.util.concurrent.ConcurrentMap;
import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.Service;
import org.apache.hadoop.yarn.MockApps;
@ -207,6 +207,7 @@ public class TestAppManager{
private ApplicationSubmissionContext asContext;
private ApplicationId appId;
@SuppressWarnings("deprecation")
@Before
public void setUp() {
long now = System.currentTimeMillis();
@ -540,6 +541,7 @@ public class TestAppManager{
Assert.assertEquals("app state doesn't match", RMAppState.FINISHED, app.getState());
}
@SuppressWarnings("deprecation")
@Test (timeout = 30000)
public void testRMAppSubmitInvalidResourceRequest() throws Exception {
asContext.setResource(Resources.createResource(

View File

@ -22,6 +22,7 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyString;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.security.PrivilegedExceptionAction;
@ -30,7 +31,6 @@ import java.util.List;
import java.util.Map;
import org.junit.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -179,6 +179,7 @@ public class TestApplicationACLs {
verifyAdministerQueueUserAccess();
}
@SuppressWarnings("deprecation")
private ApplicationId submitAppAndGetAppId(AccessControlList viewACL,
AccessControlList modifyACL) throws Exception {
SubmitApplicationRequest submitRequest = recordFactory

View File

@ -44,13 +44,12 @@ import java.util.concurrent.BrokenBarrierException;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CyclicBarrier;
import org.junit.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.authentication.util.KerberosName;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.util.KerberosName;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.yarn.MockApps;
import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
@ -87,7 +86,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
@ -100,10 +98,11 @@ import org.apache.hadoop.yarn.api.records.QueueACL;
import org.apache.hadoop.yarn.api.records.QueueInfo;
import org.apache.hadoop.yarn.api.records.ReservationDefinition;
import org.apache.hadoop.yarn.api.records.ReservationId;
import org.apache.hadoop.yarn.api.records.ReservationRequest;
import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
import org.apache.hadoop.yarn.api.records.ReservationRequests;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ReservationRequest;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
@ -138,10 +137,10 @@ import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.hadoop.yarn.util.UTCClock;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
@ -333,7 +332,7 @@ public class TestClientRMService {
mock(ApplicationSubmissionContext.class);
YarnConfiguration config = new YarnConfiguration();
RMAppAttemptImpl rmAppAttemptImpl = new RMAppAttemptImpl(attemptId,
rmContext, yarnScheduler, null, asContext, config, false);
rmContext, yarnScheduler, null, asContext, config, false, null);
ApplicationResourceUsageReport report = rmAppAttemptImpl
.getApplicationResourceUsageReport();
assertEquals(report, RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT);
@ -1061,6 +1060,7 @@ public class TestClientRMService {
return mockSubmitAppRequest(appId, name, queue, tags, false);
}
@SuppressWarnings("deprecation")
private SubmitApplicationRequest mockSubmitAppRequest(ApplicationId appId,
String name, String queue, Set<String> tags, boolean unmanaged) {
@ -1150,26 +1150,32 @@ public class TestClientRMService {
final long memorySeconds, final long vcoreSeconds) {
ApplicationSubmissionContext asContext = mock(ApplicationSubmissionContext.class);
when(asContext.getMaxAppAttempts()).thenReturn(1);
RMAppImpl app = spy(new RMAppImpl(applicationId3, rmContext, config, null,
null, queueName, asContext, yarnScheduler, null,
System.currentTimeMillis(), "YARN", null) {
@Override
public ApplicationReport createAndGetApplicationReport(
String clientUserName, boolean allowAccess) {
ApplicationReport report = super.createAndGetApplicationReport(
clientUserName, allowAccess);
ApplicationResourceUsageReport usageReport =
report.getApplicationResourceUsageReport();
usageReport.setMemorySeconds(memorySeconds);
usageReport.setVcoreSeconds(vcoreSeconds);
report.setApplicationResourceUsageReport(usageReport);
return report;
}
});
RMAppImpl app =
spy(new RMAppImpl(applicationId3, rmContext, config, null, null,
queueName, asContext, yarnScheduler, null,
System.currentTimeMillis(), "YARN", null,
BuilderUtils.newResourceRequest(
RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
Resource.newInstance(1024, 1), 1)){
@Override
public ApplicationReport createAndGetApplicationReport(
String clientUserName, boolean allowAccess) {
ApplicationReport report = super.createAndGetApplicationReport(
clientUserName, allowAccess);
ApplicationResourceUsageReport usageReport =
report.getApplicationResourceUsageReport();
usageReport.setMemorySeconds(memorySeconds);
usageReport.setVcoreSeconds(vcoreSeconds);
report.setApplicationResourceUsageReport(usageReport);
return report;
}
});
ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(
ApplicationId.newInstance(123456, 1), 1);
RMAppAttemptImpl rmAppAttemptImpl = spy(new RMAppAttemptImpl(attemptId,
rmContext, yarnScheduler, null, asContext, config, false));
rmContext, yarnScheduler, null, asContext, config, false, null));
Container container = Container.newInstance(
ContainerId.newInstance(attemptId, 1), null, "", null, null, null);
RMContainerImpl containerimpl = spy(new RMContainerImpl(container,
@ -1230,7 +1236,7 @@ public class TestClientRMService {
rm.start();
MockNM nm;
try {
nm = rm.registerNode("127.0.0.1:0", 102400, 100);
nm = rm.registerNode("127.0.0.1:1", 102400, 100);
// allow plan follower to synchronize
Thread.sleep(1050);
} catch (Exception e) {

View File

@ -127,7 +127,7 @@ public class TestRMHA {
try {
rm.getNewAppId();
rm.registerNode("127.0.0.1:0", 2048);
rm.registerNode("127.0.0.1:1", 2048);
app = rm.submitApp(1024);
attempt = app.getCurrentAppAttempt();
rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.SCHEDULED);

View File

@ -292,7 +292,7 @@ public class TestWorkPreservingRMRestart {
1e-8);
// assert user consumed resources.
assertEquals(usedResource, leafQueue.getUser(app.getUser())
.getConsumedResources());
.getTotalConsumedResources());
}
private void checkFifoQueue(SchedulerApplication schedulerApp,

View File

@ -17,13 +17,16 @@
*******************************************************************************/
package org.apache.hadoop.yarn.server.resourcemanager.reservation;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.yarn.api.records.ReservationDefinition;
@ -37,6 +40,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ReservationRequestsPBImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
@ -44,6 +48,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManag
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.junit.Assert;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public class ReservationSystemTestUtil {
@ -55,6 +61,7 @@ public class ReservationSystemTestUtil {
return ReservationId.newInstance(rand.nextLong(), rand.nextLong());
}
@SuppressWarnings("unchecked")
public CapacityScheduler mockCapacityScheduler(int numContainers)
throws IOException {
// stolen from TestCapacityScheduler
@ -68,6 +75,29 @@ public class ReservationSystemTestUtil {
new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM(), null));
RMNodeLabelsManager nlm = mock(RMNodeLabelsManager.class);
when(
nlm.getQueueResource(any(String.class), any(Set.class),
any(Resource.class))).thenAnswer(new Answer<Resource>() {
@Override
public Resource answer(InvocationOnMock invocation) throws Throwable {
Object[] args = invocation.getArguments();
return (Resource) args[2];
}
});
when(nlm.getResourceByLabel(any(String.class), any(Resource.class)))
.thenAnswer(new Answer<Resource>() {
@Override
public Resource answer(InvocationOnMock invocation) throws Throwable {
Object[] args = invocation.getArguments();
return (Resource) args[1];
}
});
mockRmContext.setNodeLabelManager(nlm);
cs.setRMContext(mockRmContext);
try {
cs.serviceInit(conf);

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.DrainDispatcher;
@ -63,6 +64,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessM
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
@ -73,6 +75,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretMan
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@ -254,7 +257,7 @@ public class TestRMAppTransitions {
RMApp application =
new RMAppImpl(applicationId, rmContext, conf, name, user, queue,
submissionContext, scheduler, masterService,
System.currentTimeMillis(), "YARN", null);
System.currentTimeMillis(), "YARN", null, null);
testAppStartState(applicationId, user, name, queue, application);
this.rmContext.getRMApps().putIfAbsent(application.getApplicationId(),
@ -914,6 +917,7 @@ public class TestRMAppTransitions {
}
}
@SuppressWarnings("deprecation")
public void testRecoverApplication(ApplicationState appState, RMState rmState)
throws Exception {
ApplicationSubmissionContext submissionContext =
@ -923,7 +927,10 @@ public class TestRMAppTransitions {
submissionContext.getApplicationName(), null,
submissionContext.getQueue(), submissionContext, null, null,
appState.getSubmitTime(), submissionContext.getApplicationType(),
submissionContext.getApplicationTags());
submissionContext.getApplicationTags(),
BuilderUtils.newResourceRequest(
RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
submissionContext.getResource(), 1));
Assert.assertEquals(RMAppState.NEW, application.getState());
application.recover(rmState);

View File

@ -40,7 +40,9 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
@ -62,7 +64,9 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
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.YarnApplicationAttemptState;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
@ -83,8 +87,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRunningOnNodeEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRunningOnNodeEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
@ -96,7 +100,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerImpl
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContainersPulledByAMEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@ -112,6 +115,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSe
import org.apache.hadoop.yarn.server.security.MasterKeyData;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
import org.junit.After;
import org.junit.Assert;
@ -122,6 +126,8 @@ import org.junit.runners.Parameterized;
import org.mockito.ArgumentCaptor;
import org.mockito.Matchers;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@RunWith(value = Parameterized.class)
public class TestRMAppAttemptTransitions {
@ -229,6 +235,7 @@ public class TestRMAppAttemptTransitions {
this.isSecurityEnabled = isSecurityEnabled;
}
@SuppressWarnings("deprecation")
@Before
public void setUp() throws Exception {
AuthenticationMethod authMethod = AuthenticationMethod.SIMPLE;
@ -300,6 +307,7 @@ public class TestRMAppAttemptTransitions {
Mockito.doReturn(resourceScheduler).when(spyRMContext).getScheduler();
final String user = MockApps.newUserName();
final String queue = MockApps.newQueue();
submissionContext = mock(ApplicationSubmissionContext.class);
when(submissionContext.getQueue()).thenReturn(queue);
@ -315,7 +323,11 @@ public class TestRMAppAttemptTransitions {
application = mock(RMAppImpl.class);
applicationAttempt =
new RMAppAttemptImpl(applicationAttemptId, spyRMContext, scheduler,
masterService, submissionContext, new Configuration(), false);
masterService, submissionContext, new Configuration(), false,
BuilderUtils.newResourceRequest(
RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
submissionContext.getResource(), 1));
when(application.getCurrentAppAttempt()).thenReturn(applicationAttempt);
when(application.getApplicationId()).thenReturn(applicationId);
spyRMContext.getRMApps().put(application.getApplicationId(), application);
@ -1399,13 +1411,16 @@ public class TestRMAppAttemptTransitions {
}
@SuppressWarnings("deprecation")
@Test
public void testContainersCleanupForLastAttempt() {
// create a failed attempt.
applicationAttempt =
new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(), spyRMContext,
scheduler, masterService, submissionContext, new Configuration(),
true);
true, BuilderUtils.newResourceRequest(
RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
submissionContext.getResource(), 1));
when(submissionContext.getKeepContainersAcrossApplicationAttempts())
.thenReturn(true);
when(submissionContext.getMaxAppAttempts()).thenReturn(1);
@ -1427,6 +1442,49 @@ public class TestRMAppAttemptTransitions {
assertFalse(transferStateFromPreviousAttempt);
verifyApplicationAttemptFinished(RMAppAttemptState.FAILED);
}
@SuppressWarnings("unchecked")
@Test
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)))
.thenAnswer(new Answer<Allocation>() {
@SuppressWarnings("rawtypes")
@Override
public Allocation answer(InvocationOnMock invocation)
throws Throwable {
ResourceRequest rr =
(ResourceRequest) ((List) invocation.getArguments()[1]).get(0);
// capacity shouldn't changed
assertEquals(Resource.newInstance(3333, 1), rr.getCapability());
assertEquals("label-expression", rr.getNodeLabelExpression());
// priority, #container, relax-locality will be changed
assertEquals(RMAppAttemptImpl.AM_CONTAINER_PRIORITY, rr.getPriority());
assertEquals(1, rr.getNumContainers());
assertEquals(ResourceRequest.ANY, rr.getResourceName());
// just return an empty allocation
List l = new ArrayList();
Set s = new HashSet();
return new Allocation(l, Resources.none(), s, s, l);
}
});
// create an attempt.
applicationAttempt =
new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(),
spyRMContext, scheduler, masterService, submissionContext,
new Configuration(), true, ResourceRequest.newInstance(
Priority.UNDEFINED, "host1", Resource.newInstance(3333, 1), 3,
false, "label-expression"));
new RMAppAttemptImpl.ScheduleTransition().transition(
(RMAppAttemptImpl) applicationAttempt, null);
}
private void verifyAMCrashAtAllocatedDiagnosticInfo(String diagnostics,
int exitCode, boolean shouldCheckURL) {

View File

@ -21,13 +21,19 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.security.PrivilegedAction;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -46,6 +52,7 @@ import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.QueueInfo;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
@ -58,6 +65,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMWithAMS;
import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager;
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.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@ -74,6 +82,8 @@ import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.Assert;
import org.junit.Test;
import com.google.common.collect.Sets;
public class TestSchedulerUtils {
private static final Log LOG = LogFactory.getLog(TestSchedulerUtils.class);
@ -173,69 +183,240 @@ public class TestSchedulerUtils {
assertEquals(1, ask.getCapability().getVirtualCores());
assertEquals(2048, ask.getCapability().getMemory());
}
@Test (timeout = 30000)
public void testValidateResourceRequest() {
public void testValidateResourceRequestWithErrorLabelsPermission()
throws IOException {
// mock queue and scheduler
YarnScheduler scheduler = mock(YarnScheduler.class);
Set<String> queueAccessibleNodeLabels = Sets.newHashSet();
QueueInfo queueInfo = mock(QueueInfo.class);
when(queueInfo.getQueueName()).thenReturn("queue");
when(queueInfo.getAccessibleNodeLabels()).thenReturn(queueAccessibleNodeLabels);
when(scheduler.getQueueInfo(any(String.class), anyBoolean(), anyBoolean()))
.thenReturn(queueInfo);
Resource maxResource = Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
// zero memory
// queue has labels, success cases
try {
// set queue accessible node labesl to [x, y]
queueAccessibleNodeLabels.clear();
queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
Resource resource = Resources.createResource(
0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource);
resReq.setNodeLabelExpression("x");
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
resReq.setNodeLabelExpression("x && y");
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
resReq.setNodeLabelExpression("y");
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
resReq.setNodeLabelExpression("");
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
resReq.setNodeLabelExpression(" ");
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
} catch (InvalidResourceRequestException e) {
e.printStackTrace();
fail("Should be valid when request labels is a subset of queue labels");
}
// queue has labels, failed cases (when ask a label not included by queue)
try {
// set queue accessible node labesl to [x, y]
queueAccessibleNodeLabels.clear();
queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
Resource resource = Resources.createResource(
0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1);
resReq.setNodeLabelExpression("z");
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
fail("Should fail");
} catch (InvalidResourceRequestException e) {
}
try {
// set queue accessible node labesl to [x, y]
queueAccessibleNodeLabels.clear();
queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
Resource resource = Resources.createResource(
0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1);
resReq.setNodeLabelExpression("x && y && z");
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
fail("Should fail");
} catch (InvalidResourceRequestException e) {
}
// queue doesn't have label, succeed (when request no label)
queueAccessibleNodeLabels.clear();
try {
// set queue accessible node labels to empty
queueAccessibleNodeLabels.clear();
Resource resource = Resources.createResource(
0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
resReq.setNodeLabelExpression("");
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
resReq.setNodeLabelExpression(" ");
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
} catch (InvalidResourceRequestException e) {
e.printStackTrace();
fail("Should be valid when request labels is empty");
}
// queue doesn't have label, failed (when request any label)
try {
// set queue accessible node labels to empty
queueAccessibleNodeLabels.clear();
Resource resource = Resources.createResource(
0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1);
resReq.setNodeLabelExpression("x");
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
fail("Should fail");
} catch (InvalidResourceRequestException e) {
}
// queue is "*", always succeeded
try {
// set queue accessible node labels to empty
queueAccessibleNodeLabels.clear();
queueAccessibleNodeLabels.add(RMNodeLabelsManager.ANY);
Resource resource = Resources.createResource(
0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1);
resReq.setNodeLabelExpression("x");
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
resReq.setNodeLabelExpression("x && y && z");
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
resReq.setNodeLabelExpression("z");
SchedulerUtils.validateResourceRequest(resReq, maxResource, "queue",
scheduler);
} catch (InvalidResourceRequestException e) {
e.printStackTrace();
fail("Should be valid when request labels is empty");
}
}
@Test (timeout = 30000)
public void testValidateResourceRequest() {
YarnScheduler mockScheduler = mock(YarnScheduler.class);
Resource maxResource =
Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
// zero memory
try {
Resource resource =
Resources.createResource(0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
} catch (InvalidResourceRequestException e) {
fail("Zero memory should be accepted");
}
// zero vcores
try {
Resource resource = Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
0);
ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource);
Resource resource =
Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0);
ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
} catch (InvalidResourceRequestException e) {
fail("Zero vcores should be accepted");
}
// max memory
try {
Resource resource = Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource);
Resource resource =
Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
} catch (InvalidResourceRequestException e) {
fail("Max memory should be accepted");
}
// max vcores
try {
Resource resource = Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource);
Resource resource =
Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
} catch (InvalidResourceRequestException e) {
fail("Max vcores should not be accepted");
}
// negative memory
try {
Resource resource = Resources.createResource(
-1,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource);
Resource resource =
Resources.createResource(-1,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
fail("Negative memory should not be accepted");
} catch (InvalidResourceRequestException e) {
// expected
@ -243,12 +424,14 @@ public class TestSchedulerUtils {
// negative vcores
try {
Resource resource = Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
-1);
ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource);
Resource resource =
Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, -1);
ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
fail("Negative vcores should not be accepted");
} catch (InvalidResourceRequestException e) {
// expected
@ -256,12 +439,15 @@ public class TestSchedulerUtils {
// more than max memory
try {
Resource resource = Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource);
Resource resource =
Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
fail("More than max memory should not be accepted");
} catch (InvalidResourceRequestException e) {
// expected
@ -269,13 +455,16 @@ public class TestSchedulerUtils {
// more than max vcores
try {
Resource resource = Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES
+ 1);
ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource);
Resource resource =
Resources
.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES + 1);
ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
fail("More than max vcores should not be accepted");
} catch (InvalidResourceRequestException e) {
// expected

View File

@ -65,6 +65,9 @@ public class TestApplicationLimits {
LeafQueue queue;
private final ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
RMContext rmContext = null;
@Before
public void setUp() throws IOException {
@ -73,7 +76,9 @@ public class TestApplicationLimits {
YarnConfiguration conf = new YarnConfiguration();
setupQueueConfiguration(csConf);
rmContext = TestUtils.getMockRMContext();
CapacitySchedulerContext csContext = mock(CapacitySchedulerContext.class);
when(csContext.getConfiguration()).thenReturn(csConf);
when(csContext.getConf()).thenReturn(conf);
@ -89,6 +94,8 @@ public class TestApplicationLimits {
thenReturn(CapacityScheduler.queueComparator);
when(csContext.getResourceCalculator()).
thenReturn(resourceCalculator);
when(csContext.getRMContext()).thenReturn(rmContext);
RMContainerTokenSecretManager containerTokenSecretManager =
new RMContainerTokenSecretManager(conf);
containerTokenSecretManager.rollMasterKey();
@ -162,6 +169,7 @@ public class TestApplicationLimits {
when(csContext.getQueueComparator()).
thenReturn(CapacityScheduler.queueComparator);
when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
when(csContext.getRMContext()).thenReturn(rmContext);
// Say cluster has 100 nodes of 16G each
Resource clusterResource = Resources.createResource(100 * 16 * GB, 100 * 16);
@ -475,6 +483,7 @@ public class TestApplicationLimits {
when(csContext.getQueueComparator()).
thenReturn(CapacityScheduler.queueComparator);
when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
when(csContext.getRMContext()).thenReturn(rmContext);
// Say cluster has 100 nodes of 16G each
Resource clusterResource = Resources.createResource(100 * 16 * GB);

View File

@ -19,38 +19,19 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.inOrder;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.reset;
import static org.mockito.Mockito.when;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.junit.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.mockito.InOrder;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public class TestCSQueueUtils {
@ -88,6 +69,8 @@ public class TestCSQueueUtils {
thenReturn(Resources.createResource(GB, 1));
when(csContext.getMaximumResourceCapability()).
thenReturn(Resources.createResource(0, 0));
RMContext rmContext = TestUtils.getMockRMContext();
when(csContext.getRMContext()).thenReturn(rmContext);
final String L1Q1 = "L1Q1";
csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {L1Q1});
@ -129,6 +112,8 @@ public class TestCSQueueUtils {
thenReturn(Resources.createResource(GB, 1));
when(csContext.getMaximumResourceCapability()).
thenReturn(Resources.createResource(16*GB, 32));
RMContext rmContext = TestUtils.getMockRMContext();
when(csContext.getRMContext()).thenReturn(rmContext);
final String L1Q1 = "L1Q1";
csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {L1Q1});
@ -174,6 +159,9 @@ public class TestCSQueueUtils {
when(csContext.getMaximumResourceCapability()).
thenReturn(Resources.createResource(16*GB, 32));
RMContext rmContext = TestUtils.getMockRMContext();
when(csContext.getRMContext()).thenReturn(rmContext);
final String L1Q1 = "L1Q1";
final String L1Q2 = "L1Q2";
final String L2Q1 = "L2Q1";

View File

@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterReque
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState;
@ -64,7 +65,6 @@ import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
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.ApplicationResourceUsageReport;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.exceptions.YarnException;
@ -84,6 +84,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.Task;
import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MockRMWithAMS;
import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.DummyRMNodeLabelsManager;
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.RMAppMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
@ -149,7 +151,14 @@ public class TestCapacityScheduler {
@Before
public void setUp() throws Exception {
resourceManager = new ResourceManager();
resourceManager = new ResourceManager() {
@Override
protected RMNodeLabelsManager createNodeLabelManager() {
RMNodeLabelsManager mgr = new DummyRMNodeLabelsManager();
mgr.init(getConfig());
return mgr;
}
};
CapacitySchedulerConfiguration csConf
= new CapacitySchedulerConfiguration();
setupQueueConfiguration(csConf);
@ -962,10 +971,7 @@ public class TestCapacityScheduler {
YarnConfiguration conf = new YarnConfiguration();
CapacityScheduler cs = new CapacityScheduler();
cs.setConf(conf);
RMContextImpl rmContext = new RMContextImpl(null, null, null, null, null,
null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM(), null);
RMContext rmContext = TestUtils.getMockRMContext();
cs.setRMContext(rmContext);
CapacitySchedulerConfiguration csConf =
new CapacitySchedulerConfiguration();
@ -1476,8 +1482,14 @@ public class TestCapacityScheduler {
@Test(expected = YarnException.class)
public void testMoveAppViolateQueueState() throws Exception {
resourceManager = new ResourceManager();
resourceManager = new ResourceManager() {
@Override
protected RMNodeLabelsManager createNodeLabelManager() {
RMNodeLabelsManager mgr = new DummyRMNodeLabelsManager();
mgr.init(getConfig());
return mgr;
}
};
CapacitySchedulerConfiguration csConf =
new CapacitySchedulerConfiguration();
setupQueueConfiguration(csConf);

View File

@ -99,6 +99,7 @@ public class TestChildQueueOrder {
thenReturn(CapacityScheduler.queueComparator);
when(csContext.getResourceCalculator()).
thenReturn(resourceComparator);
when(csContext.getRMContext()).thenReturn(rmContext);
}
private FiCaSchedulerApp getMockApplication(int appId, String user) {
@ -132,11 +133,11 @@ public class TestChildQueueOrder {
final Resource allocatedResource = Resources.createResource(allocation);
if (queue instanceof ParentQueue) {
((ParentQueue)queue).allocateResource(clusterResource,
allocatedResource);
allocatedResource, null);
} else {
FiCaSchedulerApp app1 = getMockApplication(0, "");
((LeafQueue)queue).allocateResource(clusterResource, app1,
allocatedResource);
allocatedResource, null);
}
// Next call - nothing

View File

@ -20,12 +20,14 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.SecurityUtilTestHelper;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
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.LogAggregationContext;
@ -41,19 +43,28 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.RMSecretManagerService;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.TestFifoScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.DummyRMNodeLabelsManager;
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.RMAppAttempt;
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.rmcontainer.RMContainerState;
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.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
public class TestContainerAllocation {
@ -62,12 +73,16 @@ public class TestContainerAllocation {
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 DummyRMNodeLabelsManager();
mgr.init(conf);
}
@Test(timeout = 3000000)
@ -305,4 +320,449 @@ public class TestContainerAllocation {
rm1.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.ALLOCATED);
MockRM.launchAndRegisterAM(app1, rm1, nm1);
}
private Configuration getConfigurationWithDefaultQueueLabels(
Configuration config) {
final String A = CapacitySchedulerConfiguration.ROOT + ".a";
final String B = CapacitySchedulerConfiguration.ROOT + ".b";
CapacitySchedulerConfiguration conf =
(CapacitySchedulerConfiguration) getConfigurationWithQueueLabels(config);
new CapacitySchedulerConfiguration(config);
conf.setDefaultNodeLabelExpression(A, "x");
conf.setDefaultNodeLabelExpression(B, "y");
return conf;
}
private Configuration getConfigurationWithQueueLabels(Configuration config) {
CapacitySchedulerConfiguration conf =
new CapacitySchedulerConfiguration(config);
// Define top-level queues
conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"});
final String A = CapacitySchedulerConfiguration.ROOT + ".a";
conf.setCapacity(A, 10);
conf.setMaximumCapacity(A, 15);
conf.setAccessibleNodeLabels(A, toSet("x"));
conf.setCapacityByLabel(A, "x", 100);
final String B = CapacitySchedulerConfiguration.ROOT + ".b";
conf.setCapacity(B, 20);
conf.setAccessibleNodeLabels(B, toSet("y"));
conf.setCapacityByLabel(B, "y", 100);
final String C = CapacitySchedulerConfiguration.ROOT + ".c";
conf.setCapacity(C, 70);
conf.setMaximumCapacity(C, 70);
conf.setAccessibleNodeLabels(C, RMNodeLabelsManager.EMPTY_STRING_SET);
// Define 2nd-level queues
final String A1 = A + ".a1";
conf.setQueues(A, new String[] {"a1"});
conf.setCapacity(A1, 100);
conf.setMaximumCapacity(A1, 100);
conf.setCapacityByLabel(A1, "x", 100);
final String B1 = B + ".b1";
conf.setQueues(B, new String[] {"b1"});
conf.setCapacity(B1, 100);
conf.setMaximumCapacity(B1, 100);
conf.setCapacityByLabel(B1, "y", 100);
final String C1 = C + ".c1";
conf.setQueues(C, new String[] {"c1"});
conf.setCapacity(C1, 100);
conf.setMaximumCapacity(C1, 100);
return conf;
}
private void checkTaskContainersHost(ApplicationAttemptId attemptId,
ContainerId containerId, ResourceManager rm, String host) {
YarnScheduler scheduler = rm.getRMContext().getScheduler();
SchedulerAppReport appReport = scheduler.getSchedulerAppInfo(attemptId);
Assert.assertTrue(appReport.getLiveContainers().size() > 0);
for (RMContainer c : appReport.getLiveContainers()) {
if (c.getContainerId().equals(containerId)) {
Assert.assertEquals(host, c.getAllocatedNode().getHost());
}
}
}
private <E> Set<E> toSet(E... elements) {
Set<E> set = Sets.newHashSet(elements);
return set;
}
private Configuration getComplexConfigurationWithQueueLabels(
Configuration config) {
CapacitySchedulerConfiguration conf =
new CapacitySchedulerConfiguration(config);
// Define top-level queues
conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
final String A = CapacitySchedulerConfiguration.ROOT + ".a";
conf.setCapacity(A, 10);
conf.setMaximumCapacity(A, 10);
conf.setAccessibleNodeLabels(A, toSet("x", "y"));
conf.setCapacityByLabel(A, "x", 100);
conf.setCapacityByLabel(A, "y", 50);
final String B = CapacitySchedulerConfiguration.ROOT + ".b";
conf.setCapacity(B, 90);
conf.setMaximumCapacity(B, 100);
conf.setAccessibleNodeLabels(B, toSet("y", "z"));
conf.setCapacityByLabel(B, "y", 50);
conf.setCapacityByLabel(B, "z", 100);
// Define 2nd-level queues
final String A1 = A + ".a1";
conf.setQueues(A, new String[] {"a1"});
conf.setCapacity(A1, 100);
conf.setMaximumCapacity(A1, 100);
conf.setAccessibleNodeLabels(A1, toSet("x", "y"));
conf.setDefaultNodeLabelExpression(A1, "x");
conf.setCapacityByLabel(A1, "x", 100);
conf.setCapacityByLabel(A1, "y", 100);
conf.setQueues(B, new String[] {"b1", "b2"});
final String B1 = B + ".b1";
conf.setCapacity(B1, 50);
conf.setMaximumCapacity(B1, 50);
conf.setAccessibleNodeLabels(B1, RMNodeLabelsManager.EMPTY_STRING_SET);
final String B2 = B + ".b2";
conf.setCapacity(B2, 50);
conf.setMaximumCapacity(B2, 50);
conf.setAccessibleNodeLabels(B2, toSet("y", "z"));
conf.setCapacityByLabel(B2, "y", 100);
conf.setCapacityByLabel(B2, "z", 100);
return conf;
}
@Test(timeout = 300000)
public void testContainerAllocationWithSingleUserLimits() throws Exception {
final RMNodeLabelsManager mgr = new DummyRMNodeLabelsManager();
mgr.init(conf);
// set node -> label
mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
NodeId.newInstance("h2", 0), toSet("y")));
// inject node label manager
MockRM rm1 = new MockRM(getConfigurationWithDefaultQueueLabels(conf)) {
@Override
public RMNodeLabelsManager createNodeLabelManager() {
return mgr;
}
};
rm1.getRMContext().setNodeLabelManager(mgr);
rm1.start();
MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
MockNM nm2 = rm1.registerNode("h2:1234", 8000); // label = y
MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
// launch an app to queue a1 (label = x), and check all container will
// be allocated in h1
RMApp app1 = rm1.submitApp(200, "app", "user", null, "a1");
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
// A has only 10% of x, so it can only allocate one container in label=empty
ContainerId containerId =
ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "");
Assert.assertTrue(rm1.waitForState(nm3, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
// Cannot allocate 2nd label=empty container
containerId =
ContainerId.newInstance(am1.getApplicationAttemptId(), 3);
am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "");
Assert.assertFalse(rm1.waitForState(nm3, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
// A has default user limit = 100, so it can use all resource in label = x
// We can allocate floor(8000 / 1024) = 7 containers
for (int id = 3; id <= 8; id++) {
containerId =
ContainerId.newInstance(am1.getApplicationAttemptId(), id);
am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x");
Assert.assertTrue(rm1.waitForState(nm1, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
}
rm1.close();
}
@Test(timeout = 300000)
public void testContainerAllocateWithComplexLabels() throws Exception {
/*
* Queue structure:
* root (*)
* ________________
* / \
* a x(100%), y(50%) b y(50%), z(100%)
* ________________ ______________
* / / \
* a1 (x,y) b1(no) b2(y,z)
* 100% y = 100%, z = 100%
*
* Node structure:
* h1 : x
* h2 : x, y
* h3 : y
* h4 : y, z
* h5 : NO
*
* Total resource:
* x: 4G
* y: 6G
* z: 2G
* *: 2G
*
* Resource of
* a1: x=4G, y=3G, NO=0.2G
* b1: NO=0.9G (max=1G)
* b2: y=3, z=2G, NO=0.9G (max=1G)
*
* Each node can only allocate two containers
*/
// set node -> label
mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y", "z"));
mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0),
toSet("x"), NodeId.newInstance("h2", 0), toSet("x", "y"),
NodeId.newInstance("h3", 0), toSet("y"), NodeId.newInstance("h4", 0),
toSet("y", "z"), NodeId.newInstance("h5", 0),
RMNodeLabelsManager.EMPTY_STRING_SET));
// inject node label manager
MockRM rm1 = new MockRM(getComplexConfigurationWithQueueLabels(conf)) {
@Override
public RMNodeLabelsManager createNodeLabelManager() {
return mgr;
}
};
rm1.getRMContext().setNodeLabelManager(mgr);
rm1.start();
MockNM nm1 = rm1.registerNode("h1:1234", 2048);
MockNM nm2 = rm1.registerNode("h2:1234", 2048);
MockNM nm3 = rm1.registerNode("h3:1234", 2048);
MockNM nm4 = rm1.registerNode("h4:1234", 2048);
MockNM nm5 = rm1.registerNode("h5:1234", 2048);
ContainerId containerId;
// launch an app to queue a1 (label = x), and check all container will
// be allocated in h1
RMApp app1 = rm1.submitApp(1024, "app", "user", null, "a1");
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
// request a container (label = x && y). can only allocate on nm2
am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x && y");
containerId =
ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
Assert.assertFalse(rm1.waitForState(nm1, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
Assert.assertTrue(rm1.waitForState(nm2, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
checkTaskContainersHost(am1.getApplicationAttemptId(), containerId, rm1,
"h2");
// launch an app to queue b1 (label = y), and check all container will
// be allocated in h5
RMApp app2 = rm1.submitApp(1024, "app", "user", null, "b1");
MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm5);
// request a container for AM, will succeed
// and now b1's queue capacity will be used, cannot allocate more containers
// (Maximum capacity reached)
am2.allocate("*", 1024, 1, new ArrayList<ContainerId>());
containerId = ContainerId.newInstance(am2.getApplicationAttemptId(), 2);
Assert.assertFalse(rm1.waitForState(nm4, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
Assert.assertFalse(rm1.waitForState(nm5, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
// launch an app to queue b2
RMApp app3 = rm1.submitApp(1024, "app", "user", null, "b2");
MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm5);
// request a container. try to allocate on nm1 (label = x) and nm3 (label =
// y,z). Will successfully allocate on nm3
am3.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y");
containerId = ContainerId.newInstance(am3.getApplicationAttemptId(), 2);
Assert.assertFalse(rm1.waitForState(nm1, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
Assert.assertTrue(rm1.waitForState(nm3, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
"h3");
// try to allocate container (request label = y && z) on nm3 (label = y) and
// nm4 (label = y,z). Will sucessfully allocate on nm4 only.
am3.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y && z");
containerId = ContainerId.newInstance(am3.getApplicationAttemptId(), 3);
Assert.assertFalse(rm1.waitForState(nm3, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
Assert.assertTrue(rm1.waitForState(nm4, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
"h4");
rm1.close();
}
@Test (timeout = 120000)
public void testContainerAllocateWithLabels() throws Exception {
// set node -> label
mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
NodeId.newInstance("h2", 0), toSet("y")));
// inject node label manager
MockRM rm1 = new MockRM(getConfigurationWithQueueLabels(conf)) {
@Override
public RMNodeLabelsManager createNodeLabelManager() {
return mgr;
}
};
rm1.getRMContext().setNodeLabelManager(mgr);
rm1.start();
MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
MockNM nm2 = rm1.registerNode("h2:1234", 8000); // label = y
MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
ContainerId containerId;
// launch an app to queue a1 (label = x), and check all container will
// be allocated in h1
RMApp app1 = rm1.submitApp(200, "app", "user", null, "a1");
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm3);
// request a container.
am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x");
containerId =
ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
Assert.assertFalse(rm1.waitForState(nm2, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
Assert.assertTrue(rm1.waitForState(nm1, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
checkTaskContainersHost(am1.getApplicationAttemptId(), containerId, rm1,
"h1");
// launch an app to queue b1 (label = y), and check all container will
// be allocated in h2
RMApp app2 = rm1.submitApp(200, "app", "user", null, "b1");
MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm3);
// request a container.
am2.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y");
containerId = ContainerId.newInstance(am2.getApplicationAttemptId(), 2);
Assert.assertFalse(rm1.waitForState(nm1, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
Assert.assertTrue(rm1.waitForState(nm2, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
checkTaskContainersHost(am2.getApplicationAttemptId(), containerId, rm1,
"h2");
// launch an app to queue c1 (label = ""), and check all container will
// be allocated in h3
RMApp app3 = rm1.submitApp(200, "app", "user", null, "c1");
MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm3);
// request a container.
am3.allocate("*", 1024, 1, new ArrayList<ContainerId>());
containerId = ContainerId.newInstance(am3.getApplicationAttemptId(), 2);
Assert.assertFalse(rm1.waitForState(nm2, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
Assert.assertTrue(rm1.waitForState(nm3, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
"h3");
rm1.close();
}
@Test (timeout = 120000)
public void testContainerAllocateWithDefaultQueueLabels() throws Exception {
// This test is pretty much similar to testContainerAllocateWithLabel.
// Difference is, this test doesn't specify label expression in ResourceRequest,
// instead, it uses default queue label expression
// set node -> label
mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x"),
NodeId.newInstance("h2", 0), toSet("y")));
// inject node label manager
MockRM rm1 = new MockRM(getConfigurationWithDefaultQueueLabels(conf)) {
@Override
public RMNodeLabelsManager createNodeLabelManager() {
return mgr;
}
};
rm1.getRMContext().setNodeLabelManager(mgr);
rm1.start();
MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x
MockNM nm2 = rm1.registerNode("h2:1234", 8000); // label = y
MockNM nm3 = rm1.registerNode("h3:1234", 8000); // label = <empty>
ContainerId containerId;
// launch an app to queue a1 (label = x), and check all container will
// be allocated in h1
RMApp app1 = rm1.submitApp(200, "app", "user", null, "a1");
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
// request a container.
am1.allocate("*", 1024, 1, new ArrayList<ContainerId>());
containerId =
ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
Assert.assertFalse(rm1.waitForState(nm3, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
Assert.assertTrue(rm1.waitForState(nm1, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
checkTaskContainersHost(am1.getApplicationAttemptId(), containerId, rm1,
"h1");
// launch an app to queue b1 (label = y), and check all container will
// be allocated in h2
RMApp app2 = rm1.submitApp(200, "app", "user", null, "b1");
MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm2);
// request a container.
am2.allocate("*", 1024, 1, new ArrayList<ContainerId>());
containerId = ContainerId.newInstance(am2.getApplicationAttemptId(), 2);
Assert.assertFalse(rm1.waitForState(nm3, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
Assert.assertTrue(rm1.waitForState(nm2, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
checkTaskContainersHost(am2.getApplicationAttemptId(), containerId, rm1,
"h2");
// launch an app to queue c1 (label = ""), and check all container will
// be allocated in h3
RMApp app3 = rm1.submitApp(200, "app", "user", null, "c1");
MockAM am3 = MockRM.launchAndRegisterAM(app3, rm1, nm3);
// request a container.
am3.allocate("*", 1024, 1, new ArrayList<ContainerId>());
containerId = ContainerId.newInstance(am3.getApplicationAttemptId(), 2);
Assert.assertFalse(rm1.waitForState(nm2, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
Assert.assertTrue(rm1.waitForState(nm3, containerId,
RMContainerState.ALLOCATED, 10 * 1000));
checkTaskContainersHost(am3.getApplicationAttemptId(), containerId, rm1,
"h3");
rm1.close();
}
}

View File

@ -40,10 +40,10 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.junit.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.security.UserGroupInformation;
@ -63,6 +63,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@ -82,6 +83,7 @@ import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Matchers;
@ -147,6 +149,7 @@ public class TestLeafQueue {
thenReturn(CapacityScheduler.queueComparator);
when(csContext.getResourceCalculator()).
thenReturn(resourceCalculator);
when(csContext.getRMContext()).thenReturn(rmContext);
RMContainerTokenSecretManager containerTokenSecretManager =
new RMContainerTokenSecretManager(conf);
containerTokenSecretManager.rollMasterKey();
@ -686,8 +689,9 @@ public class TestLeafQueue {
1, qb.getActiveUsersManager().getNumActiveUsers());
//get headroom
qb.assignContainers(clusterResource, node_0, false);
qb.computeUserLimitAndSetHeadroom(app_0, clusterResource,
app_0.getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability());
qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0
.getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
null);
//maxqueue 16G, userlimit 13G, - 4G used = 9G
assertEquals(9*GB,app_0.getHeadroom().getMemory());
@ -704,8 +708,9 @@ public class TestLeafQueue {
u1Priority, recordFactory)));
qb.submitApplicationAttempt(app_2, user_1);
qb.assignContainers(clusterResource, node_1, false);
qb.computeUserLimitAndSetHeadroom(app_0, clusterResource,
app_0.getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability());
qb.computeUserLimitAndSetHeadroom(app_0, clusterResource, app_0
.getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
null);
assertEquals(8*GB, qb.getUsedResources().getMemory());
assertEquals(4*GB, app_0.getCurrentConsumption().getMemory());
@ -718,8 +723,10 @@ public class TestLeafQueue {
//test case 3
qb.finishApplication(app_0.getApplicationId(), user_0);
qb.finishApplication(app_2.getApplicationId(), user_1);
qb.releaseResource(clusterResource, app_0, app_0.getResource(u0Priority));
qb.releaseResource(clusterResource, app_2, app_2.getResource(u1Priority));
qb.releaseResource(clusterResource, app_0, app_0.getResource(u0Priority),
null);
qb.releaseResource(clusterResource, app_2, app_2.getResource(u1Priority),
null);
qb.setUserLimit(50);
qb.setUserLimitFactor(1);
@ -744,8 +751,9 @@ public class TestLeafQueue {
qb.submitApplicationAttempt(app_3, user_1);
qb.assignContainers(clusterResource, node_0, false);
qb.assignContainers(clusterResource, node_0, false);
qb.computeUserLimitAndSetHeadroom(app_3, clusterResource,
app_3.getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability());
qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, app_3
.getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability(),
null);
assertEquals(4*GB, qb.getUsedResources().getMemory());
//maxqueue 16G, userlimit 7G, used (by each user) 2G, headroom 5G (both)
assertEquals(5*GB, app_3.getHeadroom().getMemory());
@ -761,10 +769,12 @@ public class TestLeafQueue {
TestUtils.createResourceRequest(ResourceRequest.ANY, 6*GB, 1, true,
u0Priority, recordFactory)));
qb.assignContainers(clusterResource, node_1, false);
qb.computeUserLimitAndSetHeadroom(app_4, clusterResource,
app_4.getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability());
qb.computeUserLimitAndSetHeadroom(app_3, clusterResource,
app_3.getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability());
qb.computeUserLimitAndSetHeadroom(app_4, clusterResource, app_4
.getResourceRequest(u0Priority, ResourceRequest.ANY).getCapability(),
null);
qb.computeUserLimitAndSetHeadroom(app_3, clusterResource, app_3
.getResourceRequest(u1Priority, ResourceRequest.ANY).getCapability(),
null);
//app3 is user1, active from last test case
@ -2272,6 +2282,7 @@ public class TestLeafQueue {
Resource clusterResource = Resources
.createResource(100 * 16 * GB, 100 * 32);
CapacitySchedulerContext csContext = mockCSContext(csConf, clusterResource);
when(csContext.getRMContext()).thenReturn(rmContext);
csConf.setFloat(CapacitySchedulerConfiguration.
MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, 0.1f);
ParentQueue root = new ParentQueue(csContext,

View File

@ -95,6 +95,7 @@ public class TestParentQueue {
thenReturn(CapacityScheduler.queueComparator);
when(csContext.getResourceCalculator()).
thenReturn(resourceComparator);
when(csContext.getRMContext()).thenReturn(rmContext);
}
private static final String A = "a";
@ -144,11 +145,11 @@ public class TestParentQueue {
final Resource allocatedResource = Resources.createResource(allocation);
if (queue instanceof ParentQueue) {
((ParentQueue)queue).allocateResource(clusterResource,
allocatedResource);
allocatedResource, null);
} else {
FiCaSchedulerApp app1 = getMockApplication(0, "");
((LeafQueue)queue).allocateResource(clusterResource, app1,
allocatedResource);
allocatedResource, null);
}
// Next call - nothing

View File

@ -27,14 +27,11 @@ import org.apache.hadoop.security.GroupMappingServiceProvider;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.SimpleGroupsMapping;
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.junit.After;
import org.junit.Assert;
import org.junit.Test;
@ -79,10 +76,7 @@ public class TestQueueMappings {
YarnConfiguration conf = new YarnConfiguration(csConf);
CapacityScheduler cs = new CapacityScheduler();
RMContextImpl rmContext = new RMContextImpl(null, null, null, null, null,
null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM(), null);
RMContext rmContext = TestUtils.getMockRMContext();
cs.setConf(conf);
cs.setRMContext(rmContext);
cs.init(conf);

View File

@ -18,23 +18,41 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import org.junit.Assert;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import com.google.common.collect.ImmutableSet;
public class TestQueueParsing {
private static final Log LOG = LogFactory.getLog(TestQueueParsing.class);
private static final double DELTA = 0.000001;
private RMNodeLabelsManager nodeLabelManager;
@Before
public void setup() {
nodeLabelManager = mock(RMNodeLabelsManager.class);
when(nodeLabelManager.containsNodeLabel(any(String.class))).thenReturn(true);
}
@Test
public void testQueueParsing() throws Exception {
CapacitySchedulerConfiguration csConf =
@ -43,15 +61,11 @@ public class TestQueueParsing {
YarnConfiguration conf = new YarnConfiguration(csConf);
CapacityScheduler capacityScheduler = new CapacityScheduler();
RMContextImpl rmContext = new RMContextImpl(null, null,
null, null, null, null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM(), null);
capacityScheduler.setConf(conf);
capacityScheduler.setRMContext(rmContext);
capacityScheduler.setRMContext(TestUtils.getMockRMContext());
capacityScheduler.init(conf);
capacityScheduler.start();
capacityScheduler.reinitialize(conf, rmContext);
capacityScheduler.reinitialize(conf, TestUtils.getMockRMContext());
CSQueue a = capacityScheduler.getQueue("a");
Assert.assertEquals(0.10, a.getAbsoluteCapacity(), DELTA);
@ -202,4 +216,241 @@ public class TestQueueParsing {
capacityScheduler.stop();
}
private void setupQueueConfigurationWithoutLabels(CapacitySchedulerConfiguration conf) {
// Define top-level queues
conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
final String A = CapacitySchedulerConfiguration.ROOT + ".a";
conf.setCapacity(A, 10);
conf.setMaximumCapacity(A, 15);
final String B = CapacitySchedulerConfiguration.ROOT + ".b";
conf.setCapacity(B, 90);
LOG.info("Setup top-level queues");
// Define 2nd-level queues
final String A1 = A + ".a1";
final String A2 = A + ".a2";
conf.setQueues(A, new String[] {"a1", "a2"});
conf.setCapacity(A1, 30);
conf.setMaximumCapacity(A1, 45);
conf.setCapacity(A2, 70);
conf.setMaximumCapacity(A2, 85);
final String B1 = B + ".b1";
final String B2 = B + ".b2";
final String B3 = B + ".b3";
conf.setQueues(B, new String[] {"b1", "b2", "b3"});
conf.setCapacity(B1, 50);
conf.setMaximumCapacity(B1, 85);
conf.setCapacity(B2, 30);
conf.setMaximumCapacity(B2, 35);
conf.setCapacity(B3, 20);
conf.setMaximumCapacity(B3, 35);
}
private void setupQueueConfigurationWithLabels(CapacitySchedulerConfiguration conf) {
// Define top-level queues
conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
final String A = CapacitySchedulerConfiguration.ROOT + ".a";
conf.setCapacity(A, 10);
conf.setMaximumCapacity(A, 15);
final String B = CapacitySchedulerConfiguration.ROOT + ".b";
conf.setCapacity(B, 90);
LOG.info("Setup top-level queues");
// Define 2nd-level queues
final String A1 = A + ".a1";
final String A2 = A + ".a2";
conf.setQueues(A, new String[] {"a1", "a2"});
conf.setAccessibleNodeLabels(A, ImmutableSet.of("red", "blue"));
conf.setCapacityByLabel(A, "red", 50);
conf.setCapacityByLabel(A, "blue", 50);
conf.setCapacity(A1, 30);
conf.setMaximumCapacity(A1, 45);
conf.setCapacityByLabel(A1, "red", 50);
conf.setCapacityByLabel(A1, "blue", 100);
conf.setCapacity(A2, 70);
conf.setMaximumCapacity(A2, 85);
conf.setAccessibleNodeLabels(A2, ImmutableSet.of("red"));
conf.setCapacityByLabel(A2, "red", 50);
final String B1 = B + ".b1";
final String B2 = B + ".b2";
final String B3 = B + ".b3";
conf.setQueues(B, new String[] {"b1", "b2", "b3"});
conf.setAccessibleNodeLabels(B, ImmutableSet.of("red", "blue"));
conf.setCapacityByLabel(B, "red", 50);
conf.setCapacityByLabel(B, "blue", 50);
conf.setCapacity(B1, 50);
conf.setMaximumCapacity(B1, 85);
conf.setCapacityByLabel(B1, "red", 50);
conf.setCapacityByLabel(B1, "blue", 50);
conf.setCapacity(B2, 30);
conf.setMaximumCapacity(B2, 35);
conf.setCapacityByLabel(B2, "red", 25);
conf.setCapacityByLabel(B2, "blue", 25);
conf.setCapacity(B3, 20);
conf.setMaximumCapacity(B3, 35);
conf.setCapacityByLabel(B3, "red", 25);
conf.setCapacityByLabel(B3, "blue", 25);
}
private void setupQueueConfigurationWithLabelsInherit(
CapacitySchedulerConfiguration conf) {
// Define top-level queues
conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
// Set A configuration
final String A = CapacitySchedulerConfiguration.ROOT + ".a";
conf.setCapacity(A, 10);
conf.setMaximumCapacity(A, 15);
conf.setQueues(A, new String[] {"a1", "a2"});
conf.setAccessibleNodeLabels(A, ImmutableSet.of("red", "blue"));
conf.setCapacityByLabel(A, "red", 100);
conf.setCapacityByLabel(A, "blue", 100);
// Set B configuraiton
final String B = CapacitySchedulerConfiguration.ROOT + ".b";
conf.setCapacity(B, 90);
conf.setAccessibleNodeLabels(B, CommonNodeLabelsManager.EMPTY_STRING_SET);
// Define 2nd-level queues
final String A1 = A + ".a1";
final String A2 = A + ".a2";
conf.setCapacity(A1, 30);
conf.setMaximumCapacity(A1, 45);
conf.setCapacityByLabel(A1, "red", 50);
conf.setCapacityByLabel(A1, "blue", 100);
conf.setCapacity(A2, 70);
conf.setMaximumCapacity(A2, 85);
conf.setAccessibleNodeLabels(A2, ImmutableSet.of("red"));
conf.setCapacityByLabel(A2, "red", 50);
}
@Test
public void testQueueParsingReinitializeWithLabels() throws IOException {
CapacitySchedulerConfiguration csConf =
new CapacitySchedulerConfiguration();
setupQueueConfigurationWithoutLabels(csConf);
YarnConfiguration conf = new YarnConfiguration(csConf);
CapacityScheduler capacityScheduler = new CapacityScheduler();
RMContextImpl rmContext =
new RMContextImpl(null, null, null, null, null, null,
new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM(), null);
rmContext.setNodeLabelManager(nodeLabelManager);
capacityScheduler.setConf(conf);
capacityScheduler.setRMContext(rmContext);
capacityScheduler.init(conf);
capacityScheduler.start();
csConf = new CapacitySchedulerConfiguration();
setupQueueConfigurationWithLabels(csConf);
conf = new YarnConfiguration(csConf);
capacityScheduler.reinitialize(conf, rmContext);
checkQueueLabels(capacityScheduler);
capacityScheduler.stop();
}
private void checkQueueLabels(CapacityScheduler capacityScheduler) {
// queue-A is red, blue
Assert.assertTrue(capacityScheduler.getQueue("a").getAccessibleNodeLabels()
.containsAll(ImmutableSet.of("red", "blue")));
// queue-A1 inherits A's configuration
Assert.assertTrue(capacityScheduler.getQueue("a1")
.getAccessibleNodeLabels().containsAll(ImmutableSet.of("red", "blue")));
// queue-A2 is "red"
Assert.assertEquals(1, capacityScheduler.getQueue("a2")
.getAccessibleNodeLabels().size());
Assert.assertTrue(capacityScheduler.getQueue("a2")
.getAccessibleNodeLabels().contains("red"));
// queue-B is "red"/"blue"
Assert.assertTrue(capacityScheduler.getQueue("b").getAccessibleNodeLabels()
.containsAll(ImmutableSet.of("red", "blue")));
// queue-B2 inherits "red"/"blue"
Assert.assertTrue(capacityScheduler.getQueue("b2")
.getAccessibleNodeLabels().containsAll(ImmutableSet.of("red", "blue")));
}
private void
checkQueueLabelsInheritConfig(CapacityScheduler capacityScheduler) {
// queue-A is red, blue
Assert.assertTrue(capacityScheduler.getQueue("a").getAccessibleNodeLabels()
.containsAll(ImmutableSet.of("red", "blue")));
// queue-A1 inherits A's configuration
Assert.assertTrue(capacityScheduler.getQueue("a1")
.getAccessibleNodeLabels().containsAll(ImmutableSet.of("red", "blue")));
// queue-A2 is "red"
Assert.assertEquals(1, capacityScheduler.getQueue("a2")
.getAccessibleNodeLabels().size());
Assert.assertTrue(capacityScheduler.getQueue("a2")
.getAccessibleNodeLabels().contains("red"));
// queue-B is "red"/"blue"
Assert.assertTrue(capacityScheduler.getQueue("b").getAccessibleNodeLabels()
.isEmpty());
}
@Test
public void testQueueParsingWithLabels() throws IOException {
YarnConfiguration conf = new YarnConfiguration();
CapacitySchedulerConfiguration csConf =
new CapacitySchedulerConfiguration(conf);
setupQueueConfigurationWithLabels(csConf);
CapacityScheduler capacityScheduler = new CapacityScheduler();
RMContextImpl rmContext =
new RMContextImpl(null, null, null, null, null, null,
new RMContainerTokenSecretManager(csConf),
new NMTokenSecretManagerInRM(csConf),
new ClientToAMTokenSecretManagerInRM(), null);
rmContext.setNodeLabelManager(nodeLabelManager);
capacityScheduler.setConf(csConf);
capacityScheduler.setRMContext(rmContext);
capacityScheduler.init(csConf);
capacityScheduler.start();
checkQueueLabels(capacityScheduler);
capacityScheduler.stop();
}
@Test
public void testQueueParsingWithLabelsInherit() throws IOException {
YarnConfiguration conf = new YarnConfiguration();
CapacitySchedulerConfiguration csConf =
new CapacitySchedulerConfiguration(conf);
setupQueueConfigurationWithLabelsInherit(csConf);
CapacityScheduler capacityScheduler = new CapacityScheduler();
RMContextImpl rmContext =
new RMContextImpl(null, null, null, null, null, null,
new RMContainerTokenSecretManager(csConf),
new NMTokenSecretManagerInRM(csConf),
new ClientToAMTokenSecretManagerInRM(), null);
rmContext.setNodeLabelManager(nodeLabelManager);
capacityScheduler.setConf(csConf);
capacityScheduler.setRMContext(rmContext);
capacityScheduler.init(csConf);
capacityScheduler.start();
checkQueueLabelsInheritConfig(capacityScheduler);
capacityScheduler.stop();
}
}

View File

@ -23,7 +23,10 @@ import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
@ -42,8 +45,7 @@ public class TestReservationQueue {
ReservationQueue reservationQueue;
@Before
public void setup() {
public void setup() throws IOException {
// setup a context / conf
csConf = new CapacitySchedulerConfiguration();
YarnConfiguration conf = new YarnConfiguration();
@ -57,6 +59,9 @@ public class TestReservationQueue {
when(csContext.getClusterResource()).thenReturn(
Resources.createResource(100 * 16 * GB, 100 * 32));
when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
RMContext mockRMContext = TestUtils.getMockRMContext();
when(csContext.getRMContext()).thenReturn(mockRMContext);
// create a queue
PlanQueue pq = new PlanQueue(csContext, "root", null, null);

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.DrainDispatcher;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
@ -121,6 +122,7 @@ public class TestReservations {
when(csContext.getQueueComparator()).thenReturn(
CapacityScheduler.queueComparator);
when(csContext.getResourceCalculator()).thenReturn(resourceCalculator);
when(csContext.getRMContext()).thenReturn(rmContext);
RMContainerTokenSecretManager containerTokenSecretManager = new RMContainerTokenSecretManager(
conf);
containerTokenSecretManager.rollMasterKey();
@ -819,7 +821,9 @@ public class TestReservations {
// allocate to queue so that the potential new capacity is greater then
// absoluteMaxCapacity
Resource capability = Resources.createResource(32 * GB, 0);
boolean res = a.assignToQueue(clusterResource, capability, app_0, true);
boolean res =
a.canAssignToThisQueue(clusterResource, capability,
CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, true);
assertFalse(res);
// now add in reservations and make sure it continues if config set
@ -836,23 +840,29 @@ public class TestReservations {
assertEquals(3 * GB, node_1.getUsedResource().getMemory());
capability = Resources.createResource(5 * GB, 0);
res = a
.assignToQueue(clusterResource, capability, app_0, true);
res =
a.canAssignToThisQueue(clusterResource, capability,
CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, true);
assertTrue(res);
// tell to not check reservations
res = a.assignToQueue(clusterResource, capability, app_0, false);
res =
a.canAssignToThisQueue(clusterResource, capability,
CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, false);
assertFalse(res);
refreshQueuesTurnOffReservationsContLook(a, csConf);
// should return false no matter what checkReservations is passed
// in since feature is off
res = a.assignToQueue(clusterResource, capability, app_0, false);
res =
a.canAssignToThisQueue(clusterResource, capability,
CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, false);
assertFalse(res);
res = a
.assignToQueue(clusterResource, capability, app_0, true);
res =
a.canAssignToThisQueue(clusterResource, capability,
CommonNodeLabelsManager.EMPTY_STRING_SET, app_0, true);
assertFalse(res);
}
@ -1000,18 +1010,18 @@ public class TestReservations {
// set limit so subtrace reservations it can continue
Resource limit = Resources.createResource(12 * GB, 0);
boolean res = a.assignToUser(clusterResource, user_0, limit, app_0,
true);
true, null);
assertTrue(res);
// tell it not to check for reservations and should fail as already over
// limit
res = a.assignToUser(clusterResource, user_0, limit, app_0, false);
res = a.assignToUser(clusterResource, user_0, limit, app_0, false, null);
assertFalse(res);
refreshQueuesTurnOffReservationsContLook(a, csConf);
// should now return false since feature off
res = a.assignToUser(clusterResource, user_0, limit, app_0, true);
res = a.assignToUser(clusterResource, user_0, limit, app_0, true, null);
assertFalse(res);
}

View File

@ -18,11 +18,14 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -43,16 +46,19 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public class TestUtils {
private static final Log LOG = LogFactory.getLog(TestUtils.class);
@ -61,7 +67,7 @@ public class TestUtils {
* Get a mock {@link RMContext} for use in test cases.
* @return a mock {@link RMContext} for use in test cases
*/
@SuppressWarnings("rawtypes")
@SuppressWarnings({ "rawtypes", "unchecked" })
public static RMContext getMockRMContext() {
// Null dispatcher
Dispatcher nullDispatcher = new Dispatcher() {
@ -93,6 +99,27 @@ public class TestUtils {
new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM(), writer);
RMNodeLabelsManager nlm = mock(RMNodeLabelsManager.class);
when(
nlm.getQueueResource(any(String.class), any(Set.class),
any(Resource.class))).thenAnswer(new Answer<Resource>() {
@Override
public Resource answer(InvocationOnMock invocation) throws Throwable {
Object[] args = invocation.getArguments();
return (Resource) args[2];
}
});
when(nlm.getResourceByLabel(any(String.class), any(Resource.class)))
.thenAnswer(new Answer<Resource>() {
@Override
public Resource answer(InvocationOnMock invocation) throws Throwable {
Object[] args = invocation.getArguments();
return (Resource) args[1];
}
});
rmContext.setNodeLabelManager(nlm);
rmContext.setSystemMetricsPublisher(mock(SystemMetricsPublisher.class));
return rmContext;
}

View File

@ -216,7 +216,7 @@ public class FairSchedulerTestBase {
RMApp rmApp = new RMAppImpl(attId.getApplicationId(), rmContext, conf,
null, null, null, ApplicationSubmissionContext.newInstance(null, null,
null, null, null, false, false, 0, amResource, null), null, null,
0, null, null);
0, null, null, null);
rmContext.getRMApps().put(attId.getApplicationId(), rmApp);
AppAddedSchedulerEvent appAddedEvent = new AppAddedSchedulerEvent(
attId.getApplicationId(), queue, user);

View File

@ -2420,7 +2420,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
RMApp application =
new RMAppImpl(applicationId, resourceManager.getRMContext(), conf, name, user,
queue, submissionContext, scheduler, masterService,
System.currentTimeMillis(), "YARN", null);
System.currentTimeMillis(), "YARN", null, null);
resourceManager.getRMContext().getRMApps().putIfAbsent(applicationId, application);
application.handle(new RMAppEvent(applicationId, RMAppEventType.START));

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager.MockAsm;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.DummyRMNodeLabelsManager;
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;
@ -46,8 +47,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.util.StringHelper;
import org.apache.hadoop.yarn.webapp.WebApps;
@ -162,21 +163,24 @@ public class TestRMWebApp {
for (RMNode node : deactivatedNodes) {
deactivatedNodesMap.put(node.getHostName(), node);
}
return new RMContextImpl(null, null, null, null,
null, null, null, null, null, null) {
@Override
public ConcurrentMap<ApplicationId, RMApp> getRMApps() {
return applicationsMaps;
}
@Override
public ConcurrentMap<String, RMNode> getInactiveRMNodes() {
return deactivatedNodesMap;
}
@Override
public ConcurrentMap<NodeId, RMNode> getRMNodes() {
return nodesMap;
}
};
RMContextImpl rmContext = new RMContextImpl(null, null, null, null,
null, null, null, null, null, null) {
@Override
public ConcurrentMap<ApplicationId, RMApp> getRMApps() {
return applicationsMaps;
}
@Override
public ConcurrentMap<String, RMNode> getInactiveRMNodes() {
return deactivatedNodesMap;
}
@Override
public ConcurrentMap<NodeId, RMNode> getRMNodes() {
return nodesMap;
}
};
rmContext.setNodeLabelManager(new DummyRMNodeLabelsManager());
return rmContext;
}
public static ResourceManager mockRm(int apps, int racks, int nodes,
@ -203,10 +207,12 @@ public class TestRMWebApp {
CapacityScheduler cs = new CapacityScheduler();
cs.setConf(new YarnConfiguration());
cs.setRMContext(new RMContextImpl(null, null, null, null, null,
RMContext rmContext = new RMContextImpl(null, null, null, null, null,
null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM(), null));
new ClientToAMTokenSecretManagerInRM(), null);
rmContext.setNodeLabelManager(new DummyRMNodeLabelsManager());
cs.setRMContext(rmContext);
cs.init(conf);
return cs;
}