YARN-8967. Change FairScheduler to use PlacementRule interface. Contributed by Wilfred Spiegelenburg.

This commit is contained in:
yufei 2019-03-25 22:47:24 -07:00
parent c99b107772
commit 5257f50abb
27 changed files with 1337 additions and 1215 deletions

View File

@ -68,6 +68,8 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.Times; import org.apache.hadoop.yarn.util.Times;
@ -418,7 +420,8 @@ private RMAppImpl createAndPopulateNewRMApp(
// We only replace the queue when it's a new application // We only replace the queue when it's a new application
if (!isRecovery) { if (!isRecovery) {
replaceQueueFromPlacementContext(placementContext, submissionContext); copyPlacementQueueToSubmissionContext(placementContext,
submissionContext);
// fail the submission if configured application timeout value is invalid // fail the submission if configured application timeout value is invalid
RMServerUtils.validateApplicationTimeouts( RMServerUtils.validateApplicationTimeouts(
@ -443,13 +446,8 @@ private RMAppImpl createAndPopulateNewRMApp(
submissionContext.setPriority(appPriority); submissionContext.setPriority(appPriority);
} }
// Since FairScheduler queue mapping is done inside scheduler, if (!isRecovery && YarnConfiguration.isAclEnabled(conf)) {
// if FairScheduler is used and the queue doesn't exist, we should not if (scheduler instanceof CapacityScheduler) {
// fail here because queue will be created inside FS. Ideally, FS queue
// mapping should be done outside scheduler too like CS.
// For now, exclude FS for the acl check.
if (!isRecovery && YarnConfiguration.isAclEnabled(conf)
&& scheduler instanceof CapacityScheduler) {
String queueName = submissionContext.getQueue(); String queueName = submissionContext.getQueue();
String appName = submissionContext.getApplicationName(); String appName = submissionContext.getApplicationName();
CSQueue csqueue = ((CapacityScheduler) scheduler).getQueue(queueName); CSQueue csqueue = ((CapacityScheduler) scheduler).getQueue(queueName);
@ -474,7 +472,34 @@ private RMAppImpl createAndPopulateNewRMApp(
null))) { null))) {
throw RPCUtil.getRemoteException(new AccessControlException( throw RPCUtil.getRemoteException(new AccessControlException(
"User " + user + " does not have permission to submit " "User " + user + " does not have permission to submit "
+ applicationId + " to queue " + submissionContext.getQueue())); + applicationId + " to queue "
+ submissionContext.getQueue()));
}
}
if (scheduler instanceof FairScheduler) {
// if we have not placed the app just skip this, the submit will be
// rejected in the scheduler.
if (placementContext != null) {
// The queue might not be created yet. Walk up the tree to check the
// parent ACL. The queueName is assured root which always exists
String queueName = submissionContext.getQueue();
FSQueue queue = ((FairScheduler) scheduler).getQueueManager().
getQueue(queueName);
while (queue == null) {
int sepIndex = queueName.lastIndexOf(".");
queueName = queueName.substring(0, sepIndex);
queue = ((FairScheduler) scheduler).getQueueManager().
getQueue(queueName);
}
if (!queue.hasAccess(QueueACL.SUBMIT_APPLICATIONS, userUgi) &&
!queue.hasAccess(QueueACL.ADMINISTER_QUEUE, userUgi)) {
throw RPCUtil.getRemoteException(new AccessControlException(
"User " + user + " does not have permission to submit " +
applicationId + " to queue " +
submissionContext.getQueue() +
" denied by ACL for queue " + queueName));
}
}
} }
} }
@ -841,34 +866,38 @@ ApplicationPlacementContext placeApplication(
// Placement could also fail if the user doesn't exist in system // Placement could also fail if the user doesn't exist in system
// skip if the user is not found during recovery. // skip if the user is not found during recovery.
if (isRecovery) { if (isRecovery) {
LOG.warn("PlaceApplication failed,skipping on recovery of rm"); LOG.warn("Application placement failed for user " + user +
" and application " + context.getApplicationId() +
", skipping placement on recovery of rm", e);
return placementContext; return placementContext;
} }
throw e; throw e;
} }
} }
if (placementContext == null && (context.getQueue() == null) || context // The submission context when created often has a queue set. In case of
.getQueue().isEmpty()) { // the FairScheduler a null placement context is still considered as a
// failure, even when a queue is provided on submit. This case handled in
// the scheduler.
if (placementContext == null && (context.getQueue() == null) ||
context.getQueue().isEmpty()) {
String msg = "Failed to place application " + context.getApplicationId() String msg = "Failed to place application " + context.getApplicationId()
+ " to queue and specified " + "queue is invalid : " + context + " in a queue and submit context queue is null or empty";
.getQueue();
LOG.error(msg); LOG.error(msg);
throw new YarnException(msg); throw new YarnException(msg);
} }
return placementContext; return placementContext;
} }
void replaceQueueFromPlacementContext( private void copyPlacementQueueToSubmissionContext(
ApplicationPlacementContext placementContext, ApplicationPlacementContext placementContext,
ApplicationSubmissionContext context) { ApplicationSubmissionContext context) {
// Set it to ApplicationSubmissionContext // Set the queue from the placement in the ApplicationSubmissionContext
//apply queue mapping only to new application submissions // Placement rule are only considered for new applications
if (placementContext != null && !StringUtils.equalsIgnoreCase( if (placementContext != null && !StringUtils.equalsIgnoreCase(
context.getQueue(), placementContext.getQueue())) { context.getQueue(), placementContext.getQueue())) {
LOG.info("Placed application=" + context.getApplicationId() + LOG.info("Placed application with ID " + context.getApplicationId() +
" to queue=" + placementContext.getQueue() + ", original queue=" " in queue: " + placementContext.getQueue() +
+ context ", original submission queue was: " + context.getQueue());
.getQueue());
context.setQueue(placementContext.getQueue()); context.setQueue(placementContext.getQueue());
} }
} }

View File

@ -58,10 +58,12 @@ QueueManager getQueueManager() {
} }
/** /**
* Set a rule to generate the parent queue dynamically. * Set a rule to generate the parent queue dynamically. The parent rule
* should only be called on rule creation when the policy is read from the
* configuration.
* @param parent A PlacementRule * @param parent A PlacementRule
*/ */
void setParentRule(PlacementRule parent) { public void setParentRule(PlacementRule parent) {
this.parentRule = parent; this.parentRule = parent;
} }
@ -69,7 +71,8 @@ void setParentRule(PlacementRule parent) {
* Get the rule that is set to generate the parent queue dynamically. * Get the rule that is set to generate the parent queue dynamically.
* @return The rule set or <code>null</code> if not set. * @return The rule set or <code>null</code> if not set.
*/ */
PlacementRule getParentRule() { @VisibleForTesting
public PlacementRule getParentRule() {
return parentRule; return parentRule;
} }
@ -149,6 +152,14 @@ boolean configuredQueue(String queueName) {
return (queue != null && !queue.isDynamic()); return (queue != null && !queue.isDynamic());
} }
/**
* Get the create flag as set during the config setup.
* @return The value of the {@link #createQueue} flag
*/
public boolean getCreateFlag() {
return createQueue;
}
/** /**
* Get the create flag from the xml configuration element. * Get the create flag from the xml configuration element.
* @param conf The FS configuration element for the queue * @param conf The FS configuration element for the queue
@ -159,7 +170,7 @@ boolean configuredQueue(String queueName) {
boolean getCreateFlag(Element conf) { boolean getCreateFlag(Element conf) {
if (conf != null) { if (conf != null) {
String create = conf.getAttribute("create"); String create = conf.getAttribute("create");
return Boolean.parseBoolean(create); return create.isEmpty() || Boolean.parseBoolean(create);
} }
return true; return true;
} }

View File

@ -28,7 +28,7 @@
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT;
/** /**
* Utility class for QueuePlacementRule. * Utility class for Capacity Scheduler queue PlacementRules.
*/ */
public final class QueuePlacementRuleUtils { public final class QueuePlacementRuleUtils {

View File

@ -23,7 +23,6 @@
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.yarn.api.records.ReservationACL; import org.apache.hadoop.yarn.api.records.ReservationACL;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
@ -94,10 +93,6 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
//Map for maximum container resource allocation per queues by queue name //Map for maximum container resource allocation per queues by queue name
private final Map<String, Resource> queueMaxContainerAllocationMap; private final Map<String, Resource> queueMaxContainerAllocationMap;
// Policy for mapping apps to queues
@VisibleForTesting
QueuePlacementPolicy placementPolicy;
//Configured queues in the alloc xml //Configured queues in the alloc xml
@VisibleForTesting @VisibleForTesting
Map<FSQueueType, Set<String>> configuredQueues; Map<FSQueueType, Set<String>> configuredQueues;
@ -107,9 +102,16 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
private final Set<String> nonPreemptableQueues; private final Set<String> nonPreemptableQueues;
/**
* Create a fully initialised configuration for the scheduler.
* @param queueProperties The list of queues and their properties from the
* configuration.
* @param allocationFileParser The allocation file parser
* @param globalReservationQueueConfig The reservation queue config
* @throws AllocationConfigurationException
*/
public AllocationConfiguration(QueueProperties queueProperties, public AllocationConfiguration(QueueProperties queueProperties,
AllocationFileParser allocationFileParser, AllocationFileParser allocationFileParser,
QueuePlacementPolicy newPlacementPolicy,
ReservationQueueConfiguration globalReservationQueueConfig) ReservationQueueConfiguration globalReservationQueueConfig)
throws AllocationConfigurationException { throws AllocationConfigurationException {
this.minQueueResources = queueProperties.getMinQueueResources(); this.minQueueResources = queueProperties.getMinQueueResources();
@ -138,14 +140,19 @@ public AllocationConfiguration(QueueProperties queueProperties,
this.resAcls = queueProperties.getReservationAcls(); this.resAcls = queueProperties.getReservationAcls();
this.reservableQueues = queueProperties.getReservableQueues(); this.reservableQueues = queueProperties.getReservableQueues();
this.globalReservationQueueConfig = globalReservationQueueConfig; this.globalReservationQueueConfig = globalReservationQueueConfig;
this.placementPolicy = newPlacementPolicy;
this.configuredQueues = queueProperties.getConfiguredQueues(); this.configuredQueues = queueProperties.getConfiguredQueues();
this.nonPreemptableQueues = queueProperties.getNonPreemptableQueues(); this.nonPreemptableQueues = queueProperties.getNonPreemptableQueues();
this.queueMaxContainerAllocationMap = this.queueMaxContainerAllocationMap =
queueProperties.getMaxContainerAllocation(); queueProperties.getMaxContainerAllocation();
} }
public AllocationConfiguration(Configuration conf) { /**
* Create a base scheduler configuration with just the defaults set.
* Should only be called to init a basic setup on scheduler init.
* @param scheduler The {@link FairScheduler} to create and initialise the
* placement policy.
*/
public AllocationConfiguration(FairScheduler scheduler) {
minQueueResources = new HashMap<>(); minQueueResources = new HashMap<>();
maxChildQueueResources = new HashMap<>(); maxChildQueueResources = new HashMap<>();
maxQueueResources = new HashMap<>(); maxQueueResources = new HashMap<>();
@ -169,8 +176,7 @@ public AllocationConfiguration(Configuration conf) {
for (FSQueueType queueType : FSQueueType.values()) { for (FSQueueType queueType : FSQueueType.values()) {
configuredQueues.put(queueType, new HashSet<>()); configuredQueues.put(queueType, new HashSet<>());
} }
placementPolicy = QueuePlacementPolicy.fromConfiguration(scheduler);
QueuePlacementPolicy.fromConfiguration(conf, configuredQueues);
nonPreemptableQueues = new HashSet<>(); nonPreemptableQueues = new HashSet<>();
queueMaxContainerAllocationMap = new HashMap<>(); queueMaxContainerAllocationMap = new HashMap<>();
} }
@ -309,10 +315,6 @@ public Map<FSQueueType, Set<String>> getConfiguredQueues() {
return configuredQueues; return configuredQueues;
} }
public QueuePlacementPolicy getPlacementPolicy() {
return placementPolicy;
}
@Override @Override
public boolean isReservable(String queue) { public boolean isReservable(String queue) {
return reservableQueues.contains(queue); return reservableQueues.contains(queue);

View File

@ -78,6 +78,7 @@ public class AllocationFileLoaderService extends AbstractService {
"(?i)(hdfs)|(file)|(s3a)|(viewfs)"; "(?i)(hdfs)|(file)|(s3a)|(viewfs)";
private final Clock clock; private final Clock clock;
private final FairScheduler scheduler;
// Last time we successfully reloaded queues // Last time we successfully reloaded queues
private volatile long lastSuccessfulReload; private volatile long lastSuccessfulReload;
@ -95,14 +96,15 @@ public class AllocationFileLoaderService extends AbstractService {
private Thread reloadThread; private Thread reloadThread;
private volatile boolean running = true; private volatile boolean running = true;
public AllocationFileLoaderService() { AllocationFileLoaderService(FairScheduler scheduler) {
this(SystemClock.getInstance()); this(SystemClock.getInstance(), scheduler);
} }
private List<Permission> defaultPermissions; private List<Permission> defaultPermissions;
public AllocationFileLoaderService(Clock clock) { AllocationFileLoaderService(Clock clock, FairScheduler scheduler) {
super(AllocationFileLoaderService.class.getName()); super(AllocationFileLoaderService.class.getName());
this.scheduler = scheduler;
this.clock = clock; this.clock = clock;
} }
@ -254,17 +256,15 @@ public synchronized void reloadAllocations()
new AllocationFileQueueParser(allocationFileParser.getQueueElements()); new AllocationFileQueueParser(allocationFileParser.getQueueElements());
QueueProperties queueProperties = queueParser.parse(); QueueProperties queueProperties = queueParser.parse();
// Load placement policy and pass it configured queues // Load placement policy
Configuration conf = getConfig(); getQueuePlacementPolicy(allocationFileParser);
QueuePlacementPolicy newPlacementPolicy =
getQueuePlacementPolicy(allocationFileParser, queueProperties, conf);
setupRootQueueProperties(allocationFileParser, queueProperties); setupRootQueueProperties(allocationFileParser, queueProperties);
ReservationQueueConfiguration globalReservationQueueConfig = ReservationQueueConfiguration globalReservationQueueConfig =
createReservationQueueConfig(allocationFileParser); createReservationQueueConfig(allocationFileParser);
AllocationConfiguration info = new AllocationConfiguration(queueProperties, AllocationConfiguration info = new AllocationConfiguration(queueProperties,
allocationFileParser, newPlacementPolicy, globalReservationQueueConfig); allocationFileParser, globalReservationQueueConfig);
lastSuccessfulReload = clock.getTime(); lastSuccessfulReload = clock.getTime();
lastReloadAttemptFailed = false; lastReloadAttemptFailed = false;
@ -272,17 +272,15 @@ public synchronized void reloadAllocations()
reloadListener.onReload(info); reloadListener.onReload(info);
} }
private QueuePlacementPolicy getQueuePlacementPolicy( private void getQueuePlacementPolicy(
AllocationFileParser allocationFileParser, AllocationFileParser allocationFileParser)
QueueProperties queueProperties, Configuration conf)
throws AllocationConfigurationException { throws AllocationConfigurationException {
if (allocationFileParser.getQueuePlacementPolicy().isPresent()) { if (allocationFileParser.getQueuePlacementPolicy().isPresent()) {
return QueuePlacementPolicy.fromXml( QueuePlacementPolicy.fromXml(
allocationFileParser.getQueuePlacementPolicy().get(), allocationFileParser.getQueuePlacementPolicy().get(),
queueProperties.getConfiguredQueues(), conf); scheduler);
} else { } else {
return QueuePlacementPolicy.fromConfiguration(conf, QueuePlacementPolicy.fromConfiguration(scheduler);
queueProperties.getConfiguredQueues());
} }
} }

View File

@ -59,6 +59,7 @@
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.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.RMCriticalThreadUncaughtExceptionHandler; import org.apache.hadoop.yarn.server.resourcemanager.RMCriticalThreadUncaughtExceptionHandler;
import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
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.reservation.ReservationConstants; import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationConstants;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@ -214,7 +215,7 @@ public class FairScheduler extends
public FairScheduler() { public FairScheduler() {
super(FairScheduler.class.getName()); super(FairScheduler.class.getName());
context = new FSContext(this); context = new FSContext(this);
allocsLoader = new AllocationFileLoaderService(); allocsLoader = new AllocationFileLoaderService(this);
queueMgr = new QueueManager(this); queueMgr = new QueueManager(this);
maxRunningEnforcer = new MaxRunningAppsEnforcer(this); maxRunningEnforcer = new MaxRunningAppsEnforcer(this);
} }
@ -223,6 +224,10 @@ public FSContext getContext() {
return context; return context;
} }
public RMContext getRMContext() {
return rmContext;
}
public boolean isAtLeastReservationThreshold( public boolean isAtLeastReservationThreshold(
ResourceCalculator resourceCalculator, Resource resource) { ResourceCalculator resourceCalculator, Resource resource) {
return Resources.greaterThanOrEqual(resourceCalculator, return Resources.greaterThanOrEqual(resourceCalculator,
@ -455,32 +460,52 @@ public int getContinuousSchedulingSleepMs() {
* configured limits, but the app will not be marked as runnable. * configured limits, but the app will not be marked as runnable.
*/ */
protected void addApplication(ApplicationId applicationId, protected void addApplication(ApplicationId applicationId,
String queueName, String user, boolean isAppRecovering) { String queueName, String user, boolean isAppRecovering,
if (queueName == null || queueName.isEmpty()) { ApplicationPlacementContext placementContext) {
String message = // If the placement was rejected the placementContext will be null.
"Reject application " + applicationId + " submitted by user " + user // We ignore placement rules on recovery.
+ " with an empty queue name."; if (!isAppRecovering && placementContext == null) {
rejectApplicationWithMessage(applicationId, message); String message = "Reject application " + applicationId +
return; " submitted by user " + user +
} " application rejected by placement rules.";
if (queueName.startsWith(".") || queueName.endsWith(".")) {
String message =
"Reject application " + applicationId + " submitted by user " + user
+ " with an illegal queue name " + queueName + ". "
+ "The queue name cannot start/end with period.";
rejectApplicationWithMessage(applicationId, message); rejectApplicationWithMessage(applicationId, message);
return; return;
} }
writeLock.lock(); writeLock.lock();
try { try {
RMApp rmApp = rmContext.getRMApps().get(applicationId); // Assign the app to the queue creating and prevent queue delete.
FSLeafQueue queue = assignToQueue(rmApp, queueName, user, applicationId); FSLeafQueue queue = queueMgr.getLeafQueue(queueName, true,
applicationId);
if (queue == null) { if (queue == null) {
rejectApplicationWithMessage(applicationId,
queueName + " is not a leaf queue");
return; return;
} }
// Enforce ACLs: 2nd check, there could be a time laps between the app
// creation in the RMAppManager and getting here. That means we could
// have a configuration change (prevent race condition)
UserGroupInformation userUgi = UserGroupInformation.createRemoteUser(
user);
if (!queue.hasAccess(QueueACL.SUBMIT_APPLICATIONS, userUgi) &&
!queue.hasAccess(QueueACL.ADMINISTER_QUEUE, userUgi)) {
String msg = "User " + user + " does not have permission to submit " +
applicationId + " to queue " + queueName;
rejectApplicationWithMessage(applicationId, msg);
queue.removeAssignedApp(applicationId);
return;
}
RMApp rmApp = rmContext.getRMApps().get(applicationId);
if (rmApp != null) {
rmApp.setQueue(queueName);
} else {
LOG.error("Couldn't find RM app for " + applicationId +
" to set queue name on");
}
if (rmApp != null && rmApp.getAMResourceRequests() != null) { if (rmApp != null && rmApp.getAMResourceRequests() != null) {
// Resources.fitsIn would always return false when queueMaxShare is 0 // Resources.fitsIn would always return false when queueMaxShare is 0
// for any resource, but only using Resources.fitsIn is not enough // for any resource, but only using Resources.fitsIn is not enough
@ -499,7 +524,7 @@ protected void addApplication(ApplicationId applicationId,
+ "it has zero amount of resource for a requested " + "it has zero amount of resource for a requested "
+ "resource! Invalid requested AM resources: %s, " + "resource! Invalid requested AM resources: %s, "
+ "maximum queue resources: %s", + "maximum queue resources: %s",
applicationId, queue.getName(), applicationId, queueName,
invalidAMResourceRequests, queue.getMaxShare()); invalidAMResourceRequests, queue.getMaxShare());
rejectApplicationWithMessage(applicationId, msg); rejectApplicationWithMessage(applicationId, msg);
queue.removeAssignedApp(applicationId); queue.removeAssignedApp(applicationId);
@ -507,27 +532,13 @@ protected void addApplication(ApplicationId applicationId,
} }
} }
// Enforce ACLs
UserGroupInformation userUgi = UserGroupInformation.createRemoteUser(
user);
if (!queue.hasAccess(QueueACL.SUBMIT_APPLICATIONS, userUgi) && !queue
.hasAccess(QueueACL.ADMINISTER_QUEUE, userUgi)) {
String msg = "User " + userUgi.getUserName()
+ " cannot submit applications to queue " + queue.getName()
+ "(requested queuename is " + queueName + ")";
rejectApplicationWithMessage(applicationId, msg);
queue.removeAssignedApp(applicationId);
return;
}
SchedulerApplication<FSAppAttempt> application = SchedulerApplication<FSAppAttempt> application =
new SchedulerApplication<FSAppAttempt>(queue, user); new SchedulerApplication<>(queue, user);
applications.put(applicationId, application); applications.put(applicationId, application);
queue.getMetrics().submitApp(user); queue.getMetrics().submitApp(user);
LOG.info("Accepted application " + applicationId + " from user: " + user LOG.info("Accepted application " + applicationId + " from user: " + user
+ ", in queue: " + queue.getName() + ", in queue: " + queueName
+ ", currently num of applications: " + applications.size()); + ", currently num of applications: " + applications.size());
if (isAppRecovering) { if (isAppRecovering) {
LOG.debug("{} is recovering. Skip notifying APP_ACCEPTED", LOG.debug("{} is recovering. Skip notifying APP_ACCEPTED",
@ -596,60 +607,6 @@ protected void addApplicationAttempt(
} }
} }
/**
* Helper method for the tests to assign the app to a queue.
*/
@VisibleForTesting
FSLeafQueue assignToQueue(RMApp rmApp, String queueName, String user) {
return assignToQueue(rmApp, queueName, user, null);
}
/**
* Helper method that attempts to assign the app to a queue. The method is
* responsible to call the appropriate event-handler if the app is rejected.
*/
private FSLeafQueue assignToQueue(RMApp rmApp, String queueName, String user,
ApplicationId applicationId) {
FSLeafQueue queue = null;
String appRejectMsg = null;
try {
QueuePlacementPolicy placementPolicy = allocConf.getPlacementPolicy();
queueName = placementPolicy.assignAppToQueue(queueName, user);
if (queueName == null) {
appRejectMsg = "Application rejected by queue placement policy";
} else {
queue = queueMgr.getLeafQueue(queueName, true, applicationId);
if (queue == null) {
appRejectMsg = queueName + " is not a leaf queue";
}
}
} catch (IllegalStateException se) {
appRejectMsg = "Unable to match app " + rmApp.getApplicationId() +
" to a queue placement policy, and no valid terminal queue " +
" placement rule is configured. Please contact an administrator " +
" to confirm that the fair scheduler configuration contains a " +
" valid terminal queue placement rule.";
} catch (InvalidQueueNameException qne) {
appRejectMsg = qne.getMessage();
} catch (IOException ioe) {
// IOException should only happen for a user without groups
appRejectMsg = "Error assigning app to a queue: " + ioe.getMessage();
}
if (appRejectMsg != null && rmApp != null) {
rejectApplicationWithMessage(rmApp.getApplicationId(), appRejectMsg);
return null;
}
if (rmApp != null) {
rmApp.setQueue(queue.getName());
} else {
LOG.error("Couldn't find RM app to set queue name on");
}
return queue;
}
private void removeApplication(ApplicationId applicationId, private void removeApplication(ApplicationId applicationId,
RMAppState finalState) { RMAppState finalState) {
SchedulerApplication<FSAppAttempt> application = applications.remove( SchedulerApplication<FSAppAttempt> application = applications.remove(
@ -1265,7 +1222,8 @@ public void handle(SchedulerEvent event) {
if (queueName != null) { if (queueName != null) {
addApplication(appAddedEvent.getApplicationId(), addApplication(appAddedEvent.getApplicationId(),
queueName, appAddedEvent.getUser(), queueName, appAddedEvent.getUser(),
appAddedEvent.getIsAppRecovering()); appAddedEvent.getIsAppRecovering(),
appAddedEvent.getPlacementContext());
} }
break; break;
case APP_REMOVED: case APP_REMOVED:
@ -1442,12 +1400,8 @@ private void initScheduler(Configuration conf) throws IOException {
// This stores per-application scheduling information // This stores per-application scheduling information
this.applications = new ConcurrentHashMap<>(); this.applications = new ConcurrentHashMap<>();
allocConf = new AllocationConfiguration(conf); allocConf = new AllocationConfiguration(this);
try { queueMgr.initialize();
queueMgr.initialize(conf);
} catch (Exception e) {
throw new IOException("Failed to start FairScheduler", e);
}
if (continuousSchedulingEnabled) { if (continuousSchedulingEnabled) {
// Continuous scheduling is deprecated log it on startup // Continuous scheduling is deprecated log it on startup

View File

@ -18,34 +18,28 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CopyOnWriteArrayList;
import javax.xml.parsers.ParserConfigurationException;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
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.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
import org.xml.sax.SAXException;
import com.google.common.annotations.VisibleForTesting;
import java.util.Iterator;
import java.util.Set;
/** /**
* Maintains a list of queues as well as scheduling parameters for each queue, * Maintains a list of queues as well as scheduling parameters for each queue,
@ -106,8 +100,7 @@ public FSParentQueue getRootQueue() {
return rootQueue; return rootQueue;
} }
public void initialize(Configuration conf) throws IOException, public void initialize() {
SAXException, AllocationConfigurationException, ParserConfigurationException {
// Policies of root and default queue are set to // Policies of root and default queue are set to
// SchedulingPolicy.DEFAULT_POLICY since the allocation file hasn't been // SchedulingPolicy.DEFAULT_POLICY since the allocation file hasn't been
// loaded yet. // loaded yet.

View File

@ -23,85 +23,261 @@
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 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.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Groups; import org.apache.hadoop.yarn.server.resourcemanager.placement.DefaultPlacementRule;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.yarn.server.resourcemanager.placement.FSPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.PrimaryGroupPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.RejectPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.SecondaryGroupExistingPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.SpecifiedPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.UserPlacementRule;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.w3c.dom.Element; import org.w3c.dom.Element;
import org.w3c.dom.Node; import org.w3c.dom.Node;
import org.w3c.dom.NodeList; import org.w3c.dom.NodeList;
import static org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementFactory.getPlacementRule;
/**
* The FairScheduler rules based policy for placing an application in a queue.
* It parses the configuration and updates the {@link
* org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager}
* with a list of {@link PlacementRule}s to execute in order.
*/
@Private @Private
@Unstable @Unstable
public class QueuePlacementPolicy { final class QueuePlacementPolicy {
private static final Map<String, Class<? extends QueuePlacementRule>> ruleClasses; private static final Logger LOG =
LoggerFactory.getLogger(QueuePlacementPolicy.class);
// Simple private class to make the rule mapping simpler.
private static final class RuleMap {
private final Class<? extends PlacementRule> ruleClass;
private final String terminal;
private RuleMap(Class<? extends PlacementRule> clazz, String terminate) {
this.ruleClass = clazz;
this.terminal = terminate;
}
}
// The list of known rules:
// key to the map is the name in the configuration.
// for each name the mapping contains the class name of the implementation
// and a flag (true, false or create) which describes the terminal state
// see the method getTerminal() for more comments.
private static final Map<String, RuleMap> RULES;
static { static {
Map<String, Class<? extends QueuePlacementRule>> map = Map<String, RuleMap> map = new HashMap<>();
new HashMap<String, Class<? extends QueuePlacementRule>>(); map.put("user", new RuleMap(UserPlacementRule.class, "create"));
map.put("user", QueuePlacementRule.User.class); map.put("primaryGroup",
map.put("primaryGroup", QueuePlacementRule.PrimaryGroup.class); new RuleMap(PrimaryGroupPlacementRule.class, "create"));
map.put("secondaryGroupExistingQueue", map.put("secondaryGroupExistingQueue",
QueuePlacementRule.SecondaryGroupExistingQueue.class); new RuleMap(SecondaryGroupExistingPlacementRule.class, "false"));
map.put("specified", QueuePlacementRule.Specified.class); map.put("specified", new RuleMap(SpecifiedPlacementRule.class, "false"));
map.put("nestedUserQueue", map.put("nestedUserQueue", new RuleMap(UserPlacementRule.class, "create"));
QueuePlacementRule.NestedUserQueue.class); map.put("default", new RuleMap(DefaultPlacementRule.class, "create"));
map.put("default", QueuePlacementRule.Default.class); map.put("reject", new RuleMap(RejectPlacementRule.class, "true"));
map.put("reject", QueuePlacementRule.Reject.class); RULES = Collections.unmodifiableMap(map);
ruleClasses = Collections.unmodifiableMap(map);
} }
private final List<QueuePlacementRule> rules; private QueuePlacementPolicy() {
private final Map<FSQueueType, Set<String>> configuredQueues; }
private final Groups groups;
public QueuePlacementPolicy(List<QueuePlacementRule> rules, /**
Map<FSQueueType, Set<String>> configuredQueues, Configuration conf) * Update the rules in the manager based on this placement policy.
* @param newRules The new list of rules to set in the manager.
* @param newTerminalState The list of terminal states for this set of rules.
* @param fs the reference to the scheduler needed in the rule on init.
* @throws AllocationConfigurationException for any errors
*/
private static void updateRuleSet(List<PlacementRule> newRules,
List<Boolean> newTerminalState,
FairScheduler fs)
throws AllocationConfigurationException { throws AllocationConfigurationException {
for (int i = 0; i < rules.size()-1; i++) { if (newRules.isEmpty()) {
if (rules.get(i).isTerminal()) { LOG.debug("Empty rule set defined, ignoring update");
return;
}
LOG.debug("Placement rule order check");
for (int i = 0; i < newTerminalState.size()-1; i++) {
if (newTerminalState.get(i)) {
throw new AllocationConfigurationException("Rules after rule " throw new AllocationConfigurationException("Rules after rule "
+ i + " in queue placement policy can never be reached"); + (i+1) + " in queue placement policy can never be reached");
} }
} }
if (!rules.get(rules.size()-1).isTerminal()) { if (!newTerminalState.get(newTerminalState.size()-1)) {
throw new AllocationConfigurationException( throw new AllocationConfigurationException(
"Could get past last queue placement rule without assigning"); "Could get past last queue placement rule without assigning");
} }
this.rules = rules; // Set the scheduler in the rule to get queues etc
this.configuredQueues = configuredQueues; LOG.debug("Initialising new rule set");
groups = new Groups(conf); try {
for (PlacementRule rule: newRules){
rule.initialize(fs);
}
} catch (IOException ioe) {
// We should never throw as we pass in a FS object, however we still
// should consider any exception here a config error.
throw new AllocationConfigurationException(
"Rule initialisation failed with exception", ioe);
}
// Update the placement manager with the new rule list.
// We only get here when all rules are OK.
fs.getRMContext().getQueuePlacementManager().updateRules(newRules);
LOG.debug("PlacementManager active with new rule set");
} }
/** /**
* Builds a QueuePlacementPolicy from an xml element. * Builds a QueuePlacementPolicy from a xml element.
* @param confElement the placement policy xml snippet from the
* {@link FairSchedulerConfiguration}
* @param fs the reference to the scheduler needed in the rule on init.
* @throws AllocationConfigurationException for any errors
*/ */
public static QueuePlacementPolicy fromXml(Element el, static void fromXml(Element confElement, FairScheduler fs)
Map<FSQueueType, Set<String>> configuredQueues, Configuration conf)
throws AllocationConfigurationException { throws AllocationConfigurationException {
List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>(); LOG.debug("Reloading placement policy from allocation config");
NodeList elements = el.getChildNodes(); if (confElement == null || !confElement.hasChildNodes()) {
throw new AllocationConfigurationException(
"Empty configuration for QueuePlacementPolicy is not allowed");
}
List<PlacementRule> newRules = new ArrayList<>();
List<Boolean> newTerminalState = new ArrayList<>();
NodeList elements = confElement.getChildNodes();
for (int i = 0; i < elements.getLength(); i++) { for (int i = 0; i < elements.getLength(); i++) {
Node node = elements.item(i); Node node = elements.item(i);
if (node instanceof Element) { if (node instanceof Element &&
QueuePlacementRule rule = createAndInitializeRule(node); node.getNodeName().equalsIgnoreCase("rule")) {
rules.add(rule); String name = ((Element) node).getAttribute("name");
LOG.debug("Creating new rule: {}", name);
PlacementRule rule = createRule((Element)node);
// The only child node that we currently know is a parent rule
PlacementRule parentRule = null;
String parentName = null;
Element child = getParentRuleElement(node);
if (child != null) {
parentName = child.getAttribute("name");
parentRule = getParentRule(child, fs);
}
// Need to make sure that the nestedUserQueue has a parent for
// backwards compatibility
if (name.equalsIgnoreCase("nestedUserQueue") && parentRule == null) {
throw new AllocationConfigurationException("Rule '" + name
+ "' must have a parent rule set");
}
newRules.add(rule);
if (parentRule == null) {
newTerminalState.add(
getTerminal(RULES.get(name).terminal, rule));
} else {
((FSPlacementRule)rule).setParentRule(parentRule);
newTerminalState.add(
getTerminal(RULES.get(name).terminal, rule) &&
getTerminal(RULES.get(parentName).terminal, parentRule));
} }
} }
return new QueuePlacementPolicy(rules, configuredQueues, conf); }
updateRuleSet(newRules, newTerminalState, fs);
} }
/** /**
* Create and initialize a rule given a xml node * Find the element that defines the parent rule.
* @param node * @param node the xml node to check for a parent rule
* @return QueuePlacementPolicy * @return {@link Element} that describes the parent rule or
* @throws AllocationConfigurationException * <code>null</code> if none is found
*/ */
public static QueuePlacementRule createAndInitializeRule(Node node) private static Element getParentRuleElement(Node node)
throws AllocationConfigurationException {
Element parent = null;
// walk over the node list
if (node.hasChildNodes()) {
NodeList childList = node.getChildNodes();
for (int j = 0; j < childList.getLength(); j++) {
Node child = childList.item(j);
if (child instanceof Element &&
child.getNodeName().equalsIgnoreCase("rule")) {
if (parent != null) {
LOG.warn("Rule '{}' has multiple parent rules defined, only the " +
"last parent rule will be used",
((Element) node).getAttribute("name"));
}
parent = ((Element) child);
}
}
}
// sanity check the rule that is configured
if (parent != null) {
String parentName = parent.getAttribute("name");
if (parentName.equals("reject") ||
parentName.equals("nestedUserQueue")) {
throw new AllocationConfigurationException("Rule '"
+ parentName
+ "' is not allowed as a parent rule for any rule");
}
}
return parent;
}
/**
* Retrieve the configured parent rule from the xml config.
* @param parent the xml element that contains the name of the rule to add.
* @param fs the reference to the scheduler needed in the rule on init.
* @return {@link PlacementRule} to set as a parent
* @throws AllocationConfigurationException for any error
*/
private static PlacementRule getParentRule(Element parent,
FairScheduler fs)
throws AllocationConfigurationException {
LOG.debug("Creating new parent rule: {}", parent.getAttribute("name"));
PlacementRule parentRule = createRule(parent);
// Init the rule, we do not want to add it to the list of the
// placement manager
try {
parentRule.initialize(fs);
} catch (IOException ioe) {
// We should never throw as we pass in a FS object, however we
// still should consider any exception here a config error.
throw new AllocationConfigurationException(
"Parent Rule initialisation failed with exception", ioe);
}
return parentRule;
}
/**
* Returns the terminal status of the rule based on the definition and the
* create flag set in the rule.
* @param terminal The definition of the terminal flag
* @param rule The rule to check
* @return <code>true</code> if the rule is terminal <code>false</code> in
* all other cases.
*/
private static Boolean getTerminal(String terminal, PlacementRule rule) {
switch (terminal) {
case "true": // rule is always terminal
return true;
case "false": // rule is never terminal
return false;
default: // rule is terminal based on the create flag
return ((FSPlacementRule)rule).getCreateFlag();
}
}
/**
* Create a rule from a given a xml node.
* @param element the xml element to create the rule from
* @return PlacementRule
* @throws AllocationConfigurationException for any error
*/
@SuppressWarnings("unchecked")
private static PlacementRule createRule(Element element)
throws AllocationConfigurationException { throws AllocationConfigurationException {
Element element = (Element) node;
String ruleName = element.getAttribute("name"); String ruleName = element.getAttribute("name");
if ("".equals(ruleName)) { if ("".equals(ruleName)) {
@ -109,72 +285,54 @@ public static QueuePlacementRule createAndInitializeRule(Node node)
+ "rule element"); + "rule element");
} }
Class<? extends QueuePlacementRule> clazz = ruleClasses.get(ruleName); Class<? extends PlacementRule> ruleClass = null;
if (clazz == null) { if (RULES.containsKey(ruleName)) {
ruleClass = RULES.get(ruleName).ruleClass;
}
if (ruleClass == null) {
throw new AllocationConfigurationException("No rule class found for " throw new AllocationConfigurationException("No rule class found for "
+ ruleName); + ruleName);
} }
QueuePlacementRule rule = ReflectionUtils.newInstance(clazz, null); return getPlacementRule(ruleClass, element);
rule.initializeFromXml(element);
return rule;
} }
/** /**
* Build a simple queue placement policy from the allow-undeclared-pools and * Build a simple queue placement policy from the configuration options
* user-as-default-queue configuration options. * {@link FairSchedulerConfiguration#ALLOW_UNDECLARED_POOLS} and
* {@link FairSchedulerConfiguration#USER_AS_DEFAULT_QUEUE}.
* @param fs the reference to the scheduler needed in the rule on init.
*/ */
public static QueuePlacementPolicy fromConfiguration(Configuration conf, static void fromConfiguration(FairScheduler fs) {
Map<FSQueueType, Set<String>> configuredQueues) { LOG.debug("Creating base placement policy from config");
Configuration conf = fs.getConfig();
boolean create = conf.getBoolean( boolean create = conf.getBoolean(
FairSchedulerConfiguration.ALLOW_UNDECLARED_POOLS, FairSchedulerConfiguration.ALLOW_UNDECLARED_POOLS,
FairSchedulerConfiguration.DEFAULT_ALLOW_UNDECLARED_POOLS); FairSchedulerConfiguration.DEFAULT_ALLOW_UNDECLARED_POOLS);
boolean userAsDefaultQueue = conf.getBoolean( boolean userAsDefaultQueue = conf.getBoolean(
FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE,
FairSchedulerConfiguration.DEFAULT_USER_AS_DEFAULT_QUEUE); FairSchedulerConfiguration.DEFAULT_USER_AS_DEFAULT_QUEUE);
List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>(); List<PlacementRule> newRules = new ArrayList<>();
rules.add(new QueuePlacementRule.Specified().initialize(create, null)); List<Boolean> newTerminalState = new ArrayList<>();
Class<? extends PlacementRule> clazz =
RULES.get("specified").ruleClass;
newRules.add(getPlacementRule(clazz, create));
newTerminalState.add(false);
if (userAsDefaultQueue) { if (userAsDefaultQueue) {
rules.add(new QueuePlacementRule.User().initialize(create, null)); clazz = RULES.get("user").ruleClass;
newRules.add(getPlacementRule(clazz, create));
newTerminalState.add(create);
} }
if (!userAsDefaultQueue || !create) { if (!userAsDefaultQueue || !create) {
rules.add(new QueuePlacementRule.Default().initialize(true, null)); clazz = RULES.get("default").ruleClass;
newRules.add(getPlacementRule(clazz, true));
newTerminalState.add(true);
} }
try { try {
return new QueuePlacementPolicy(rules, configuredQueues, conf); updateRuleSet(newRules, newTerminalState, fs);
} catch (AllocationConfigurationException ex) { } catch (AllocationConfigurationException ex) {
throw new RuntimeException("Should never hit exception when loading" + throw new RuntimeException("Should never hit exception when loading" +
"placement policy from conf", ex); "placement policy from conf", ex);
} }
} }
/**
* Applies this rule to an app with the given requested queue and user/group
* information.
*
* @param requestedQueue
* The queue specified in the ApplicationSubmissionContext
* @param user
* The user submitting the app
* @return
* The name of the queue to assign the app to. Or null if the app should
* be rejected.
* @throws IOException
* If an exception is encountered while getting the user's groups
*/
public String assignAppToQueue(String requestedQueue, String user)
throws IOException {
for (QueuePlacementRule rule : rules) {
String queue = rule.assignAppToQueue(requestedQueue, user, groups,
configuredQueues);
if (queue == null || !queue.isEmpty()) {
return queue;
}
}
throw new IllegalStateException("Should have applied a rule before " +
"reaching here");
}
public List<QueuePlacementRule> getRules() {
return rules;
}
} }

View File

@ -1,366 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.security.Groups;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.w3c.dom.Element;
import org.w3c.dom.NamedNodeMap;
import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
import com.google.common.annotations.VisibleForTesting;
@Private
@Unstable
public abstract class QueuePlacementRule {
protected boolean create;
public static final Logger LOG =
LoggerFactory.getLogger(QueuePlacementRule.class.getName());
/**
* Initializes the rule with any arguments.
*
* @param args
* Additional attributes of the rule's xml element other than create.
*/
public QueuePlacementRule initialize(boolean create, Map<String, String> args) {
this.create = create;
return this;
}
/**
*
* @param requestedQueue
* The queue explicitly requested.
* @param user
* The user submitting the app.
* @param groups
* The groups of the user submitting the app.
* @param configuredQueues
* The queues specified in the scheduler configuration.
* @return
* The queue to place the app into. An empty string indicates that we should
* continue to the next rule, and null indicates that the app should be rejected.
*/
public String assignAppToQueue(String requestedQueue, String user,
Groups groups, Map<FSQueueType, Set<String>> configuredQueues)
throws IOException {
String queue = getQueueForApp(requestedQueue, user, groups,
configuredQueues);
if (create || configuredQueues.get(FSQueueType.LEAF).contains(queue)
|| configuredQueues.get(FSQueueType.PARENT).contains(queue)) {
return queue;
} else {
return "";
}
}
public void initializeFromXml(Element el)
throws AllocationConfigurationException {
boolean create = true;
NamedNodeMap attributes = el.getAttributes();
Map<String, String> args = new HashMap<String, String>();
for (int i = 0; i < attributes.getLength(); i++) {
Node node = attributes.item(i);
String key = node.getNodeName();
String value = node.getNodeValue();
if (key.equals("create")) {
create = Boolean.parseBoolean(value);
} else {
args.put(key, value);
}
}
initialize(create, args);
}
/**
* Returns true if this rule never tells the policy to continue.
*/
public abstract boolean isTerminal();
/**
* Applies this rule to an app with the given requested queue and user/group
* information.
*
* @param requestedQueue
* The queue specified in the ApplicationSubmissionContext
* @param user
* The user submitting the app.
* @param groups
* The groups of the user submitting the app.
* @return
* The name of the queue to assign the app to, or null to empty string
* continue to the next rule.
*/
protected abstract String getQueueForApp(String requestedQueue, String user,
Groups groups, Map<FSQueueType, Set<String>> configuredQueues)
throws IOException;
/**
* Places apps in queues by username of the submitter
*/
public static class User extends QueuePlacementRule {
@Override
protected String getQueueForApp(String requestedQueue, String user,
Groups groups, Map<FSQueueType, Set<String>> configuredQueues) {
return "root." + cleanName(user);
}
@Override
public boolean isTerminal() {
return create;
}
}
/**
* Places apps in queues by primary group of the submitter
*/
public static class PrimaryGroup extends QueuePlacementRule {
@Override
protected String getQueueForApp(String requestedQueue, String user,
Groups groups, Map<FSQueueType, Set<String>> configuredQueues)
throws IOException {
final List<String> groupList = groups.getGroups(user);
if (groupList.isEmpty()) {
throw new IOException("No groups returned for user " + user);
}
return "root." + cleanName(groupList.get(0));
}
@Override
public boolean isTerminal() {
return create;
}
}
/**
* Places apps in queues by secondary group of the submitter
*
* Match will be made on first secondary group that exist in
* queues
*/
public static class SecondaryGroupExistingQueue extends QueuePlacementRule {
@Override
protected String getQueueForApp(String requestedQueue, String user,
Groups groups, Map<FSQueueType, Set<String>> configuredQueues)
throws IOException {
List<String> groupNames = groups.getGroups(user);
for (int i = 1; i < groupNames.size(); i++) {
String group = cleanName(groupNames.get(i));
if (configuredQueues.get(FSQueueType.LEAF).contains("root." + group)
|| configuredQueues.get(FSQueueType.PARENT).contains(
"root." + group)) {
return "root." + group;
}
}
return "";
}
@Override
public boolean isTerminal() {
return false;
}
}
/**
* Places apps in queues with name of the submitter under the queue
* returned by the nested rule.
*/
public static class NestedUserQueue extends QueuePlacementRule {
@VisibleForTesting
QueuePlacementRule nestedRule;
/**
* Parse xml and instantiate the nested rule
*/
@Override
public void initializeFromXml(Element el)
throws AllocationConfigurationException {
NodeList elements = el.getChildNodes();
for (int i = 0; i < elements.getLength(); i++) {
Node node = elements.item(i);
if (node instanceof Element) {
Element element = (Element) node;
if ("rule".equals(element.getTagName())) {
QueuePlacementRule rule = QueuePlacementPolicy
.createAndInitializeRule(node);
if (rule == null) {
throw new AllocationConfigurationException(
"Unable to create nested rule in nestedUserQueue rule");
}
this.nestedRule = rule;
break;
} else {
continue;
}
}
}
if (this.nestedRule == null) {
throw new AllocationConfigurationException(
"No nested rule specified in <nestedUserQueue> rule");
}
super.initializeFromXml(el);
}
@Override
protected String getQueueForApp(String requestedQueue, String user,
Groups groups, Map<FSQueueType, Set<String>> configuredQueues)
throws IOException {
// Apply the nested rule
String queueName = nestedRule.assignAppToQueue(requestedQueue, user,
groups, configuredQueues);
if (queueName != null && queueName.length() != 0) {
if (!queueName.startsWith("root.")) {
queueName = "root." + queueName;
}
// Verify if the queue returned by the nested rule is an configured leaf queue,
// if yes then skip to next rule in the queue placement policy
if (configuredQueues.get(FSQueueType.LEAF).contains(queueName)) {
return "";
}
return queueName + "." + cleanName(user);
}
return queueName;
}
@Override
public boolean isTerminal() {
return false;
}
}
/**
* Places apps in queues by requested queue of the submitter
*/
public static class Specified extends QueuePlacementRule {
@Override
protected String getQueueForApp(String requestedQueue, String user,
Groups groups, Map<FSQueueType, Set<String>> configuredQueues) {
if (requestedQueue.equals(YarnConfiguration.DEFAULT_QUEUE_NAME)) {
return "";
} else {
if (!requestedQueue.startsWith("root.")) {
requestedQueue = "root." + requestedQueue;
}
return requestedQueue;
}
}
@Override
public boolean isTerminal() {
return false;
}
}
/**
* Places apps in the specified default queue. If no default queue is
* specified the app is placed in root.default queue.
*/
public static class Default extends QueuePlacementRule {
@VisibleForTesting
String defaultQueueName;
@Override
public QueuePlacementRule initialize(boolean create,
Map<String, String> args) {
if (defaultQueueName == null) {
defaultQueueName = "root." + YarnConfiguration.DEFAULT_QUEUE_NAME;
}
return super.initialize(create, args);
}
@Override
public void initializeFromXml(Element el)
throws AllocationConfigurationException {
defaultQueueName = el.getAttribute("queue");
if (defaultQueueName != null && !defaultQueueName.isEmpty()) {
if (!defaultQueueName.startsWith("root.")) {
defaultQueueName = "root." + defaultQueueName;
}
} else {
defaultQueueName = "root." + YarnConfiguration.DEFAULT_QUEUE_NAME;
}
super.initializeFromXml(el);
}
@Override
protected String getQueueForApp(String requestedQueue, String user,
Groups groups, Map<FSQueueType, Set<String>> configuredQueues) {
return defaultQueueName;
}
@Override
public boolean isTerminal() {
return true;
}
}
/**
* Rejects all apps
*/
public static class Reject extends QueuePlacementRule {
@Override
public String assignAppToQueue(String requestedQueue, String user,
Groups groups, Map<FSQueueType, Set<String>> configuredQueues) {
return null;
}
@Override
protected String getQueueForApp(String requestedQueue, String user,
Groups groups, Map<FSQueueType, Set<String>> configuredQueues) {
throw new UnsupportedOperationException();
}
@Override
public boolean isTerminal() {
return true;
}
}
/**
* Replace the periods in the username or groupname with "_dot_" and
* remove trailing and leading whitespace.
*/
protected String cleanName(String name) {
name = name.trim();
if (name.contains(".")) {
String converted = name.replaceAll("\\.", "_dot_");
LOG.warn("Name " + name + " is converted to " + converted
+ " when it is used as a queue name.");
return converted;
} else {
return name;
}
}
}

View File

@ -169,8 +169,7 @@ public static final class Builder {
private Set<String> nonPreemptableQueues = new HashSet<>(); private Set<String> nonPreemptableQueues = new HashSet<>();
// Remember all queue names so we can display them on web UI, etc. // Remember all queue names so we can display them on web UI, etc.
// configuredQueues is segregated based on whether it is a leaf queue // configuredQueues is segregated based on whether it is a leaf queue
// or a parent queue. This information is used for creating queues // or a parent queue. This information is used for creating queues.
// and also for making queue placement decisions(QueuePlacementRule.java).
private Map<FSQueueType, Set<String>> configuredQueues = new HashMap<>(); private Map<FSQueueType, Set<String>> configuredQueues = new HashMap<>();
Builder() { Builder() {

View File

@ -18,32 +18,30 @@
package org.apache.hadoop.yarn.server.resourcemanager; package org.apache.hadoop.yarn.server.resourcemanager;
import static junit.framework.TestCase.assertTrue;
import static org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException.InvalidResourceType.GREATER_THEN_MAX_ALLOCATION; import static org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException.InvalidResourceType.GREATER_THEN_MAX_ALLOCATION;
import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.matches;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.File; import java.io.File;
import java.io.FileWriter; import java.io.FileWriter;
import java.io.IOException; import java.io.IOException;
import java.io.PrintWriter; import java.io.PrintWriter;
import java.util.HashMap; import java.util.Collections;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.yarn.MockApps; import org.apache.hadoop.yarn.MockApps;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
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.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
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.ResourceRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException; import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext; import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager; import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@ -51,39 +49,34 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.AfterClass; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.BeforeClass; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
/** /**
* Testing applications being retired from RM with fair scheduler. * Testing RMAppManager application submission with fair scheduler.
*
*/ */
public class TestAppManagerWithFairScheduler extends AppManagerTestBase{ public class TestAppManagerWithFairScheduler extends AppManagerTestBase {
private static final String TEST_FOLDER = "test-queues"; private static final String TEST_FOLDER = "test-queues";
private static YarnConfiguration conf = new YarnConfiguration(); private static YarnConfiguration conf = new YarnConfiguration();
private PlacementManager placementMgr;
@BeforeClass private TestRMAppManager rmAppManager;
public static void setup() throws IOException { private RMContext rmContext;
String allocFile = private static String allocFileName =
GenericTestUtils.getTestDir(TEST_FOLDER).getAbsolutePath(); GenericTestUtils.getTestDir(TEST_FOLDER).getAbsolutePath();
int queueMaxAllocation = 512; @Before
public void setup() throws IOException {
PrintWriter out = new PrintWriter(new FileWriter(allocFile)); // Basic config with one queue (override in test if needed)
PrintWriter out = new PrintWriter(new FileWriter(allocFileName));
out.println("<?xml version=\"1.0\"?>"); out.println("<?xml version=\"1.0\"?>");
out.println("<allocations>"); out.println("<allocations>");
out.println(" <queue name=\"queueA\">"); out.println(" <queue name=\"test\">");
out.println(" <maxContainerAllocation>" + queueMaxAllocation
+ " mb 1 vcores" + "</maxContainerAllocation>");
out.println(" </queue>");
out.println(" <queue name=\"queueB\">");
out.println(" </queue>"); out.println(" </queue>");
out.println("</allocations>"); out.println("</allocations>");
out.close(); out.close();
@ -91,87 +84,205 @@ public static void setup() throws IOException {
conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class, conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class,
ResourceScheduler.class); ResourceScheduler.class);
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, allocFile); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, allocFileName);
placementMgr = mock(PlacementManager.class);
MockRM mockRM = new MockRM(conf);
rmContext = mockRM.getRMContext();
rmContext.setQueuePlacementManager(placementMgr);
ApplicationMasterService masterService = new ApplicationMasterService(
rmContext, rmContext.getScheduler());
rmAppManager = new TestRMAppManager(rmContext,
new ClientToAMTokenSecretManagerInRM(), rmContext.getScheduler(),
masterService, new ApplicationACLsManager(conf), conf);
} }
@AfterClass @After
public static void teardown(){ public void teardown(){
File allocFile = GenericTestUtils.getTestDir(TEST_FOLDER); File allocFile = GenericTestUtils.getTestDir(TEST_FOLDER);
allocFile.delete(); allocFile.delete();
} }
@Test @Test
public void testQueueSubmitWithHighQueueContainerSize() public void testQueueSubmitWithHighQueueContainerSize()
throws YarnException { throws YarnException, IOException {
int maxAlloc =
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB;
// scheduler config with a limited queue
PrintWriter out = new PrintWriter(new FileWriter(allocFileName));
out.println("<?xml version=\"1.0\"?>");
out.println("<allocations>");
out.println(" <queue name=\"root\">");
out.println(" <queue name=\"limited\">");
out.println(" <maxContainerAllocation>" + maxAlloc + " mb 1 vcores");
out.println(" </maxContainerAllocation>");
out.println(" </queue>");
out.println(" <queue name=\"unlimited\">");
out.println(" </queue>");
out.println(" </queue>");
out.println("</allocations>");
out.close();
rmContext.getScheduler().reinitialize(conf, rmContext);
ApplicationId appId = MockApps.newAppID(1); ApplicationId appId = MockApps.newAppID(1);
RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); Resource res = Resources.createResource(maxAlloc + 1);
ApplicationSubmissionContext asContext = createAppSubmitCtx(appId, res);
Resource resource = Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
ApplicationSubmissionContext asContext =
recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
asContext.setApplicationId(appId);
asContext.setResource(resource);
asContext.setPriority(Priority.newInstance(0));
asContext.setAMContainerSpec(mockContainerLaunchContext(recordFactory));
asContext.setQueue("queueA");
QueueInfo mockDefaultQueueInfo = mock(QueueInfo.class);
// Setup a PlacementManager returns a new queue
PlacementManager placementMgr = mock(PlacementManager.class);
doAnswer(new Answer<ApplicationPlacementContext>() {
@Override
public ApplicationPlacementContext answer(InvocationOnMock invocation)
throws Throwable {
return new ApplicationPlacementContext("queueA");
}
}).when(placementMgr).placeApplication(
any(ApplicationSubmissionContext.class), matches("test1"));
doAnswer(new Answer<ApplicationPlacementContext>() {
@Override
public ApplicationPlacementContext answer(InvocationOnMock invocation)
throws Throwable {
return new ApplicationPlacementContext("queueB");
}
}).when(placementMgr).placeApplication(
any(ApplicationSubmissionContext.class), matches("test2"));
MockRM newMockRM = new MockRM(conf);
RMContext newMockRMContext = newMockRM.getRMContext();
newMockRMContext.setQueuePlacementManager(placementMgr);
ApplicationMasterService masterService = new ApplicationMasterService(
newMockRMContext, newMockRMContext.getScheduler());
TestRMAppManager newAppMonitor = new TestRMAppManager(newMockRMContext,
new ClientToAMTokenSecretManagerInRM(), newMockRMContext.getScheduler(),
masterService, new ApplicationACLsManager(conf), conf);
// only user test has permission to submit to 'test' queue
// Submit to limited queue
when(placementMgr.placeApplication(any(), any()))
.thenReturn(new ApplicationPlacementContext("limited"));
try { try {
newAppMonitor.submitApplication(asContext, "test1"); rmAppManager.submitApplication(asContext, "test");
Assert.fail("Test should fail on too high allocation!"); Assert.fail("Test should fail on too high allocation!");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
Assert.assertEquals(GREATER_THEN_MAX_ALLOCATION, Assert.assertEquals(GREATER_THEN_MAX_ALLOCATION,
e.getInvalidResourceType()); e.getInvalidResourceType());
} }
// Should not throw exception // submit same app but now place it in the unlimited queue
newAppMonitor.submitApplication(asContext, "test2"); when(placementMgr.placeApplication(any(), any()))
.thenReturn(new ApplicationPlacementContext("root.unlimited"));
rmAppManager.submitApplication(asContext, "test");
} }
private static ContainerLaunchContext mockContainerLaunchContext( @Test
RecordFactory recordFactory) { public void testQueueSubmitWithPermissionLimits()
ContainerLaunchContext amContainer = recordFactory.newRecordInstance( throws YarnException, IOException {
ContainerLaunchContext.class);
amContainer conf.set(YarnConfiguration.YARN_ACL_ENABLE, "true");
.setApplicationACLs(new HashMap<ApplicationAccessType, String>());
return amContainer; PrintWriter out = new PrintWriter(new FileWriter(allocFileName));
out.println("<?xml version=\"1.0\"?>");
out.println("<allocations>");
out.println(" <queue name=\"root\">");
out.println(" <aclSubmitApps> </aclSubmitApps>");
out.println(" <aclAdministerApps> </aclAdministerApps>");
out.println(" <queue name=\"noaccess\">");
out.println(" </queue>");
out.println(" <queue name=\"submitonly\">");
out.println(" <aclSubmitApps>test </aclSubmitApps>");
out.println(" <aclAdministerApps> </aclAdministerApps>");
out.println(" </queue>");
out.println(" <queue name=\"adminonly\">");
out.println(" <aclSubmitApps> </aclSubmitApps>");
out.println(" <aclAdministerApps>test </aclAdministerApps>");
out.println(" </queue>");
out.println(" </queue>");
out.println("</allocations>");
out.close();
rmContext.getScheduler().reinitialize(conf, rmContext);
ApplicationId appId = MockApps.newAppID(1);
Resource res = Resources.createResource(1024, 1);
ApplicationSubmissionContext asContext = createAppSubmitCtx(appId, res);
// Submit to no access queue
when(placementMgr.placeApplication(any(), any()))
.thenReturn(new ApplicationPlacementContext("noaccess"));
try {
rmAppManager.submitApplication(asContext, "test");
Assert.fail("Test should have failed with access denied");
} catch (YarnException e) {
assertTrue("Access exception not found",
e.getCause() instanceof AccessControlException);
}
// Submit to submit access queue
when(placementMgr.placeApplication(any(), any()))
.thenReturn(new ApplicationPlacementContext("submitonly"));
rmAppManager.submitApplication(asContext, "test");
// Submit second app to admin access queue
appId = MockApps.newAppID(2);
asContext = createAppSubmitCtx(appId, res);
when(placementMgr.placeApplication(any(), any()))
.thenReturn(new ApplicationPlacementContext("adminonly"));
rmAppManager.submitApplication(asContext, "test");
}
@Test
public void testQueueSubmitWithRootPermission()
throws YarnException, IOException {
conf.set(YarnConfiguration.YARN_ACL_ENABLE, "true");
PrintWriter out = new PrintWriter(new FileWriter(allocFileName));
out.println("<?xml version=\"1.0\"?>");
out.println("<allocations>");
out.println(" <queue name=\"root\">");
out.println(" <queue name=\"noaccess\">");
out.println(" <aclSubmitApps> </aclSubmitApps>");
out.println(" <aclAdministerApps> </aclAdministerApps>");
out.println(" </queue>");
out.println(" </queue>");
out.println("</allocations>");
out.close();
rmContext.getScheduler().reinitialize(conf, rmContext);
ApplicationId appId = MockApps.newAppID(1);
Resource res = Resources.createResource(1024, 1);
ApplicationSubmissionContext asContext = createAppSubmitCtx(appId, res);
// Submit to noaccess queue should be allowed by root ACL
when(placementMgr.placeApplication(any(), any()))
.thenReturn(new ApplicationPlacementContext("noaccess"));
rmAppManager.submitApplication(asContext, "test");
}
@Test
public void testQueueSubmitWithAutoCreateQueue()
throws YarnException, IOException {
conf.set(YarnConfiguration.YARN_ACL_ENABLE, "true");
PrintWriter out = new PrintWriter(new FileWriter(allocFileName));
out.println("<?xml version=\"1.0\"?>");
out.println("<allocations>");
out.println(" <queue name=\"root\">");
out.println(" <aclSubmitApps> </aclSubmitApps>");
out.println(" <aclAdministerApps> </aclAdministerApps>");
out.println(" <queue name=\"noaccess\" type=\"parent\">");
out.println(" </queue>");
out.println(" <queue name=\"submitonly\" type=\"parent\">");
out.println(" <aclSubmitApps>test </aclSubmitApps>");
out.println(" </queue>");
out.println(" </queue>");
out.println("</allocations>");
out.close();
rmContext.getScheduler().reinitialize(conf, rmContext);
ApplicationId appId = MockApps.newAppID(1);
Resource res = Resources.createResource(1024, 1);
ApplicationSubmissionContext asContext = createAppSubmitCtx(appId, res);
// Submit to noaccess parent with non existent child queue
when(placementMgr.placeApplication(any(), any()))
.thenReturn(new ApplicationPlacementContext("root.noaccess.child"));
try {
rmAppManager.submitApplication(asContext, "test");
Assert.fail("Test should have failed with access denied");
} catch (YarnException e) {
assertTrue("Access exception not found",
e.getCause() instanceof AccessControlException);
}
// Submit to submitonly parent with non existent child queue
when(placementMgr.placeApplication(any(), any()))
.thenReturn(new ApplicationPlacementContext("root.submitonly.child"));
rmAppManager.submitApplication(asContext, "test");
}
private ApplicationSubmissionContext createAppSubmitCtx(ApplicationId appId,
Resource res) {
ApplicationSubmissionContext asContext =
Records.newRecord(ApplicationSubmissionContext.class);
asContext.setApplicationId(appId);
ResourceRequest resReg =
ResourceRequest.newInstance(Priority.newInstance(0),
ResourceRequest.ANY, res, 1);
asContext.setAMContainerResourceRequests(
Collections.singletonList(resReg));
asContext.setAMContainerSpec(mock(ContainerLaunchContext.class));
asContext.setQueue("default");
return asContext;
} }
} }

View File

@ -466,7 +466,7 @@ private void verifyInvalidQueueWithAcl() throws Exception {
if (conf.get(YarnConfiguration.RM_SCHEDULER) if (conf.get(YarnConfiguration.RM_SCHEDULER)
.equals(FairScheduler.class.getName())) { .equals(FairScheduler.class.getName())) {
Assert.assertTrue(appReport.getDiagnostics() Assert.assertTrue(appReport.getDiagnostics()
.contains("Application rejected by queue placement policy")); .contains("user owner application rejected by placement rules."));
} else { } else {
Assert.assertTrue(appReport.getDiagnostics() Assert.assertTrue(appReport.getDiagnostics()
.contains("submitted by user owner to unknown queue: InvalidQueue")); .contains("submitted by user owner to unknown queue: InvalidQueue"));

View File

@ -30,6 +30,7 @@
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.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException; import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent; import org.apache.hadoop.yarn.server.resourcemanager.reservation.planning.ReservationAgent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@ -38,6 +39,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
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.scheduler.fair.FairScheduler;
import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.Clock;
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;
@ -106,8 +108,14 @@ protected void testPlanFollower(boolean isMove) throws PlanningException,
ApplicationId appId = ApplicationId.newInstance(0, 1); ApplicationId appId = ApplicationId.newInstance(0, 1);
ApplicationAttemptId appAttemptId_0 = ApplicationAttemptId appAttemptId_0 =
ApplicationAttemptId.newInstance(appId, 0); ApplicationAttemptId.newInstance(appId, 0);
AppAddedSchedulerEvent addAppEvent = AppAddedSchedulerEvent addAppEvent;
new AppAddedSchedulerEvent(appId, q.getQueueName(), user_0); if (scheduler instanceof FairScheduler) {
addAppEvent = new AppAddedSchedulerEvent(appId, q.getQueueName(),
user_0, new ApplicationPlacementContext("dedicated"));
} else {
addAppEvent = new AppAddedSchedulerEvent(appId, q.getQueueName(),
user_0);
}
scheduler.handle(addAppEvent); scheduler.handle(addAppEvent);
AppAttemptAddedSchedulerEvent appAttemptAddedEvent = AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
new AppAttemptAddedSchedulerEvent(appAttemptId_0, false); new AppAttemptAddedSchedulerEvent(appAttemptId_0, false);

View File

@ -81,6 +81,7 @@
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.NullRMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
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;
@ -979,13 +980,16 @@ public static void waitSchedulerApplicationAttemptStopped(
public static SchedulerApplication<SchedulerApplicationAttempt> public static SchedulerApplication<SchedulerApplicationAttempt>
verifyAppAddedAndRemovedFromScheduler( verifyAppAddedAndRemovedFromScheduler(
Map<ApplicationId, SchedulerApplication<SchedulerApplicationAttempt>> applications, Map<ApplicationId, SchedulerApplication<SchedulerApplicationAttempt>>
applications,
EventHandler<SchedulerEvent> handler, String queueName) { EventHandler<SchedulerEvent> handler, String queueName) {
ApplicationPlacementContext apc =
new ApplicationPlacementContext(queueName);
ApplicationId appId = ApplicationId appId =
ApplicationId.newInstance(System.currentTimeMillis(), 1); ApplicationId.newInstance(System.currentTimeMillis(), 1);
AppAddedSchedulerEvent appAddedEvent = AppAddedSchedulerEvent appAddedEvent =
new AppAddedSchedulerEvent(appId, queueName, "user"); new AppAddedSchedulerEvent(appId, queueName, "user", apc);
handler.handle(appAddedEvent); handler.handle(appAddedEvent);
SchedulerApplication<SchedulerApplicationAttempt> app = SchedulerApplication<SchedulerApplicationAttempt> app =
applications.get(appId); applications.get(appId);

View File

@ -35,6 +35,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes; import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
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.rmapp.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.RMAppEvent;
@ -177,7 +178,11 @@ protected ApplicationAttemptId createSchedulingRequest(
Collection<ResourceRequest> requests, String queueId, String userId) { Collection<ResourceRequest> requests, String queueId, String userId) {
ApplicationAttemptId id = ApplicationAttemptId id =
createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++); createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
scheduler.addApplication(id.getApplicationId(), queueId, userId, false); // This fakes the placement which is not part of the scheduler anymore
ApplicationPlacementContext placementCtx =
new ApplicationPlacementContext(queueId);
scheduler.addApplication(id.getApplicationId(), queueId, userId, false,
placementCtx);
// This conditional is for testAclSubmitApplication where app is rejected // This conditional is for testAclSubmitApplication where app is rejected
// and no app is added. // and no app is added.
if (scheduler.getSchedulerApplications() if (scheduler.getSchedulerApplications()
@ -212,10 +217,15 @@ protected ApplicationAttemptId createSchedulingRequest(String queueId,
String userId, List<ResourceRequest> ask) { String userId, List<ResourceRequest> ask) {
ApplicationAttemptId id = createAppAttemptId(this.APP_ID++, ApplicationAttemptId id = createAppAttemptId(this.APP_ID++,
this.ATTEMPT_ID++); this.ATTEMPT_ID++);
scheduler.addApplication(id.getApplicationId(), queueId, userId, false); // This fakes the placement which is not part of the scheduler anymore
ApplicationPlacementContext placementCtx =
new ApplicationPlacementContext(queueId);
scheduler.addApplication(id.getApplicationId(), queueId, userId, false,
placementCtx);
// This conditional is for testAclSubmitApplication where app is rejected // This conditional is for testAclSubmitApplication where app is rejected
// and no app is added. // and no app is added.
if (scheduler.getSchedulerApplications().containsKey(id.getApplicationId())) { if (scheduler.getSchedulerApplications().containsKey(
id.getApplicationId())) {
scheduler.addApplicationAttempt(id, false, false); scheduler.addApplicationAttempt(id, false, false);
} }
@ -298,8 +308,11 @@ private void addApplication(String queue, String user, ApplicationId appId) {
resourceManager.getRMContext().getRMApps().get(appId).handle(event); resourceManager.getRMContext().getRMApps().get(appId).handle(event);
event = new RMAppEvent(appId, RMAppEventType.APP_ACCEPTED); event = new RMAppEvent(appId, RMAppEventType.APP_ACCEPTED);
resourceManager.getRMContext().getRMApps().get(appId).handle(event); resourceManager.getRMContext().getRMApps().get(appId).handle(event);
// This fakes the placement which is not part of the scheduler anymore
ApplicationPlacementContext placementCtx =
new ApplicationPlacementContext(queue);
AppAddedSchedulerEvent appAddedEvent = new AppAddedSchedulerEvent( AppAddedSchedulerEvent appAddedEvent = new AppAddedSchedulerEvent(
appId, queue, user); appId, queue, user, placementCtx);
scheduler.handle(appAddedEvent); scheduler.handle(appAddedEvent);
} }

View File

@ -25,14 +25,23 @@
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
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.server.resourcemanager.placement.DefaultPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.FSPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.PrimaryGroupPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.SpecifiedPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.UserPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueuePlacementRule.NestedUserQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileWriter; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfile.AllocationFileWriter;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
import org.apache.hadoop.yarn.util.ControlledClock; import org.apache.hadoop.yarn.util.ControlledClock;
import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.hadoop.yarn.util.resource.CustomResourceTypesConfigurationProvider; import org.apache.hadoop.yarn.util.resource.CustomResourceTypesConfigurationProvider;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.io.File; import java.io.File;
import java.io.FileOutputStream; import java.io.FileOutputStream;
@ -52,7 +61,12 @@
import static org.junit.Assert.assertNull; import static org.junit.Assert.assertNull;
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.Mockito.mock;
import static org.mockito.Mockito.when;
/**
* Test loading the allocation file for the FairScheduler.
*/
public class TestAllocationFileLoaderService { public class TestAllocationFileLoaderService {
private static final String A_CUSTOM_RESOURCE = "a-custom-resource"; private static final String A_CUSTOM_RESOURCE = "a-custom-resource";
@ -64,10 +78,28 @@ public class TestAllocationFileLoaderService {
"test-queues").getAbsolutePath(); "test-queues").getAbsolutePath();
private static final String TEST_FAIRSCHED_XML = "test-fair-scheduler.xml"; private static final String TEST_FAIRSCHED_XML = "test-fair-scheduler.xml";
private FairScheduler scheduler;
private Configuration conf;
@Before
public void setup() {
SystemClock clock = SystemClock.getInstance();
PlacementManager placementManager = new PlacementManager();
FairSchedulerConfiguration fsConf = new FairSchedulerConfiguration();
RMContext rmContext = mock(RMContext.class);
when(rmContext.getQueuePlacementManager()).thenReturn(placementManager);
scheduler = mock(FairScheduler.class);
conf = new YarnConfiguration();
when(scheduler.getClock()).thenReturn(clock);
when(scheduler.getConf()).thenReturn(fsConf);
when(scheduler.getConfig()).thenReturn(conf);
when(scheduler.getRMContext()).thenReturn(rmContext);
}
@Test @Test
public void testGetAllocationFileFromFileSystem() public void testGetAllocationFileFromFileSystem()
throws IOException, URISyntaxException { throws IOException, URISyntaxException {
Configuration conf = new YarnConfiguration();
File baseDir = File baseDir =
new File(TEST_DIR + Path.SEPARATOR + "getAllocHDFS").getAbsoluteFile(); new File(TEST_DIR + Path.SEPARATOR + "getAllocHDFS").getAbsoluteFile();
FileUtil.fullyDelete(baseDir); FileUtil.fullyDelete(baseDir);
@ -83,7 +115,8 @@ public void testGetAllocationFileFromFileSystem()
fs.copyFromLocalFile(new Path(fschedURL.toURI()), new Path(fsAllocPath)); fs.copyFromLocalFile(new Path(fschedURL.toURI()), new Path(fsAllocPath));
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, fsAllocPath); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, fsAllocPath);
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
Path allocationFile = allocLoader.getAllocationFile(conf); Path allocationFile = allocLoader.getAllocationFile(conf);
assertEquals(fsAllocPath, allocationFile.toString()); assertEquals(fsAllocPath, allocationFile.toString());
assertTrue(fs.exists(allocationFile)); assertTrue(fs.exists(allocationFile));
@ -94,9 +127,9 @@ public void testGetAllocationFileFromFileSystem()
@Test (expected = UnsupportedFileSystemException.class) @Test (expected = UnsupportedFileSystemException.class)
public void testDenyGetAllocationFileFromUnsupportedFileSystem() public void testDenyGetAllocationFileFromUnsupportedFileSystem()
throws UnsupportedFileSystemException { throws UnsupportedFileSystemException {
Configuration conf = new YarnConfiguration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, "badfs:///badfile"); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, "badfs:///badfile");
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
allocLoader.getAllocationFile(conf); allocLoader.getAllocationFile(conf);
} }
@ -104,12 +137,11 @@ public void testDenyGetAllocationFileFromUnsupportedFileSystem()
@Test @Test
public void testGetAllocationFileFromClasspath() { public void testGetAllocationFileFromClasspath() {
try { try {
Configuration conf = new Configuration();
FileSystem fs = FileSystem.get(conf); FileSystem fs = FileSystem.get(conf);
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, conf.set(FairSchedulerConfiguration.ALLOCATION_FILE,
TEST_FAIRSCHED_XML); TEST_FAIRSCHED_XML);
AllocationFileLoaderService allocLoader = AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(); new AllocationFileLoaderService(scheduler);
Path allocationFile = allocLoader.getAllocationFile(conf); Path allocationFile = allocLoader.getAllocationFile(conf);
assertEquals(TEST_FAIRSCHED_XML, allocationFile.getName()); assertEquals(TEST_FAIRSCHED_XML, allocationFile.getName());
assertTrue(fs.exists(allocationFile)); assertTrue(fs.exists(allocationFile));
@ -135,11 +167,10 @@ public void testReload() throws Exception {
ControlledClock clock = new ControlledClock(); ControlledClock clock = new ControlledClock();
clock.setTime(0); clock.setTime(0);
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService( AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(
clock); clock, scheduler);
allocLoader.reloadIntervalMs = 5; allocLoader.reloadIntervalMs = 5;
allocLoader.init(conf); allocLoader.init(conf);
ReloadListener confHolder = new ReloadListener(); ReloadListener confHolder = new ReloadListener();
@ -148,10 +179,10 @@ public void testReload() throws Exception {
AllocationConfiguration allocConf = confHolder.allocConf; AllocationConfiguration allocConf = confHolder.allocConf;
// Verify conf // Verify conf
QueuePlacementPolicy policy = allocConf.getPlacementPolicy(); List<PlacementRule> rules = scheduler.getRMContext()
List<QueuePlacementRule> rules = policy.getRules(); .getQueuePlacementManager().getPlacementRules();
assertEquals(1, rules.size()); assertEquals(1, rules.size());
assertEquals(QueuePlacementRule.Default.class, rules.get(0).getClass()); assertEquals(DefaultPlacementRule.class, rules.get(0).getClass());
assertEquals(1, allocConf.getQueueMaxApps("root.queueA")); assertEquals(1, allocConf.getQueueMaxApps("root.queueA"));
assertEquals(2, allocConf.getConfiguredQueues().get(FSQueueType.LEAF) assertEquals(2, allocConf.getConfiguredQueues().get(FSQueueType.LEAF)
.size()); .size());
@ -160,6 +191,7 @@ public void testReload() throws Exception {
assertTrue(allocConf.getConfiguredQueues().get(FSQueueType.LEAF) assertTrue(allocConf.getConfiguredQueues().get(FSQueueType.LEAF)
.contains("root.queueB")); .contains("root.queueB"));
// reset the conf so we can detect the reload
confHolder.allocConf = null; confHolder.allocConf = null;
// Modify file and advance the clock // Modify file and advance the clock
@ -174,7 +206,6 @@ public void testReload() throws Exception {
out.println(" <rule name='nestedUserQueue' >"); out.println(" <rule name='nestedUserQueue' >");
out.println(" <rule name='primaryGroup' />"); out.println(" <rule name='primaryGroup' />");
out.println(" </rule>"); out.println(" </rule>");
out.println(" <rule name='default' />");
out.println(" </queuePlacementPolicy>"); out.println(" </queuePlacementPolicy>");
out.println("</allocations>"); out.println("</allocations>");
out.close(); out.close();
@ -189,15 +220,13 @@ public void testReload() throws Exception {
// Verify conf // Verify conf
allocConf = confHolder.allocConf; allocConf = confHolder.allocConf;
policy = allocConf.getPlacementPolicy(); rules = scheduler.getRMContext().getQueuePlacementManager()
rules = policy.getRules(); .getPlacementRules();
assertEquals(3, rules.size()); assertEquals(2, rules.size());
assertEquals(QueuePlacementRule.Specified.class, rules.get(0).getClass()); assertEquals(SpecifiedPlacementRule.class, rules.get(0).getClass());
assertEquals(QueuePlacementRule.NestedUserQueue.class, rules.get(1) assertEquals(UserPlacementRule.class, rules.get(1).getClass());
.getClass()); assertEquals(PrimaryGroupPlacementRule.class,
assertEquals(QueuePlacementRule.PrimaryGroup.class, ((FSPlacementRule)(rules.get(1))).getParentRule().getClass());
((NestedUserQueue) (rules.get(1))).nestedRule.getClass());
assertEquals(QueuePlacementRule.Default.class, rules.get(2).getClass());
assertEquals(3, allocConf.getQueueMaxApps("root.queueB")); assertEquals(3, allocConf.getQueueMaxApps("root.queueB"));
assertEquals(1, allocConf.getConfiguredQueues().get(FSQueueType.LEAF) assertEquals(1, allocConf.getConfiguredQueues().get(FSQueueType.LEAF)
.size()); .size());
@ -207,11 +236,11 @@ public void testReload() throws Exception {
@Test @Test
public void testAllocationFileParsing() throws Exception { public void testAllocationFileParsing() throws Exception {
Configuration conf = new YarnConfiguration();
CustomResourceTypesConfigurationProvider. CustomResourceTypesConfigurationProvider.
initResourceTypes(A_CUSTOM_RESOURCE); initResourceTypes(A_CUSTOM_RESOURCE);
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
AllocationFileWriter AllocationFileWriter
.create() .create()
@ -455,9 +484,9 @@ public void testAllocationFileParsing() throws Exception {
@Test @Test
public void testBackwardsCompatibleAllocationFileParsing() throws Exception { public void testBackwardsCompatibleAllocationFileParsing() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
out.println("<?xml version=\"1.0\"?>"); out.println("<?xml version=\"1.0\"?>");
@ -569,7 +598,6 @@ public void testBackwardsCompatibleAllocationFileParsing() throws Exception {
@Test @Test
public void testSimplePlacementPolicyFromConf() throws Exception { public void testSimplePlacementPolicyFromConf() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
conf.setBoolean(FairSchedulerConfiguration.ALLOW_UNDECLARED_POOLS, false); conf.setBoolean(FairSchedulerConfiguration.ALLOW_UNDECLARED_POOLS, false);
conf.setBoolean(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, false); conf.setBoolean(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, false);
@ -580,19 +608,20 @@ public void testSimplePlacementPolicyFromConf() throws Exception {
out.println("</allocations>"); out.println("</allocations>");
out.close(); out.close();
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
allocLoader.init(conf); allocLoader.init(conf);
ReloadListener confHolder = new ReloadListener(); ReloadListener confHolder = new ReloadListener();
allocLoader.setReloadListener(confHolder); allocLoader.setReloadListener(confHolder);
allocLoader.reloadAllocations(); allocLoader.reloadAllocations();
AllocationConfiguration allocConf = confHolder.allocConf;
QueuePlacementPolicy placementPolicy = allocConf.getPlacementPolicy(); List<PlacementRule> rules = scheduler.getRMContext()
List<QueuePlacementRule> rules = placementPolicy.getRules(); .getQueuePlacementManager().getPlacementRules();
assertEquals(2, rules.size()); assertEquals(2, rules.size());
assertEquals(QueuePlacementRule.Specified.class, rules.get(0).getClass()); assertEquals(SpecifiedPlacementRule.class, rules.get(0).getClass());
assertEquals(false, rules.get(0).create); assertFalse("Create flag was not set to false",
assertEquals(QueuePlacementRule.Default.class, rules.get(1).getClass()); ((FSPlacementRule)rules.get(0)).getCreateFlag());
assertEquals(DefaultPlacementRule.class, rules.get(1).getClass());
} }
/** /**
@ -601,7 +630,6 @@ public void testSimplePlacementPolicyFromConf() throws Exception {
*/ */
@Test (expected = AllocationConfigurationException.class) @Test (expected = AllocationConfigurationException.class)
public void testQueueAlongsideRoot() throws Exception { public void testQueueAlongsideRoot() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
@ -614,7 +642,8 @@ public void testQueueAlongsideRoot() throws Exception {
out.println("</allocations>"); out.println("</allocations>");
out.close(); out.close();
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
allocLoader.init(conf); allocLoader.init(conf);
ReloadListener confHolder = new ReloadListener(); ReloadListener confHolder = new ReloadListener();
allocLoader.setReloadListener(confHolder); allocLoader.setReloadListener(confHolder);
@ -627,7 +656,6 @@ public void testQueueAlongsideRoot() throws Exception {
*/ */
@Test (expected = AllocationConfigurationException.class) @Test (expected = AllocationConfigurationException.class)
public void testQueueNameContainingPeriods() throws Exception { public void testQueueNameContainingPeriods() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
@ -638,7 +666,8 @@ public void testQueueNameContainingPeriods() throws Exception {
out.println("</allocations>"); out.println("</allocations>");
out.close(); out.close();
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
allocLoader.init(conf); allocLoader.init(conf);
ReloadListener confHolder = new ReloadListener(); ReloadListener confHolder = new ReloadListener();
allocLoader.setReloadListener(confHolder); allocLoader.setReloadListener(confHolder);
@ -651,7 +680,6 @@ public void testQueueNameContainingPeriods() throws Exception {
*/ */
@Test (expected = AllocationConfigurationException.class) @Test (expected = AllocationConfigurationException.class)
public void testQueueNameContainingOnlyWhitespace() throws Exception { public void testQueueNameContainingOnlyWhitespace() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
@ -662,7 +690,8 @@ public void testQueueNameContainingOnlyWhitespace() throws Exception {
out.println("</allocations>"); out.println("</allocations>");
out.close(); out.close();
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
allocLoader.init(conf); allocLoader.init(conf);
ReloadListener confHolder = new ReloadListener(); ReloadListener confHolder = new ReloadListener();
allocLoader.setReloadListener(confHolder); allocLoader.setReloadListener(confHolder);
@ -671,7 +700,6 @@ public void testQueueNameContainingOnlyWhitespace() throws Exception {
@Test @Test
public void testParentTagWithReservation() throws Exception { public void testParentTagWithReservation() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
@ -684,7 +712,8 @@ public void testParentTagWithReservation() throws Exception {
out.println("</allocations>"); out.println("</allocations>");
out.close(); out.close();
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
allocLoader.init(conf); allocLoader.init(conf);
ReloadListener confHolder = new ReloadListener(); ReloadListener confHolder = new ReloadListener();
allocLoader.setReloadListener(confHolder); allocLoader.setReloadListener(confHolder);
@ -700,7 +729,6 @@ public void testParentTagWithReservation() throws Exception {
@Test @Test
public void testParentWithReservation() throws Exception { public void testParentWithReservation() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
@ -715,7 +743,8 @@ public void testParentWithReservation() throws Exception {
out.println("</allocations>"); out.println("</allocations>");
out.close(); out.close();
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
allocLoader.init(conf); allocLoader.init(conf);
ReloadListener confHolder = new ReloadListener(); ReloadListener confHolder = new ReloadListener();
allocLoader.setReloadListener(confHolder); allocLoader.setReloadListener(confHolder);
@ -731,7 +760,6 @@ public void testParentWithReservation() throws Exception {
@Test @Test
public void testParentTagWithChild() throws Exception { public void testParentTagWithChild() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
@ -744,7 +772,8 @@ public void testParentTagWithChild() throws Exception {
out.println("</allocations>"); out.println("</allocations>");
out.close(); out.close();
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
allocLoader.init(conf); allocLoader.init(conf);
ReloadListener confHolder = new ReloadListener(); ReloadListener confHolder = new ReloadListener();
allocLoader.setReloadListener(confHolder); allocLoader.setReloadListener(confHolder);
@ -763,7 +792,6 @@ public void testParentTagWithChild() throws Exception {
*/ */
@Test (expected = AllocationConfigurationException.class) @Test (expected = AllocationConfigurationException.class)
public void testQueueNameContainingNBWhitespace() throws Exception { public void testQueueNameContainingNBWhitespace() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
PrintWriter out = new PrintWriter(new OutputStreamWriter( PrintWriter out = new PrintWriter(new OutputStreamWriter(
@ -775,7 +803,8 @@ public void testQueueNameContainingNBWhitespace() throws Exception {
out.println("</allocations>"); out.println("</allocations>");
out.close(); out.close();
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
allocLoader.init(conf); allocLoader.init(conf);
ReloadListener confHolder = new ReloadListener(); ReloadListener confHolder = new ReloadListener();
allocLoader.setReloadListener(confHolder); allocLoader.setReloadListener(confHolder);
@ -787,17 +816,18 @@ public void testQueueNameContainingNBWhitespace() throws Exception {
*/ */
@Test (expected = AllocationConfigurationException.class) @Test (expected = AllocationConfigurationException.class)
public void testDefaultQueueSchedulingModeIsFIFO() throws Exception { public void testDefaultQueueSchedulingModeIsFIFO() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
out.println("<?xml version=\"1.0\"?>"); out.println("<?xml version=\"1.0\"?>");
out.println("<allocations>"); out.println("<allocations>");
out.println("<defaultQueueSchedulingPolicy>fifo</defaultQueueSchedulingPolicy>"); out.println("<defaultQueueSchedulingPolicy>fifo" +
"</defaultQueueSchedulingPolicy>");
out.println("</allocations>"); out.println("</allocations>");
out.close(); out.close();
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
allocLoader.init(conf); allocLoader.init(conf);
ReloadListener confHolder = new ReloadListener(); ReloadListener confHolder = new ReloadListener();
allocLoader.setReloadListener(confHolder); allocLoader.setReloadListener(confHolder);
@ -806,7 +836,6 @@ public void testDefaultQueueSchedulingModeIsFIFO() throws Exception {
@Test @Test
public void testReservableQueue() throws Exception { public void testReservableQueue() throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
@ -823,7 +852,8 @@ public void testReservableQueue() throws Exception {
out.println("</allocations>"); out.println("</allocations>");
out.close(); out.close();
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
allocLoader.init(conf); allocLoader.init(conf);
ReloadListener confHolder = new ReloadListener(); ReloadListener confHolder = new ReloadListener();
allocLoader.setReloadListener(confHolder); allocLoader.setReloadListener(confHolder);
@ -845,11 +875,9 @@ public void testReservableQueue() throws Exception {
assertTrue(allocConf.getMoveOnExpiry(reservableQueueName)); assertTrue(allocConf.getMoveOnExpiry(reservableQueueName));
assertEquals(ReservationSchedulerConfiguration.DEFAULT_RESERVATION_WINDOW, assertEquals(ReservationSchedulerConfiguration.DEFAULT_RESERVATION_WINDOW,
allocConf.getReservationWindow(reservableQueueName)); allocConf.getReservationWindow(reservableQueueName));
assertEquals(100, allocConf.getInstantaneousMaxCapacity assertEquals(100,
(reservableQueueName), allocConf.getInstantaneousMaxCapacity(reservableQueueName), 0.0001);
0.0001); assertEquals("DummyAgentName",
assertEquals(
"DummyAgentName",
allocConf.getReservationAgent(reservableQueueName)); allocConf.getReservationAgent(reservableQueueName));
assertEquals(100, allocConf.getAverageCapacity(reservableQueueName), 0.001); assertEquals(100, allocConf.getAverageCapacity(reservableQueueName), 0.001);
assertFalse(allocConf.getShowReservationAsQueues(reservableQueueName)); assertFalse(allocConf.getShowReservationAsQueues(reservableQueueName));
@ -865,12 +893,11 @@ public void testReservableQueue() throws Exception {
/** /**
* Verify that you can't have dynamic user queue and reservable queue on * Verify that you can't have dynamic user queue and reservable queue on
* the same queue * the same queue.
*/ */
@Test (expected = AllocationConfigurationException.class) @Test (expected = AllocationConfigurationException.class)
public void testReservableCannotBeCombinedWithDynamicUserQueue() public void testReservableCannotBeCombinedWithDynamicUserQueue()
throws Exception { throws Exception {
Configuration conf = new Configuration();
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE)); PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
@ -883,7 +910,8 @@ public void testReservableCannotBeCombinedWithDynamicUserQueue()
out.println("</allocations>"); out.println("</allocations>");
out.close(); out.close();
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService(); AllocationFileLoaderService allocLoader =
new AllocationFileLoaderService(scheduler);
allocLoader.init(conf); allocLoader.init(conf);
ReloadListener confHolder = new ReloadListener(); ReloadListener confHolder = new ReloadListener();
allocLoader.setReloadListener(confHolder); allocLoader.setReloadListener(confHolder);
@ -891,7 +919,7 @@ public void testReservableCannotBeCombinedWithDynamicUserQueue()
} }
private class ReloadListener implements AllocationFileLoaderService.Listener { private class ReloadListener implements AllocationFileLoaderService.Listener {
public AllocationConfiguration allocConf; private AllocationConfiguration allocConf;
@Override @Override
public void onReload(AllocationConfiguration info) { public void onReload(AllocationConfiguration info) {

View File

@ -34,6 +34,7 @@
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes; import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
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.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
@ -47,7 +48,7 @@
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
/* /**
* This class is to test the fair scheduler functionality of * This class is to test the fair scheduler functionality of
* deciding the number of runnable application under various conditions. * deciding the number of runnable application under various conditions.
*/ */
@ -78,7 +79,7 @@ public void testUserAsDefaultQueue() throws Exception {
conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "true"); conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "true");
scheduler.reinitialize(conf, resourceManager.getRMContext()); scheduler.reinitialize(conf, resourceManager.getRMContext());
ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1); ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1);
createApplicationWithAMResource(appAttemptId, "default", "user1", null); createApplicationWithAMResource(appAttemptId, "root.user1", "user1", null);
assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true) assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true)
.getNumRunnableApps()); .getNumRunnableApps());
assertEquals(0, scheduler.getQueueManager().getLeafQueue("default", true) assertEquals(0, scheduler.getQueueManager().getLeafQueue("default", true)
@ -110,9 +111,11 @@ public void testNotUserAsDefaultQueue() throws Exception {
@Test @Test
public void testAppAdditionAndRemoval() throws Exception { public void testAppAdditionAndRemoval() throws Exception {
ApplicationAttemptId attemptId = createAppAttemptId(1, 1); ApplicationAttemptId attemptId = createAppAttemptId(1, 1);
ApplicationPlacementContext apc =
new ApplicationPlacementContext("user1");
AppAddedSchedulerEvent appAddedEvent = AppAddedSchedulerEvent appAddedEvent =
new AppAddedSchedulerEvent(attemptId.getApplicationId(), "default", new AppAddedSchedulerEvent(attemptId.getApplicationId(), "user1",
"user1"); "user1", apc);
scheduler.handle(appAddedEvent); scheduler.handle(appAddedEvent);
AppAttemptAddedSchedulerEvent attemptAddedEvent = AppAttemptAddedSchedulerEvent attemptAddedEvent =
new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false); new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false);
@ -149,7 +152,7 @@ public void testPreemptionVariablesForQueueCreatedRuntime() throws Exception {
// User1 submits one application // User1 submits one application
ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1); ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1);
createApplicationWithAMResource(appAttemptId, "default", "user1", null); createApplicationWithAMResource(appAttemptId, "user1", "user1", null);
// The user1 queue should inherit the configurations from the root queue // The user1 queue should inherit the configurations from the root queue
FSLeafQueue userQueue = FSLeafQueue userQueue =
@ -184,12 +187,14 @@ public void testDontAllowUndeclaredPools() throws Exception {
FSLeafQueue jerryQueue = queueManager.getLeafQueue("jerry", false); FSLeafQueue jerryQueue = queueManager.getLeafQueue("jerry", false);
FSLeafQueue defaultQueue = queueManager.getLeafQueue("default", false); FSLeafQueue defaultQueue = queueManager.getLeafQueue("default", false);
// NOTE: placement is not inside the scheduler anymore need to fake it here.
// The scheduling request contains the fake placing
// Should get put into jerry // Should get put into jerry
createSchedulingRequest(1024, "jerry", "someuser"); createSchedulingRequest(1024, "jerry", "someuser");
assertEquals(1, jerryQueue.getNumRunnableApps()); assertEquals(1, jerryQueue.getNumRunnableApps());
// Should get forced into default // Should get forced into default
createSchedulingRequest(1024, "newqueue", "someuser"); createSchedulingRequest(1024, "default", "someuser");
assertEquals(1, jerryQueue.getNumRunnableApps()); assertEquals(1, jerryQueue.getNumRunnableApps());
assertEquals(1, defaultQueue.getNumRunnableApps()); assertEquals(1, defaultQueue.getNumRunnableApps());
@ -200,7 +205,7 @@ public void testDontAllowUndeclaredPools() throws Exception {
assertEquals(2, defaultQueue.getNumRunnableApps()); assertEquals(2, defaultQueue.getNumRunnableApps());
// Should get put into jerry because of user-as-default-queue // Should get put into jerry because of user-as-default-queue
createSchedulingRequest(1024, "default", "jerry"); createSchedulingRequest(1024, "jerry", "jerry");
assertEquals(2, jerryQueue.getNumRunnableApps()); assertEquals(2, jerryQueue.getNumRunnableApps());
assertEquals(2, defaultQueue.getNumRunnableApps()); assertEquals(2, defaultQueue.getNumRunnableApps());
} }

View File

@ -29,6 +29,7 @@
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes; import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
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.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ClusterNodeTracker; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ClusterNodeTracker;
@ -113,14 +114,18 @@ public void testBasic() throws InterruptedException {
1, Resources.createResource(4096, 4), 1, host); 1, Resources.createResource(4096, 4), 1, host);
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
NodeUpdateSchedulerEvent nodeUpdateEvent = new NodeUpdateSchedulerEvent(node1); NodeUpdateSchedulerEvent nodeUpdateEvent =
new NodeUpdateSchedulerEvent(node1);
scheduler.handle(nodeUpdateEvent); scheduler.handle(nodeUpdateEvent);
ApplicationAttemptId appAttemptId = ApplicationAttemptId appAttemptId =
createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++); createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
createMockRMApp(appAttemptId); createMockRMApp(appAttemptId);
scheduler.addApplication(appAttemptId.getApplicationId(), "queue11", "user11", false); ApplicationPlacementContext placementCtx =
new ApplicationPlacementContext("queue11");
scheduler.addApplication(appAttemptId.getApplicationId(), "queue11",
"user11", false, placementCtx);
scheduler.addApplicationAttempt(appAttemptId, false, false); scheduler.addApplicationAttempt(appAttemptId, false, false);
List<ResourceRequest> ask = new ArrayList<>(); List<ResourceRequest> ask = new ArrayList<>();
ask.add(createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true)); ask.add(createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true));
@ -148,7 +153,8 @@ public void testSortedNodes() throws Exception {
scheduler.handle(nodeEvent2); scheduler.handle(nodeEvent2);
// available resource // available resource
Assert.assertEquals(scheduler.getClusterResource().getMemorySize(), 16 * 1024); Assert.assertEquals(scheduler.getClusterResource().getMemorySize(),
16 * 1024);
Assert.assertEquals(scheduler.getClusterResource().getVirtualCores(), 16); Assert.assertEquals(scheduler.getClusterResource().getVirtualCores(), 16);
// send application request // send application request
@ -156,14 +162,17 @@ public void testSortedNodes() throws Exception {
createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++); createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
createMockRMApp(appAttemptId); createMockRMApp(appAttemptId);
scheduler.addApplication(appAttemptId.getApplicationId(), ApplicationPlacementContext placementCtx =
"queue11", "user11", false); new ApplicationPlacementContext("queue11");
scheduler.addApplication(appAttemptId.getApplicationId(), "queue11",
"user11", false, placementCtx);
scheduler.addApplicationAttempt(appAttemptId, false, false); scheduler.addApplicationAttempt(appAttemptId, false, false);
List<ResourceRequest> ask = new ArrayList<>(); List<ResourceRequest> ask = new ArrayList<>();
ResourceRequest request = ResourceRequest request =
createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true); createResourceRequest(1024, 1, ResourceRequest.ANY, 1, 1, true);
ask.add(request); ask.add(request);
scheduler.allocate(appAttemptId, ask, null, new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS); scheduler.allocate(appAttemptId, ask, null, new ArrayList<>(), null, null,
NULL_UPDATE_REQUESTS);
triggerSchedulingAttempt(); triggerSchedulingAttempt();
FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId); FSAppAttempt app = scheduler.getSchedulerApp(appAttemptId);
@ -174,10 +183,11 @@ public void testSortedNodes() throws Exception {
createResourceRequest(1024, 1, ResourceRequest.ANY, 2, 1, true); createResourceRequest(1024, 1, ResourceRequest.ANY, 2, 1, true);
ask.clear(); ask.clear();
ask.add(request); ask.add(request);
scheduler.allocate(appAttemptId, ask, null, new ArrayList<ContainerId>(), null, null, NULL_UPDATE_REQUESTS); scheduler.allocate(appAttemptId, ask, null, new ArrayList<>(), null, null,
NULL_UPDATE_REQUESTS);
triggerSchedulingAttempt(); triggerSchedulingAttempt();
checkAppConsumption(app, Resources.createResource(2048,2)); checkAppConsumption(app, Resources.createResource(2048, 2));
// 2 containers should be assigned to 2 nodes // 2 containers should be assigned to 2 nodes
Set<NodeId> nodes = new HashSet<NodeId>(); Set<NodeId> nodes = new HashSet<NodeId>();
@ -353,8 +363,10 @@ public void testFairSchedulerContinuousSchedulingInitTime() throws Exception {
id11 = createAppAttemptId(1, 1); id11 = createAppAttemptId(1, 1);
createMockRMApp(id11); createMockRMApp(id11);
priority = Priority.newInstance(priorityValue); priority = Priority.newInstance(priorityValue);
ApplicationPlacementContext placementCtx =
new ApplicationPlacementContext("root.queue1");
scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1", scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1",
false); false, placementCtx);
scheduler.addApplicationAttempt(id11, false, false); scheduler.addApplicationAttempt(id11, false, false);
fsAppAttempt = scheduler.getApplicationAttempt(id11); fsAppAttempt = scheduler.getApplicationAttempt(id11);

View File

@ -35,6 +35,7 @@
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
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.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.QueueMetrics;
@ -298,8 +299,10 @@ public void testHeadroomWithBlackListedNodes() {
assertEquals(0, clusterUsage.getVirtualCores()); assertEquals(0, clusterUsage.getVirtualCores());
ApplicationAttemptId id11 = createAppAttemptId(1, 1); ApplicationAttemptId id11 = createAppAttemptId(1, 1);
createMockRMApp(id11); createMockRMApp(id11);
ApplicationPlacementContext placementCtx =
new ApplicationPlacementContext("default");
scheduler.addApplication(id11.getApplicationId(), scheduler.addApplication(id11.getApplicationId(),
"default", "user1", false); "default", "user1", false, placementCtx);
scheduler.addApplicationAttempt(id11, false, false); scheduler.addApplicationAttempt(id11, false, false);
assertNotNull(scheduler.getSchedulerApplications().get(id11. assertNotNull(scheduler.getSchedulerApplications().get(id11.
getApplicationId())); getApplicationId()));

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
import org.apache.hadoop.yarn.util.SystemClock; import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
import org.junit.Before; import org.junit.Before;
@ -29,23 +31,27 @@
public class TestFSParentQueue { public class TestFSParentQueue {
private FairSchedulerConfiguration conf;
private QueueManager queueManager; private QueueManager queueManager;
@Before @Before
public void setUp() throws Exception { public void setUp() {
conf = new FairSchedulerConfiguration(); FairSchedulerConfiguration conf = new FairSchedulerConfiguration();
RMContext rmContext = mock(RMContext.class);
SystemClock clock = SystemClock.getInstance();
PlacementManager placementManager = new PlacementManager();
FairScheduler scheduler = mock(FairScheduler.class); FairScheduler scheduler = mock(FairScheduler.class);
AllocationConfiguration allocConf = new AllocationConfiguration(conf); when(scheduler.getRMContext()).thenReturn(rmContext);
when(scheduler.getAllocationConfiguration()).thenReturn(allocConf); when(scheduler.getConfig()).thenReturn(conf);
when(scheduler.getConf()).thenReturn(conf); when(scheduler.getConf()).thenReturn(conf);
when(scheduler.getResourceCalculator()).thenReturn( when(scheduler.getResourceCalculator()).thenReturn(
new DefaultResourceCalculator()); new DefaultResourceCalculator());
SystemClock clock = SystemClock.getInstance();
when(scheduler.getClock()).thenReturn(clock); when(scheduler.getClock()).thenReturn(clock);
when(rmContext.getQueuePlacementManager()).thenReturn(placementManager);
AllocationConfiguration allocConf = new AllocationConfiguration(scheduler);
when(scheduler.getAllocationConfiguration()).thenReturn(allocConf);
queueManager = new QueueManager(scheduler); queueManager = new QueueManager(scheduler);
FSQueueMetrics.forQueue("root", null, true, conf); FSQueueMetrics.forQueue("root", null, true, conf);
queueManager.initialize(conf); queueManager.initialize();
} }
@Test @Test

View File

@ -58,6 +58,9 @@
import org.apache.hadoop.yarn.server.resourcemanager.NodeManager; import org.apache.hadoop.yarn.server.resourcemanager.NodeManager;
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.placement.ApplicationPlacementContext;
import org.apache.hadoop.yarn.server.resourcemanager.placement.DefaultPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore; import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@ -85,7 +88,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.QueuePlacementRule.Default;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.BuilderUtils;
@ -108,10 +110,7 @@
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.HashMap;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -426,8 +425,11 @@ public void testNormalizationUsingQueueMaximumAllocation()
private void allocateAppAttempt(String queueName, int id, int memorySize) { private void allocateAppAttempt(String queueName, int id, int memorySize) {
ApplicationAttemptId id11 = createAppAttemptId(id, id); ApplicationAttemptId id11 = createAppAttemptId(id, id);
createMockRMApp(id11); createMockRMApp(id11);
ApplicationPlacementContext placementCtx =
new ApplicationPlacementContext(queueName);
scheduler.addApplication(id11.getApplicationId(), queueName, "user1", scheduler.addApplication(id11.getApplicationId(), queueName, "user1",
false); false, placementCtx);
scheduler.addApplicationAttempt(id11, false, false); scheduler.addApplicationAttempt(id11, false, false);
List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>(); List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
ResourceRequest request1 = ResourceRequest request1 =
@ -1404,8 +1406,10 @@ public void testRackLocalAppReservationThreshold() throws Exception {
createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++); createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
createMockRMApp(attemptId); createMockRMApp(attemptId);
ApplicationPlacementContext placementCtx =
new ApplicationPlacementContext("queue1");
scheduler.addApplication(attemptId.getApplicationId(), "queue1", "user1", scheduler.addApplication(attemptId.getApplicationId(), "queue1", "user1",
false); false, placementCtx);
scheduler.addApplicationAttempt(attemptId, false, false); scheduler.addApplicationAttempt(attemptId, false, false);
List<ResourceRequest> asks = new ArrayList<ResourceRequest>(); List<ResourceRequest> asks = new ArrayList<ResourceRequest>();
asks.add(createResourceRequest(2048, node2.getRackName(), 1, 1, false)); asks.add(createResourceRequest(2048, node2.getRackName(), 1, 1, false));
@ -1814,10 +1818,12 @@ public void testEmptyQueueName() throws Exception {
// only default queue // only default queue
assertEquals(1, scheduler.getQueueManager().getLeafQueues().size()); assertEquals(1, scheduler.getQueueManager().getLeafQueues().size());
// submit app with empty queue // Submit app with empty queue
// Submit fails before we reach the placement check.
ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1); ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1);
AppAddedSchedulerEvent appAddedEvent = AppAddedSchedulerEvent appAddedEvent =
new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "", "user1"); new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "",
"user1");
scheduler.handle(appAddedEvent); scheduler.handle(appAddedEvent);
// submission rejected // submission rejected
@ -1835,20 +1841,24 @@ public void testQueueuNameWithPeriods() throws Exception {
// only default queue // only default queue
assertEquals(1, scheduler.getQueueManager().getLeafQueues().size()); assertEquals(1, scheduler.getQueueManager().getLeafQueues().size());
// submit app with queue name (.A) // Submit app with queue name (.A)
// Submit fails before we reach the placement check.
ApplicationAttemptId appAttemptId1 = createAppAttemptId(1, 1); ApplicationAttemptId appAttemptId1 = createAppAttemptId(1, 1);
AppAddedSchedulerEvent appAddedEvent1 = AppAddedSchedulerEvent appAddedEvent1 =
new AppAddedSchedulerEvent(appAttemptId1.getApplicationId(), ".A", "user1"); new AppAddedSchedulerEvent(appAttemptId1.getApplicationId(), ".A",
"user1");
scheduler.handle(appAddedEvent1); scheduler.handle(appAddedEvent1);
// submission rejected // submission rejected
assertEquals(1, scheduler.getQueueManager().getLeafQueues().size()); assertEquals(1, scheduler.getQueueManager().getLeafQueues().size());
assertNull(scheduler.getSchedulerApp(appAttemptId1)); assertNull(scheduler.getSchedulerApp(appAttemptId1));
assertEquals(0, resourceManager.getRMContext().getRMApps().size()); assertEquals(0, resourceManager.getRMContext().getRMApps().size());
// submit app with queue name (A.) // Submit app with queue name (A.)
// Submit fails before we reach the placement check.
ApplicationAttemptId appAttemptId2 = createAppAttemptId(2, 1); ApplicationAttemptId appAttemptId2 = createAppAttemptId(2, 1);
AppAddedSchedulerEvent appAddedEvent2 = AppAddedSchedulerEvent appAddedEvent2 =
new AppAddedSchedulerEvent(appAttemptId2.getApplicationId(), "A.", "user1"); new AppAddedSchedulerEvent(appAttemptId2.getApplicationId(), "A.",
"user1");
scheduler.handle(appAddedEvent2); scheduler.handle(appAddedEvent2);
// submission rejected // submission rejected
assertEquals(1, scheduler.getQueueManager().getLeafQueues().size()); assertEquals(1, scheduler.getQueueManager().getLeafQueues().size());
@ -1856,9 +1866,11 @@ public void testQueueuNameWithPeriods() throws Exception {
assertEquals(0, resourceManager.getRMContext().getRMApps().size()); assertEquals(0, resourceManager.getRMContext().getRMApps().size());
// submit app with queue name (A.B) // submit app with queue name (A.B)
// Submit does not fail we must have a placement context.
ApplicationAttemptId appAttemptId3 = createAppAttemptId(3, 1); ApplicationAttemptId appAttemptId3 = createAppAttemptId(3, 1);
AppAddedSchedulerEvent appAddedEvent3 = AppAddedSchedulerEvent appAddedEvent3 =
new AppAddedSchedulerEvent(appAttemptId3.getApplicationId(), "A.B", "user1"); new AppAddedSchedulerEvent(appAttemptId3.getApplicationId(), "A.B",
"user1", new ApplicationPlacementContext("A.B"));
scheduler.handle(appAddedEvent3); scheduler.handle(appAddedEvent3);
// submission accepted // submission accepted
assertEquals(2, scheduler.getQueueManager().getLeafQueues().size()); assertEquals(2, scheduler.getQueueManager().getLeafQueues().size());
@ -1866,123 +1878,6 @@ public void testQueueuNameWithPeriods() throws Exception {
assertEquals(0, resourceManager.getRMContext().getRMApps().size()); assertEquals(0, resourceManager.getRMContext().getRMApps().size());
} }
@Test
public void testAssignToQueue() throws Exception {
conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "true");
scheduler.init(conf);
scheduler.start();
scheduler.reinitialize(conf, resourceManager.getRMContext());
RMApp rmApp1 = new MockRMApp(0, 0, RMAppState.NEW);
RMApp rmApp2 = new MockRMApp(1, 1, RMAppState.NEW);
FSLeafQueue queue1 = scheduler.assignToQueue(rmApp1, "default", "asterix");
FSLeafQueue queue2 = scheduler.assignToQueue(rmApp2, "notdefault", "obelix");
// assert FSLeafQueue's name is the correct name is the one set in the RMApp
assertEquals(rmApp1.getQueue(), queue1.getName());
assertEquals("root.asterix", rmApp1.getQueue());
assertEquals(rmApp2.getQueue(), queue2.getName());
assertEquals("root.notdefault", rmApp2.getQueue());
}
@Test
public void testAssignToBadDefaultQueue() throws Exception {
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
out.println("<?xml version=\"1.0\"?>");
out.println("<allocations>");
out.println("<queuePlacementPolicy>");
out.println("<rule name=\"specified\" create=\"false\" />");
out.println("<rule name=\"default\" create=\"false\" />");
out.println("</queuePlacementPolicy>");
out.println("</allocations>");
out.close();
scheduler.init(conf);
scheduler.start();
scheduler.reinitialize(conf, resourceManager.getRMContext());
RMApp rmApp1 = new MockRMApp(0, 0, RMAppState.NEW);
try {
FSLeafQueue queue1 = scheduler.assignToQueue(rmApp1, "default",
"asterix");
} catch (IllegalStateException ise) {
fail("Bad queue placement policy terminal rule should not throw " +
"exception ");
}
}
@Test
public void testAssignToNonLeafQueueReturnsNull() throws Exception {
conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "true");
scheduler.init(conf);
scheduler.start();
scheduler.reinitialize(conf, resourceManager.getRMContext());
scheduler.getQueueManager().getLeafQueue("root.child1.granchild", true);
scheduler.getQueueManager().getLeafQueue("root.child2", true);
RMApp rmApp1 = new MockRMApp(0, 0, RMAppState.NEW);
RMApp rmApp2 = new MockRMApp(1, 1, RMAppState.NEW);
// Trying to assign to non leaf queue would return null
assertNull(scheduler.assignToQueue(rmApp1, "root.child1", "tintin"));
assertNotNull(scheduler.assignToQueue(rmApp2, "root.child2", "snowy"));
}
@Test
public void testQueuePlacementWithPolicy() throws Exception {
conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
SimpleGroupsMapping.class, GroupMappingServiceProvider.class);
scheduler.init(conf);
scheduler.start();
scheduler.reinitialize(conf, resourceManager.getRMContext());
ApplicationAttemptId appId;
List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>();
rules.add(new QueuePlacementRule.Specified().initialize(true, null));
rules.add(new QueuePlacementRule.User().initialize(false, null));
rules.add(new QueuePlacementRule.PrimaryGroup().initialize(false, null));
rules.add(new QueuePlacementRule.SecondaryGroupExistingQueue().initialize(false, null));
rules.add(new QueuePlacementRule.Default().initialize(true, null));
Set<String> queues = Sets.newHashSet("root.user1", "root.user3group",
"root.user4subgroup1", "root.user4subgroup2" , "root.user5subgroup2");
Map<FSQueueType, Set<String>> configuredQueues = new HashMap<FSQueueType, Set<String>>();
configuredQueues.put(FSQueueType.LEAF, queues);
configuredQueues.put(FSQueueType.PARENT, new HashSet<String>());
scheduler.getAllocationConfiguration().placementPolicy =
new QueuePlacementPolicy(rules, configuredQueues, conf);
appId = createSchedulingRequest(1024, "somequeue", "user1");
assertEquals("root.somequeue", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "user1");
assertEquals("root.user1", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "user3");
assertEquals("root.user3group", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "user4");
assertEquals("root.user4subgroup1", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "user5");
assertEquals("root.user5subgroup2", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "otheruser");
assertEquals("root.default", scheduler.getSchedulerApp(appId).getQueueName());
// test without specified as first rule
rules = new ArrayList<QueuePlacementRule>();
rules.add(new QueuePlacementRule.User().initialize(false, null));
rules.add(new QueuePlacementRule.Specified().initialize(true, null));
rules.add(new QueuePlacementRule.Default().initialize(true, null));
scheduler.getAllocationConfiguration().placementPolicy =
new QueuePlacementPolicy(rules, configuredQueues, conf);
appId = createSchedulingRequest(1024, "somequeue", "user1");
assertEquals("root.user1", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "somequeue", "otheruser");
assertEquals("root.somequeue", scheduler.getSchedulerApp(appId).getQueueName());
appId = createSchedulingRequest(1024, "default", "otheruser");
assertEquals("root.default", scheduler.getSchedulerApp(appId).getQueueName());
}
@Test @Test
public void testFairShareWithMinAlloc() throws Exception { public void testFairShareWithMinAlloc() throws Exception {
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
@ -2028,38 +1923,6 @@ else if (p.getName().equals("root.queueB")) {
} }
} }
@Test
public void testNestedUserQueue() throws IOException {
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
SimpleGroupsMapping.class, GroupMappingServiceProvider.class);
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
out.println("<?xml version=\"1.0\"?>");
out.println("<allocations>");
out.println("<queue name=\"user1group\" type=\"parent\">");
out.println("<minResources>1024mb,0vcores</minResources>");
out.println("</queue>");
out.println("<queuePlacementPolicy>");
out.println("<rule name=\"specified\" create=\"false\" />");
out.println("<rule name=\"nestedUserQueue\">");
out.println(" <rule name=\"primaryGroup\" create=\"false\" />");
out.println("</rule>");
out.println("<rule name=\"default\" />");
out.println("</queuePlacementPolicy>");
out.println("</allocations>");
out.close();
scheduler.init(conf);
scheduler.start();
scheduler.reinitialize(conf, resourceManager.getRMContext());
RMApp rmApp1 = new MockRMApp(0, 0, RMAppState.NEW);
FSLeafQueue user1Leaf = scheduler.assignToQueue(rmApp1, "root.default",
"user1");
assertEquals("root.user1group.user1", user1Leaf.getName());
}
@Test @Test
public void testFairShareAndWeightsInNestedUserQueueRule() throws Exception { public void testFairShareAndWeightsInNestedUserQueueRule() throws Exception {
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE); conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
@ -2228,7 +2091,7 @@ public void testSteadyFairShareWithQueueCreatedRuntime() throws Exception {
// Submit one application // Submit one application
ApplicationAttemptId appAttemptId1 = createAppAttemptId(1, 1); ApplicationAttemptId appAttemptId1 = createAppAttemptId(1, 1);
createApplicationWithAMResource(appAttemptId1, "default", "user1", null); createApplicationWithAMResource(appAttemptId1, "user1", "user1", null);
assertEquals(3072, scheduler.getQueueManager() assertEquals(3072, scheduler.getQueueManager()
.getLeafQueue("default", false).getSteadyFairShare().getMemorySize()); .getLeafQueue("default", false).getSteadyFairShare().getMemorySize());
assertEquals(3072, scheduler.getQueueManager() assertEquals(3072, scheduler.getQueueManager()
@ -2249,8 +2112,10 @@ public void testQueueDemandCalculation() throws Exception {
// First ask, queue1 requests 1 large (minReqSize * 2). // First ask, queue1 requests 1 large (minReqSize * 2).
ApplicationAttemptId id11 = createAppAttemptId(1, 1); ApplicationAttemptId id11 = createAppAttemptId(1, 1);
createMockRMApp(id11); createMockRMApp(id11);
ApplicationPlacementContext placementCtx =
new ApplicationPlacementContext("root.queue1");
scheduler.addApplication(id11.getApplicationId(), scheduler.addApplication(id11.getApplicationId(),
"root.queue1", "user1", false); "root.queue1", "user1", false, placementCtx);
scheduler.addApplicationAttempt(id11, false, false); scheduler.addApplicationAttempt(id11, false, false);
List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>(); List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
ResourceRequest request1 = createResourceRequest(minReqSize * 2, ResourceRequest request1 = createResourceRequest(minReqSize * 2,
@ -2262,8 +2127,9 @@ public void testQueueDemandCalculation() throws Exception {
// Second ask, queue2 requests 1 large. // Second ask, queue2 requests 1 large.
ApplicationAttemptId id21 = createAppAttemptId(2, 1); ApplicationAttemptId id21 = createAppAttemptId(2, 1);
createMockRMApp(id21); createMockRMApp(id21);
placementCtx = new ApplicationPlacementContext("root.queue2");
scheduler.addApplication(id21.getApplicationId(), scheduler.addApplication(id21.getApplicationId(),
"root.queue2", "user1", false); "root.queue2", "user1", false, placementCtx);
scheduler.addApplicationAttempt(id21, false, false); scheduler.addApplicationAttempt(id21, false, false);
List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>(); List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
ResourceRequest request2 = createResourceRequest(2 * minReqSize, ResourceRequest request2 = createResourceRequest(2 * minReqSize,
@ -2279,7 +2145,7 @@ public void testQueueDemandCalculation() throws Exception {
ApplicationAttemptId id22 = createAppAttemptId(2, 2); ApplicationAttemptId id22 = createAppAttemptId(2, 2);
createMockRMApp(id22); createMockRMApp(id22);
scheduler.addApplication(id22.getApplicationId(), scheduler.addApplication(id22.getApplicationId(),
"root.queue2", "user1", false); "root.queue2", "user1", false, placementCtx);
scheduler.addApplicationAttempt(id22, false, false); scheduler.addApplicationAttempt(id22, false, false);
List<ResourceRequest> ask3 = new ArrayList<ResourceRequest>(); List<ResourceRequest> ask3 = new ArrayList<ResourceRequest>();
ResourceRequest request4 = createResourceRequest(minReqSize, ResourceRequest request4 = createResourceRequest(minReqSize,
@ -2886,8 +2752,10 @@ public void testMultipleNodesSingleRackRequest() throws Exception {
createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++); createAppAttemptId(this.APP_ID++, this.ATTEMPT_ID++);
createMockRMApp(attemptId); createMockRMApp(attemptId);
ApplicationPlacementContext placementCtx =
new ApplicationPlacementContext("queue1");
scheduler.addApplication(attemptId.getApplicationId(), "queue1", "user1", scheduler.addApplication(attemptId.getApplicationId(), "queue1", "user1",
false); false, placementCtx);
scheduler.addApplicationAttempt(attemptId, false, false); scheduler.addApplicationAttempt(attemptId, false, false);
// 1 request with 2 nodes on the same rack. another request with 1 node on // 1 request with 2 nodes on the same rack. another request with 1 node on
@ -2900,7 +2768,7 @@ public void testMultipleNodesSingleRackRequest() throws Exception {
asks.add(createResourceRequest(1024, node3.getRackName(), 1, 1, true)); asks.add(createResourceRequest(1024, node3.getRackName(), 1, 1, true));
asks.add(createResourceRequest(1024, ResourceRequest.ANY, 1, 2, true)); asks.add(createResourceRequest(1024, ResourceRequest.ANY, 1, 2, true));
scheduler.allocate(attemptId, asks, null, new ArrayList<ContainerId>(), null, scheduler.allocate(attemptId, asks, null, new ArrayList<>(), null,
null, NULL_UPDATE_REQUESTS); null, NULL_UPDATE_REQUESTS);
// node 1 checks in // node 1 checks in
@ -3202,10 +3070,11 @@ public void testNotAllowSubmitApplication() throws Exception {
submissionContext.setApplicationId(applicationId); submissionContext.setApplicationId(applicationId);
submissionContext.setAMContainerSpec(clc); submissionContext.setAMContainerSpec(clc);
RMApp application = RMApp application =
new RMAppImpl(applicationId, resourceManager.getRMContext(), conf, name, user, new RMAppImpl(applicationId, resourceManager.getRMContext(), conf,
queue, submissionContext, scheduler, masterService, name, user, queue, submissionContext, scheduler, masterService,
System.currentTimeMillis(), "YARN", null, 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));
final int MAX_TRIES=20; final int MAX_TRIES=20;
@ -3222,16 +3091,22 @@ public void testNotAllowSubmitApplication() throws Exception {
ApplicationAttemptId attId = ApplicationAttemptId attId =
ApplicationAttemptId.newInstance(applicationId, this.ATTEMPT_ID++); ApplicationAttemptId.newInstance(applicationId, this.ATTEMPT_ID++);
scheduler.addApplication(attId.getApplicationId(), queue, user, false); ApplicationPlacementContext placementCtx =
new ApplicationPlacementContext(queue);
scheduler.addApplication(attId.getApplicationId(), queue, user, false,
placementCtx);
numTries = 0; numTries = 0;
while (application.getFinishTime() == 0 && numTries < MAX_TRIES) { while (application.getFinishTime() == 0 && numTries < MAX_TRIES) {
try { try {
Thread.sleep(100); Thread.sleep(100);
} catch (InterruptedException ex) {ex.printStackTrace();} } catch (InterruptedException ex) {
ex.printStackTrace();
}
numTries++; numTries++;
} }
assertEquals(FinalApplicationStatus.FAILED, application.getFinalApplicationStatus()); assertEquals(FinalApplicationStatus.FAILED,
application.getFinalApplicationStatus());
} }
@Test @Test
@ -3845,7 +3720,7 @@ public void testQueueMaxAMShare() throws Exception {
assertEquals("Queue queue1's fair share should be 0", 0, queue1 assertEquals("Queue queue1's fair share should be 0", 0, queue1
.getFairShare().getMemorySize()); .getFairShare().getMemorySize());
createSchedulingRequest(1 * 1024, "root.default", "user1"); createSchedulingRequest(1 * 1024, "default", "user1");
scheduler.update(); scheduler.update();
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
@ -4559,8 +4434,10 @@ public void testSchedulingOnRemovedNode() throws Exception {
ApplicationAttemptId id11 = createAppAttemptId(1, 1); ApplicationAttemptId id11 = createAppAttemptId(1, 1);
createMockRMApp(id11); createMockRMApp(id11);
ApplicationPlacementContext placementCtx =
new ApplicationPlacementContext("root.queue1");
scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1", scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1",
false); false, placementCtx);
scheduler.addApplicationAttempt(id11, false, false); scheduler.addApplicationAttempt(id11, false, false);
List<ResourceRequest> ask1 = new ArrayList<>(); List<ResourceRequest> ask1 = new ArrayList<>();
@ -4611,12 +4488,12 @@ public void testDefaultRuleInitializesProperlyWhenPolicyNotConfigured()
scheduler.start(); scheduler.start();
scheduler.reinitialize(conf, resourceManager.getRMContext()); scheduler.reinitialize(conf, resourceManager.getRMContext());
List<QueuePlacementRule> rules = List<PlacementRule> rules = scheduler.getRMContext()
scheduler.allocConf.placementPolicy.getRules(); .getQueuePlacementManager().getPlacementRules();
for (QueuePlacementRule rule : rules) { for (PlacementRule rule : rules) {
if (rule instanceof Default) { if (rule instanceof DefaultPlacementRule) {
Default defaultRule = (Default) rule; DefaultPlacementRule defaultRule = (DefaultPlacementRule) rule;
assertNotNull(defaultRule.defaultQueueName); assertNotNull(defaultRule.defaultQueueName);
} }
} }
@ -4686,7 +4563,7 @@ public void testGetAppsInQueue() throws Exception {
ApplicationAttemptId appAttId2 = ApplicationAttemptId appAttId2 =
createSchedulingRequest(1024, 1, "queue1.subqueue2", "user1"); createSchedulingRequest(1024, 1, "queue1.subqueue2", "user1");
ApplicationAttemptId appAttId3 = ApplicationAttemptId appAttId3 =
createSchedulingRequest(1024, 1, "default", "user1"); createSchedulingRequest(1024, 1, "user1", "user1");
List<ApplicationAttemptId> apps = List<ApplicationAttemptId> apps =
scheduler.getAppsInQueue("queue1.subqueue1"); scheduler.getAppsInQueue("queue1.subqueue1");
@ -4888,11 +4765,13 @@ public void testDoubleRemoval() throws Exception {
scheduler.reinitialize(conf, resourceManager.getRMContext()); scheduler.reinitialize(conf, resourceManager.getRMContext());
ApplicationAttemptId attemptId = createAppAttemptId(1, 1); ApplicationAttemptId attemptId = createAppAttemptId(1, 1);
// The placement rule will add the app to the user based queue but the // The placement rule should add it to the user based queue but the
// passed in queue must exist. // passed in queue must exist.
ApplicationPlacementContext apc =
new ApplicationPlacementContext(testUser);
AppAddedSchedulerEvent appAddedEvent = AppAddedSchedulerEvent appAddedEvent =
new AppAddedSchedulerEvent(attemptId.getApplicationId(), testUser, new AppAddedSchedulerEvent(attemptId.getApplicationId(), testUser,
testUser); testUser, apc);
scheduler.handle(appAddedEvent); scheduler.handle(appAddedEvent);
AppAttemptAddedSchedulerEvent attemptAddedEvent = AppAttemptAddedSchedulerEvent attemptAddedEvent =
new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false); new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false);
@ -4936,9 +4815,11 @@ public void testMoveAfterRemoval() throws Exception {
scheduler.reinitialize(conf, resourceManager.getRMContext()); scheduler.reinitialize(conf, resourceManager.getRMContext());
ApplicationAttemptId attemptId = createAppAttemptId(1, 1); ApplicationAttemptId attemptId = createAppAttemptId(1, 1);
ApplicationPlacementContext apc =
new ApplicationPlacementContext(testUser);
AppAddedSchedulerEvent appAddedEvent = AppAddedSchedulerEvent appAddedEvent =
new AppAddedSchedulerEvent(attemptId.getApplicationId(), testUser, new AppAddedSchedulerEvent(attemptId.getApplicationId(), testUser,
testUser); testUser, apc);
scheduler.handle(appAddedEvent); scheduler.handle(appAddedEvent);
AppAttemptAddedSchedulerEvent attemptAddedEvent = AppAttemptAddedSchedulerEvent attemptAddedEvent =
new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false); new AppAttemptAddedSchedulerEvent(createAppAttemptId(1, 1), false);
@ -4990,8 +4871,10 @@ public void testQueueNameWithTrailingSpace() throws Exception {
// submit app with queue name "A" // submit app with queue name "A"
ApplicationAttemptId appAttemptId1 = createAppAttemptId(1, 1); ApplicationAttemptId appAttemptId1 = createAppAttemptId(1, 1);
ApplicationPlacementContext apc =
new ApplicationPlacementContext("A");
AppAddedSchedulerEvent appAddedEvent1 = new AppAddedSchedulerEvent( AppAddedSchedulerEvent appAddedEvent1 = new AppAddedSchedulerEvent(
appAttemptId1.getApplicationId(), "A", "user1"); appAttemptId1.getApplicationId(), "A", "user1", apc);
scheduler.handle(appAddedEvent1); scheduler.handle(appAddedEvent1);
// submission accepted // submission accepted
assertEquals(2, scheduler.getQueueManager().getLeafQueues().size()); assertEquals(2, scheduler.getQueueManager().getLeafQueues().size());
@ -5008,9 +4891,15 @@ public void testQueueNameWithTrailingSpace() throws Exception {
// submit app with queue name "A " // submit app with queue name "A "
ApplicationAttemptId appAttemptId2 = createAppAttemptId(2, 1); ApplicationAttemptId appAttemptId2 = createAppAttemptId(2, 1);
apc = new ApplicationPlacementContext("A ");
AppAddedSchedulerEvent appAddedEvent2 = new AppAddedSchedulerEvent( AppAddedSchedulerEvent appAddedEvent2 = new AppAddedSchedulerEvent(
appAttemptId2.getApplicationId(), "A ", "user1"); appAttemptId2.getApplicationId(), "A ", "user1", apc);
try {
scheduler.handle(appAddedEvent2); scheduler.handle(appAddedEvent2);
Assert.fail("Submit should have failed with InvalidQueueNameException");
} catch (InvalidQueueNameException iqne) {
// expected ignore: rules should have filtered this out
}
// submission rejected // submission rejected
assertEquals(2, scheduler.getQueueManager().getLeafQueues().size()); assertEquals(2, scheduler.getQueueManager().getLeafQueues().size());
assertNull(scheduler.getSchedulerApplications().get(appAttemptId2. assertNull(scheduler.getSchedulerApplications().get(appAttemptId2.
@ -5019,8 +4908,9 @@ public void testQueueNameWithTrailingSpace() throws Exception {
// submit app with queue name "B.C" // submit app with queue name "B.C"
ApplicationAttemptId appAttemptId3 = createAppAttemptId(3, 1); ApplicationAttemptId appAttemptId3 = createAppAttemptId(3, 1);
apc = new ApplicationPlacementContext("B.C");
AppAddedSchedulerEvent appAddedEvent3 = new AppAddedSchedulerEvent( AppAddedSchedulerEvent appAddedEvent3 = new AppAddedSchedulerEvent(
appAttemptId3.getApplicationId(), "B.C", "user1"); appAttemptId3.getApplicationId(), "B.C", "user1", apc);
scheduler.handle(appAddedEvent3); scheduler.handle(appAddedEvent3);
// submission accepted // submission accepted
assertEquals(3, scheduler.getQueueManager().getLeafQueues().size()); assertEquals(3, scheduler.getQueueManager().getLeafQueues().size());
@ -5037,9 +4927,14 @@ public void testQueueNameWithTrailingSpace() throws Exception {
// submit app with queue name "A\u00a0" (non-breaking space) // submit app with queue name "A\u00a0" (non-breaking space)
ApplicationAttemptId appAttemptId4 = createAppAttemptId(4, 1); ApplicationAttemptId appAttemptId4 = createAppAttemptId(4, 1);
apc = new ApplicationPlacementContext("A\u00a0");
AppAddedSchedulerEvent appAddedEvent4 = new AppAddedSchedulerEvent( AppAddedSchedulerEvent appAddedEvent4 = new AppAddedSchedulerEvent(
appAttemptId4.getApplicationId(), "A\u00a0", "user1"); appAttemptId4.getApplicationId(), "A\u00a0", "user1", apc);
try {
scheduler.handle(appAddedEvent4); scheduler.handle(appAddedEvent4);
} catch (InvalidQueueNameException iqne) {
// expected ignore: rules should have filtered this out
}
// submission rejected // submission rejected
assertEquals(3, scheduler.getQueueManager().getLeafQueues().size()); assertEquals(3, scheduler.getQueueManager().getLeafQueues().size());
assertNull(scheduler.getSchedulerApplications().get(appAttemptId4. assertNull(scheduler.getSchedulerApplications().get(appAttemptId4.
@ -5075,17 +4970,17 @@ public void testUserAsDefaultQueueWithLeadingTrailingSpaceUserName()
scheduler.init(conf); scheduler.init(conf);
scheduler.start(); scheduler.start();
scheduler.reinitialize(conf, resourceManager.getRMContext()); scheduler.reinitialize(conf, resourceManager.getRMContext());
ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1); ApplicationAttemptId attId1 = createAppAttemptId(1, 1);
createApplicationWithAMResource(appAttemptId, "default", " user1", null); createApplicationWithAMResource(attId1, "root.user1", " user1", null);
assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true) assertEquals(1, scheduler.getQueueManager().getLeafQueue("user1", true)
.getNumRunnableApps()); .getNumRunnableApps());
assertEquals(0, scheduler.getQueueManager().getLeafQueue("default", true) assertEquals(0, scheduler.getQueueManager().getLeafQueue("default", true)
.getNumRunnableApps()); .getNumRunnableApps());
assertEquals("root.user1", resourceManager.getRMContext().getRMApps() assertEquals("root.user1", resourceManager.getRMContext().getRMApps()
.get(appAttemptId.getApplicationId()).getQueue()); .get(attId1.getApplicationId()).getQueue());
ApplicationAttemptId attId2 = createAppAttemptId(2, 1); ApplicationAttemptId attId2 = createAppAttemptId(2, 1);
createApplicationWithAMResource(attId2, "default", "user1 ", null); createApplicationWithAMResource(attId2, "root.user1", "user1 ", null);
assertEquals(2, scheduler.getQueueManager().getLeafQueue("user1", true) assertEquals(2, scheduler.getQueueManager().getLeafQueue("user1", true)
.getNumRunnableApps()); .getNumRunnableApps());
assertEquals(0, scheduler.getQueueManager().getLeafQueue("default", true) assertEquals(0, scheduler.getQueueManager().getLeafQueue("default", true)
@ -5094,7 +4989,7 @@ public void testUserAsDefaultQueueWithLeadingTrailingSpaceUserName()
.get(attId2.getApplicationId()).getQueue()); .get(attId2.getApplicationId()).getQueue());
ApplicationAttemptId attId3 = createAppAttemptId(3, 1); ApplicationAttemptId attId3 = createAppAttemptId(3, 1);
createApplicationWithAMResource(attId3, "default", "user1", null); createApplicationWithAMResource(attId3, "root.user1", "user1", null);
assertEquals(3, scheduler.getQueueManager().getLeafQueue("user1", true) assertEquals(3, scheduler.getQueueManager().getLeafQueue("user1", true)
.getNumRunnableApps()); .getNumRunnableApps());
assertEquals(0, scheduler.getQueueManager().getLeafQueue("default", true) assertEquals(0, scheduler.getQueueManager().getLeafQueue("default", true)
@ -5526,8 +5421,10 @@ public void testCompletedContainerOnRemovedNode() throws IOException {
// Create application attempt // Create application attempt
ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1); ApplicationAttemptId appAttemptId = createAppAttemptId(1, 1);
createMockRMApp(appAttemptId); createMockRMApp(appAttemptId);
ApplicationPlacementContext placementCtx =
new ApplicationPlacementContext("root.queue1");
scheduler.addApplication(appAttemptId.getApplicationId(), "root.queue1", scheduler.addApplication(appAttemptId.getApplicationId(), "root.queue1",
"user1", false); "user1", false, placementCtx);
scheduler.addApplicationAttempt(appAttemptId, false, false); scheduler.addApplicationAttempt(appAttemptId, false, false);
// Create container request that goes to a specific node. // Create container request that goes to a specific node.
@ -5612,7 +5509,7 @@ private void testAppRejectedToQueueWithZeroCapacityOfResource(String resource)
ResourceRequest amReqs = ResourceRequest.newBuilder() ResourceRequest amReqs = ResourceRequest.newBuilder()
.capability(Resource.newInstance(5 * GB, 3)).build(); .capability(Resource.newInstance(5 * GB, 3)).build();
createApplicationWithAMResource(appAttemptId1, "queueA", "user1", createApplicationWithAMResource(appAttemptId1, "root.queueA", "user1",
Resource.newInstance(GB, 1), Lists.newArrayList(amReqs)); Resource.newInstance(GB, 1), Lists.newArrayList(amReqs));
scheduler.update(); scheduler.update();

View File

@ -21,6 +21,8 @@
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.api.records.ResourceInformation;
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.placement.PlacementManager;
import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.apache.hadoop.yarn.util.resource.ResourceUtils;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
@ -33,6 +35,8 @@
import static org.apache.hadoop.yarn.util.resource.ResourceUtils.MAXIMUM_ALLOCATION; import static org.apache.hadoop.yarn.util.resource.ResourceUtils.MAXIMUM_ALLOCATION;
import static org.apache.hadoop.yarn.util.resource.ResourceUtils.UNITS; import static org.apache.hadoop.yarn.util.resource.ResourceUtils.UNITS;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class TestFairSchedulerWithMultiResourceTypes public class TestFairSchedulerWithMultiResourceTypes
extends FairSchedulerTestBase { extends FairSchedulerTestBase {
@ -44,6 +48,11 @@ public void setUp() throws IOException {
scheduler = new FairScheduler(); scheduler = new FairScheduler();
conf = createConfiguration(); conf = createConfiguration();
initResourceTypes(conf); initResourceTypes(conf);
// since this runs outside of the normal context we need to set one
RMContext rmContext = mock(RMContext.class);
PlacementManager placementManager = new PlacementManager();
when(rmContext.getQueuePlacementManager()).thenReturn(placementManager);
scheduler.setRMContext(rmContext);
} }
@After @After

View File

@ -27,10 +27,10 @@
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
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.server.resourcemanager.placement.PlacementManager;
import org.apache.hadoop.yarn.util.ControlledClock; import org.apache.hadoop.yarn.util.ControlledClock;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
import org.junit.Before; import org.junit.Before;
@ -46,26 +46,27 @@ public class TestMaxRunningAppsEnforcer {
private FairScheduler scheduler; private FairScheduler scheduler;
@Before @Before
public void setup() throws Exception { public void setup() {
Configuration conf = new Configuration(); FairSchedulerConfiguration conf = new FairSchedulerConfiguration();
PlacementManager placementManager = new PlacementManager();
rmContext = mock(RMContext.class);
when(rmContext.getQueuePlacementManager()).thenReturn(placementManager);
when(rmContext.getEpoch()).thenReturn(0L);
clock = new ControlledClock(); clock = new ControlledClock();
scheduler = mock(FairScheduler.class); scheduler = mock(FairScheduler.class);
when(scheduler.getConf()).thenReturn( when(scheduler.getConf()).thenReturn(conf);
new FairSchedulerConfiguration(conf)); when(scheduler.getConfig()).thenReturn(conf);
when(scheduler.getClock()).thenReturn(clock); when(scheduler.getClock()).thenReturn(clock);
AllocationConfiguration allocConf = new AllocationConfiguration(
conf);
when(scheduler.getAllocationConfiguration()).thenReturn(allocConf);
when(scheduler.getResourceCalculator()).thenReturn( when(scheduler.getResourceCalculator()).thenReturn(
new DefaultResourceCalculator()); new DefaultResourceCalculator());
when(scheduler.getRMContext()).thenReturn(rmContext);
AllocationConfiguration allocConf = new AllocationConfiguration(scheduler);
when(scheduler.getAllocationConfiguration()).thenReturn(allocConf);
queueManager = new QueueManager(scheduler); queueManager = new QueueManager(scheduler);
queueManager.initialize(conf); queueManager.initialize();
userMaxApps = allocConf.userMaxApps; userMaxApps = allocConf.userMaxApps;
maxAppsEnforcer = new MaxRunningAppsEnforcer(scheduler); maxAppsEnforcer = new MaxRunningAppsEnforcer(scheduler);
appNum = 0; appNum = 0;
rmContext = mock(RMContext.class);
when(rmContext.getEpoch()).thenReturn(0L);
} }
private FSAppAttempt addApp(FSLeafQueue queue, String user) { private FSAppAttempt addApp(FSLeafQueue queue, String user) {

View File

@ -26,6 +26,7 @@
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.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
import org.apache.hadoop.yarn.util.SystemClock; import org.apache.hadoop.yarn.util.SystemClock;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
@ -37,35 +38,43 @@
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
/**
* Test the {@link FairScheduler} queue manager correct queue hierarchies
* management (create, delete and type changes).
*/
public class TestQueueManager { public class TestQueueManager {
private FairSchedulerConfiguration conf;
private QueueManager queueManager; private QueueManager queueManager;
private FairScheduler scheduler; private FairScheduler scheduler;
@Before @Before
public void setUp() throws Exception { public void setUp() {
conf = new FairSchedulerConfiguration(); PlacementManager placementManager = new PlacementManager();
scheduler = mock(FairScheduler.class); FairSchedulerConfiguration conf = new FairSchedulerConfiguration();
RMContext rmContext = mock(RMContext.class);
when(rmContext.getQueuePlacementManager()).thenReturn(placementManager);
SystemClock clock = SystemClock.getInstance();
AllocationConfiguration allocConf = new AllocationConfiguration(conf); scheduler = mock(FairScheduler.class);
when(scheduler.getConf()).thenReturn(conf);
when(scheduler.getConfig()).thenReturn(conf);
when(scheduler.getRMContext()).thenReturn(rmContext);
when(scheduler.getResourceCalculator()).thenReturn(
new DefaultResourceCalculator());
when(scheduler.getClock()).thenReturn(clock);
AllocationConfiguration allocConf =
new AllocationConfiguration(scheduler);
when(scheduler.getAllocationConfiguration()).thenReturn(allocConf);
// Set up some queues to test default child max resource inheritance // Set up some queues to test default child max resource inheritance
allocConf.configuredQueues.get(FSQueueType.PARENT).add("root.test"); allocConf.configuredQueues.get(FSQueueType.PARENT).add("root.test");
allocConf.configuredQueues.get(FSQueueType.LEAF).add("root.test.childA"); allocConf.configuredQueues.get(FSQueueType.LEAF).add("root.test.childA");
allocConf.configuredQueues.get(FSQueueType.PARENT).add("root.test.childB"); allocConf.configuredQueues.get(FSQueueType.PARENT).add("root.test.childB");
when(scheduler.getAllocationConfiguration()).thenReturn(allocConf);
when(scheduler.getConf()).thenReturn(conf);
when(scheduler.getResourceCalculator()).thenReturn(
new DefaultResourceCalculator());
SystemClock clock = SystemClock.getInstance();
when(scheduler.getClock()).thenReturn(clock);
queueManager = new QueueManager(scheduler); queueManager = new QueueManager(scheduler);
FSQueueMetrics.forQueue("root", null, true, conf); FSQueueMetrics.forQueue("root", null, true, conf);
queueManager.initialize(conf); queueManager.initialize();
queueManager.updateAllocationConfiguration(allocConf); queueManager.updateAllocationConfiguration(allocConf);
} }
@ -118,7 +127,8 @@ public void testReloadTurnsLeafQueueIntoParent() {
*/ */
@Test @Test
public void testReloadTurnsLeafToParentWithNoLeaf() { public void testReloadTurnsLeafToParentWithNoLeaf() {
AllocationConfiguration allocConf = new AllocationConfiguration(conf); AllocationConfiguration allocConf =
new AllocationConfiguration(scheduler);
// Create a leaf queue1 // Create a leaf queue1
allocConf.configuredQueues.get(FSQueueType.LEAF).add("root.queue1"); allocConf.configuredQueues.get(FSQueueType.LEAF).add("root.queue1");
queueManager.updateAllocationConfiguration(allocConf); queueManager.updateAllocationConfiguration(allocConf);
@ -130,7 +140,7 @@ public void testReloadTurnsLeafToParentWithNoLeaf() {
FSLeafQueue q1 = queueManager.getLeafQueue("queue1", false); FSLeafQueue q1 = queueManager.getLeafQueue("queue1", false);
ApplicationId appId = ApplicationId.newInstance(0, 0); ApplicationId appId = ApplicationId.newInstance(0, 0);
q1.addAssignedApp(appId); q1.addAssignedApp(appId);
allocConf = new AllocationConfiguration(conf); allocConf = new AllocationConfiguration(scheduler);
allocConf.configuredQueues.get(FSQueueType.PARENT) allocConf.configuredQueues.get(FSQueueType.PARENT)
.add("root.queue1"); .add("root.queue1");
@ -169,7 +179,8 @@ public void testCheckQueueNodeName() {
private void updateConfiguredLeafQueues(QueueManager queueMgr, private void updateConfiguredLeafQueues(QueueManager queueMgr,
String... confLeafQueues) { String... confLeafQueues) {
AllocationConfiguration allocConf = new AllocationConfiguration(conf); AllocationConfiguration allocConf =
new AllocationConfiguration(scheduler);
allocConf.configuredQueues.get(FSQueueType.LEAF) allocConf.configuredQueues.get(FSQueueType.LEAF)
.addAll(Sets.newHashSet(confLeafQueues)); .addAll(Sets.newHashSet(confLeafQueues));
queueMgr.updateAllocationConfiguration(allocConf); queueMgr.updateAllocationConfiguration(allocConf);

View File

@ -18,44 +18,82 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException; import java.nio.charset.StandardCharsets;
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 javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilder;
import javax.xml.parsers.DocumentBuilderFactory; import javax.xml.parsers.DocumentBuilderFactory;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.security.GroupMappingServiceProvider; import org.apache.hadoop.security.GroupMappingServiceProvider;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
import org.apache.hadoop.yarn.server.resourcemanager.placement.DefaultPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.FSPlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementRule;
import org.apache.hadoop.yarn.server.resourcemanager.placement.UserPlacementRule;
import org.apache.hadoop.yarn.util.SystemClock;
import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import org.w3c.dom.Document; import org.w3c.dom.Document;
import org.w3c.dom.Element; import org.w3c.dom.Element;
/**
* Tests for the queue placement policy for the {@link FairScheduler}.
*/
public class TestQueuePlacementPolicy { public class TestQueuePlacementPolicy {
private final static Configuration conf = new Configuration(); private final static FairSchedulerConfiguration CONF =
private Map<FSQueueType, Set<String>> configuredQueues; new FairSchedulerConfiguration();
// Base setup needed, policy is an intermediate object
private PlacementManager placementManager;
private FairScheduler scheduler;
private QueueManager queueManager;
// Locals used in each assignment
private ApplicationSubmissionContext asc;
private ApplicationPlacementContext context;
@BeforeClass @BeforeClass
public static void setup() { public static void setup() {
conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING, CONF.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
SimpleGroupsMapping.class, GroupMappingServiceProvider.class); SimpleGroupsMapping.class, GroupMappingServiceProvider.class);
} }
@Before @Before
public void initTest() { public void initTest() {
configuredQueues = new HashMap<FSQueueType, Set<String>>(); SystemClock clock = SystemClock.getInstance();
for (FSQueueType type : FSQueueType.values()) { RMContext rmContext = mock(RMContext.class);
configuredQueues.put(type, new HashSet<String>()); placementManager = new PlacementManager();
scheduler = mock(FairScheduler.class);
when(scheduler.getClock()).thenReturn(clock);
when(scheduler.getRMContext()).thenReturn(rmContext);
when(scheduler.getConfig()).thenReturn(CONF);
when(scheduler.getConf()).thenReturn(CONF);
when(rmContext.getQueuePlacementManager()).thenReturn(placementManager);
AllocationConfiguration allocConf = new AllocationConfiguration(scheduler);
when(scheduler.getAllocationConfiguration()).thenReturn(allocConf);
queueManager = new QueueManager(scheduler);
queueManager.initialize();
when(scheduler.getQueueManager()).thenReturn(queueManager);
} }
@After
public void cleanTest() {
placementManager = null;
queueManager = null;
scheduler = null;
} }
@Test @Test
@ -65,13 +103,16 @@ public void testSpecifiedUserPolicy() throws Exception {
sb.append(" <rule name='specified' />"); sb.append(" <rule name='specified' />");
sb.append(" <rule name='user' />"); sb.append(" <rule name='user' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
QueuePlacementPolicy policy = parse(sb.toString()); createPolicy(sb.toString());
assertEquals("root.specifiedq",
policy.assignAppToQueue("specifiedq", "someuser")); asc = newAppSubmissionContext("specifiedq");
assertEquals("root.someuser", context = placementManager.placeApplication(asc, "someuser");
policy.assignAppToQueue("default", "someuser")); assertEquals("root.specifiedq", context.getQueue());
assertEquals("root.otheruser", asc = newAppSubmissionContext("default");
policy.assignAppToQueue("default", "otheruser")); context = placementManager.placeApplication(asc, "someuser");
assertEquals("root.someuser", context.getQueue());
context = placementManager.placeApplication(asc, "otheruser");
assertEquals("root.otheruser", context.getQueue());
} }
@Test @Test
@ -82,13 +123,22 @@ public void testNoCreate() throws Exception {
sb.append(" <rule name='user' create=\"false\" />"); sb.append(" <rule name='user' create=\"false\" />");
sb.append(" <rule name='default' />"); sb.append(" <rule name='default' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
createPolicy(sb.toString());
configuredQueues.get(FSQueueType.LEAF).add("root.someuser"); createQueue(FSQueueType.LEAF, "root.someuser");
QueuePlacementPolicy policy = parse(sb.toString());
assertEquals("root.specifiedq", policy.assignAppToQueue("specifiedq", "someuser")); asc = newAppSubmissionContext("specifiedq");
assertEquals("root.someuser", policy.assignAppToQueue("default", "someuser")); context = placementManager.placeApplication(asc, "someuser");
assertEquals("root.specifiedq", policy.assignAppToQueue("specifiedq", "otheruser")); assertEquals("root.specifiedq", context.getQueue());
assertEquals("root.default", policy.assignAppToQueue("default", "otheruser")); asc = newAppSubmissionContext("default");
context = placementManager.placeApplication(asc, "someuser");
assertEquals("root.someuser", context.getQueue());
asc = newAppSubmissionContext("specifiedq");
context = placementManager.placeApplication(asc, "otheruser");
assertEquals("root.specifiedq", context.getQueue());
asc = newAppSubmissionContext("default");
context = placementManager.placeApplication(asc, "otheruser");
assertEquals("root.default", context.getQueue());
} }
@Test @Test
@ -98,63 +148,64 @@ public void testSpecifiedThenReject() throws Exception {
sb.append(" <rule name='specified' />"); sb.append(" <rule name='specified' />");
sb.append(" <rule name='reject' />"); sb.append(" <rule name='reject' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
QueuePlacementPolicy policy = parse(sb.toString()); createPolicy(sb.toString());
assertEquals("root.specifiedq", asc = newAppSubmissionContext("specifiedq");
policy.assignAppToQueue("specifiedq", "someuser")); context = placementManager.placeApplication(asc, "someuser");
assertEquals(null, policy.assignAppToQueue("default", "someuser")); assertEquals("root.specifiedq", context.getQueue());
asc = newAppSubmissionContext("default");
context = placementManager.placeApplication(asc, "someuser");
assertNull("Assignment should have been rejected and was not", context);
} }
@Test (expected = AllocationConfigurationException.class) @Test
public void testOmittedTerminalRule() throws Exception { public void testOmittedTerminalRule() {
StringBuffer sb = new StringBuffer(); StringBuffer sb = new StringBuffer();
sb.append("<queuePlacementPolicy>"); sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='specified' />"); sb.append(" <rule name='specified' />");
sb.append(" <rule name='user' create=\"false\" />"); sb.append(" <rule name='user' create=\"false\" />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
parse(sb.toString()); assertIfExceptionThrown(sb);
} }
@Test (expected = AllocationConfigurationException.class) @Test
public void testTerminalRuleInMiddle() throws Exception { public void testTerminalRuleInMiddle() {
StringBuffer sb = new StringBuffer(); StringBuffer sb = new StringBuffer();
sb.append("<queuePlacementPolicy>"); sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='specified' />"); sb.append(" <rule name='specified' />");
sb.append(" <rule name='default' />"); sb.append(" <rule name='default' />");
sb.append(" <rule name='user' />"); sb.append(" <rule name='user' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
parse(sb.toString()); assertIfExceptionThrown(sb);
} }
@Test @Test
public void testTerminals() throws Exception { public void testTerminals() {
// Should make it through without an exception // The default rule is no longer considered terminal when the create flag
// is false. The throw now happens when configuring not when assigning the
// application
StringBuffer sb = new StringBuffer(); StringBuffer sb = new StringBuffer();
sb.append("<queuePlacementPolicy>"); sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='secondaryGroupExistingQueue' create='true'/>"); sb.append(" <rule name='secondaryGroupExistingQueue' create='true'/>");
sb.append(" <rule name='default' queue='otherdefault' create='false'/>"); sb.append(" <rule name='default' queue='otherdefault' create='false'/>");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
QueuePlacementPolicy policy = parse(sb.toString()); assertIfExceptionThrown(sb);
try {
policy.assignAppToQueue("root.otherdefault", "user1");
fail("Expect exception from having default rule with create=\'false\'");
} catch (IllegalStateException se) {
}
} }
@Test @Test
public void testDefaultRuleWithQueueAttribute() throws Exception { public void testDefaultRuleWithQueueAttribute() throws Exception {
// This test covers the use case where we would like default rule // This test covers the use case where we would like default rule
// to point to a different queue by default rather than root.default // to point to a different queue by default rather than root.default
configuredQueues.get(FSQueueType.LEAF).add("root.someDefaultQueue"); createQueue(FSQueueType.LEAF, "root.someDefaultQueue");
StringBuffer sb = new StringBuffer(); StringBuffer sb = new StringBuffer();
sb.append("<queuePlacementPolicy>"); sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='specified' create='false' />"); sb.append(" <rule name='specified' create='false' />");
sb.append(" <rule name='default' queue='root.someDefaultQueue'/>"); sb.append(" <rule name='default' queue='root.someDefaultQueue'/>");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
QueuePlacementPolicy policy = parse(sb.toString()); createPolicy(sb.toString());
assertEquals("root.someDefaultQueue", asc = newAppSubmissionContext("default");
policy.assignAppToQueue("root.default", "user1")); context = placementManager.placeApplication(asc, "user1");
assertEquals("root.someDefaultQueue", context.getQueue());
} }
@Test @Test
@ -162,30 +213,108 @@ public void testNestedUserQueueParsingErrors() {
// No nested rule specified in hierarchical user queue // No nested rule specified in hierarchical user queue
StringBuffer sb = new StringBuffer(); StringBuffer sb = new StringBuffer();
sb.append("<queuePlacementPolicy>"); sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='specified' />");
sb.append(" <rule name='nestedUserQueue'/>"); sb.append(" <rule name='nestedUserQueue'/>");
sb.append(" <rule name='default' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
assertIfExceptionThrown(sb); assertIfExceptionThrown(sb);
// Specified nested rule is not a QueuePlacementRule // Specified nested rule is not a FSPlacementRule
sb = new StringBuffer(); sb = new StringBuffer();
sb.append("<queuePlacementPolicy>"); sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='specified' />");
sb.append(" <rule name='nestedUserQueue'>"); sb.append(" <rule name='nestedUserQueue'>");
sb.append(" <rule name='unknownRule'/>"); sb.append(" <rule name='unknownRule'/>");
sb.append(" </rule>"); sb.append(" </rule>");
sb.append(" <rule name='default' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
assertIfExceptionThrown(sb); assertIfExceptionThrown(sb);
// Parent rule is rule that cannot be one: reject or nestedUserQueue
sb = new StringBuffer();
sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='nestedUserQueue'>");
sb.append(" <rule name='reject'/>");
sb.append(" </rule>");
sb.append("</queuePlacementPolicy>");
assertIfExceptionThrown(sb);
// If the parent rule does not have the create flag the nested rule is not
// terminal
sb = new StringBuffer();
sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='nestedUserQueue'>");
sb.append(" <rule name='primaryGroup' create='false'/>");
sb.append(" </rule>");
sb.append("</queuePlacementPolicy>");
assertIfExceptionThrown(sb);
}
@Test
public void testMultipleParentRules() throws Exception {
StringBuffer sb = new StringBuffer();
sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='nestedUserQueue'>");
sb.append(" <rule name='primaryGroup'/>");
sb.append(" <rule name='default'/>");
sb.append(" </rule>");
sb.append("</queuePlacementPolicy>");
createPolicy(sb.toString());
PlacementRule nested = placementManager.getPlacementRules().get(0);
if (nested instanceof UserPlacementRule) {
PlacementRule parent = ((FSPlacementRule)nested).getParentRule();
assertTrue("Nested rule should have been Default rule",
parent instanceof DefaultPlacementRule);
} else {
fail("Policy parsing failed: rule with multiple parents not set");
}
}
@Test
public void testBrokenRules() throws Exception {
// broken rule should fail configuring
StringBuffer sb = new StringBuffer();
sb.append("<queuePlacementPolicy>");
sb.append(" <rule />");
sb.append("</queuePlacementPolicy>");
assertIfExceptionThrown(sb);
// policy without rules ignoring policy
sb = new StringBuffer();
sb.append("<queuePlacementPolicy>");
sb.append(" <notarule />");
sb.append("</queuePlacementPolicy>");
createPolicy(sb.toString());
// broken rule should fail configuring
sb = new StringBuffer();
sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='user'>");
sb.append(" <rule />");
sb.append(" </rule>");
sb.append("</queuePlacementPolicy>");
assertIfExceptionThrown(sb);
// parent rule not set to something known: no parent rule is required
// required case is only for nestedUserQueue tested earlier
sb = new StringBuffer();
sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='user'>");
sb.append(" <notarule />");
sb.append(" </rule>");
sb.append("</queuePlacementPolicy>");
createPolicy(sb.toString());
} }
private void assertIfExceptionThrown(StringBuffer sb) { private void assertIfExceptionThrown(StringBuffer sb) {
Throwable th = null; Throwable th = null;
try { try {
parse(sb.toString()); createPolicy(sb.toString());
} catch (Exception e) { } catch (Exception e) {
th = e; th = e;
} }
@ -193,6 +322,17 @@ private void assertIfExceptionThrown(StringBuffer sb) {
assertTrue(th instanceof AllocationConfigurationException); assertTrue(th instanceof AllocationConfigurationException);
} }
private void assertIfExceptionThrown(String user) {
Throwable th = null;
try {
placementManager.placeApplication(asc, user);
} catch (Exception e) {
th = e;
}
assertTrue(th instanceof YarnException);
}
@Test @Test
public void testNestedUserQueueParsing() throws Exception { public void testNestedUserQueueParsing() throws Exception {
StringBuffer sb = new StringBuffer(); StringBuffer sb = new StringBuffer();
@ -201,17 +341,9 @@ public void testNestedUserQueueParsing() throws Exception {
sb.append(" <rule name='nestedUserQueue'>"); sb.append(" <rule name='nestedUserQueue'>");
sb.append(" <rule name='primaryGroup'/>"); sb.append(" <rule name='primaryGroup'/>");
sb.append(" </rule>"); sb.append(" </rule>");
sb.append(" <rule name='default' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
Throwable th = null; createPolicy(sb.toString());
try {
parse(sb.toString());
} catch (Exception e) {
th = e;
}
assertNull(th);
} }
@Test @Test
@ -222,24 +354,26 @@ public void testNestedUserQueuePrimaryGroup() throws Exception {
sb.append(" <rule name='nestedUserQueue'>"); sb.append(" <rule name='nestedUserQueue'>");
sb.append(" <rule name='primaryGroup'/>"); sb.append(" <rule name='primaryGroup'/>");
sb.append(" </rule>"); sb.append(" </rule>");
sb.append(" <rule name='default' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
// User queue would be created under primary group queue // User queue would be created under primary group queue
QueuePlacementPolicy policy = parse(sb.toString()); createPolicy(sb.toString());
assertEquals("root.user1group.user1", asc = newAppSubmissionContext("default");
policy.assignAppToQueue("root.default", "user1")); context = placementManager.placeApplication(asc, "user1");
assertEquals("root.user1group.user1", context.getQueue());
// Other rules above and below hierarchical user queue rule should work as // Other rules above and below hierarchical user queue rule should work as
// usual // usual
configuredQueues.get(FSQueueType.LEAF).add("root.specifiedq"); createQueue(FSQueueType.LEAF, "root.specifiedq");
// test if specified rule(above nestedUserQueue rule) works ok // test if specified rule(above nestedUserQueue rule) works ok
assertEquals("root.specifiedq", asc = newAppSubmissionContext("root.specifiedq");
policy.assignAppToQueue("root.specifiedq", "user2")); context = placementManager.placeApplication(asc, "user2");
assertEquals("root.specifiedq", context.getQueue());
// test if default rule(below nestedUserQueue rule) works // Submit should fail if we cannot create the queue
configuredQueues.get(FSQueueType.LEAF).add("root.user3group"); createQueue(FSQueueType.LEAF, "root.user3group");
assertEquals("root.default", asc = newAppSubmissionContext("default");
policy.assignAppToQueue("root.default", "user3")); context = placementManager.placeApplication(asc, "user3");
assertNull("Submission should have failed and did not", context);
} }
@Test @Test
@ -253,18 +387,18 @@ public void testNestedUserQueuePrimaryGroupNoCreate() throws Exception {
sb.append(" <rule name='default' />"); sb.append(" <rule name='default' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
QueuePlacementPolicy policy = parse(sb.toString()); createPolicy(sb.toString());
// Should return root.default since primary group 'root.user1group' is not // Should return root.default since primary group 'root.user1group' is not
// configured // configured
assertEquals("root.default", asc = newAppSubmissionContext("default");
policy.assignAppToQueue("root.default", "user1")); context = placementManager.placeApplication(asc, "user1");
assertEquals("root.default", context.getQueue());
// Let's configure primary group and check if user queue is created // Let's configure primary group and check if user queue is created
configuredQueues.get(FSQueueType.PARENT).add("root.user1group"); createQueue(FSQueueType.PARENT, "root.user1group");
policy = parse(sb.toString()); context = placementManager.placeApplication(asc, "user1");
assertEquals("root.user1group.user1", assertEquals("root.user1group.user1", context.getQueue());
policy.assignAppToQueue("root.default", "user1"));
// Both Primary group and nestedUserQueue rule has create='false' // Both Primary group and nestedUserQueue rule has create='false'
sb = new StringBuffer(); sb = new StringBuffer();
@ -277,16 +411,16 @@ public void testNestedUserQueuePrimaryGroupNoCreate() throws Exception {
// Should return root.default since primary group and user queue for user 2 // Should return root.default since primary group and user queue for user 2
// are not configured. // are not configured.
assertEquals("root.default", asc = newAppSubmissionContext("default");
policy.assignAppToQueue("root.default", "user2")); context = placementManager.placeApplication(asc, "user2");
assertEquals("root.default", context.getQueue());
// Now configure both primary group and the user queue for user2 // Now configure both primary group and the user queue for user2
configuredQueues.get(FSQueueType.PARENT).add("root.user2group"); createQueue(FSQueueType.LEAF, "root.user2group.user2");
configuredQueues.get(FSQueueType.LEAF).add("root.user2group.user2");
policy = parse(sb.toString());
assertEquals("root.user2group.user2", // Try placing the same app again
policy.assignAppToQueue("root.default", "user2")); context = placementManager.placeApplication(asc, "user2");
assertEquals("root.user2group.user2", context.getQueue());
} }
@Test @Test
@ -299,17 +433,18 @@ public void testNestedUserQueueSecondaryGroup() throws Exception {
sb.append(" <rule name='default' />"); sb.append(" <rule name='default' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
QueuePlacementPolicy policy = parse(sb.toString()); createPolicy(sb.toString());
// Should return root.default since secondary groups are not configured // Should return root.default since secondary groups are not configured
assertEquals("root.default", asc = newAppSubmissionContext("default");
policy.assignAppToQueue("root.default", "user1")); context = placementManager.placeApplication(asc, "user1");
assertEquals("root.default", context.getQueue());
// configure secondary group for user1 // configure secondary group for user1
configuredQueues.get(FSQueueType.PARENT).add("root.user1subgroup1"); createQueue(FSQueueType.PARENT, "root.user1subgroup1");
policy = parse(sb.toString()); createPolicy(sb.toString());
// user queue created should be created under secondary group // user queue created should be created under secondary group
assertEquals("root.user1subgroup1.user1", context = placementManager.placeApplication(asc, "user1");
policy.assignAppToQueue("root.default", "user1")); assertEquals("root.user1subgroup1.user1", context.getQueue());
} }
@Test @Test
@ -325,33 +460,66 @@ public void testNestedUserQueueSpecificRule() throws Exception {
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
// Let's create couple of parent queues // Let's create couple of parent queues
configuredQueues.get(FSQueueType.PARENT).add("root.parent1"); createQueue(FSQueueType.PARENT, "root.parent1");
configuredQueues.get(FSQueueType.PARENT).add("root.parent2"); createQueue(FSQueueType.PARENT, "root.parent2");
QueuePlacementPolicy policy = parse(sb.toString()); createPolicy(sb.toString());
assertEquals("root.parent1.user1", asc = newAppSubmissionContext("root.parent1");
policy.assignAppToQueue("root.parent1", "user1")); context = placementManager.placeApplication(asc, "user1");
assertEquals("root.parent2.user2", assertEquals("root.parent1.user1", context.getQueue());
policy.assignAppToQueue("root.parent2", "user2")); asc = newAppSubmissionContext("root.parent2");
context = placementManager.placeApplication(asc, "user2");
assertEquals("root.parent2.user2", context.getQueue());
} }
@Test @Test
public void testNestedUserQueueDefaultRule() throws Exception { public void testNestedUserQueueDefaultRule() throws Exception {
// This test covers the use case where we would like user queues to be // This test covers the use case where we would like user queues to be
// created under a default parent queue // created under a default parent queue
configuredQueues.get(FSQueueType.PARENT).add("root.parentq");
StringBuffer sb = new StringBuffer(); StringBuffer sb = new StringBuffer();
sb.append("<queuePlacementPolicy>"); sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='specified' create='false' />"); sb.append(" <rule name='specified' create='false' />");
sb.append(" <rule name='nestedUserQueue'>"); sb.append(" <rule name='nestedUserQueue'>");
sb.append(" <rule name='default' queue='root.parentq'/>"); sb.append(" <rule name='default' queue='root.parent'/>");
sb.append(" </rule>");
sb.append("</queuePlacementPolicy>");
createPolicy(sb.toString());
asc = newAppSubmissionContext("default");
context = placementManager.placeApplication(asc, "user1");
assertEquals("root.parent.user1", context.getQueue());
// Same as above but now with the create flag false for the parent
createQueue(FSQueueType.PARENT, "root.parent");
sb = new StringBuffer();
sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='specified' create='false' />");
sb.append(" <rule name='nestedUserQueue'>");
sb.append(" <rule name='default' queue='root.parent' create='false'/>");
sb.append(" </rule>"); sb.append(" </rule>");
sb.append(" <rule name='default' />"); sb.append(" <rule name='default' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
QueuePlacementPolicy policy = parse(sb.toString()); createPolicy(sb.toString());
assertEquals("root.parentq.user1", asc = newAppSubmissionContext("default");
policy.assignAppToQueue("root.default", "user1")); context = placementManager.placeApplication(asc, "user1");
assertEquals("root.parent.user1", context.getQueue());
// Parent queue returned is already a configured LEAF, should fail and the
// context is null.
createQueue(FSQueueType.LEAF, "root.parent");
sb = new StringBuffer();
sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='specified' create='false' />");
sb.append(" <rule name='nestedUserQueue'>");
sb.append(" <rule name='default' queue='root.parent' />");
sb.append(" </rule>");
sb.append("</queuePlacementPolicy>");
createPolicy(sb.toString());
asc = newAppSubmissionContext("default");
context = placementManager.placeApplication(asc, "user1");
assertNull("Submission should have failed and did not", context);
} }
@Test @Test
@ -361,9 +529,10 @@ public void testUserContainsPeriod() throws Exception {
sb.append("<queuePlacementPolicy>"); sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='user' />"); sb.append(" <rule name='user' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
QueuePlacementPolicy policy = parse(sb.toString()); createPolicy(sb.toString());
assertEquals("root.first_dot_last", asc = newAppSubmissionContext("default");
policy.assignAppToQueue("default", "first.last")); context = placementManager.placeApplication(asc, "first.last");
assertEquals("root.first_dot_last", context.getQueue());
sb = new StringBuffer(); sb = new StringBuffer();
sb.append("<queuePlacementPolicy>"); sb.append("<queuePlacementPolicy>");
@ -371,11 +540,14 @@ public void testUserContainsPeriod() throws Exception {
sb.append(" <rule name='nestedUserQueue'>"); sb.append(" <rule name='nestedUserQueue'>");
sb.append(" <rule name='default'/>"); sb.append(" <rule name='default'/>");
sb.append(" </rule>"); sb.append(" </rule>");
sb.append(" <rule name='default' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
policy = parse(sb.toString()); // specified create is false, bypass the rule
assertEquals("root.default.first_dot_last", // default rule has create which requires a PARENT queue: remove the LEAF
policy.assignAppToQueue("root.default", "first.last")); queueManager.removeLeafQueue("root.default");
createPolicy(sb.toString());
asc = newAppSubmissionContext("default");
context = placementManager.placeApplication(asc, "first_dot_last");
assertEquals("root.default.first_dot_last", context.getQueue());
} }
@Test @Test
@ -386,22 +558,22 @@ public void testGroupContainsPeriod() throws Exception {
sb.append(" <rule name='nestedUserQueue'>"); sb.append(" <rule name='nestedUserQueue'>");
sb.append(" <rule name='primaryGroup'/>"); sb.append(" <rule name='primaryGroup'/>");
sb.append(" </rule>"); sb.append(" </rule>");
sb.append(" <rule name='default' />");
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING, CONF.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
PeriodGroupsMapping.class, GroupMappingServiceProvider.class); PeriodGroupsMapping.class, GroupMappingServiceProvider.class);
// User queue would be created under primary group queue, and the period // User queue would be created under primary group queue, and the period
// in the group name should be converted into _dot_ // in the group name should be converted into _dot_
QueuePlacementPolicy policy = parse(sb.toString()); createPolicy(sb.toString());
assertEquals("root.user1_dot_group.user1", asc = newAppSubmissionContext("default");
policy.assignAppToQueue("root.default", "user1")); context = placementManager.placeApplication(asc, "user1");
assertEquals("root.user1_dot_group.user1", context.getQueue());
conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING, CONF.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
SimpleGroupsMapping.class, GroupMappingServiceProvider.class); SimpleGroupsMapping.class, GroupMappingServiceProvider.class);
} }
@Test(expected=IOException.class) @Test
public void testEmptyGroupsPrimaryGroupRule() throws Exception { public void testEmptyGroupsPrimaryGroupRule() throws Exception {
StringBuffer sb = new StringBuffer(); StringBuffer sb = new StringBuffer();
sb.append("<queuePlacementPolicy>"); sb.append("<queuePlacementPolicy>");
@ -410,20 +582,67 @@ public void testEmptyGroupsPrimaryGroupRule() throws Exception {
sb.append("</queuePlacementPolicy>"); sb.append("</queuePlacementPolicy>");
// Add a static mapping that returns empty groups for users // Add a static mapping that returns empty groups for users
conf.setStrings(CommonConfigurationKeys CONF.setStrings(CommonConfigurationKeys
.HADOOP_USER_GROUP_STATIC_OVERRIDES, "emptygroupuser="); .HADOOP_USER_GROUP_STATIC_OVERRIDES, "emptygroupuser=");
QueuePlacementPolicy policy = parse(sb.toString()); createPolicy(sb.toString());
policy.assignAppToQueue(null, "emptygroupuser"); asc = newAppSubmissionContext("root.fake");
assertIfExceptionThrown("emptygroupuser");
} }
private QueuePlacementPolicy parse(String str) throws Exception { @Test
public void testSpecifiedQueueWithSpaces() throws Exception {
StringBuffer sb = new StringBuffer();
sb.append("<queuePlacementPolicy>");
sb.append(" <rule name='specified'/>");
sb.append(" <rule name='default'/>");
sb.append("</queuePlacementPolicy>");
createPolicy(sb.toString());
asc = newAppSubmissionContext("A ");
assertIfExceptionThrown("user1");
asc = newAppSubmissionContext("A\u00a0");
assertIfExceptionThrown("user1");
}
private void createPolicy(String str)
throws AllocationConfigurationException {
// Read and parse the allocations file. // Read and parse the allocations file.
Element root = null;
try {
DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory
.newInstance(); .newInstance();
docBuilderFactory.setIgnoringComments(true); docBuilderFactory.setIgnoringComments(true);
DocumentBuilder builder = docBuilderFactory.newDocumentBuilder(); DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
Document doc = builder.parse(IOUtils.toInputStream(str)); Document doc = builder.parse(IOUtils.toInputStream(str,
Element root = doc.getDocumentElement(); StandardCharsets.UTF_8));
return QueuePlacementPolicy.fromXml(root, configuredQueues, conf); root = doc.getDocumentElement();
} catch (Exception ex) {
// Don't really want to test the xml parsing side,
// let it fail with a null config below.
}
QueuePlacementPolicy.fromXml(root, scheduler);
}
private ApplicationSubmissionContext newAppSubmissionContext(String queue) {
ApplicationId appId = ApplicationId.newInstance(1L, 1);
Priority prio = Priority.UNDEFINED;
Resource resource = Resource.newInstance(1, 1);
ContainerLaunchContext amContainer =
ContainerLaunchContext.newInstance(null, null, null, null, null, null);
return ApplicationSubmissionContext.newInstance(appId, "test", queue,
prio, amContainer, false, false, 1, resource, "testing");
}
private void createQueue(FSQueueType type, String name) {
// Create a queue as if it is in the config.
FSQueue queue = queueManager.createQueue(name, type);
assertNotNull("Queue not created", queue);
// walk up the list till we have a non dynamic queue
// root is always non dynamic
do {
queue.setDynamic(false);
queue = queue.parent;
} while (queue.isDynamic());
} }
} }

View File

@ -30,6 +30,8 @@
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hadoop.yarn.api.records.Priority; 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.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
@ -40,6 +42,8 @@
import static org.junit.Assert.assertNull; import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class TestSchedulingPolicy { public class TestSchedulingPolicy {
private static final Logger LOG = private static final Logger LOG =
@ -53,6 +57,11 @@ public class TestSchedulingPolicy {
public void setUp() throws Exception { public void setUp() throws Exception {
scheduler = new FairScheduler(); scheduler = new FairScheduler();
conf = new FairSchedulerConfiguration(); conf = new FairSchedulerConfiguration();
// since this runs outside of the normal context we need to set one
RMContext rmContext = mock(RMContext.class);
PlacementManager placementManager = new PlacementManager();
when(rmContext.getQueuePlacementManager()).thenReturn(placementManager);
scheduler.setRMContext(rmContext);
} }
public void testParseSchedulingPolicy() public void testParseSchedulingPolicy()

View File

@ -19,6 +19,8 @@
package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao; package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.AllocationConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.AllocationConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FSQueue;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
@ -37,19 +39,25 @@
public class TestFairSchedulerQueueInfo { public class TestFairSchedulerQueueInfo {
@Test @Test
public void testEmptyChildQueues() throws Exception { public void testEmptyChildQueues() {
FairSchedulerConfiguration conf = new FairSchedulerConfiguration(); FairSchedulerConfiguration fsConf = new FairSchedulerConfiguration();
RMContext rmContext = mock(RMContext.class);
PlacementManager placementManager = new PlacementManager();
SystemClock clock = SystemClock.getInstance();
FairScheduler scheduler = mock(FairScheduler.class); FairScheduler scheduler = mock(FairScheduler.class);
AllocationConfiguration allocConf = new AllocationConfiguration(conf); when(scheduler.getConf()).thenReturn(fsConf);
when(scheduler.getAllocationConfiguration()).thenReturn(allocConf); when(scheduler.getConfig()).thenReturn(fsConf);
when(scheduler.getConf()).thenReturn(conf); when(scheduler.getRMContext()).thenReturn(rmContext);
when(scheduler.getClusterResource()).thenReturn(Resource.newInstance(1, 1)); when(rmContext.getQueuePlacementManager()).thenReturn(placementManager);
when(scheduler.getClusterResource()).thenReturn(
Resource.newInstance(1, 1));
when(scheduler.getResourceCalculator()).thenReturn( when(scheduler.getResourceCalculator()).thenReturn(
new DefaultResourceCalculator()); new DefaultResourceCalculator());
SystemClock clock = SystemClock.getInstance();
when(scheduler.getClock()).thenReturn(clock); when(scheduler.getClock()).thenReturn(clock);
AllocationConfiguration allocConf = new AllocationConfiguration(scheduler);
when(scheduler.getAllocationConfiguration()).thenReturn(allocConf);
QueueManager queueManager = new QueueManager(scheduler); QueueManager queueManager = new QueueManager(scheduler);
queueManager.initialize(conf); queueManager.initialize();
FSQueue testQueue = queueManager.getLeafQueue("test", true); FSQueue testQueue = queueManager.getLeafQueue("test", true);
FairSchedulerQueueInfo queueInfo = FairSchedulerQueueInfo queueInfo =