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 f2ea555ac6c06a3f2f6559731f48711fff05d3f1)
This commit is contained in:
Vinod Kumar Vavilapalli 2014-10-15 18:33:06 -07:00
parent 7b53ab7b14
commit fe80fd5e88
57 changed files with 2870 additions and 799 deletions

View File

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

View File

@ -33,6 +33,7 @@
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Set;
@Private @Private
@Unstable @Unstable
@ -147,4 +148,8 @@ public String getNodeManagerVersion() {
return node.getNodeManagerVersion(); return node.getNodeManagerVersion();
} }
@Override
public Set<String> getNodeLabels() {
return null;
}
} }

View File

@ -102,6 +102,12 @@ Release 2.6.0 - UNRELEASED
YARN-2656. Made RM web services authentication filter support proxy user. YARN-2656. Made RM web services authentication filter support proxy user.
(Varun Vasudev and Zhijie Shen via zjshen) (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 IMPROVEMENTS
YARN-2242. Improve exception information on AM launch crashes. (Li Lu YARN-2242. Improve exception information on AM launch crashes. (Li Lu

View File

@ -188,6 +188,23 @@
</Or> </Or>
<Bug pattern="IS2_INCONSISTENT_SYNC" /> <Bug pattern="IS2_INCONSISTENT_SYNC" />
</Match> </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 --> <!-- Inconsistent sync warning - scheduleAsynchronously is only initialized once and never changed -->
<Match> <Match>
<Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler" /> <Class name="org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler" />

View File

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

View File

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

View File

@ -27,6 +27,7 @@
import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; 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.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.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem; import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@ -108,6 +109,10 @@ void setRMApplicationHistoryWriter(
boolean isWorkPreservingRecoveryEnabled(); boolean isWorkPreservingRecoveryEnabled();
RMNodeLabelsManager getNodeLabelManager();
public void setNodeLabelManager(RMNodeLabelsManager mgr);
long getEpoch(); long getEpoch();
ReservationSystem getReservationSystem(); ReservationSystem getReservationSystem();

View File

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

View File

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

View File

@ -67,6 +67,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher; 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.SchedulingEditPolicy;
import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor; 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.NullRMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState; import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
@ -320,6 +321,10 @@ protected AMLivelinessMonitor createAMLivelinessMonitor() {
return new AMLivelinessMonitor(this.rmDispatcher); return new AMLivelinessMonitor(this.rmDispatcher);
} }
protected RMNodeLabelsManager createNodeLabelManager() {
return new RMNodeLabelsManager();
}
protected DelegationTokenRenewer createDelegationTokenRenewer() { protected DelegationTokenRenewer createDelegationTokenRenewer() {
return new DelegationTokenRenewer(); return new DelegationTokenRenewer();
} }
@ -399,6 +404,10 @@ protected void serviceInit(Configuration configuration) throws Exception {
AMLivelinessMonitor amFinishingMonitor = createAMLivelinessMonitor(); AMLivelinessMonitor amFinishingMonitor = createAMLivelinessMonitor();
addService(amFinishingMonitor); addService(amFinishingMonitor);
rmContext.setAMFinishingMonitor(amFinishingMonitor); rmContext.setAMFinishingMonitor(amFinishingMonitor);
RMNodeLabelsManager nlm = createNodeLabelManager();
addService(nlm);
rmContext.setNodeLabelManager(nlm);
boolean isRecoveryEnabled = conf.getBoolean( boolean isRecoveryEnabled = conf.getBoolean(
YarnConfiguration.RECOVERY_ENABLED, YarnConfiguration.RECOVERY_ENABLED,
@ -962,7 +971,7 @@ protected void startWepApp() {
* instance of {@link RMActiveServices} and initializes it. * instance of {@link RMActiveServices} and initializes it.
* @throws Exception * @throws Exception
*/ */
void createAndInitActiveServices() throws Exception { protected void createAndInitActiveServices() throws Exception {
activeServices = new RMActiveServices(); activeServices = new RMActiveServices();
activeServices.init(conf); activeServices.init(conf);
} }

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -0,0 +1,448 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.yarn.api.records.QueueACL;
import org.apache.hadoop.yarn.api.records.QueueInfo;
import org.apache.hadoop.yarn.api.records.QueueState;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.common.collect.Sets;
public abstract class AbstractCSQueue implements CSQueue {
CSQueue parent;
final String queueName;
float capacity;
float maximumCapacity;
float absoluteCapacity;
float absoluteMaxCapacity;
float absoluteUsedCapacity = 0.0f;
float usedCapacity = 0.0f;
volatile int numContainers;
final Resource minimumAllocation;
final Resource maximumAllocation;
QueueState state;
final QueueMetrics metrics;
final ResourceCalculator resourceCalculator;
Set<String> accessibleLabels;
RMNodeLabelsManager labelManager;
String defaultLabelExpression;
Resource usedResources = Resources.createResource(0, 0);
QueueInfo queueInfo;
Map<String, Float> absoluteCapacityByNodeLabels;
Map<String, Float> capacitiyByNodeLabels;
Map<String, Resource> usedResourcesByNodeLabels = new HashMap<String, Resource>();
Map<String, Float> absoluteMaxCapacityByNodeLabels;
Map<String, Float> maxCapacityByNodeLabels;
Map<QueueACL, AccessControlList> acls =
new HashMap<QueueACL, AccessControlList>();
boolean reservationsContinueLooking;
private final RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);
public AbstractCSQueue(CapacitySchedulerContext cs,
String queueName, CSQueue parent, CSQueue old) throws IOException {
this.minimumAllocation = cs.getMinimumResourceCapability();
this.maximumAllocation = cs.getMaximumResourceCapability();
this.labelManager = cs.getRMContext().getNodeLabelManager();
this.parent = parent;
this.queueName = queueName;
this.resourceCalculator = cs.getResourceCalculator();
this.queueInfo = recordFactory.newRecordInstance(QueueInfo.class);
// must be called after parent and queueName is set
this.metrics = old != null ? old.getMetrics() :
QueueMetrics.forQueue(getQueuePath(), parent,
cs.getConfiguration().getEnableUserMetrics(),
cs.getConf());
// get labels
this.accessibleLabels = cs.getConfiguration().getAccessibleNodeLabels(getQueuePath());
this.defaultLabelExpression = cs.getConfiguration()
.getDefaultNodeLabelExpression(getQueuePath());
this.queueInfo.setQueueName(queueName);
// inherit from parent if labels not set
if (this.accessibleLabels == null && parent != null) {
this.accessibleLabels = parent.getAccessibleNodeLabels();
SchedulerUtils.checkIfLabelInClusterNodeLabels(labelManager,
this.accessibleLabels);
}
// inherit from parent if labels not set
if (this.defaultLabelExpression == null && parent != null
&& this.accessibleLabels.containsAll(parent.getAccessibleNodeLabels())) {
this.defaultLabelExpression = parent.getDefaultNodeLabelExpression();
}
// set capacity by labels
capacitiyByNodeLabels =
cs.getConfiguration().getNodeLabelCapacities(getQueuePath(), accessibleLabels,
labelManager);
// set maximum capacity by labels
maxCapacityByNodeLabels =
cs.getConfiguration().getMaximumNodeLabelCapacities(getQueuePath(),
accessibleLabels, labelManager);
}
@Override
public synchronized float getCapacity() {
return capacity;
}
@Override
public synchronized float getAbsoluteCapacity() {
return absoluteCapacity;
}
@Override
public float getAbsoluteMaximumCapacity() {
return absoluteMaxCapacity;
}
@Override
public synchronized float getAbsoluteUsedCapacity() {
return absoluteUsedCapacity;
}
@Override
public float getMaximumCapacity() {
return maximumCapacity;
}
@Override
public synchronized float getUsedCapacity() {
return usedCapacity;
}
@Override
public synchronized Resource getUsedResources() {
return usedResources;
}
public synchronized int getNumContainers() {
return numContainers;
}
@Override
public synchronized QueueState getState() {
return state;
}
@Override
public QueueMetrics getMetrics() {
return metrics;
}
@Override
public String getQueueName() {
return queueName;
}
@Override
public synchronized CSQueue getParent() {
return parent;
}
@Override
public synchronized void setParent(CSQueue newParentQueue) {
this.parent = (ParentQueue)newParentQueue;
}
public Set<String> getAccessibleNodeLabels() {
return accessibleLabels;
}
@Override
public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
synchronized (this) {
if (acls.get(acl).isUserAllowed(user)) {
return true;
}
}
if (parent != null) {
return parent.hasAccess(acl, user);
}
return false;
}
@Override
public synchronized void setUsedCapacity(float usedCapacity) {
this.usedCapacity = usedCapacity;
}
@Override
public synchronized void setAbsoluteUsedCapacity(float absUsedCapacity) {
this.absoluteUsedCapacity = absUsedCapacity;
}
/**
* Set maximum capacity - used only for testing.
* @param maximumCapacity new max capacity
*/
synchronized void setMaxCapacity(float maximumCapacity) {
// Sanity check
CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
float absMaxCapacity =
CSQueueUtils.computeAbsoluteMaximumCapacity(maximumCapacity, parent);
CSQueueUtils.checkAbsoluteCapacity(getQueueName(), absoluteCapacity,
absMaxCapacity);
this.maximumCapacity = maximumCapacity;
this.absoluteMaxCapacity = absMaxCapacity;
}
@Override
public float getAbsActualCapacity() {
// for now, simply return actual capacity = guaranteed capacity for parent
// queue
return absoluteCapacity;
}
@Override
public String getDefaultNodeLabelExpression() {
return defaultLabelExpression;
}
synchronized void setupQueueConfigs(Resource clusterResource, float capacity,
float absoluteCapacity, float maximumCapacity, float absoluteMaxCapacity,
QueueState state, Map<QueueACL, AccessControlList> acls,
Set<String> labels, String defaultLabelExpression,
Map<String, Float> nodeLabelCapacities,
Map<String, Float> maximumNodeLabelCapacities,
boolean reservationContinueLooking)
throws IOException {
// Sanity check
CSQueueUtils.checkMaxCapacity(getQueueName(), capacity, maximumCapacity);
CSQueueUtils.checkAbsoluteCapacity(getQueueName(), absoluteCapacity,
absoluteMaxCapacity);
this.capacity = capacity;
this.absoluteCapacity = absoluteCapacity;
this.maximumCapacity = maximumCapacity;
this.absoluteMaxCapacity = absoluteMaxCapacity;
this.state = state;
this.acls = acls;
// set labels
this.accessibleLabels = labels;
// set label expression
this.defaultLabelExpression = defaultLabelExpression;
// copy node label capacity
this.capacitiyByNodeLabels = new HashMap<String, Float>(nodeLabelCapacities);
this.maxCapacityByNodeLabels =
new HashMap<String, Float>(maximumNodeLabelCapacities);
this.queueInfo.setAccessibleNodeLabels(this.accessibleLabels);
this.queueInfo.setCapacity(this.capacity);
this.queueInfo.setMaximumCapacity(this.maximumCapacity);
this.queueInfo.setQueueState(this.state);
this.queueInfo.setDefaultNodeLabelExpression(this.defaultLabelExpression);
// Update metrics
CSQueueUtils.updateQueueStatistics(
resourceCalculator, this, parent, clusterResource, minimumAllocation);
// Check if labels of this queue is a subset of parent queue, only do this
// when we not root
if (parent != null && parent.getParent() != null) {
if (parent.getAccessibleNodeLabels() != null
&& !parent.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
// if parent isn't "*", child shouldn't be "*" too
if (this.getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) {
throw new IOException("Parent's accessible queue is not ANY(*), "
+ "but child's accessible queue is *");
} else {
Set<String> diff =
Sets.difference(this.getAccessibleNodeLabels(),
parent.getAccessibleNodeLabels());
if (!diff.isEmpty()) {
throw new IOException("Some labels of child queue is not a subset "
+ "of parent queue, these labels=["
+ StringUtils.join(diff, ",") + "]");
}
}
}
}
// calculate absolute capacity by each node label
this.absoluteCapacityByNodeLabels =
CSQueueUtils.computeAbsoluteCapacityByNodeLabels(
this.capacitiyByNodeLabels, parent);
// calculate maximum capacity by each node label
this.absoluteMaxCapacityByNodeLabels =
CSQueueUtils.computeAbsoluteMaxCapacityByNodeLabels(
maximumNodeLabelCapacities, parent);
// check absoluteMaximumNodeLabelCapacities is valid
CSQueueUtils.checkAbsoluteCapacitiesByLabel(getQueueName(),
absoluteCapacityByNodeLabels, absoluteCapacityByNodeLabels);
this.reservationsContinueLooking = reservationContinueLooking;
}
@Private
public Resource getMaximumAllocation() {
return maximumAllocation;
}
@Private
public Resource getMinimumAllocation() {
return minimumAllocation;
}
synchronized void allocateResource(Resource clusterResource,
Resource resource, Set<String> nodeLabels) {
Resources.addTo(usedResources, resource);
// Update usedResources by labels
if (nodeLabels == null || nodeLabels.isEmpty()) {
if (!usedResourcesByNodeLabels.containsKey(RMNodeLabelsManager.NO_LABEL)) {
usedResourcesByNodeLabels.put(RMNodeLabelsManager.NO_LABEL,
Resources.createResource(0));
}
Resources.addTo(usedResourcesByNodeLabels.get(RMNodeLabelsManager.NO_LABEL),
resource);
} else {
for (String label : Sets.intersection(accessibleLabels, nodeLabels)) {
if (!usedResourcesByNodeLabels.containsKey(label)) {
usedResourcesByNodeLabels.put(label, Resources.createResource(0));
}
Resources.addTo(usedResourcesByNodeLabels.get(label), resource);
}
}
++numContainers;
CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(),
clusterResource, minimumAllocation);
}
protected synchronized void releaseResource(Resource clusterResource,
Resource resource, Set<String> nodeLabels) {
// Update queue metrics
Resources.subtractFrom(usedResources, resource);
// Update usedResources by labels
if (null == nodeLabels || nodeLabels.isEmpty()) {
if (!usedResourcesByNodeLabels.containsKey(RMNodeLabelsManager.NO_LABEL)) {
usedResourcesByNodeLabels.put(RMNodeLabelsManager.NO_LABEL,
Resources.createResource(0));
}
Resources.subtractFrom(
usedResourcesByNodeLabels.get(RMNodeLabelsManager.NO_LABEL), resource);
} else {
for (String label : Sets.intersection(accessibleLabels, nodeLabels)) {
if (!usedResourcesByNodeLabels.containsKey(label)) {
usedResourcesByNodeLabels.put(label, Resources.createResource(0));
}
Resources.subtractFrom(usedResourcesByNodeLabels.get(label), resource);
}
}
CSQueueUtils.updateQueueStatistics(resourceCalculator, this, getParent(),
clusterResource, minimumAllocation);
--numContainers;
}
@Private
public float getCapacityByNodeLabel(String label) {
if (null == parent) {
return 1f;
}
if (StringUtils.equals(label, RMNodeLabelsManager.NO_LABEL)) {
return getCapacity();
}
if (!capacitiyByNodeLabels.containsKey(label)) {
return 0;
} else {
return capacitiyByNodeLabels.get(label);
}
}
@Private
public float getAbsoluteCapacityByNodeLabel(String label) {
if (null == parent) {
return 1;
}
if (StringUtils.equals(label, RMNodeLabelsManager.NO_LABEL)) {
return getAbsoluteCapacity();
}
if (!absoluteMaxCapacityByNodeLabels.containsKey(label)) {
return 0;
} else {
return absoluteMaxCapacityByNodeLabels.get(label);
}
}
@Private
public float getAbsoluteMaximumCapacityByNodeLabel(String label) {
if (StringUtils.equals(label, RMNodeLabelsManager.NO_LABEL)) {
return getAbsoluteMaximumCapacity();
}
return getAbsoluteCapacityByNodeLabel(label);
}
@Private
public boolean getReservationContinueLooking() {
return reservationsContinueLooking;
}
@Private
public Map<QueueACL, AccessControlList> getACLs() {
return acls;
}
}

View File

@ -72,9 +72,18 @@ public interface CSQueue
/** /**
* Get the configured <em>capacity</em> of the queue. * Get the configured <em>capacity</em> of the queue.
* @return queue capacity * @return configured queue capacity
*/ */
public float getCapacity(); 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 * Get capacity of the parent of the queue as a function of the
@ -105,29 +114,32 @@ public interface CSQueue
*/ */
public float getAbsoluteUsedCapacity(); 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. * 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); public void setUsedCapacity(float usedCapacity);
/** /**
* Set absolute used capacity of the queue. * 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); public void setAbsoluteUsedCapacity(float absUsedCapacity);
/** /**
* Get the currently utilized resources in the cluster * Get the current used capacity of nodes without label(s) of the queue
* by the queue and children (if any). * and it's children (if any).
* @return used resources by the queue and it's children * @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(); public Resource getUsedResources();
@ -259,4 +271,25 @@ public void detachContainer(Resource clusterResource,
*/ */
public void attachContainer(Resource clusterResource, public void attachContainer(Resource clusterResource,
FiCaSchedulerApp application, RMContainer container); FiCaSchedulerApp application, RMContainer container);
/**
* Get absolute capacity by label of this queue can use
* @param nodeLabel
* @return absolute capacity by label of this queue can use
*/
public float getAbsoluteCapacityByNodeLabel(String nodeLabel);
/**
* Get absolute max capacity by label of this queue can use
* @param nodeLabel
* @return absolute capacity by label of this queue can use
*/
public float getAbsoluteMaximumCapacityByNodeLabel(String nodeLabel);
/**
* Get capacity by node label
* @param nodeLabel
* @return capacity by node label
*/
public float getCapacityByNodeLabel(String nodeLabel);
} }

View File

@ -17,9 +17,12 @@
*/ */
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.utils.Lock; import org.apache.hadoop.yarn.server.utils.Lock;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@ -40,7 +43,7 @@ public static void checkMaxCapacity(String queueName,
} }
} }
public static void checkAbsoluteCapacities(String queueName, public static void checkAbsoluteCapacity(String queueName,
float absCapacity, float absMaxCapacity) { float absCapacity, float absMaxCapacity) {
if (absMaxCapacity < (absCapacity - EPSILON)) { if (absMaxCapacity < (absCapacity - EPSILON)) {
throw new IllegalArgumentException("Illegal call to setMaxCapacity. " throw new IllegalArgumentException("Illegal call to setMaxCapacity. "
@ -49,6 +52,23 @@ public static void checkAbsoluteCapacities(String queueName,
+ ")"); + ")");
} }
} }
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( public static float computeAbsoluteMaximumCapacity(
float maximumCapacity, CSQueue parent) { float maximumCapacity, CSQueue parent) {
@ -56,6 +76,39 @@ public static float computeAbsoluteMaximumCapacity(
(parent == null) ? 1.0f : parent.getAbsoluteMaximumCapacity(); (parent == null) ? 1.0f : parent.getAbsoluteMaximumCapacity();
return (parentAbsMaxCapacity * maximumCapacity); 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( public static int computeMaxActiveApplications(
ResourceCalculator calculator, ResourceCalculator calculator,

View File

@ -20,7 +20,15 @@
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; 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.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -53,8 +61,13 @@
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes; import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext; 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.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.RMAppAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@ -191,6 +204,7 @@ public Configuration getConf() {
private boolean scheduleAsynchronously; private boolean scheduleAsynchronously;
private AsyncScheduleThread asyncSchedulerThread; private AsyncScheduleThread asyncSchedulerThread;
private RMNodeLabelsManager labelManager;
/** /**
* EXPERT * EXPERT
@ -275,6 +289,8 @@ private synchronized void initScheduler(Configuration configuration) throws
this.applications = this.applications =
new ConcurrentHashMap<ApplicationId, new ConcurrentHashMap<ApplicationId,
SchedulerApplication<FiCaSchedulerApp>>(); SchedulerApplication<FiCaSchedulerApp>>();
this.labelManager = rmContext.getNodeLabelManager();
initializeQueues(this.conf); initializeQueues(this.conf);
scheduleAsynchronously = this.conf.getScheduleAynschronously(); scheduleAsynchronously = this.conf.getScheduleAynschronously();
@ -446,7 +462,7 @@ private void initializeQueues(CapacitySchedulerConfiguration conf)
root = root =
parseQueue(this, conf, null, CapacitySchedulerConfiguration.ROOT, parseQueue(this, conf, null, CapacitySchedulerConfiguration.ROOT,
queues, queues, noop); queues, queues, noop);
labelManager.reinitializeQueueLabels(getQueueToLabels());
LOG.info("Initialized root queue " + root); LOG.info("Initialized root queue " + root);
initializeQueueMappings(); initializeQueueMappings();
} }
@ -469,10 +485,19 @@ private void reinitializeQueues(CapacitySchedulerConfiguration conf)
// Re-configure queues // Re-configure queues
root.reinitialize(newRoot, clusterResource); root.reinitialize(newRoot, clusterResource);
initializeQueueMappings(); initializeQueueMappings();
// Re-calculate headroom for active applications // Re-calculate headroom for active applications
root.updateClusterResource(clusterResource); 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 @@ private void addNewQueues(
@Lock(CapacityScheduler.class) @Lock(CapacityScheduler.class)
static CSQueue parseQueue( static CSQueue parseQueue(
CapacitySchedulerContext csContext, CapacitySchedulerContext csContext,
CapacitySchedulerConfiguration conf, CapacitySchedulerConfiguration conf,
CSQueue parent, String queueName, Map<String, CSQueue> queues, CSQueue parent, String queueName, Map<String, CSQueue> queues,
Map<String, CSQueue> oldQueues, Map<String, CSQueue> oldQueues,
@ -1094,11 +1119,18 @@ public void handle(SchedulerEvent event) {
} }
private synchronized void addNode(RMNode nodeManager) { 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, this.nodes.put(nodeManager.getNodeID(), new FiCaSchedulerNode(nodeManager,
usePortForNodeName)); usePortForNodeName));
Resources.addTo(clusterResource, nodeManager.getTotalCapability()); Resources.addTo(clusterResource, nodeManager.getTotalCapability());
root.updateClusterResource(clusterResource); root.updateClusterResource(clusterResource);
int numNodes = numNodeManagers.incrementAndGet(); int numNodes = numNodeManagers.incrementAndGet();
LOG.info("Added node " + nodeManager.getNodeAddress() + LOG.info("Added node " + nodeManager.getNodeAddress() +
" clusterResource: " + clusterResource); " clusterResource: " + clusterResource);
@ -1108,6 +1140,11 @@ private synchronized void addNode(RMNode nodeManager) {
} }
private synchronized void removeNode(RMNode nodeInfo) { 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()); FiCaSchedulerNode node = nodes.get(nodeInfo.getNodeID());
if (node == null) { if (node == null) {
return; return;
@ -1141,6 +1178,7 @@ private synchronized void removeNode(RMNode nodeInfo) {
} }
this.nodes.remove(nodeInfo.getNodeID()); this.nodes.remove(nodeInfo.getNodeID());
LOG.info("Removed node " + nodeInfo.getNodeAddress() + LOG.info("Removed node " + nodeInfo.getNodeAddress() +
" clusterResource: " + clusterResource); " clusterResource: " + clusterResource);
} }

View File

@ -18,7 +18,15 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -31,10 +39,14 @@
import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.api.records.QueueState;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration; 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.DefaultResourceCalculator;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.common.collect.ImmutableSet;
public class CapacitySchedulerConfiguration extends Configuration { public class CapacitySchedulerConfiguration extends Configuration {
private static final Log LOG = private static final Log LOG =
@ -83,6 +95,12 @@ public class CapacitySchedulerConfiguration extends Configuration {
public static final String STATE = "state"; public static final String STATE = "state";
@Private @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 public static final String RESERVE_CONT_LOOK_ALL_NODES = PREFIX
+ "reservations-continue-look-all-nodes"; + "reservations-continue-look-all-nodes";
@ -268,6 +286,10 @@ private String getQueuePrefix(String queue) {
return queueName; return queueName;
} }
private String getNodeLabelPrefix(String queue, String label) {
return getQueuePrefix(queue) + ACCESSIBLE_NODE_LABELS + DOT + label + DOT;
}
public int getMaximumSystemApplications() { public int getMaximumSystemApplications() {
int maxApplications = int maxApplications =
getInt(MAXIMUM_SYSTEM_APPLICATIONS, DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS); getInt(MAXIMUM_SYSTEM_APPLICATIONS, DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS);
@ -343,6 +365,15 @@ public void setMaximumCapacity(String queue, float maxCapacity) {
", maxCapacity=" + maxCapacity); ", 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) { public int getUserLimit(String queue) {
int userLimit = getInt(getQueuePrefix(queue) + USER_LIMIT, int userLimit = getInt(getQueuePrefix(queue) + USER_LIMIT,
DEFAULT_USER_LIMIT); DEFAULT_USER_LIMIT);
@ -372,6 +403,121 @@ public QueueState getState(String queue) {
QueueState.valueOf(state.toUpperCase()) : QueueState.RUNNING; 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 * 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 * after the reservation limit was hit. The node heart beating in could

View File

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

View File

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

View File

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

View File

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

View File

@ -25,6 +25,7 @@
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ConcurrentSkipListMap;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
@ -187,6 +188,18 @@ public void recoverContainer(Resource clusterResource,
updateAppHeadRoom(schedulerAttempt); updateAppHeadRoom(schedulerAttempt);
updateAvailableResourcesMetrics(); 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() { public FifoScheduler() {

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -127,7 +127,7 @@ private void checkActiveRMFunctionality() throws Exception {
try { try {
rm.getNewAppId(); rm.getNewAppId();
rm.registerNode("127.0.0.1:0", 2048); rm.registerNode("127.0.0.1:1", 2048);
app = rm.submitApp(1024); app = rm.submitApp(1024);
attempt = app.getCurrentAppAttempt(); attempt = app.getCurrentAppAttempt();
rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.SCHEDULED); rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.SCHEDULED);

View File

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

View File

@ -17,13 +17,16 @@
*******************************************************************************/ *******************************************************************************/
package org.apache.hadoop.yarn.server.resourcemanager.reservation; 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.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.Map; import java.util.Map;
import java.util.Random; import java.util.Random;
import java.util.Set;
import java.util.TreeMap; import java.util.TreeMap;
import org.apache.hadoop.yarn.api.records.ReservationDefinition; import org.apache.hadoop.yarn.api.records.ReservationDefinition;
@ -37,6 +40,7 @@
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; 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.CapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
@ -44,6 +48,8 @@
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.junit.Assert; import org.junit.Assert;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public class ReservationSystemTestUtil { public class ReservationSystemTestUtil {
@ -55,6 +61,7 @@ public static ReservationId getNewReservationId() {
return ReservationId.newInstance(rand.nextLong(), rand.nextLong()); return ReservationId.newInstance(rand.nextLong(), rand.nextLong());
} }
@SuppressWarnings("unchecked")
public CapacityScheduler mockCapacityScheduler(int numContainers) public CapacityScheduler mockCapacityScheduler(int numContainers)
throws IOException { throws IOException {
// stolen from TestCapacityScheduler // stolen from TestCapacityScheduler
@ -68,6 +75,29 @@ public CapacityScheduler mockCapacityScheduler(int numContainers)
new RMContainerTokenSecretManager(conf), new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf), new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM(), null)); 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); cs.setRMContext(mockRmContext);
try { try {
cs.serviceInit(conf); cs.serviceInit(conf);

View File

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

View File

@ -40,7 +40,9 @@
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Set;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
@ -62,7 +64,9 @@
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.NodeId; 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.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState; import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.event.EventHandler;
@ -83,8 +87,8 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType; 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.RMAppFailedAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl; 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.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.RMAppAttemptContainerAllocatedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerFinishedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
@ -96,7 +100,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent; 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.RMNodeEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeFinishedContainersPulledByAMEvent; 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.rmnode.RMNodeImpl;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@ -112,6 +115,7 @@
import org.apache.hadoop.yarn.server.security.MasterKeyData; import org.apache.hadoop.yarn.server.security.MasterKeyData;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils; 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.apache.hadoop.yarn.webapp.util.WebAppUtils;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
@ -122,6 +126,8 @@
import org.mockito.ArgumentCaptor; import org.mockito.ArgumentCaptor;
import org.mockito.Matchers; import org.mockito.Matchers;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@RunWith(value = Parameterized.class) @RunWith(value = Parameterized.class)
public class TestRMAppAttemptTransitions { public class TestRMAppAttemptTransitions {
@ -229,6 +235,7 @@ public TestRMAppAttemptTransitions(Boolean isSecurityEnabled) {
this.isSecurityEnabled = isSecurityEnabled; this.isSecurityEnabled = isSecurityEnabled;
} }
@SuppressWarnings("deprecation")
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
AuthenticationMethod authMethod = AuthenticationMethod.SIMPLE; AuthenticationMethod authMethod = AuthenticationMethod.SIMPLE;
@ -300,6 +307,7 @@ public void setUp() throws Exception {
Mockito.doReturn(resourceScheduler).when(spyRMContext).getScheduler(); Mockito.doReturn(resourceScheduler).when(spyRMContext).getScheduler();
final String user = MockApps.newUserName();
final String queue = MockApps.newQueue(); final String queue = MockApps.newQueue();
submissionContext = mock(ApplicationSubmissionContext.class); submissionContext = mock(ApplicationSubmissionContext.class);
when(submissionContext.getQueue()).thenReturn(queue); when(submissionContext.getQueue()).thenReturn(queue);
@ -315,7 +323,11 @@ public void setUp() throws Exception {
application = mock(RMAppImpl.class); application = mock(RMAppImpl.class);
applicationAttempt = applicationAttempt =
new RMAppAttemptImpl(applicationAttemptId, spyRMContext, scheduler, 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.getCurrentAppAttempt()).thenReturn(applicationAttempt);
when(application.getApplicationId()).thenReturn(applicationId); when(application.getApplicationId()).thenReturn(applicationId);
spyRMContext.getRMApps().put(application.getApplicationId(), application); spyRMContext.getRMApps().put(application.getApplicationId(), application);
@ -1399,13 +1411,16 @@ public void testFailedToFailed() {
} }
@SuppressWarnings("deprecation")
@Test @Test
public void testContainersCleanupForLastAttempt() { public void testContainersCleanupForLastAttempt() {
// create a failed attempt. // create a failed attempt.
applicationAttempt = applicationAttempt =
new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(), spyRMContext, new RMAppAttemptImpl(applicationAttempt.getAppAttemptId(), spyRMContext,
scheduler, masterService, submissionContext, new Configuration(), scheduler, masterService, submissionContext, new Configuration(),
true); true, BuilderUtils.newResourceRequest(
RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
submissionContext.getResource(), 1));
when(submissionContext.getKeepContainersAcrossApplicationAttempts()) when(submissionContext.getKeepContainersAcrossApplicationAttempts())
.thenReturn(true); .thenReturn(true);
when(submissionContext.getMaxAppAttempts()).thenReturn(1); when(submissionContext.getMaxAppAttempts()).thenReturn(1);
@ -1427,6 +1442,49 @@ scheduler, masterService, submissionContext, new Configuration(),
assertFalse(transferStateFromPreviousAttempt); assertFalse(transferStateFromPreviousAttempt);
verifyApplicationAttemptFinished(RMAppAttemptState.FAILED); 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, private void verifyAMCrashAtAllocatedDiagnosticInfo(String diagnostics,
int exitCode, boolean shouldCheckURL) { int exitCode, boolean shouldCheckURL) {

View File

@ -21,13 +21,19 @@
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; 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.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.security.PrivilegedAction; import java.security.PrivilegedAction;
import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Set;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -46,6 +52,7 @@
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.Priority; 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.Resource;
import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest; import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
@ -58,6 +65,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.MockNM; 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.MockRMWithAMS;
import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager; 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.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; 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.RMAppAttempt;
@ -74,6 +82,8 @@
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import com.google.common.collect.Sets;
public class TestSchedulerUtils { public class TestSchedulerUtils {
private static final Log LOG = LogFactory.getLog(TestSchedulerUtils.class); private static final Log LOG = LogFactory.getLog(TestSchedulerUtils.class);
@ -173,69 +183,240 @@ public void testNormalizeRequestWithDominantResourceCalculator() {
assertEquals(1, ask.getCapability().getVirtualCores()); assertEquals(1, ask.getCapability().getVirtualCores());
assertEquals(2048, ask.getCapability().getMemory()); assertEquals(2048, ask.getCapability().getMemory());
} }
@Test (timeout = 30000) @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( Resource maxResource = Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
// zero memory // queue has labels, success cases
try { try {
// set queue accessible node labesl to [x, y]
queueAccessibleNodeLabels.clear();
queueAccessibleNodeLabels.addAll(Arrays.asList("x", "y"));
Resource resource = Resources.createResource( Resource resource = Resources.createResource(
0, 0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1); 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) { } catch (InvalidResourceRequestException e) {
fail("Zero memory should be accepted"); fail("Zero memory should be accepted");
} }
// zero vcores // zero vcores
try { try {
Resource resource = Resources.createResource( Resource resource =
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, Resources.createResource(
0); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0);
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq =
mock(Priority.class), ResourceRequest.ANY, resource, 1); BuilderUtils.newResourceRequest(mock(Priority.class),
SchedulerUtils.validateResourceRequest(resReq, maxResource); ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
fail("Zero vcores should be accepted"); fail("Zero vcores should be accepted");
} }
// max memory // max memory
try { try {
Resource resource = Resources.createResource( Resource resource =
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
ResourceRequest resReq = BuilderUtils.newResourceRequest( YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
mock(Priority.class), ResourceRequest.ANY, resource, 1); ResourceRequest resReq =
SchedulerUtils.validateResourceRequest(resReq, maxResource); BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
fail("Max memory should be accepted"); fail("Max memory should be accepted");
} }
// max vcores // max vcores
try { try {
Resource resource = Resources.createResource( Resource resource =
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
ResourceRequest resReq = BuilderUtils.newResourceRequest( YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
mock(Priority.class), ResourceRequest.ANY, resource, 1); ResourceRequest resReq =
SchedulerUtils.validateResourceRequest(resReq, maxResource); BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
fail("Max vcores should not be accepted"); fail("Max vcores should not be accepted");
} }
// negative memory // negative memory
try { try {
Resource resource = Resources.createResource( Resource resource =
-1, Resources.createResource(-1,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq =
mock(Priority.class), ResourceRequest.ANY, resource, 1); BuilderUtils.newResourceRequest(mock(Priority.class),
SchedulerUtils.validateResourceRequest(resReq, maxResource); ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
fail("Negative memory should not be accepted"); fail("Negative memory should not be accepted");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
// expected // expected
@ -243,12 +424,14 @@ public void testValidateResourceRequest() {
// negative vcores // negative vcores
try { try {
Resource resource = Resources.createResource( Resource resource =
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, Resources.createResource(
-1); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, -1);
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq =
mock(Priority.class), ResourceRequest.ANY, resource, 1); BuilderUtils.newResourceRequest(mock(Priority.class),
SchedulerUtils.validateResourceRequest(resReq, maxResource); ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
fail("Negative vcores should not be accepted"); fail("Negative vcores should not be accepted");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
// expected // expected
@ -256,12 +439,15 @@ public void testValidateResourceRequest() {
// more than max memory // more than max memory
try { try {
Resource resource = Resources.createResource( Resource resource =
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1, Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1,
ResourceRequest resReq = BuilderUtils.newResourceRequest( YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
mock(Priority.class), ResourceRequest.ANY, resource, 1); ResourceRequest resReq =
SchedulerUtils.validateResourceRequest(resReq, maxResource); BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
fail("More than max memory should not be accepted"); fail("More than max memory should not be accepted");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
// expected // expected
@ -269,13 +455,16 @@ public void testValidateResourceRequest() {
// more than max vcores // more than max vcores
try { try {
Resource resource = Resources.createResource( Resource resource =
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, Resources
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES .createResource(
+ 1); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
ResourceRequest resReq = BuilderUtils.newResourceRequest( YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES + 1);
mock(Priority.class), ResourceRequest.ANY, resource, 1); ResourceRequest resReq =
SchedulerUtils.validateResourceRequest(resReq, maxResource); BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1);
SchedulerUtils.validateResourceRequest(resReq, maxResource, null,
mockScheduler);
fail("More than max vcores should not be accepted"); fail("More than max vcores should not be accepted");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
// expected // expected

View File

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

View File

@ -19,38 +19,19 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import static org.junit.Assert.assertEquals; 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.mock;
import static org.mockito.Mockito.reset;
import static org.mockito.Mockito.when; 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.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration; 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.DefaultResourceCalculator;
import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.After;
import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.mockito.InOrder;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public class TestCSQueueUtils { public class TestCSQueueUtils {
@ -88,6 +69,8 @@ public void runInvalidDivisorTest(boolean useDominant) throws Exception {
thenReturn(Resources.createResource(GB, 1)); thenReturn(Resources.createResource(GB, 1));
when(csContext.getMaximumResourceCapability()). when(csContext.getMaximumResourceCapability()).
thenReturn(Resources.createResource(0, 0)); thenReturn(Resources.createResource(0, 0));
RMContext rmContext = TestUtils.getMockRMContext();
when(csContext.getRMContext()).thenReturn(rmContext);
final String L1Q1 = "L1Q1"; final String L1Q1 = "L1Q1";
csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {L1Q1}); csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {L1Q1});
@ -129,6 +112,8 @@ public void testAbsoluteMaxAvailCapacityNoUse() throws Exception {
thenReturn(Resources.createResource(GB, 1)); thenReturn(Resources.createResource(GB, 1));
when(csContext.getMaximumResourceCapability()). when(csContext.getMaximumResourceCapability()).
thenReturn(Resources.createResource(16*GB, 32)); thenReturn(Resources.createResource(16*GB, 32));
RMContext rmContext = TestUtils.getMockRMContext();
when(csContext.getRMContext()).thenReturn(rmContext);
final String L1Q1 = "L1Q1"; final String L1Q1 = "L1Q1";
csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {L1Q1}); csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {L1Q1});
@ -174,6 +159,9 @@ public void testAbsoluteMaxAvailCapacityWithUse() throws Exception {
when(csContext.getMaximumResourceCapability()). when(csContext.getMaximumResourceCapability()).
thenReturn(Resources.createResource(16*GB, 32)); thenReturn(Resources.createResource(16*GB, 32));
RMContext rmContext = TestUtils.getMockRMContext();
when(csContext.getRMContext()).thenReturn(rmContext);
final String L1Q1 = "L1Q1"; final String L1Q1 = "L1Q1";
final String L1Q2 = "L1Q2"; final String L1Q2 = "L1Q2";
final String L2Q1 = "L2Q1"; final String L2Q1 = "L2Q1";

View File

@ -52,6 +52,7 @@
import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId; 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.Container;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerState;
@ -64,7 +65,6 @@
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceOption; import org.apache.hadoop.yarn.api.records.ResourceOption;
import org.apache.hadoop.yarn.api.records.ResourceRequest; 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.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
@ -84,6 +84,8 @@
import org.apache.hadoop.yarn.server.resourcemanager.Task; 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.MockRMWithAMS;
import org.apache.hadoop.yarn.server.resourcemanager.TestAMAuthorization.MyContainerManager; 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.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
@ -149,7 +151,14 @@ public class TestCapacityScheduler {
@Before @Before
public void setUp() throws Exception { 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 CapacitySchedulerConfiguration csConf
= new CapacitySchedulerConfiguration(); = new CapacitySchedulerConfiguration();
setupQueueConfiguration(csConf); setupQueueConfiguration(csConf);
@ -962,10 +971,7 @@ public void testNumClusterNodes() throws Exception {
YarnConfiguration conf = new YarnConfiguration(); YarnConfiguration conf = new YarnConfiguration();
CapacityScheduler cs = new CapacityScheduler(); CapacityScheduler cs = new CapacityScheduler();
cs.setConf(conf); cs.setConf(conf);
RMContextImpl rmContext = new RMContextImpl(null, null, null, null, null, RMContext rmContext = TestUtils.getMockRMContext();
null, new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM(), null);
cs.setRMContext(rmContext); cs.setRMContext(rmContext);
CapacitySchedulerConfiguration csConf = CapacitySchedulerConfiguration csConf =
new CapacitySchedulerConfiguration(); new CapacitySchedulerConfiguration();
@ -1476,8 +1482,14 @@ public void testMoveAppSuccess() throws Exception {
@Test(expected = YarnException.class) @Test(expected = YarnException.class)
public void testMoveAppViolateQueueState() throws Exception { 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 = CapacitySchedulerConfiguration csConf =
new CapacitySchedulerConfiguration(); new CapacitySchedulerConfiguration();
setupQueueConfiguration(csConf); setupQueueConfiguration(csConf);

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -18,11 +18,14 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; 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.doReturn;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy; import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
import java.util.Set;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -43,16 +46,19 @@
import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl; import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter; 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.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.rmcontainer.ContainerAllocationExpirer;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; 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.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; 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.AMRMTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM; 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.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; 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.apache.hadoop.yarn.util.resource.Resources;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
public class TestUtils { public class TestUtils {
private static final Log LOG = LogFactory.getLog(TestUtils.class); 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. * Get a mock {@link RMContext} for use in test cases.
* @return 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() { public static RMContext getMockRMContext() {
// Null dispatcher // Null dispatcher
Dispatcher nullDispatcher = new Dispatcher() { Dispatcher nullDispatcher = new Dispatcher() {
@ -93,6 +99,27 @@ public EventHandler getEventHandler() {
new RMContainerTokenSecretManager(conf), new RMContainerTokenSecretManager(conf),
new NMTokenSecretManagerInRM(conf), new NMTokenSecretManagerInRM(conf),
new ClientToAMTokenSecretManagerInRM(), writer); 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)); rmContext.setSystemMetricsPublisher(mock(SystemMetricsPublisher.class));
return rmContext; return rmContext;
} }

View File

@ -216,7 +216,7 @@ protected void createApplicationWithAMResource(ApplicationAttemptId attId,
RMApp rmApp = new RMAppImpl(attId.getApplicationId(), rmContext, conf, RMApp rmApp = new RMAppImpl(attId.getApplicationId(), rmContext, conf,
null, null, null, ApplicationSubmissionContext.newInstance(null, null, null, null, null, ApplicationSubmissionContext.newInstance(null, null,
null, null, null, false, false, 0, amResource, null), 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); rmContext.getRMApps().put(attId.getApplicationId(), rmApp);
AppAddedSchedulerEvent appAddedEvent = new AppAddedSchedulerEvent( AppAddedSchedulerEvent appAddedEvent = new AppAddedSchedulerEvent(
attId.getApplicationId(), queue, user); attId.getApplicationId(), queue, user);

View File

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

View File

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