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:
parent
08eeb3e5b6
commit
e8e3a36213
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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" />
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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() +
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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));
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue