Revert "YARN-8468. Enable the use of queue based maximum container allocation limit and implement it in FairScheduler. Contributed by Antal Bálint Steinbach."

This reverts commit ce4a0898df.
This commit is contained in:
Weiwei Yang 2018-10-10 21:41:00 +08:00
parent 323b76bccf
commit a0060cf8ee
32 changed files with 264 additions and 1142 deletions

View File

@ -38,7 +38,6 @@ import org.apache.hadoop.yarn.api.records.ExecutionType;
import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest; import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueACL;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.event.EventHandler;
@ -87,7 +86,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
private int maxCompletedAppsInMemory; private int maxCompletedAppsInMemory;
private int maxCompletedAppsInStateStore; private int maxCompletedAppsInStateStore;
protected int completedAppsInStateStore = 0; protected int completedAppsInStateStore = 0;
protected LinkedList<ApplicationId> completedApps = new LinkedList<ApplicationId>(); private LinkedList<ApplicationId> completedApps = new LinkedList<ApplicationId>();
private final RMContext rmContext; private final RMContext rmContext;
private final ApplicationMasterService masterService; private final ApplicationMasterService masterService;
@ -527,13 +526,13 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
// Normalize all requests // Normalize all requests
String queue = submissionContext.getQueue(); String queue = submissionContext.getQueue();
Resource maxAllocation = scheduler.getMaximumResourceCapability(queue);
for (ResourceRequest amReq : amReqs) { for (ResourceRequest amReq : amReqs) {
SchedulerUtils.normalizeAndValidateRequest(amReq, maxAllocation, SchedulerUtils.normalizeAndValidateRequest(amReq,
queue, scheduler, isRecovery, rmContext, null); scheduler.getMaximumResourceCapability(queue),
queue, scheduler, isRecovery, rmContext);
amReq.setCapability(scheduler.getNormalizedResource( amReq.setCapability(
amReq.getCapability(), maxAllocation)); scheduler.getNormalizedResource(amReq.getCapability()));
} }
return amReqs; return amReqs;
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {

View File

@ -97,7 +97,7 @@ public class RMServerUtils {
"INCORRECT_CONTAINER_VERSION_ERROR"; "INCORRECT_CONTAINER_VERSION_ERROR";
private static final String INVALID_CONTAINER_ID = private static final String INVALID_CONTAINER_ID =
"INVALID_CONTAINER_ID"; "INVALID_CONTAINER_ID";
public static final String RESOURCE_OUTSIDE_ALLOWED_RANGE = private static final String RESOURCE_OUTSIDE_ALLOWED_RANGE =
"RESOURCE_OUTSIDE_ALLOWED_RANGE"; "RESOURCE_OUTSIDE_ALLOWED_RANGE";
protected static final RecordFactory RECORD_FACTORY = protected static final RecordFactory RECORD_FACTORY =
@ -235,7 +235,7 @@ public class RMServerUtils {
* requested memory/vcore is non-negative and not greater than max * requested memory/vcore is non-negative and not greater than max
*/ */
public static void normalizeAndValidateRequests(List<ResourceRequest> ask, public static void normalizeAndValidateRequests(List<ResourceRequest> ask,
Resource maximumAllocation, String queueName, YarnScheduler scheduler, Resource maximumResource, String queueName, YarnScheduler scheduler,
RMContext rmContext) throws InvalidResourceRequestException { RMContext rmContext) throws InvalidResourceRequestException {
// Get queue from scheduler // Get queue from scheduler
QueueInfo queueInfo = null; QueueInfo queueInfo = null;
@ -247,7 +247,7 @@ public class RMServerUtils {
} }
for (ResourceRequest resReq : ask) { for (ResourceRequest resReq : ask) {
SchedulerUtils.normalizeAndValidateRequest(resReq, maximumAllocation, SchedulerUtils.normalizeAndvalidateRequest(resReq, maximumResource,
queueName, scheduler, rmContext, queueInfo); queueName, scheduler, rmContext, queueInfo);
} }
} }
@ -338,8 +338,7 @@ public class RMServerUtils {
return false; return false;
} }
ResourceScheduler scheduler = rmContext.getScheduler(); ResourceScheduler scheduler = rmContext.getScheduler();
request.setCapability(scheduler request.setCapability(scheduler.getNormalizedResource(request.getCapability()));
.getNormalizedResource(request.getCapability(), maximumAllocation));
return true; return true;
} }

View File

@ -1149,12 +1149,11 @@ public abstract class AbstractYarnScheduler
} }
@Override @Override
public Resource getNormalizedResource(Resource requestedResource, public Resource getNormalizedResource(Resource requestedResource) {
Resource maxResourceCapability) {
return SchedulerUtils.getNormalizedResource(requestedResource, return SchedulerUtils.getNormalizedResource(requestedResource,
getResourceCalculator(), getResourceCalculator(),
getMinimumResourceCapability(), getMinimumResourceCapability(),
maxResourceCapability, getMaximumResourceCapability(),
getMinimumResourceCapability()); getMinimumResourceCapability());
} }
@ -1164,20 +1163,8 @@ public abstract class AbstractYarnScheduler
* @param asks resource requests * @param asks resource requests
*/ */
protected void normalizeResourceRequests(List<ResourceRequest> asks) { protected void normalizeResourceRequests(List<ResourceRequest> asks) {
normalizeResourceRequests(asks, null); for (ResourceRequest ask: asks) {
} ask.setCapability(getNormalizedResource(ask.getCapability()));
/**
* Normalize a list of resource requests
* using queue maximum resource allocations.
* @param asks resource requests
*/
protected void normalizeResourceRequests(List<ResourceRequest> asks,
String queueName) {
Resource maxAllocation = getMaximumResourceCapability(queueName);
for (ResourceRequest ask : asks) {
ask.setCapability(
getNormalizedResource(ask.getCapability(), maxAllocation));
} }
} }

View File

@ -63,27 +63,27 @@ public class SchedulerUtils {
private static final Log LOG = LogFactory.getLog(SchedulerUtils.class); private static final Log LOG = LogFactory.getLog(SchedulerUtils.class);
private static final RecordFactory recordFactory = private static final RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null); RecordFactoryProvider.getRecordFactory(null);
public static final String RELEASED_CONTAINER = public static final String RELEASED_CONTAINER =
"Container released by application"; "Container released by application";
public static final String UPDATED_CONTAINER = public static final String UPDATED_CONTAINER =
"Temporary container killed by application for ExeutionType update"; "Temporary container killed by application for ExeutionType update";
public static final String LOST_CONTAINER = public static final String LOST_CONTAINER =
"Container released on a *lost* node"; "Container released on a *lost* node";
public static final String PREEMPTED_CONTAINER = public static final String PREEMPTED_CONTAINER =
"Container preempted by scheduler"; "Container preempted by scheduler";
public static final String COMPLETED_APPLICATION = public static final String COMPLETED_APPLICATION =
"Container of a completed application"; "Container of a completed application";
public static final String EXPIRED_CONTAINER = public static final String EXPIRED_CONTAINER =
"Container expired since it was unused"; "Container expired since it was unused";
public static final String UNRESERVED_CONTAINER = public static final String UNRESERVED_CONTAINER =
"Container reservation no longer required."; "Container reservation no longer required.";
@ -98,7 +98,7 @@ public class SchedulerUtils {
*/ */
public static ContainerStatus createAbnormalContainerStatus( public static ContainerStatus createAbnormalContainerStatus(
ContainerId containerId, String diagnostics) { ContainerId containerId, String diagnostics) {
return createAbnormalContainerStatus(containerId, return createAbnormalContainerStatus(containerId,
ContainerExitStatus.ABORTED, diagnostics); ContainerExitStatus.ABORTED, diagnostics);
} }
@ -126,14 +126,14 @@ public class SchedulerUtils {
*/ */
public static ContainerStatus createPreemptedContainerStatus( public static ContainerStatus createPreemptedContainerStatus(
ContainerId containerId, String diagnostics) { ContainerId containerId, String diagnostics) {
return createAbnormalContainerStatus(containerId, return createAbnormalContainerStatus(containerId,
ContainerExitStatus.PREEMPTED, diagnostics); ContainerExitStatus.PREEMPTED, diagnostics);
} }
/** /**
* Utility to create a {@link ContainerStatus} during exceptional * Utility to create a {@link ContainerStatus} during exceptional
* circumstances. * circumstances.
* *
* @param containerId {@link ContainerId} of returned/released/lost container. * @param containerId {@link ContainerId} of returned/released/lost container.
* @param diagnostics diagnostic message * @param diagnostics diagnostic message
* @return <code>ContainerStatus</code> for an returned/released/lost * @return <code>ContainerStatus</code> for an returned/released/lost
@ -141,7 +141,7 @@ public class SchedulerUtils {
*/ */
private static ContainerStatus createAbnormalContainerStatus( private static ContainerStatus createAbnormalContainerStatus(
ContainerId containerId, int exitStatus, String diagnostics) { ContainerId containerId, int exitStatus, String diagnostics) {
ContainerStatus containerStatus = ContainerStatus containerStatus =
recordFactory.newRecordInstance(ContainerStatus.class); recordFactory.newRecordInstance(ContainerStatus.class);
containerStatus.setContainerId(containerId); containerStatus.setContainerId(containerId);
containerStatus.setDiagnostics(diagnostics); containerStatus.setDiagnostics(diagnostics);
@ -217,7 +217,16 @@ public class SchedulerUtils {
} }
public static void normalizeAndValidateRequest(ResourceRequest resReq, public static void normalizeAndValidateRequest(ResourceRequest resReq,
Resource maximumAllocation, String queueName, YarnScheduler scheduler, Resource maximumResource, String queueName, YarnScheduler scheduler,
boolean isRecovery, RMContext rmContext)
throws InvalidResourceRequestException {
normalizeAndValidateRequest(resReq, maximumResource, queueName, scheduler,
isRecovery, rmContext, null);
}
public static void normalizeAndValidateRequest(ResourceRequest resReq,
Resource maximumResource, String queueName, YarnScheduler scheduler,
boolean isRecovery, RMContext rmContext, QueueInfo queueInfo) boolean isRecovery, RMContext rmContext, QueueInfo queueInfo)
throws InvalidResourceRequestException { throws InvalidResourceRequestException {
Configuration conf = rmContext.getYarnConfiguration(); Configuration conf = rmContext.getYarnConfiguration();
@ -247,30 +256,37 @@ public class SchedulerUtils {
SchedulerUtils.normalizeNodeLabelExpressionInRequest(resReq, queueInfo); SchedulerUtils.normalizeNodeLabelExpressionInRequest(resReq, queueInfo);
if (!isRecovery) { if (!isRecovery) {
validateResourceRequest(resReq, maximumAllocation, queueInfo, rmContext); validateResourceRequest(resReq, maximumResource, queueInfo, rmContext);
} }
} }
public static void normalizeAndValidateRequest(ResourceRequest resReq, public static void normalizeAndvalidateRequest(ResourceRequest resReq,
Resource maximumAllocation, String queueName, YarnScheduler scheduler, Resource maximumResource, String queueName, YarnScheduler scheduler,
RMContext rmContext) throws InvalidResourceRequestException {
normalizeAndvalidateRequest(resReq, maximumResource, queueName, scheduler,
rmContext, null);
}
public static void normalizeAndvalidateRequest(ResourceRequest resReq,
Resource maximumResource, String queueName, YarnScheduler scheduler,
RMContext rmContext, QueueInfo queueInfo) RMContext rmContext, QueueInfo queueInfo)
throws InvalidResourceRequestException { throws InvalidResourceRequestException {
normalizeAndValidateRequest(resReq, maximumAllocation, queueName, scheduler, normalizeAndValidateRequest(resReq, maximumResource, queueName, scheduler,
false, rmContext, queueInfo); false, rmContext, queueInfo);
} }
/** /**
* Utility method to validate a resource request, by insuring that the * Utility method to validate a resource request, by insuring that the
* requested memory/vcore is non-negative and not greater than max * requested memory/vcore is non-negative and not greater than max
* *
* @throws InvalidResourceRequestException when there is invalid request * @throws InvalidResourceRequestException when there is invalid request
*/ */
private static void validateResourceRequest(ResourceRequest resReq, private static void validateResourceRequest(ResourceRequest resReq,
Resource maximumAllocation, QueueInfo queueInfo, RMContext rmContext) Resource maximumResource, QueueInfo queueInfo, RMContext rmContext)
throws InvalidResourceRequestException { throws InvalidResourceRequestException {
final Resource requestedResource = resReq.getCapability(); final Resource requestedResource = resReq.getCapability();
checkResourceRequestAgainstAvailableResource(requestedResource, checkResourceRequestAgainstAvailableResource(requestedResource,
maximumAllocation); maximumResource);
String labelExp = resReq.getNodeLabelExpression(); String labelExp = resReq.getNodeLabelExpression();
// we don't allow specify label expression other than resourceName=ANY now // we don't allow specify label expression other than resourceName=ANY now
@ -409,7 +425,7 @@ public class SchedulerUtils {
if (!str.trim().isEmpty()) { if (!str.trim().isEmpty()) {
// check queue label // check queue label
if (queueLabels == null) { if (queueLabels == null) {
return false; return false;
} else { } else {
if (!queueLabels.contains(str) if (!queueLabels.contains(str)
&& !queueLabels.contains(RMNodeLabelsManager.ANY)) { && !queueLabels.contains(RMNodeLabelsManager.ANY)) {

View File

@ -380,17 +380,12 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
SchedulerNode getSchedulerNode(NodeId nodeId); SchedulerNode getSchedulerNode(NodeId nodeId);
/** /**
* Normalize a resource request using scheduler level maximum resource or * Normalize a resource request.
* queue based maximum resource.
* *
* @param requestedResource the resource to be normalized * @param requestedResource the resource to be normalized
* @param maxResourceCapability Maximum container allocation value, if null or
* empty scheduler level maximum container allocation value will be
* used
* @return the normalized resource * @return the normalized resource
*/ */
Resource getNormalizedResource(Resource requestedResource, Resource getNormalizedResource(Resource requestedResource);
Resource maxResourceCapability);
/** /**
* Verify whether a submitted application lifetime is valid as per configured * Verify whether a submitted application lifetime is valid as per configured

View File

@ -1107,12 +1107,10 @@ public class CapacityScheduler extends
if (asks == null) { if (asks == null) {
return; return;
} }
Resource maxAllocation = getMaximumResourceCapability();
for (SchedulingRequest ask: asks) { for (SchedulingRequest ask: asks) {
ResourceSizing sizing = ask.getResourceSizing(); ResourceSizing sizing = ask.getResourceSizing();
if (sizing != null && sizing.getResources() != null) { if (sizing != null && sizing.getResources() != null) {
sizing.setResources( sizing.setResources(getNormalizedResource(sizing.getResources()));
getNormalizedResource(sizing.getResources(), maxAllocation));
} }
} }
} }
@ -2433,9 +2431,6 @@ public class CapacityScheduler extends
@Override @Override
public Resource getMaximumResourceCapability(String queueName) { public Resource getMaximumResourceCapability(String queueName) {
if(queueName == null || queueName.isEmpty()) {
return getMaximumResourceCapability();
}
CSQueue queue = getQueue(queueName); CSQueue queue = getQueue(queueName);
if (queue == null) { if (queue == null) {
LOG.error("Unknown queue: " + queueName); LOG.error("Unknown queue: " + queueName);

View File

@ -175,19 +175,11 @@ public class PlacementConstraintProcessor extends AbstractPlacementProcessor {
private void dispatchRequestsForPlacement(ApplicationAttemptId appAttemptId, private void dispatchRequestsForPlacement(ApplicationAttemptId appAttemptId,
List<SchedulingRequest> schedulingRequests) { List<SchedulingRequest> schedulingRequests) {
if (schedulingRequests != null && !schedulingRequests.isEmpty()) { if (schedulingRequests != null && !schedulingRequests.isEmpty()) {
SchedulerApplicationAttempt appAttempt =
scheduler.getApplicationAttempt(appAttemptId);
String queueName = null;
if(appAttempt != null) {
queueName = appAttempt.getQueueName();
}
Resource maxAllocation =
scheduler.getMaximumResourceCapability(queueName);
// Normalize the Requests before dispatching // Normalize the Requests before dispatching
schedulingRequests.forEach(req -> { schedulingRequests.forEach(req -> {
Resource reqResource = req.getResourceSizing().getResources(); Resource reqResource = req.getResourceSizing().getResources();
req.getResourceSizing().setResources( req.getResourceSizing()
this.scheduler.getNormalizedResource(reqResource, maxAllocation)); .setResources(this.scheduler.getNormalizedResource(reqResource));
}); });
this.placementDispatcher.dispatch(new BatchedRequests(iteratorType, this.placementDispatcher.dispatch(new BatchedRequests(iteratorType,
appAttemptId.getApplicationId(), schedulingRequests, 1)); appAttemptId.getApplicationId(), schedulingRequests, 1));

View File

@ -93,9 +93,6 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
private final SchedulingPolicy defaultSchedulingPolicy; private final SchedulingPolicy defaultSchedulingPolicy;
//Map for maximum container resource allocation per queues by queue name
private final Map<String, Resource> queueMaxContainerAllocationMap;
// Policy for mapping apps to queues // Policy for mapping apps to queues
@VisibleForTesting @VisibleForTesting
QueuePlacementPolicy placementPolicy; QueuePlacementPolicy placementPolicy;
@ -143,8 +140,6 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
this.placementPolicy = newPlacementPolicy; this.placementPolicy = newPlacementPolicy;
this.configuredQueues = queueProperties.getConfiguredQueues(); this.configuredQueues = queueProperties.getConfiguredQueues();
this.nonPreemptableQueues = queueProperties.getNonPreemptableQueues(); this.nonPreemptableQueues = queueProperties.getNonPreemptableQueues();
this.queueMaxContainerAllocationMap =
queueProperties.getMaxContainerAllocation();
} }
public AllocationConfiguration(Configuration conf) { public AllocationConfiguration(Configuration conf) {
@ -174,7 +169,6 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
placementPolicy = placementPolicy =
QueuePlacementPolicy.fromConfiguration(conf, configuredQueues); QueuePlacementPolicy.fromConfiguration(conf, configuredQueues);
nonPreemptableQueues = new HashSet<>(); nonPreemptableQueues = new HashSet<>();
queueMaxContainerAllocationMap = new HashMap<>();
} }
/** /**
@ -298,12 +292,6 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
return maxQueueResource; return maxQueueResource;
} }
@VisibleForTesting
Resource getQueueMaxContainerAllocation(String queue) {
Resource resource = queueMaxContainerAllocationMap.get(queue);
return resource == null ? Resources.unbounded() : resource;
}
/** /**
* Get the maximum resource allocation for children of the given queue. * Get the maximum resource allocation for children of the given queue.
* *
@ -407,7 +395,6 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
queue.setMaxRunningApps(getQueueMaxApps(name)); queue.setMaxRunningApps(getQueueMaxApps(name));
queue.setMaxAMShare(getQueueMaxAMShare(name)); queue.setMaxAMShare(getQueueMaxAMShare(name));
queue.setMaxChildQueueResource(getMaxChildResources(name)); queue.setMaxChildQueueResource(getMaxChildResources(name));
queue.setMaxContainerAllocation(getQueueMaxContainerAllocation(name));
// Set queue metrics. // Set queue metrics.
queue.getMetrics().setMinShare(queue.getMinShare()); queue.getMetrics().setMinShare(queue.getMinShare());

View File

@ -530,16 +530,6 @@ public class FSLeafQueue extends FSQueue {
this.weights = weight; this.weights = weight;
} }
@Override
public Resource getMaximumContainerAllocation() {
if (maxContainerAllocation.equals(Resources.unbounded())
&& getParent() != null) {
return getParent().getMaximumContainerAllocation();
} else {
return maxContainerAllocation;
}
}
/** /**
* Helper method to compute the amount of minshare starvation. * Helper method to compute the amount of minshare starvation.
* *

View File

@ -59,20 +59,7 @@ public class FSParentQueue extends FSQueue {
FSParentQueue parent) { FSParentQueue parent) {
super(name, scheduler, parent); super(name, scheduler, parent);
} }
@Override
public Resource getMaximumContainerAllocation() {
if (getName().equals("root")) {
return maxContainerAllocation;
}
if (maxContainerAllocation.equals(Resources.unbounded())
&& getParent() != null) {
return getParent().getMaximumContainerAllocation();
} else {
return maxContainerAllocation;
}
}
void addChildQueue(FSQueue child) { void addChildQueue(FSQueue child) {
writeLock.lock(); writeLock.lock();
try { try {

View File

@ -83,7 +83,6 @@ public abstract class FSQueue implements Queue, Schedulable {
private long minSharePreemptionTimeout = Long.MAX_VALUE; private long minSharePreemptionTimeout = Long.MAX_VALUE;
private float fairSharePreemptionThreshold = 0.5f; private float fairSharePreemptionThreshold = 0.5f;
private boolean preemptable = true; private boolean preemptable = true;
protected Resource maxContainerAllocation;
public FSQueue(String name, FairScheduler scheduler, FSParentQueue parent) { public FSQueue(String name, FairScheduler scheduler, FSParentQueue parent) {
this.name = name; this.name = name;
@ -163,12 +162,6 @@ public abstract class FSQueue implements Queue, Schedulable {
this.maxShare = maxShare; this.maxShare = maxShare;
} }
public void setMaxContainerAllocation(Resource maxContainerAllocation){
this.maxContainerAllocation = maxContainerAllocation;
}
public abstract Resource getMaximumContainerAllocation();
@Override @Override
public Resource getMaxShare() { public Resource getMaxShare() {
Resource maxResource = maxShare.getResource(scheduler.getClusterResource()); Resource maxResource = maxShare.getResource(scheduler.getClusterResource());

View File

@ -186,7 +186,6 @@ public class FairScheduler extends
protected long rackLocalityDelayMs; // Delay for rack locality protected long rackLocalityDelayMs; // Delay for rack locality
protected boolean assignMultiple; // Allocate multiple containers per protected boolean assignMultiple; // Allocate multiple containers per
// heartbeat // heartbeat
@VisibleForTesting @VisibleForTesting
boolean maxAssignDynamic; boolean maxAssignDynamic;
protected int maxAssign; // Max containers to assign per heartbeat protected int maxAssign; // Max containers to assign per heartbeat
@ -222,12 +221,12 @@ public class FairScheduler extends
private void validateConf(FairSchedulerConfiguration config) { private void validateConf(FairSchedulerConfiguration config) {
// validate scheduler memory allocation setting // validate scheduler memory allocation setting
int minMem = int minMem = config.getInt(
config.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
int maxMem = int maxMem = config.getInt(
config.getInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB); YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
if (minMem < 0 || minMem > maxMem) { if (minMem < 0 || minMem > maxMem) {
throw new YarnRuntimeException("Invalid resource scheduler memory" throw new YarnRuntimeException("Invalid resource scheduler memory"
@ -249,12 +248,12 @@ public class FairScheduler extends
} }
// validate scheduler vcores allocation setting // validate scheduler vcores allocation setting
int minVcores = int minVcores = config.getInt(
config.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
int maxVcores = int maxVcores = config.getInt(
config.getInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
if (minVcores < 0 || minVcores > maxVcores) { if (minVcores < 0 || minVcores > maxVcores) {
throw new YarnRuntimeException("Invalid resource scheduler vcores" throw new YarnRuntimeException("Invalid resource scheduler vcores"
@ -813,35 +812,14 @@ public class FairScheduler extends
} }
@Override @Override
public Resource getNormalizedResource(Resource requestedResource, public Resource getNormalizedResource(Resource requestedResource) {
Resource maxResourceCapability) {
return SchedulerUtils.getNormalizedResource(requestedResource, return SchedulerUtils.getNormalizedResource(requestedResource,
DOMINANT_RESOURCE_CALCULATOR, DOMINANT_RESOURCE_CALCULATOR,
minimumAllocation, minimumAllocation,
maxResourceCapability, getMaximumResourceCapability(),
incrAllocation); incrAllocation);
} }
@Override
public Resource getMaximumResourceCapability(String queueName) {
if(queueName == null || queueName.isEmpty()) {
return getMaximumResourceCapability();
}
FSQueue queue = queueMgr.getQueue(queueName);
Resource schedulerLevelMaxResourceCapability =
getMaximumResourceCapability();
if (queue == null) {
return schedulerLevelMaxResourceCapability;
}
Resource queueMaxResourceCapability = queue.getMaximumContainerAllocation();
if (queueMaxResourceCapability.equals(Resources.unbounded())) {
return schedulerLevelMaxResourceCapability;
} else {
return Resources.componentwiseMin(schedulerLevelMaxResourceCapability,
queueMaxResourceCapability);
}
}
@VisibleForTesting @VisibleForTesting
@Override @Override
public void killContainer(RMContainer container) { public void killContainer(RMContainer container) {
@ -881,7 +859,7 @@ public class FairScheduler extends
handleContainerUpdates(application, updateRequests); handleContainerUpdates(application, updateRequests);
// Sanity check // Sanity check
normalizeResourceRequests(ask, application.getQueueName()); normalizeResourceRequests(ask);
// TODO, normalize SchedulingRequest // TODO, normalize SchedulingRequest

View File

@ -51,8 +51,6 @@ public class AllocationFileQueueParser {
private static final String MAX_CHILD_RESOURCES = "maxChildResources"; private static final String MAX_CHILD_RESOURCES = "maxChildResources";
private static final String MAX_RUNNING_APPS = "maxRunningApps"; private static final String MAX_RUNNING_APPS = "maxRunningApps";
private static final String MAX_AMSHARE = "maxAMShare"; private static final String MAX_AMSHARE = "maxAMShare";
public static final String MAX_CONTAINER_ALLOCATION =
"maxContainerAllocation";
private static final String WEIGHT = "weight"; private static final String WEIGHT = "weight";
private static final String MIN_SHARE_PREEMPTION_TIMEOUT = private static final String MIN_SHARE_PREEMPTION_TIMEOUT =
"minSharePreemptionTimeout"; "minSharePreemptionTimeout";
@ -157,11 +155,6 @@ public class AllocationFileQueueParser {
float val = Float.parseFloat(text); float val = Float.parseFloat(text);
val = Math.min(val, 1.0f); val = Math.min(val, 1.0f);
builder.queueMaxAMShares(queueName, val); builder.queueMaxAMShares(queueName, val);
} else if (MAX_CONTAINER_ALLOCATION.equals(field.getTagName())) {
String text = getTrimmedTextData(field);
ConfigurableResource val =
FairSchedulerConfiguration.parseResourceConfigValue(text);
builder.queueMaxContainerAllocation(queueName, val.getResource());
} else if (WEIGHT.equals(field.getTagName())) { } else if (WEIGHT.equals(field.getTagName())) {
String text = getTrimmedTextData(field); String text = getTrimmedTextData(field);
double val = Double.parseDouble(text); double val = Double.parseDouble(text);

View File

@ -53,7 +53,6 @@ public class QueueProperties {
private final Set<String> reservableQueues; private final Set<String> reservableQueues;
private final Set<String> nonPreemptableQueues; private final Set<String> nonPreemptableQueues;
private final Map<FSQueueType, Set<String>> configuredQueues; private final Map<FSQueueType, Set<String>> configuredQueues;
private final Map<String, Resource> queueMaxContainerAllocation;
QueueProperties(Builder builder) { QueueProperties(Builder builder) {
this.reservableQueues = builder.reservableQueues; this.reservableQueues = builder.reservableQueues;
@ -71,7 +70,6 @@ public class QueueProperties {
this.maxChildQueueResources = builder.maxChildQueueResources; this.maxChildQueueResources = builder.maxChildQueueResources;
this.reservationAcls = builder.reservationAcls; this.reservationAcls = builder.reservationAcls;
this.queueAcls = builder.queueAcls; this.queueAcls = builder.queueAcls;
this.queueMaxContainerAllocation = builder.queueMaxContainerAllocation;
} }
public Map<FSQueueType, Set<String>> getConfiguredQueues() { public Map<FSQueueType, Set<String>> getConfiguredQueues() {
@ -135,11 +133,7 @@ public class QueueProperties {
return nonPreemptableQueues; return nonPreemptableQueues;
} }
public Map<String, Resource> getMaxContainerAllocation() { /**
return queueMaxContainerAllocation;
}
/**
* Builder class for {@link QueueProperties}. * Builder class for {@link QueueProperties}.
* All methods are adding queue properties to the maps of this builder * All methods are adding queue properties to the maps of this builder
* keyed by the queue's name except some methods * keyed by the queue's name except some methods
@ -155,7 +149,6 @@ public class QueueProperties {
new HashMap<>(); new HashMap<>();
private Map<String, Integer> queueMaxApps = new HashMap<>(); private Map<String, Integer> queueMaxApps = new HashMap<>();
private Map<String, Float> queueMaxAMShares = new HashMap<>(); private Map<String, Float> queueMaxAMShares = new HashMap<>();
private Map<String, Resource> queueMaxContainerAllocation = new HashMap<>();
private Map<String, Float> queueWeights = new HashMap<>(); private Map<String, Float> queueWeights = new HashMap<>();
private Map<String, SchedulingPolicy> queuePolicies = new HashMap<>(); private Map<String, SchedulingPolicy> queuePolicies = new HashMap<>();
private Map<String, Long> minSharePreemptionTimeouts = new HashMap<>(); private Map<String, Long> minSharePreemptionTimeouts = new HashMap<>();
@ -260,12 +253,6 @@ public class QueueProperties {
return this; return this;
} }
public Builder queueMaxContainerAllocation(String queueName,
Resource value) {
queueMaxContainerAllocation.put(queueName, value);
return this;
}
public void configuredQueues(FSQueueType queueType, String queueName) { public void configuredQueues(FSQueueType queueType, String queueName) {
this.configuredQueues.get(queueType).add(queueName); this.configuredQueues.get(queueType).add(queueName);
} }
@ -288,5 +275,6 @@ public class QueueProperties {
public QueueProperties build() { public QueueProperties build() {
return new QueueProperties(this); return new QueueProperties(this);
} }
} }
} }

View File

@ -78,8 +78,6 @@ public class FairSchedulerPage extends RmView {
__("Num Pending Applications:", qinfo.getNumPendingApplications()). __("Num Pending Applications:", qinfo.getNumPendingApplications()).
__("Min Resources:", qinfo.getMinResources().toString()). __("Min Resources:", qinfo.getMinResources().toString()).
__("Max Resources:", qinfo.getMaxResources().toString()). __("Max Resources:", qinfo.getMaxResources().toString()).
__("Max Container Allocation:",
qinfo.getMaxContainerAllocation().toString()).
__("Reserved Resources:", qinfo.getReservedResources().toString()); __("Reserved Resources:", qinfo.getReservedResources().toString());
int maxApps = qinfo.getMaxApplications(); int maxApps = qinfo.getMaxApplications();
if (maxApps < Integer.MAX_VALUE) { if (maxApps < Integer.MAX_VALUE) {
@ -109,8 +107,6 @@ public class FairSchedulerPage extends RmView {
__("Used Resources:", qinfo.getUsedResources().toString()). __("Used Resources:", qinfo.getUsedResources().toString()).
__("Min Resources:", qinfo.getMinResources().toString()). __("Min Resources:", qinfo.getMinResources().toString()).
__("Max Resources:", qinfo.getMaxResources().toString()). __("Max Resources:", qinfo.getMaxResources().toString()).
__("Max Container Allocation:",
qinfo.getMaxContainerAllocation().toString()).
__("Reserved Resources:", qinfo.getReservedResources().toString()); __("Reserved Resources:", qinfo.getReservedResources().toString());
int maxApps = qinfo.getMaxApplications(); int maxApps = qinfo.getMaxApplications();
if (maxApps < Integer.MAX_VALUE) { if (maxApps < Integer.MAX_VALUE) {

View File

@ -60,7 +60,6 @@ public class FairSchedulerQueueInfo {
private ResourceInfo fairResources; private ResourceInfo fairResources;
private ResourceInfo clusterResources; private ResourceInfo clusterResources;
private ResourceInfo reservedResources; private ResourceInfo reservedResources;
private ResourceInfo maxContainerAllocation;
private long allocatedContainers; private long allocatedContainers;
private long reservedContainers; private long reservedContainers;
@ -100,8 +99,6 @@ public class FairSchedulerQueueInfo {
maxResources = new ResourceInfo( maxResources = new ResourceInfo(
Resources.componentwiseMin(queue.getMaxShare(), Resources.componentwiseMin(queue.getMaxShare(),
scheduler.getClusterResource())); scheduler.getClusterResource()));
maxContainerAllocation =
new ResourceInfo(scheduler.getMaximumResourceCapability(queueName));
reservedResources = new ResourceInfo(queue.getReservedResource()); reservedResources = new ResourceInfo(queue.getReservedResource());
fractionMemSteadyFairShare = fractionMemSteadyFairShare =
@ -189,11 +186,7 @@ public class FairSchedulerQueueInfo {
public ResourceInfo getMaxResources() { public ResourceInfo getMaxResources() {
return maxResources; return maxResources;
} }
public ResourceInfo getMaxContainerAllocation() {
return maxContainerAllocation;
}
public ResourceInfo getReservedResources() { public ResourceInfo getReservedResources() {
return reservedResources; return reservedResources;
} }

View File

@ -1,107 +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;
import static java.util.stream.Collectors.toSet;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.mockito.ArgumentCaptor;
/**
* Base class for AppManager related test.
*
*/
public class AppManagerTestBase {
// Extend and make the functions we want to test public
protected class TestRMAppManager extends RMAppManager {
private final RMStateStore stateStore;
public TestRMAppManager(RMContext context, Configuration conf) {
super(context, null, null, new ApplicationACLsManager(conf), conf);
this.stateStore = context.getStateStore();
}
public TestRMAppManager(RMContext context,
ClientToAMTokenSecretManagerInRM clientToAMSecretManager,
YarnScheduler scheduler, ApplicationMasterService masterService,
ApplicationACLsManager applicationACLsManager, Configuration conf) {
super(context, scheduler, masterService, applicationACLsManager, conf);
this.stateStore = context.getStateStore();
}
public void checkAppNumCompletedLimit() {
super.checkAppNumCompletedLimit();
}
public void finishApplication(ApplicationId appId) {
super.finishApplication(appId);
}
public int getCompletedAppsListSize() {
return super.getCompletedAppsListSize();
}
public int getNumberOfCompletedAppsInStateStore() {
return this.completedAppsInStateStore;
}
public List<ApplicationId> getCompletedApps() {
return completedApps;
}
public Set<ApplicationId> getFirstNCompletedApps(int n) {
return getCompletedApps().stream().limit(n).collect(toSet());
}
public Set<ApplicationId> getCompletedAppsWithEvenIdsInRange(int n) {
return getCompletedApps().stream().limit(n)
.filter(app -> app.getId() % 2 == 0).collect(toSet());
}
public Set<ApplicationId> getRemovedAppsFromStateStore(int numRemoves) {
ArgumentCaptor<RMApp> argumentCaptor =
ArgumentCaptor.forClass(RMApp.class);
verify(stateStore, times(numRemoves))
.removeApplication(argumentCaptor.capture());
return argumentCaptor.getAllValues().stream().map(RMApp::getApplicationId)
.collect(toSet());
}
public void submitApplication(
ApplicationSubmissionContext submissionContext, String user)
throws YarnException {
super.submitApplication(submissionContext, System.currentTimeMillis(),
user);
}
}
}

View File

@ -513,14 +513,6 @@ public class MockRM extends ResourceManager {
return submitApp(masterMemory, false); return submitApp(masterMemory, false);
} }
public RMApp submitApp(int masterMemory, String queue) throws Exception {
return submitApp(masterMemory, "",
UserGroupInformation.getCurrentUser().getShortUserName(), null, false,
queue, super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS),
null);
}
public RMApp submitApp(int masterMemory, Priority priority) throws Exception { public RMApp submitApp(int masterMemory, Priority priority) throws Exception {
Resource resource = Resource.newInstance(masterMemory, 0); Resource resource = Resource.newInstance(masterMemory, 0);
return submitApp(resource, "", UserGroupInformation.getCurrentUser() return submitApp(resource, "", UserGroupInformation.getCurrentUser()

View File

@ -105,7 +105,6 @@ import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import static org.mockito.Matchers.anyString;
import org.mockito.invocation.InvocationOnMock; import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer; import org.mockito.stubbing.Answer;
@ -238,7 +237,7 @@ public class TestAppManager{
protected void addToCompletedApps(TestRMAppManager appMonitor, RMContext rmContext) { protected void addToCompletedApps(TestRMAppManager appMonitor, RMContext rmContext) {
for (RMApp app : rmContext.getRMApps().values()) { for (RMApp app : rmContext.getRMApps().values()) {
if (app.getState() == RMAppState.FINISHED if (app.getState() == RMAppState.FINISHED
|| app.getState() == RMAppState.KILLED || app.getState() == RMAppState.KILLED
|| app.getState() == RMAppState.FAILED) { || app.getState() == RMAppState.FAILED) {
appMonitor.finishApplication(app.getApplicationId()); appMonitor.finishApplication(app.getApplicationId());
} }
@ -1075,21 +1074,17 @@ public class TestAppManager{
Resources.createResource( Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB)); YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB));
when(scheduler.getMaximumResourceCapability(anyString())).thenReturn(
Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB));
ResourceCalculator rs = mock(ResourceCalculator.class); ResourceCalculator rs = mock(ResourceCalculator.class);
when(scheduler.getResourceCalculator()).thenReturn(rs); when(scheduler.getResourceCalculator()).thenReturn(rs);
when(scheduler.getNormalizedResource(any(), any())) when(scheduler.getNormalizedResource(any()))
.thenAnswer(new Answer<Resource>() { .thenAnswer(new Answer<Resource>() {
@Override @Override
public Resource answer(InvocationOnMock invocationOnMock) public Resource answer(InvocationOnMock invocationOnMock)
throws Throwable { throws Throwable {
return (Resource) invocationOnMock.getArguments()[0]; return (Resource) invocationOnMock.getArguments()[0];
} }
}); });
return scheduler; return scheduler;
} }

View File

@ -1,175 +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;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.matches;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import java.io.File;
import java.io.FileWriter;
import java.io.IOException;
import java.io.PrintWriter;
import java.util.HashMap;
import org.apache.hadoop.test.GenericTestUtils;
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.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
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.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
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.PlacementManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
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.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
/**
* Testing applications being retired from RM with fair scheduler.
*
*/
public class TestAppManagerWithFairScheduler extends AppManagerTestBase{
private static final String TEST_FOLDER = "test-queues";
private static YarnConfiguration conf = new YarnConfiguration();
@BeforeClass
public static void setup() throws IOException {
String allocFile =
GenericTestUtils.getTestDir(TEST_FOLDER).getAbsolutePath();
int queueMaxAllocation = 512;
PrintWriter out = new PrintWriter(new FileWriter(allocFile));
out.println("<?xml version=\"1.0\"?>");
out.println("<allocations>");
out.println(" <queue name=\"queueA\">");
out.println(" <maxContainerAllocation>" + queueMaxAllocation
+ " mb 1 vcores" + "</maxContainerAllocation>");
out.println(" </queue>");
out.println(" <queue name=\"queueB\">");
out.println(" </queue>");
out.println("</allocations>");
out.close();
conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class,
ResourceScheduler.class);
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, allocFile);
}
@AfterClass
public static void teardown(){
File allocFile = GenericTestUtils.getTestDir(TEST_FOLDER);
allocFile.delete();
}
@Test
public void testQueueSubmitWithHighQueueContainerSize()
throws YarnException {
ApplicationId appId = MockApps.newAppID(1);
RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
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
try {
newAppMonitor.submitApplication(asContext, "test1");
Assert.fail("Test should fail on too high allocation!");
} catch (InvalidResourceRequestException e) {
// Should throw exception
}
// Should not throw exception
newAppMonitor.submitApplication(asContext, "test2");
}
private static ContainerLaunchContext mockContainerLaunchContext(
RecordFactory recordFactory) {
ContainerLaunchContext amContainer = recordFactory.newRecordInstance(
ContainerLaunchContext.class);
amContainer
.setApplicationACLs(new HashMap<ApplicationAccessType, String>());
return amContainer;
}
}

View File

@ -347,9 +347,9 @@ public class TestClientRMService {
@Test @Test
public void testGetApplicationReport() throws Exception { public void testGetApplicationReport() throws Exception {
ResourceScheduler scheduler = mock(ResourceScheduler.class); YarnScheduler yarnScheduler = mock(YarnScheduler.class);
RMContext rmContext = mock(RMContext.class); RMContext rmContext = mock(RMContext.class);
mockRMContext(scheduler, rmContext); mockRMContext(yarnScheduler, rmContext);
ApplicationId appId1 = getApplicationId(1); ApplicationId appId1 = getApplicationId(1);
@ -358,7 +358,7 @@ public class TestClientRMService {
mockAclsManager.checkAccess(UserGroupInformation.getCurrentUser(), mockAclsManager.checkAccess(UserGroupInformation.getCurrentUser(),
ApplicationAccessType.VIEW_APP, null, appId1)).thenReturn(true); ApplicationAccessType.VIEW_APP, null, appId1)).thenReturn(true);
ClientRMService rmService = new ClientRMService(rmContext, scheduler, ClientRMService rmService = new ClientRMService(rmContext, yarnScheduler,
null, mockAclsManager, null, null); null, mockAclsManager, null, null);
try { try {
RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
@ -441,9 +441,9 @@ public class TestClientRMService {
public void testGetApplicationResourceUsageReportDummy() throws YarnException, public void testGetApplicationResourceUsageReportDummy() throws YarnException,
IOException { IOException {
ApplicationAttemptId attemptId = getApplicationAttemptId(1); ApplicationAttemptId attemptId = getApplicationAttemptId(1);
ResourceScheduler scheduler = mockResourceScheduler(); YarnScheduler yarnScheduler = mockYarnScheduler();
RMContext rmContext = mock(RMContext.class); RMContext rmContext = mock(RMContext.class);
mockRMContext(scheduler, rmContext); mockRMContext(yarnScheduler, rmContext);
when(rmContext.getDispatcher().getEventHandler()).thenReturn( when(rmContext.getDispatcher().getEventHandler()).thenReturn(
new EventHandler<Event>() { new EventHandler<Event>() {
public void handle(Event event) { public void handle(Event event) {
@ -453,7 +453,7 @@ public class TestClientRMService {
mock(ApplicationSubmissionContext.class); mock(ApplicationSubmissionContext.class);
YarnConfiguration config = new YarnConfiguration(); YarnConfiguration config = new YarnConfiguration();
RMAppAttemptImpl rmAppAttemptImpl = new RMAppAttemptImpl(attemptId, RMAppAttemptImpl rmAppAttemptImpl = new RMAppAttemptImpl(attemptId,
rmContext, scheduler, null, asContext, config, null, null); rmContext, yarnScheduler, null, asContext, config, null, null);
ApplicationResourceUsageReport report = rmAppAttemptImpl ApplicationResourceUsageReport report = rmAppAttemptImpl
.getApplicationResourceUsageReport(); .getApplicationResourceUsageReport();
assertEquals(report, RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT); assertEquals(report, RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT);
@ -522,14 +522,14 @@ public class TestClientRMService {
} }
public ClientRMService createRMService() throws IOException, YarnException { public ClientRMService createRMService() throws IOException, YarnException {
ResourceScheduler scheduler = mockResourceScheduler(); YarnScheduler yarnScheduler = mockYarnScheduler();
RMContext rmContext = mock(RMContext.class); RMContext rmContext = mock(RMContext.class);
mockRMContext(scheduler, rmContext); mockRMContext(yarnScheduler, rmContext);
ConcurrentHashMap<ApplicationId, RMApp> apps = getRMApps(rmContext, ConcurrentHashMap<ApplicationId, RMApp> apps = getRMApps(rmContext,
scheduler); yarnScheduler);
when(rmContext.getRMApps()).thenReturn(apps); when(rmContext.getRMApps()).thenReturn(apps);
when(rmContext.getYarnConfiguration()).thenReturn(new Configuration()); when(rmContext.getYarnConfiguration()).thenReturn(new Configuration());
RMAppManager appManager = new RMAppManager(rmContext, scheduler, null, RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler, null,
mock(ApplicationACLsManager.class), new Configuration()); mock(ApplicationACLsManager.class), new Configuration());
when(rmContext.getDispatcher().getEventHandler()).thenReturn( when(rmContext.getDispatcher().getEventHandler()).thenReturn(
new EventHandler<Event>() { new EventHandler<Event>() {
@ -543,7 +543,7 @@ public class TestClientRMService {
mockQueueACLsManager.checkAccess(any(UserGroupInformation.class), mockQueueACLsManager.checkAccess(any(UserGroupInformation.class),
any(QueueACL.class), any(RMApp.class), any(String.class), any(QueueACL.class), any(RMApp.class), any(String.class),
any())).thenReturn(true); any())).thenReturn(true);
return new ClientRMService(rmContext, scheduler, appManager, return new ClientRMService(rmContext, yarnScheduler, appManager,
mockAclsManager, mockQueueACLsManager, null); mockAclsManager, mockQueueACLsManager, null);
} }
@ -892,9 +892,9 @@ public class TestClientRMService {
@Test @Test
public void testGetQueueInfo() throws Exception { public void testGetQueueInfo() throws Exception {
ResourceScheduler scheduler = mock(ResourceScheduler.class); YarnScheduler yarnScheduler = mock(YarnScheduler.class);
RMContext rmContext = mock(RMContext.class); RMContext rmContext = mock(RMContext.class);
mockRMContext(scheduler, rmContext); mockRMContext(yarnScheduler, rmContext);
ApplicationACLsManager mockAclsManager = mock(ApplicationACLsManager.class); ApplicationACLsManager mockAclsManager = mock(ApplicationACLsManager.class);
QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class); QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class);
@ -906,7 +906,7 @@ public class TestClientRMService {
any(ApplicationAccessType.class), anyString(), any(ApplicationAccessType.class), anyString(),
any(ApplicationId.class))).thenReturn(true); any(ApplicationId.class))).thenReturn(true);
ClientRMService rmService = new ClientRMService(rmContext, scheduler, ClientRMService rmService = new ClientRMService(rmContext, yarnScheduler,
null, mockAclsManager, mockQueueACLsManager, null); null, mockAclsManager, mockQueueACLsManager, null);
GetQueueInfoRequest request = recordFactory GetQueueInfoRequest request = recordFactory
.newRecordInstance(GetQueueInfoRequest.class); .newRecordInstance(GetQueueInfoRequest.class);
@ -945,7 +945,7 @@ public class TestClientRMService {
any(ApplicationAccessType.class), anyString(), any(ApplicationAccessType.class), anyString(),
any(ApplicationId.class))).thenReturn(false); any(ApplicationId.class))).thenReturn(false);
ClientRMService rmService1 = new ClientRMService(rmContext, scheduler, ClientRMService rmService1 = new ClientRMService(rmContext, yarnScheduler,
null, mockAclsManager1, mockQueueACLsManager1, null); null, mockAclsManager1, mockQueueACLsManager1, null);
request.setQueueName("testqueue"); request.setQueueName("testqueue");
request.setIncludeApplications(true); request.setIncludeApplications(true);
@ -959,12 +959,12 @@ public class TestClientRMService {
@Test (timeout = 30000) @Test (timeout = 30000)
@SuppressWarnings ("rawtypes") @SuppressWarnings ("rawtypes")
public void testAppSubmit() throws Exception { public void testAppSubmit() throws Exception {
ResourceScheduler scheduler = mockResourceScheduler(); YarnScheduler yarnScheduler = mockYarnScheduler();
RMContext rmContext = mock(RMContext.class); RMContext rmContext = mock(RMContext.class);
mockRMContext(scheduler, rmContext); mockRMContext(yarnScheduler, rmContext);
RMStateStore stateStore = mock(RMStateStore.class); RMStateStore stateStore = mock(RMStateStore.class);
when(rmContext.getStateStore()).thenReturn(stateStore); when(rmContext.getStateStore()).thenReturn(stateStore);
RMAppManager appManager = new RMAppManager(rmContext, scheduler, RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler,
null, mock(ApplicationACLsManager.class), new Configuration()); null, mock(ApplicationACLsManager.class), new Configuration());
when(rmContext.getDispatcher().getEventHandler()).thenReturn( when(rmContext.getDispatcher().getEventHandler()).thenReturn(
new EventHandler<Event>() { new EventHandler<Event>() {
@ -986,7 +986,7 @@ public class TestClientRMService {
any())) any()))
.thenReturn(true); .thenReturn(true);
ClientRMService rmService = ClientRMService rmService =
new ClientRMService(rmContext, scheduler, appManager, new ClientRMService(rmContext, yarnScheduler, appManager,
mockAclsManager, mockQueueACLsManager, null); mockAclsManager, mockQueueACLsManager, null);
rmService.init(new Configuration()); rmService.init(new Configuration());
@ -1070,15 +1070,15 @@ public class TestClientRMService {
* 2. Test each of the filters * 2. Test each of the filters
*/ */
// Basic setup // Basic setup
ResourceScheduler scheduler = mockResourceScheduler(); YarnScheduler yarnScheduler = mockYarnScheduler();
RMContext rmContext = mock(RMContext.class); RMContext rmContext = mock(RMContext.class);
mockRMContext(scheduler, rmContext); mockRMContext(yarnScheduler, rmContext);
RMStateStore stateStore = mock(RMStateStore.class); RMStateStore stateStore = mock(RMStateStore.class);
when(rmContext.getStateStore()).thenReturn(stateStore); when(rmContext.getStateStore()).thenReturn(stateStore);
doReturn(mock(RMTimelineCollectorManager.class)).when(rmContext) doReturn(mock(RMTimelineCollectorManager.class)).when(rmContext)
.getRMTimelineCollectorManager(); .getRMTimelineCollectorManager();
RMAppManager appManager = new RMAppManager(rmContext, scheduler, RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler,
null, mock(ApplicationACLsManager.class), new Configuration()); null, mock(ApplicationACLsManager.class), new Configuration());
when(rmContext.getDispatcher().getEventHandler()).thenReturn( when(rmContext.getDispatcher().getEventHandler()).thenReturn(
new EventHandler<Event>() { new EventHandler<Event>() {
@ -1092,7 +1092,7 @@ public class TestClientRMService {
any())) any()))
.thenReturn(true); .thenReturn(true);
ClientRMService rmService = ClientRMService rmService =
new ClientRMService(rmContext, scheduler, appManager, new ClientRMService(rmContext, yarnScheduler, appManager,
mockAclsManager, mockQueueACLsManager, null); mockAclsManager, mockQueueACLsManager, null);
rmService.init(new Configuration()); rmService.init(new Configuration());
@ -1223,12 +1223,12 @@ public class TestClientRMService {
public void testConcurrentAppSubmit() public void testConcurrentAppSubmit()
throws IOException, InterruptedException, BrokenBarrierException, throws IOException, InterruptedException, BrokenBarrierException,
YarnException { YarnException {
ResourceScheduler scheduler = mockResourceScheduler(); YarnScheduler yarnScheduler = mockYarnScheduler();
RMContext rmContext = mock(RMContext.class); RMContext rmContext = mock(RMContext.class);
mockRMContext(scheduler, rmContext); mockRMContext(yarnScheduler, rmContext);
RMStateStore stateStore = mock(RMStateStore.class); RMStateStore stateStore = mock(RMStateStore.class);
when(rmContext.getStateStore()).thenReturn(stateStore); when(rmContext.getStateStore()).thenReturn(stateStore);
RMAppManager appManager = new RMAppManager(rmContext, scheduler, RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler,
null, mock(ApplicationACLsManager.class), new Configuration()); null, mock(ApplicationACLsManager.class), new Configuration());
final ApplicationId appId1 = getApplicationId(100); final ApplicationId appId1 = getApplicationId(100);
@ -1265,7 +1265,7 @@ public class TestClientRMService {
.getRMTimelineCollectorManager(); .getRMTimelineCollectorManager();
final ClientRMService rmService = final ClientRMService rmService =
new ClientRMService(rmContext, scheduler, appManager, null, null, new ClientRMService(rmContext, yarnScheduler, appManager, null, null,
null); null);
rmService.init(new Configuration()); rmService.init(new Configuration());
@ -1324,7 +1324,7 @@ public class TestClientRMService {
return submitRequest; return submitRequest;
} }
private void mockRMContext(ResourceScheduler scheduler, RMContext rmContext) private void mockRMContext(YarnScheduler yarnScheduler, RMContext rmContext)
throws IOException { throws IOException {
Dispatcher dispatcher = mock(Dispatcher.class); Dispatcher dispatcher = mock(Dispatcher.class);
when(rmContext.getDispatcher()).thenReturn(dispatcher); when(rmContext.getDispatcher()).thenReturn(dispatcher);
@ -1346,21 +1346,22 @@ public class TestClientRMService {
queueConfigsByPartition.put("*", queueConfigs); queueConfigsByPartition.put("*", queueConfigs);
queInfo.setQueueConfigurations(queueConfigsByPartition); queInfo.setQueueConfigurations(queueConfigsByPartition);
when(scheduler.getQueueInfo(eq("testqueue"), anyBoolean(), anyBoolean())) when(yarnScheduler.getQueueInfo(eq("testqueue"), anyBoolean(), anyBoolean()))
.thenReturn(queInfo); .thenReturn(queInfo);
when(scheduler.getQueueInfo(eq("nonexistentqueue"), anyBoolean(), when(yarnScheduler.getQueueInfo(eq("nonexistentqueue"), anyBoolean(), anyBoolean()))
anyBoolean())).thenThrow(new IOException("queue does not exist")); .thenThrow(new IOException("queue does not exist"));
RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class); RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer); when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
SystemMetricsPublisher publisher = mock(SystemMetricsPublisher.class); SystemMetricsPublisher publisher = mock(SystemMetricsPublisher.class);
when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher); when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
when(rmContext.getYarnConfiguration()).thenReturn(new YarnConfiguration()); when(rmContext.getYarnConfiguration()).thenReturn(new YarnConfiguration());
ConcurrentHashMap<ApplicationId, RMApp> apps = ConcurrentHashMap<ApplicationId, RMApp> apps = getRMApps(rmContext,
getRMApps(rmContext, scheduler); yarnScheduler);
when(rmContext.getRMApps()).thenReturn(apps); when(rmContext.getRMApps()).thenReturn(apps);
when(scheduler.getAppsInQueue(eq("testqueue"))).thenReturn( when(yarnScheduler.getAppsInQueue(eq("testqueue"))).thenReturn(
getSchedulerApps(apps)); getSchedulerApps(apps));
when(rmContext.getScheduler()).thenReturn(scheduler); ResourceScheduler rs = mock(ResourceScheduler.class);
when(rmContext.getScheduler()).thenReturn(rs);
} }
private ConcurrentHashMap<ApplicationId, RMApp> getRMApps( private ConcurrentHashMap<ApplicationId, RMApp> getRMApps(
@ -1464,32 +1465,31 @@ public class TestClientRMService {
return app; return app;
} }
private static ResourceScheduler mockResourceScheduler() private static YarnScheduler mockYarnScheduler() throws YarnException {
throws YarnException { YarnScheduler yarnScheduler = mock(YarnScheduler.class);
ResourceScheduler scheduler = mock(ResourceScheduler.class); when(yarnScheduler.getMinimumResourceCapability()).thenReturn(
when(scheduler.getMinimumResourceCapability()).thenReturn(
Resources.createResource( Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB)); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB));
when(scheduler.getMaximumResourceCapability()).thenReturn( when(yarnScheduler.getMaximumResourceCapability()).thenReturn(
Resources.createResource( Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB)); YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB));
when(scheduler.getMaximumResourceCapability(anyString())).thenReturn( when(yarnScheduler.getMaximumResourceCapability(any(String.class)))
Resources.createResource( .thenReturn(Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB)); YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB));
when(scheduler.getAppsInQueue(QUEUE_1)).thenReturn( when(yarnScheduler.getAppsInQueue(QUEUE_1)).thenReturn(
Arrays.asList(getApplicationAttemptId(101), getApplicationAttemptId(102))); Arrays.asList(getApplicationAttemptId(101), getApplicationAttemptId(102)));
when(scheduler.getAppsInQueue(QUEUE_2)).thenReturn( when(yarnScheduler.getAppsInQueue(QUEUE_2)).thenReturn(
Arrays.asList(getApplicationAttemptId(103))); Arrays.asList(getApplicationAttemptId(103)));
ApplicationAttemptId attemptId = getApplicationAttemptId(1); ApplicationAttemptId attemptId = getApplicationAttemptId(1);
when(scheduler.getAppResourceUsageReport(attemptId)).thenReturn(null); when(yarnScheduler.getAppResourceUsageReport(attemptId)).thenReturn(null);
ResourceCalculator rs = mock(ResourceCalculator.class); ResourceCalculator rs = mock(ResourceCalculator.class);
when(scheduler.getResourceCalculator()).thenReturn(rs); when(yarnScheduler.getResourceCalculator()).thenReturn(rs);
when(scheduler.checkAndGetApplicationPriority(any(Priority.class), when(yarnScheduler.checkAndGetApplicationPriority(any(Priority.class),
any(UserGroupInformation.class), anyString(), any(ApplicationId.class))) any(UserGroupInformation.class), anyString(), any(ApplicationId.class)))
.thenReturn(Priority.newInstance(0)); .thenReturn(Priority.newInstance(0));
return scheduler; return yarnScheduler;
} }
private ResourceManager setupResourceManager() { private ResourceManager setupResourceManager() {
@ -2186,15 +2186,15 @@ public class TestClientRMService {
* Submit 3 applications alternately in two queues * Submit 3 applications alternately in two queues
*/ */
// Basic setup // Basic setup
ResourceScheduler scheduler = mockResourceScheduler(); YarnScheduler yarnScheduler = mockYarnScheduler();
RMContext rmContext = mock(RMContext.class); RMContext rmContext = mock(RMContext.class);
mockRMContext(scheduler, rmContext); mockRMContext(yarnScheduler, rmContext);
RMStateStore stateStore = mock(RMStateStore.class); RMStateStore stateStore = mock(RMStateStore.class);
when(rmContext.getStateStore()).thenReturn(stateStore); when(rmContext.getStateStore()).thenReturn(stateStore);
doReturn(mock(RMTimelineCollectorManager.class)).when(rmContext) doReturn(mock(RMTimelineCollectorManager.class)).when(rmContext)
.getRMTimelineCollectorManager(); .getRMTimelineCollectorManager();
RMAppManager appManager = new RMAppManager(rmContext, scheduler, null, RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler, null,
mock(ApplicationACLsManager.class), new Configuration()); mock(ApplicationACLsManager.class), new Configuration());
when(rmContext.getDispatcher().getEventHandler()) when(rmContext.getDispatcher().getEventHandler())
.thenReturn(new EventHandler<Event>() { .thenReturn(new EventHandler<Event>() {
@ -2213,7 +2213,7 @@ public class TestClientRMService {
when(appAclsManager.checkAccess(eq(UserGroupInformation.getCurrentUser()), when(appAclsManager.checkAccess(eq(UserGroupInformation.getCurrentUser()),
any(ApplicationAccessType.class), any(String.class), any(ApplicationAccessType.class), any(String.class),
any(ApplicationId.class))).thenReturn(false); any(ApplicationId.class))).thenReturn(false);
ClientRMService rmService = new ClientRMService(rmContext, scheduler, ClientRMService rmService = new ClientRMService(rmContext, yarnScheduler,
appManager, appAclsManager, queueAclsManager, null); appManager, appAclsManager, queueAclsManager, null);
rmService.init(new Configuration()); rmService.init(new Configuration());

View File

@ -18,8 +18,16 @@
package org.apache.hadoop.yarn.server.resourcemanager; package org.apache.hadoop.yarn.server.resourcemanager;
import static org.apache.hadoop.yarn.api.records.ContainerUpdateType.INCREASE_RESOURCE; import org.apache.hadoop.yarn.api.records.NodeId;
import static org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils.RESOURCE_OUTSIDE_ALLOWED_RANGE; import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
@ -29,97 +37,7 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.UpdateContainerError;
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
import org.apache.hadoop.yarn.api.records.impl.pb.UpdateContainerRequestPBImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
public class TestRMServerUtils { public class TestRMServerUtils {
@Test
public void testValidateAndSplitUpdateResourceRequests() {
List<UpdateContainerRequest> updateRequests = new ArrayList<>();
int containerVersion = 10;
int resource = 10;
Resource maxAllocation = Resource.newInstance(resource, resource);
UpdateContainerRequestPBImpl updateContainerRequestPBFail =
new UpdateContainerRequestPBImpl();
updateContainerRequestPBFail.setContainerVersion(containerVersion);
updateContainerRequestPBFail
.setCapability(Resource.newInstance(resource + 1, resource + 1));
updateContainerRequestPBFail
.setContainerId(Mockito.mock(ContainerId.class));
ContainerId containerIdOk = Mockito.mock(ContainerId.class);
Resource capabilityOk = Resource.newInstance(resource - 1, resource - 1);
UpdateContainerRequestPBImpl updateContainerRequestPBOk =
new UpdateContainerRequestPBImpl();
updateContainerRequestPBOk.setContainerVersion(containerVersion);
updateContainerRequestPBOk.setCapability(capabilityOk);
updateContainerRequestPBOk.setContainerUpdateType(INCREASE_RESOURCE);
updateContainerRequestPBOk.setContainerId(containerIdOk);
updateRequests.add(updateContainerRequestPBOk);
updateRequests.add(updateContainerRequestPBFail);
Dispatcher dispatcher = Mockito.mock(Dispatcher.class);
RMContext rmContext = Mockito.mock(RMContext.class);
ResourceScheduler scheduler = Mockito.mock(ResourceScheduler.class);
Mockito.when(rmContext.getScheduler()).thenReturn(scheduler);
Mockito.when(rmContext.getDispatcher()).thenReturn(dispatcher);
RMContainer rmContainer = Mockito.mock(RMContainer.class);
Mockito.when(scheduler.getRMContainer(Mockito.any()))
.thenReturn(rmContainer);
Container container = Mockito.mock(Container.class);
Mockito.when(container.getVersion()).thenReturn(containerVersion);
Mockito.when(rmContainer.getContainer()).thenReturn(container);
Mockito.when(scheduler.getNormalizedResource(capabilityOk, maxAllocation))
.thenReturn(capabilityOk);
AllocateRequest allocateRequest =
AllocateRequest.newInstance(1, 0.5f, new ArrayList<ResourceRequest>(),
new ArrayList<ContainerId>(), updateRequests, null);
List<UpdateContainerError> updateErrors = new ArrayList<>();
ContainerUpdates containerUpdates =
RMServerUtils.validateAndSplitUpdateResourceRequests(rmContext,
allocateRequest, maxAllocation, updateErrors);
Assert.assertEquals(1, updateErrors.size());
Assert.assertEquals(resource + 1, updateErrors.get(0)
.getUpdateContainerRequest().getCapability().getMemorySize());
Assert.assertEquals(resource + 1, updateErrors.get(0)
.getUpdateContainerRequest().getCapability().getVirtualCores());
Assert.assertEquals(RESOURCE_OUTSIDE_ALLOWED_RANGE,
updateErrors.get(0).getReason());
Assert.assertEquals(1, containerUpdates.getIncreaseRequests().size());
UpdateContainerRequest increaseRequest =
containerUpdates.getIncreaseRequests().get(0);
Assert.assertEquals(capabilityOk.getVirtualCores(),
increaseRequest.getCapability().getVirtualCores());
Assert.assertEquals(capabilityOk.getMemorySize(),
increaseRequest.getCapability().getMemorySize());
Assert.assertEquals(containerIdOk, increaseRequest.getContainerId());
}
@Test @Test
public void testGetApplicableNodeCountForAMLocality() throws Exception { public void testGetApplicableNodeCountForAMLocality() throws Exception {
List<NodeId> rack1Nodes = new ArrayList<>(); List<NodeId> rack1Nodes = new ArrayList<>();

View File

@ -99,7 +99,6 @@ import org.junit.Test;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import org.junit.rules.ExpectedException; import org.junit.rules.ExpectedException;
import org.mockito.Mockito;
public class TestSchedulerUtils { public class TestSchedulerUtils {
@ -166,12 +165,12 @@ public class TestSchedulerUtils {
@Test (timeout = 30000) @Test (timeout = 30000)
public void testNormalizeRequest() { public void testNormalizeRequest() {
ResourceCalculator resourceCalculator = new DefaultResourceCalculator(); ResourceCalculator resourceCalculator = new DefaultResourceCalculator();
final int minMemory = 1024; final int minMemory = 1024;
final int maxMemory = 8192; final int maxMemory = 8192;
Resource minResource = Resources.createResource(minMemory, 0); Resource minResource = Resources.createResource(minMemory, 0);
Resource maxResource = Resources.createResource(maxMemory, 0); Resource maxResource = Resources.createResource(maxMemory, 0);
ResourceRequest ask = new ResourceRequestPBImpl(); ResourceRequest ask = new ResourceRequestPBImpl();
// case negative memory // case negative memory
@ -231,11 +230,11 @@ public class TestSchedulerUtils {
@Test (timeout = 30000) @Test (timeout = 30000)
public void testNormalizeRequestWithDominantResourceCalculator() { public void testNormalizeRequestWithDominantResourceCalculator() {
ResourceCalculator resourceCalculator = new DominantResourceCalculator(); ResourceCalculator resourceCalculator = new DominantResourceCalculator();
Resource minResource = Resources.createResource(1024, 1); Resource minResource = Resources.createResource(1024, 1);
Resource maxResource = Resources.createResource(10240, 10); Resource maxResource = Resources.createResource(10240, 10);
Resource clusterResource = Resources.createResource(10 * 1024, 10); Resource clusterResource = Resources.createResource(10 * 1024, 10);
ResourceRequest ask = new ResourceRequestPBImpl(); ResourceRequest ask = new ResourceRequestPBImpl();
// case negative memory/vcores // case negative memory/vcores
@ -260,12 +259,12 @@ public class TestSchedulerUtils {
assertEquals(1, ask.getCapability().getVirtualCores()); assertEquals(1, ask.getCapability().getVirtualCores());
assertEquals(2048, ask.getCapability().getMemorySize()); assertEquals(2048, ask.getCapability().getMemorySize());
} }
@Test(timeout = 30000) @Test(timeout = 30000)
public void testValidateResourceRequestWithErrorLabelsPermission() public void testValidateResourceRequestWithErrorLabelsPermission()
throws IOException { throws IOException {
// mock queue and scheduler // mock queue and scheduler
ResourceScheduler scheduler = mock(ResourceScheduler.class); YarnScheduler scheduler = mock(YarnScheduler.class);
Set<String> queueAccessibleNodeLabels = Sets.newHashSet(); Set<String> queueAccessibleNodeLabels = Sets.newHashSet();
QueueInfo queueInfo = mock(QueueInfo.class); QueueInfo queueInfo = mock(QueueInfo.class);
when(queueInfo.getQueueName()).thenReturn("queue"); when(queueInfo.getQueueName()).thenReturn("queue");
@ -274,8 +273,6 @@ public class TestSchedulerUtils {
when(scheduler.getQueueInfo(any(String.class), anyBoolean(), anyBoolean())) when(scheduler.getQueueInfo(any(String.class), anyBoolean(), anyBoolean()))
.thenReturn(queueInfo); .thenReturn(queueInfo);
when(rmContext.getScheduler()).thenReturn(scheduler);
Resource maxResource = Resources.createResource( Resource maxResource = Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
@ -294,20 +291,20 @@ public class TestSchedulerUtils {
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1); mock(Priority.class), ResourceRequest.ANY, resource, 1);
resReq.setNodeLabelExpression("x"); resReq.setNodeLabelExpression("x");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
resReq.setNodeLabelExpression("y"); resReq.setNodeLabelExpression("y");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
resReq.setNodeLabelExpression(""); resReq.setNodeLabelExpression("");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
resReq.setNodeLabelExpression(" "); resReq.setNodeLabelExpression(" ");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
e.printStackTrace(); e.printStackTrace();
fail("Should be valid when request labels is a subset of queue labels"); fail("Should be valid when request labels is a subset of queue labels");
@ -315,7 +312,7 @@ public class TestSchedulerUtils {
rmContext.getNodeLabelManager().removeFromClusterNodeLabels( rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
Arrays.asList("x", "y")); Arrays.asList("x", "y"));
} }
// same as above, but cluster node labels don't contains label being // same as above, but cluster node labels don't contains label being
// requested. should fail // requested. should fail
try { try {
@ -328,13 +325,13 @@ public class TestSchedulerUtils {
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1); mock(Priority.class), ResourceRequest.ANY, resource, 1);
resReq.setNodeLabelExpression("x"); resReq.setNodeLabelExpression("x");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
fail("Should fail"); fail("Should fail");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
} }
// queue has labels, failed cases (when ask a label not included by queue) // queue has labels, failed cases (when ask a label not included by queue)
try { try {
// set queue accessible node labesl to [x, y] // set queue accessible node labesl to [x, y]
@ -343,22 +340,22 @@ public class TestSchedulerUtils {
rmContext.getNodeLabelManager().addToCluserNodeLabels( rmContext.getNodeLabelManager().addToCluserNodeLabels(
ImmutableSet.of(NodeLabel.newInstance("x"), ImmutableSet.of(NodeLabel.newInstance("x"),
NodeLabel.newInstance("y"))); NodeLabel.newInstance("y")));
Resource resource = Resources.createResource( Resource resource = Resources.createResource(
0, 0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1); mock(Priority.class), ResourceRequest.ANY, resource, 1);
resReq.setNodeLabelExpression("z"); resReq.setNodeLabelExpression("z");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
fail("Should fail"); fail("Should fail");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
} finally { } finally {
rmContext.getNodeLabelManager().removeFromClusterNodeLabels( rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
Arrays.asList("x", "y")); Arrays.asList("x", "y"));
} }
// we don't allow specify more than two node labels in a single expression // we don't allow specify more than two node labels in a single expression
// now // now
try { try {
@ -368,43 +365,43 @@ public class TestSchedulerUtils {
rmContext.getNodeLabelManager().addToCluserNodeLabels( rmContext.getNodeLabelManager().addToCluserNodeLabels(
ImmutableSet.of(NodeLabel.newInstance("x"), ImmutableSet.of(NodeLabel.newInstance("x"),
NodeLabel.newInstance("y"))); NodeLabel.newInstance("y")));
Resource resource = Resources.createResource( Resource resource = Resources.createResource(
0, 0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1); mock(Priority.class), ResourceRequest.ANY, resource, 1);
resReq.setNodeLabelExpression("x && y"); resReq.setNodeLabelExpression("x && y");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
fail("Should fail"); fail("Should fail");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
} finally { } finally {
rmContext.getNodeLabelManager().removeFromClusterNodeLabels( rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
Arrays.asList("x", "y")); Arrays.asList("x", "y"));
} }
// queue doesn't have label, succeed (when request no label) // queue doesn't have label, succeed (when request no label)
queueAccessibleNodeLabels.clear(); queueAccessibleNodeLabels.clear();
try { try {
// set queue accessible node labels to empty // set queue accessible node labels to empty
queueAccessibleNodeLabels.clear(); queueAccessibleNodeLabels.clear();
Resource resource = Resources.createResource( Resource resource = Resources.createResource(
0, 0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1); mock(Priority.class), ResourceRequest.ANY, resource, 1);
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
resReq.setNodeLabelExpression(""); resReq.setNodeLabelExpression("");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
resReq.setNodeLabelExpression(" "); resReq.setNodeLabelExpression(" ");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
e.printStackTrace(); e.printStackTrace();
fail("Should be valid when request labels is empty"); fail("Should be valid when request labels is empty");
@ -414,18 +411,18 @@ public class TestSchedulerUtils {
try { try {
// set queue accessible node labels to empty // set queue accessible node labels to empty
queueAccessibleNodeLabels.clear(); queueAccessibleNodeLabels.clear();
rmContext.getNodeLabelManager().addToCluserNodeLabels( rmContext.getNodeLabelManager().addToCluserNodeLabels(
ImmutableSet.of(NodeLabel.newInstance("x"))); ImmutableSet.of(NodeLabel.newInstance("x")));
Resource resource = Resources.createResource( Resource resource = Resources.createResource(
0, 0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1); mock(Priority.class), ResourceRequest.ANY, resource, 1);
resReq.setNodeLabelExpression("x"); resReq.setNodeLabelExpression("x");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
fail("Should fail"); fail("Should fail");
} catch (InvalidLabelResourceRequestException e) { } catch (InvalidLabelResourceRequestException e) {
invalidlabelexception=true; invalidlabelexception=true;
@ -441,27 +438,27 @@ public class TestSchedulerUtils {
// set queue accessible node labels to empty // set queue accessible node labels to empty
queueAccessibleNodeLabels.clear(); queueAccessibleNodeLabels.clear();
queueAccessibleNodeLabels.add(RMNodeLabelsManager.ANY); queueAccessibleNodeLabels.add(RMNodeLabelsManager.ANY);
rmContext.getNodeLabelManager().addToCluserNodeLabels( rmContext.getNodeLabelManager().addToCluserNodeLabels(
ImmutableSet.of(NodeLabel.newInstance("x"), ImmutableSet.of(NodeLabel.newInstance("x"),
NodeLabel.newInstance("y"), NodeLabel.newInstance("z"))); NodeLabel.newInstance("y"), NodeLabel.newInstance("z")));
Resource resource = Resources.createResource( Resource resource = Resources.createResource(
0, 0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1); mock(Priority.class), ResourceRequest.ANY, resource, 1);
resReq.setNodeLabelExpression("x"); resReq.setNodeLabelExpression("x");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
resReq.setNodeLabelExpression("y"); resReq.setNodeLabelExpression("y");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
resReq.setNodeLabelExpression("z"); resReq.setNodeLabelExpression("z");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
e.printStackTrace(); e.printStackTrace();
fail("Should be valid when queue can access any labels"); fail("Should be valid when queue can access any labels");
@ -469,25 +466,25 @@ public class TestSchedulerUtils {
rmContext.getNodeLabelManager().removeFromClusterNodeLabels( rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
Arrays.asList("x", "y", "z")); Arrays.asList("x", "y", "z"));
} }
// same as above, but cluster node labels don't contains label, should fail // same as above, but cluster node labels don't contains label, should fail
try { try {
// set queue accessible node labels to empty // set queue accessible node labels to empty
queueAccessibleNodeLabels.clear(); queueAccessibleNodeLabels.clear();
queueAccessibleNodeLabels.add(RMNodeLabelsManager.ANY); queueAccessibleNodeLabels.add(RMNodeLabelsManager.ANY);
Resource resource = Resources.createResource( Resource resource = Resources.createResource(
0, 0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1); mock(Priority.class), ResourceRequest.ANY, resource, 1);
resReq.setNodeLabelExpression("x"); resReq.setNodeLabelExpression("x");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
fail("Should fail"); fail("Should fail");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
} }
// we don't allow resource name other than ANY and specify label // we don't allow resource name other than ANY and specify label
try { try {
// set queue accessible node labesl to [x, y] // set queue accessible node labesl to [x, y]
@ -496,22 +493,22 @@ public class TestSchedulerUtils {
rmContext.getNodeLabelManager().addToCluserNodeLabels( rmContext.getNodeLabelManager().addToCluserNodeLabels(
ImmutableSet.of(NodeLabel.newInstance("x"), ImmutableSet.of(NodeLabel.newInstance("x"),
NodeLabel.newInstance("y"))); NodeLabel.newInstance("y")));
Resource resource = Resources.createResource( Resource resource = Resources.createResource(
0, 0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), "rack", resource, 1); mock(Priority.class), "rack", resource, 1);
resReq.setNodeLabelExpression("x"); resReq.setNodeLabelExpression("x");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
fail("Should fail"); fail("Should fail");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
} finally { } finally {
rmContext.getNodeLabelManager().removeFromClusterNodeLabels( rmContext.getNodeLabelManager().removeFromClusterNodeLabels(
Arrays.asList("x", "y")); Arrays.asList("x", "y"));
} }
// we don't allow resource name other than ANY and specify label even if // we don't allow resource name other than ANY and specify label even if
// queue has accessible label = * // queue has accessible label = *
try { try {
@ -521,15 +518,15 @@ public class TestSchedulerUtils {
.asList(CommonNodeLabelsManager.ANY)); .asList(CommonNodeLabelsManager.ANY));
rmContext.getNodeLabelManager().addToCluserNodeLabels( rmContext.getNodeLabelManager().addToCluserNodeLabels(
ImmutableSet.of(NodeLabel.newInstance("x"))); ImmutableSet.of(NodeLabel.newInstance("x")));
Resource resource = Resources.createResource( Resource resource = Resources.createResource(
0, 0,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), "rack", resource, 1); mock(Priority.class), "rack", resource, 1);
resReq.setNodeLabelExpression("x"); resReq.setNodeLabelExpression("x");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
fail("Should fail"); fail("Should fail");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
} finally { } finally {
@ -541,8 +538,8 @@ public class TestSchedulerUtils {
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq1 = BuilderUtils ResourceRequest resReq1 = BuilderUtils
.newResourceRequest(mock(Priority.class), "*", resource, 1, "x"); .newResourceRequest(mock(Priority.class), "*", resource, 1, "x");
normalizeAndvalidateRequest(resReq1, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq1, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
fail("Should fail"); fail("Should fail");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
assertEquals("Invalid label resource request, cluster do not contain , " assertEquals("Invalid label resource request, cluster do not contain , "
@ -556,8 +553,8 @@ public class TestSchedulerUtils {
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq1 = BuilderUtils ResourceRequest resReq1 = BuilderUtils
.newResourceRequest(mock(Priority.class), "*", resource, 1, "x"); .newResourceRequest(mock(Priority.class), "*", resource, 1, "x");
normalizeAndvalidateRequest(resReq1, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq1, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
Assert.assertEquals(RMNodeLabelsManager.NO_LABEL, Assert.assertEquals(RMNodeLabelsManager.NO_LABEL,
resReq1.getNodeLabelExpression()); resReq1.getNodeLabelExpression());
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
@ -567,21 +564,14 @@ public class TestSchedulerUtils {
} }
@Test (timeout = 30000) @Test (timeout = 30000)
public void testValidateResourceRequest() throws IOException { public void testValidateResourceRequest() {
ResourceScheduler mockScheduler = mock(ResourceScheduler.class); YarnScheduler mockScheduler = mock(YarnScheduler.class);
QueueInfo queueInfo = mock(QueueInfo.class);
when(queueInfo.getQueueName()).thenReturn("queue");
Resource maxResource = Resource maxResource =
Resources.createResource( Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
when(rmContext.getScheduler()).thenReturn(mockScheduler);
when(mockScheduler.getQueueInfo(Mockito.anyString(), Mockito.anyBoolean(),
Mockito.anyBoolean())).thenReturn(queueInfo);
// zero memory // zero memory
try { try {
Resource resource = Resource resource =
@ -590,8 +580,8 @@ public class TestSchedulerUtils {
ResourceRequest resReq = ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class), BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1); ResourceRequest.ANY, resource, 1);
normalizeAndvalidateRequest(resReq, null, SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
mockScheduler, rmContext, maxResource); mockScheduler, rmContext);
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
fail("Zero memory should be accepted"); fail("Zero memory should be accepted");
} }
@ -604,8 +594,8 @@ public class TestSchedulerUtils {
ResourceRequest resReq = ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class), BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1); ResourceRequest.ANY, resource, 1);
normalizeAndvalidateRequest(resReq, null, SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
mockScheduler, rmContext, maxResource); mockScheduler, rmContext);
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
fail("Zero vcores should be accepted"); fail("Zero vcores should be accepted");
} }
@ -619,8 +609,8 @@ public class TestSchedulerUtils {
ResourceRequest resReq = ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class), BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1); ResourceRequest.ANY, resource, 1);
normalizeAndvalidateRequest(resReq, null, SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
mockScheduler, rmContext, maxResource); mockScheduler, rmContext);
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
fail("Max memory should be accepted"); fail("Max memory should be accepted");
} }
@ -634,8 +624,8 @@ public class TestSchedulerUtils {
ResourceRequest resReq = ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class), BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1); ResourceRequest.ANY, resource, 1);
normalizeAndvalidateRequest(resReq, null, SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
mockScheduler, rmContext, maxResource); mockScheduler, rmContext);
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
fail("Max vcores should not be accepted"); fail("Max vcores should not be accepted");
} }
@ -648,8 +638,8 @@ public class TestSchedulerUtils {
ResourceRequest resReq = ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class), BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1); ResourceRequest.ANY, resource, 1);
normalizeAndvalidateRequest(resReq, null, SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
mockScheduler, rmContext, maxResource); mockScheduler, rmContext);
fail("Negative memory should not be accepted"); fail("Negative memory should not be accepted");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
// expected // expected
@ -663,8 +653,8 @@ public class TestSchedulerUtils {
ResourceRequest resReq = ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class), BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1); ResourceRequest.ANY, resource, 1);
normalizeAndvalidateRequest(resReq, null, SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
mockScheduler, rmContext, maxResource); mockScheduler, rmContext);
fail("Negative vcores should not be accepted"); fail("Negative vcores should not be accepted");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
// expected // expected
@ -679,8 +669,8 @@ public class TestSchedulerUtils {
ResourceRequest resReq = ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class), BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1); ResourceRequest.ANY, resource, 1);
normalizeAndvalidateRequest(resReq, null, SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
mockScheduler, rmContext, maxResource); mockScheduler, rmContext);
fail("More than max memory should not be accepted"); fail("More than max memory should not be accepted");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
// expected // expected
@ -694,14 +684,14 @@ public class TestSchedulerUtils {
ResourceRequest resReq = ResourceRequest resReq =
BuilderUtils.newResourceRequest(mock(Priority.class), BuilderUtils.newResourceRequest(mock(Priority.class),
ResourceRequest.ANY, resource, 1); ResourceRequest.ANY, resource, 1);
normalizeAndvalidateRequest(resReq, null, SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
mockScheduler, rmContext, maxResource); mockScheduler, rmContext);
fail("More than max vcores should not be accepted"); fail("More than max vcores should not be accepted");
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
// expected // expected
} }
} }
@Test @Test
public void testValidateResourceBlacklistRequest() throws Exception { public void testValidateResourceBlacklistRequest() throws Exception {
@ -763,7 +753,7 @@ public class TestSchedulerUtils {
} }
rm.stop(); rm.stop();
Assert.assertTrue( Assert.assertTrue(
"Didn't not catch InvalidResourceBlacklistRequestException", error); "Didn't not catch InvalidResourceBlacklistRequestException", error);
} }
@ -803,12 +793,12 @@ public class TestSchedulerUtils {
ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x"); ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x");
Assert.assertEquals(ContainerExitStatus.PREEMPTED, cd.getExitStatus()); Assert.assertEquals(ContainerExitStatus.PREEMPTED, cd.getExitStatus());
} }
@Test (timeout = 30000) @Test (timeout = 30000)
public void testNormalizeNodeLabelExpression() public void testNormalizeNodeLabelExpression()
throws IOException { throws IOException {
// mock queue and scheduler // mock queue and scheduler
ResourceScheduler scheduler = mock(ResourceScheduler.class); YarnScheduler scheduler = mock(YarnScheduler.class);
Set<String> queueAccessibleNodeLabels = Sets.newHashSet(); Set<String> queueAccessibleNodeLabels = Sets.newHashSet();
QueueInfo queueInfo = mock(QueueInfo.class); QueueInfo queueInfo = mock(QueueInfo.class);
when(queueInfo.getQueueName()).thenReturn("queue"); when(queueInfo.getQueueName()).thenReturn("queue");
@ -816,13 +806,11 @@ public class TestSchedulerUtils {
when(queueInfo.getDefaultNodeLabelExpression()).thenReturn(" x "); when(queueInfo.getDefaultNodeLabelExpression()).thenReturn(" x ");
when(scheduler.getQueueInfo(any(String.class), anyBoolean(), anyBoolean())) when(scheduler.getQueueInfo(any(String.class), anyBoolean(), anyBoolean()))
.thenReturn(queueInfo); .thenReturn(queueInfo);
Resource maxResource = Resources.createResource( Resource maxResource = Resources.createResource(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
when(rmContext.getScheduler()).thenReturn(scheduler);
// queue has labels, success cases // queue has labels, success cases
try { try {
// set queue accessible node labels to [x, y] // set queue accessible node labels to [x, y]
@ -836,13 +824,13 @@ public class TestSchedulerUtils {
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
ResourceRequest resReq = BuilderUtils.newResourceRequest( ResourceRequest resReq = BuilderUtils.newResourceRequest(
mock(Priority.class), ResourceRequest.ANY, resource, 1); mock(Priority.class), ResourceRequest.ANY, resource, 1);
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
Assert.assertEquals("x", resReq.getNodeLabelExpression()); Assert.assertEquals("x", resReq.getNodeLabelExpression());
resReq.setNodeLabelExpression(" y "); resReq.setNodeLabelExpression(" y ");
normalizeAndvalidateRequest(resReq, "queue", SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
scheduler, rmContext, maxResource); scheduler, rmContext);
Assert.assertEquals("y", resReq.getNodeLabelExpression()); Assert.assertEquals("y", resReq.getNodeLabelExpression());
} catch (InvalidResourceRequestException e) { } catch (InvalidResourceRequestException e) {
e.printStackTrace(); e.printStackTrace();
@ -1019,7 +1007,7 @@ public class TestSchedulerUtils {
Assert.assertNull(applications.get(appId)); Assert.assertNull(applications.get(appId));
return app; return app;
} }
private static RMContext getMockRMContext() { private static RMContext getMockRMContext() {
RMContext rmContext = mock(RMContext.class); RMContext rmContext = mock(RMContext.class);
RMNodeLabelsManager nlm = new NullRMNodeLabelsManager(); RMNodeLabelsManager nlm = new NullRMNodeLabelsManager();
@ -1031,14 +1019,6 @@ public class TestSchedulerUtils {
return rmContext; return rmContext;
} }
private static void normalizeAndvalidateRequest(ResourceRequest resReq,
String queueName, YarnScheduler scheduler, RMContext rmContext,
Resource maxAllocation)
throws InvalidResourceRequestException {
SchedulerUtils.normalizeAndValidateRequest(resReq, maxAllocation, queueName,
scheduler, rmContext, null);
}
private static class InvalidResourceRequestExceptionMessageGenerator { private static class InvalidResourceRequestExceptionMessageGenerator {
private StringBuilder sb; private StringBuilder sb;

View File

@ -18,8 +18,6 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.MAXIMUM_ALLOCATION_MB;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.MAXIMUM_ALLOCATION_VCORES;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
@ -840,41 +838,6 @@ public class TestCapacityScheduler extends CapacitySchedulerTestBase {
assertEquals(CapacitySchedulerConfiguration.MAXIMUM_CAPACITY_VALUE,conf.getNonLabeledQueueMaximumCapacity(A),delta); assertEquals(CapacitySchedulerConfiguration.MAXIMUM_CAPACITY_VALUE,conf.getNonLabeledQueueMaximumCapacity(A),delta);
} }
@Test
public void testQueueMaximumAllocations() {
CapacityScheduler scheduler = new CapacityScheduler();
scheduler.setConf(new YarnConfiguration());
scheduler.setRMContext(resourceManager.getRMContext());
CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
setupQueueConfiguration(conf);
conf.set(CapacitySchedulerConfiguration.getQueuePrefix(A1)
+ MAXIMUM_ALLOCATION_MB, "1024");
conf.set(CapacitySchedulerConfiguration.getQueuePrefix(A1)
+ MAXIMUM_ALLOCATION_VCORES, "1");
scheduler.init(conf);
scheduler.start();
Resource maxAllocationForQueue =
scheduler.getMaximumResourceCapability("a1");
Resource maxAllocation1 = scheduler.getMaximumResourceCapability("");
Resource maxAllocation2 = scheduler.getMaximumResourceCapability(null);
Resource maxAllocation3 = scheduler.getMaximumResourceCapability();
Assert.assertEquals(maxAllocation1, maxAllocation2);
Assert.assertEquals(maxAllocation1, maxAllocation3);
Assert.assertEquals(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
maxAllocation1.getMemorySize());
Assert.assertEquals(
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
maxAllocation1.getVirtualCores());
Assert.assertEquals(1024, maxAllocationForQueue.getMemorySize());
Assert.assertEquals(1, maxAllocationForQueue.getVirtualCores());
}
@Test @Test
public void testRefreshQueues() throws Exception { public void testRefreshQueues() throws Exception {
@ -4049,7 +4012,7 @@ public class TestCapacityScheduler extends CapacitySchedulerTestBase {
private void setMaxAllocMb(CapacitySchedulerConfiguration conf, private void setMaxAllocMb(CapacitySchedulerConfiguration conf,
String queueName, int maxAllocMb) { String queueName, int maxAllocMb) {
String propName = CapacitySchedulerConfiguration.getQueuePrefix(queueName) String propName = CapacitySchedulerConfiguration.getQueuePrefix(queueName)
+ MAXIMUM_ALLOCATION_MB; + CapacitySchedulerConfiguration.MAXIMUM_ALLOCATION_MB;
conf.setInt(propName, maxAllocMb); conf.setInt(propName, maxAllocMb);
} }

View File

@ -75,7 +75,6 @@ public class FairSchedulerTestBase {
public static final float TEST_RESERVATION_THRESHOLD = 0.09f; public static final float TEST_RESERVATION_THRESHOLD = 0.09f;
private static final int SLEEP_DURATION = 10; private static final int SLEEP_DURATION = 10;
private static final int SLEEP_RETRIES = 1000; private static final int SLEEP_RETRIES = 1000;
protected static final int RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE = 10240;
final static ContainerUpdates NULL_UPDATE_REQUESTS = final static ContainerUpdates NULL_UPDATE_REQUESTS =
new ContainerUpdates(); new ContainerUpdates();
@ -92,8 +91,7 @@ public class FairSchedulerTestBase {
conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0); conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0);
conf.setInt(FairSchedulerConfiguration.RM_SCHEDULER_INCREMENT_ALLOCATION_MB, conf.setInt(FairSchedulerConfiguration.RM_SCHEDULER_INCREMENT_ALLOCATION_MB,
1024); 1024);
conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 10240);
RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE);
conf.setBoolean(FairSchedulerConfiguration.ASSIGN_MULTIPLE, false); conf.setBoolean(FairSchedulerConfiguration.ASSIGN_MULTIPLE, false);
conf.setLong(FairSchedulerConfiguration.UPDATE_INTERVAL_MS, 10); conf.setLong(FairSchedulerConfiguration.UPDATE_INTERVAL_MS, 10);
conf.setFloat(FairSchedulerConfiguration.PREEMPTION_THRESHOLD, 0f); conf.setFloat(FairSchedulerConfiguration.PREEMPTION_THRESHOLD, 0f);

View File

@ -24,7 +24,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.yarn.api.records.QueueACL; import org.apache.hadoop.yarn.api.records.QueueACL;
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.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.QueuePlacementRule.NestedUserQueue;
@ -32,9 +31,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.allocationfi
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.resource.ResourceUtils;
import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.resource.Resources;
import org.apache.hadoop.yarn.util.resource.TestResourceUtils;
import org.junit.Test; import org.junit.Test;
import java.io.File; import java.io.File;
import java.io.FileOutputStream; import java.io.FileOutputStream;
@ -45,7 +42,6 @@ import java.io.PrintWriter;
import java.net.URISyntaxException; import java.net.URISyntaxException;
import java.net.URL; import java.net.URL;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.List; import java.util.List;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
@ -205,7 +201,7 @@ public class TestAllocationFileLoaderService {
@Test @Test
public void testAllocationFileParsing() throws Exception { public void testAllocationFileParsing() throws Exception {
Configuration conf = new YarnConfiguration(); 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();
@ -248,7 +244,6 @@ public class TestAllocationFileLoaderService {
.fairSharePreemptionTimeout(120) .fairSharePreemptionTimeout(120)
.minSharePreemptionTimeout(50) .minSharePreemptionTimeout(50)
.fairSharePreemptionThreshold(0.6) .fairSharePreemptionThreshold(0.6)
.maxContainerAllocation("512mb,16vcores")
// Create hierarchical queues G,H, with different min/fair // Create hierarchical queues G,H, with different min/fair
// share preemption timeouts and preemption thresholds. // share preemption timeouts and preemption thresholds.
// Also add a child default to make sure it doesn't impact queue H. // Also add a child default to make sure it doesn't impact queue H.
@ -256,7 +251,6 @@ public class TestAllocationFileLoaderService {
.fairSharePreemptionTimeout(180) .fairSharePreemptionTimeout(180)
.minSharePreemptionTimeout(40) .minSharePreemptionTimeout(40)
.fairSharePreemptionThreshold(0.7) .fairSharePreemptionThreshold(0.7)
.maxContainerAllocation("1024mb,8vcores")
.buildSubQueue() .buildSubQueue()
.buildQueue() .buildQueue()
// Set default limit of apps per queue to 15 // Set default limit of apps per queue to 15
@ -382,28 +376,6 @@ public class TestAllocationFileLoaderService {
assertEquals("alice,bob admins", queueConf.getQueueAcl("root.queueC", assertEquals("alice,bob admins", queueConf.getQueueAcl("root.queueC",
QueueACL.SUBMIT_APPLICATIONS).getAclString()); QueueACL.SUBMIT_APPLICATIONS).getAclString());
Resource expectedResourceWithCustomType = Resources.createResource(512, 16);
assertEquals(Resources.unbounded(),
queueConf.getQueueMaxContainerAllocation(
"root." + YarnConfiguration.DEFAULT_QUEUE_NAME));
assertEquals(Resources.unbounded(),
queueConf.getQueueMaxContainerAllocation("root.queueA"));
assertEquals(Resources.unbounded(),
queueConf.getQueueMaxContainerAllocation("root.queueB"));
assertEquals(Resources.unbounded(),
queueConf.getQueueMaxContainerAllocation("root.queueC"));
assertEquals(Resources.unbounded(),
queueConf.getQueueMaxContainerAllocation("root.queueD"));
assertEquals(Resources.unbounded(),
queueConf.getQueueMaxContainerAllocation("root.queueE"));
assertEquals(Resources.unbounded(),
queueConf.getQueueMaxContainerAllocation("root.queueF"));
assertEquals(expectedResourceWithCustomType,
queueConf.getQueueMaxContainerAllocation("root.queueG"));
assertEquals(Resources.createResource(1024, 8),
queueConf.getQueueMaxContainerAllocation("root.queueG.queueH"));
assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root")); assertEquals(120000, queueConf.getMinSharePreemptionTimeout("root"));
assertEquals(-1, queueConf.getMinSharePreemptionTimeout("root." + assertEquals(-1, queueConf.getMinSharePreemptionTimeout("root." +
YarnConfiguration.DEFAULT_QUEUE_NAME)); YarnConfiguration.DEFAULT_QUEUE_NAME));

View File

@ -1,170 +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.File;
import java.io.FileWriter;
import java.io.IOException;
import java.io.PrintWriter;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Test Application master service using Fair scheduler.
*/
public class TestApplicationMasterServiceWithFS {
private static final Log LOG =
LogFactory.getLog(TestApplicationMasterServiceWithFS.class);
private static final int GB = 1024;
private static final int MEMORY_ALLOCATION = 3 * GB;
private static final String TEST_FOLDER = "test-queues";
private AllocateResponse allocateResponse;
private static YarnConfiguration configuration;
@BeforeClass
public static void setup() throws IOException {
String allocFile =
GenericTestUtils.getTestDir(TEST_FOLDER).getAbsolutePath();
configuration = new YarnConfiguration();
configuration.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class,
ResourceScheduler.class);
configuration.set(FairSchedulerConfiguration.ALLOCATION_FILE, allocFile);
PrintWriter out = new PrintWriter(new FileWriter(allocFile));
out.println("<?xml version=\"1.0\"?>");
out.println("<allocations>");
out.println(" <queue name=\"queueA\">");
out.println(
" <maxContainerAllocation>2048 mb 1 vcores</maxContainerAllocation>");
out.println(" </queue>");
out.println(" <queue name=\"queueB\">");
out.println(
" <maxContainerAllocation>3072 mb 1 vcores</maxContainerAllocation>");
out.println(" </queue>");
out.println(" <queue name=\"queueC\">");
out.println(" </queue>");
out.println("</allocations>");
out.close();
}
@AfterClass
public static void teardown(){
File allocFile = GenericTestUtils.getTestDir(TEST_FOLDER);
allocFile.delete();
}
@Test(timeout = 3000000)
public void testQueueLevelContainerAllocationFail() throws Exception {
MockRM rm = new MockRM(configuration);
rm.start();
// Register node1
MockNM nm1 = rm.registerNode("127.0.0.1:1234", 6 * GB);
// Submit an application
RMApp app1 = rm.submitApp(2 * GB, "queueA");
// kick the scheduling
nm1.nodeHeartbeat(true);
RMAppAttempt attempt1 = app1.getCurrentAppAttempt();
MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
am1.registerAppAttempt();
am1.addRequests(new String[] { "127.0.0.1" }, MEMORY_ALLOCATION, 1, 1);
try {
allocateResponse = am1.schedule(); // send the request
Assert.fail();
} catch (Exception e) {
Assert.assertTrue(e instanceof InvalidResourceRequestException);
} finally {
rm.stop();
}
}
@Test(timeout = 3000000)
public void testQueueLevelContainerAllocationSuccess() throws Exception {
testFairSchedulerContainerAllocationSuccess("queueB");
}
@Test(timeout = 3000000)
public void testSchedulerLevelContainerAllocationSuccess() throws Exception {
testFairSchedulerContainerAllocationSuccess("queueC");
}
private void testFairSchedulerContainerAllocationSuccess(String queueName)
throws Exception {
MockRM rm = new MockRM(configuration);
rm.start();
// Register node1
MockNM nm1 = rm.registerNode("127.0.0.1:1234", 6 * GB);
// Submit an application
RMApp app1 = rm.submitApp(2 * GB, queueName);
// kick the scheduling
nm1.nodeHeartbeat(true);
RMAppAttempt attempt1 = app1.getCurrentAppAttempt();
MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
am1.registerAppAttempt();
am1.addRequests(new String[] { "127.0.0.1" }, MEMORY_ALLOCATION, 1, 1);
allocateResponse = am1.schedule(); // send the request
((FairScheduler) rm.getResourceScheduler()).update();
// kick the scheduler
nm1.nodeHeartbeat(true);
GenericTestUtils.waitFor(() -> {
LOG.info("Waiting for containers to be created for app 1");
try {
allocateResponse = am1.schedule();
} catch (Exception e) {
Assert.fail("Allocation should be successful");
}
return allocateResponse.getAllocatedContainers().size() > 0;
}, 1000, 10000);
Container allocatedContainer =
allocateResponse.getAllocatedContainers().get(0);
Assert.assertEquals(MEMORY_ALLOCATION,
allocatedContainer.getResource().getMemorySize());
Assert.assertEquals(1, allocatedContainer.getResource().getVirtualCores());
rm.stop();
}
}

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotEquals;
@ -194,6 +193,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
} }
} }
// TESTS
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
@Test(timeout=2000) @Test(timeout=2000)
public void testLoadConfigurationOnInitialize() throws IOException { public void testLoadConfigurationOnInitialize() throws IOException {
@ -332,111 +333,6 @@ public class TestFairScheduler extends FairSchedulerTestBase {
} }
} }
@Test
public void testQueueMaximumCapacityAllocations() throws IOException {
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
int tooHighQueueAllocation = RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE +1;
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
out.println("<?xml version=\"1.0\"?>");
out.println("<allocations>");
out.println(" <queue name=\"queueA\">");
out.println(
" <maxContainerAllocation>512 mb 1 vcores</maxContainerAllocation>");
out.println(" </queue>");
out.println(" <queue name=\"queueB\">");
out.println(" </queue>");
out.println(" <queue name=\"queueC\">");
out.println(
" <maxContainerAllocation>2048 mb 3 vcores</maxContainerAllocation>");
out.println(" <queue name=\"queueD\">");
out.println(" </queue>");
out.println(" </queue>");
out.println(" <queue name=\"queueE\">");
out.println(" <maxContainerAllocation>" + tooHighQueueAllocation
+ " mb 1 vcores</maxContainerAllocation>");
out.println(" </queue>");
out.println("</allocations>");
out.close();
scheduler.init(conf);
Assert.assertEquals(1, scheduler.getMaximumResourceCapability("root.queueA")
.getVirtualCores());
Assert.assertEquals(512,
scheduler.getMaximumResourceCapability("root.queueA").getMemorySize());
Assert.assertEquals(DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
scheduler.getMaximumResourceCapability("root.queueB")
.getVirtualCores());
Assert.assertEquals(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE,
scheduler.getMaximumResourceCapability("root.queueB").getMemorySize());
Assert.assertEquals(3, scheduler.getMaximumResourceCapability("root.queueC")
.getVirtualCores());
Assert.assertEquals(2048,
scheduler.getMaximumResourceCapability("root.queueC").getMemorySize());
Assert.assertEquals(3, scheduler
.getMaximumResourceCapability("root.queueC.queueD").getVirtualCores());
Assert.assertEquals(2048, scheduler
.getMaximumResourceCapability("root.queueC.queueD").getMemorySize());
Assert.assertEquals(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, scheduler
.getMaximumResourceCapability("root.queueE").getMemorySize());
}
@Test
public void testNormalizationUsingQueueMaximumAllocation()
throws IOException {
int queueMaxAllocation = 4096;
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(" <queue name=\"queueA\">");
out.println(" <maxContainerAllocation>" + queueMaxAllocation
+ " mb 1 vcores" + "</maxContainerAllocation>");
out.println(" </queue>");
out.println(" <queue name=\"queueB\">");
out.println(" </queue>");
out.println("</allocations>");
out.close();
scheduler.init(conf);
scheduler.start();
scheduler.reinitialize(conf, resourceManager.getRMContext());
allocateAppAttempt("root.queueA", 1, queueMaxAllocation + 1024);
allocateAppAttempt("root.queueB", 2,
RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE + 1024);
scheduler.update();
FSQueue queueToCheckA = scheduler.getQueueManager().getQueue("root.queueA");
FSQueue queueToCheckB = scheduler.getQueueManager().getQueue("root.queueB");
assertEquals(queueMaxAllocation, queueToCheckA.getDemand().getMemorySize());
assertEquals(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE,
queueToCheckB.getDemand().getMemorySize());
}
private void allocateAppAttempt(String queueName, int id, int memorySize) {
ApplicationAttemptId id11 = createAppAttemptId(id, id);
createMockRMApp(id11);
scheduler.addApplication(id11.getApplicationId(), queueName, "user1",
false);
scheduler.addApplicationAttempt(id11, false, false);
List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
ResourceRequest request1 =
createResourceRequest(memorySize, ResourceRequest.ANY, 1, 1, true);
ask1.add(request1);
scheduler.allocate(id11, ask1, null, new ArrayList<ContainerId>(), null,
null, NULL_UPDATE_REQUESTS);
}
/** /**
* Test fair shares when max resources are set but are too high to impact * Test fair shares when max resources are set but are too high to impact
* the shares. * the shares.
@ -1415,9 +1311,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
// New node satisfies resource request // New node satisfies resource request
scheduler.update(); scheduler.update();
scheduler.handle(new NodeUpdateSchedulerEvent(node4)); scheduler.handle(new NodeUpdateSchedulerEvent(node4));
assertEquals(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, assertEquals(10240, scheduler.getQueueManager().getQueue("queue1").
scheduler.getQueueManager().getQueue("queue1").getResourceUsage() getResourceUsage().getMemorySize());
.getMemorySize());
scheduler.handle(new NodeUpdateSchedulerEvent(node1)); scheduler.handle(new NodeUpdateSchedulerEvent(node1));
scheduler.handle(new NodeUpdateSchedulerEvent(node2)); scheduler.handle(new NodeUpdateSchedulerEvent(node2));
@ -4199,12 +4094,12 @@ public class TestFairScheduler extends FairSchedulerTestBase {
scheduler.start(); scheduler.start();
scheduler.reinitialize(conf, resourceManager.getRMContext()); scheduler.reinitialize(conf, resourceManager.getRMContext());
RMNode node1 = MockNodes.newNodeInfo(1, RMNode node1 =
Resources.createResource(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, 10), MockNodes.newNodeInfo(1, Resources.createResource(10240, 10),
1, "127.0.0.1"); 1, "127.0.0.1");
RMNode node2 = MockNodes.newNodeInfo(1, RMNode node2 =
Resources.createResource(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, 10), MockNodes.newNodeInfo(1, Resources.createResource(10240, 10),
2, "127.0.0.2"); 2, "127.0.0.2");
RMNode node3 = RMNode node3 =
MockNodes.newNodeInfo(1, Resources.createResource(5120, 5), MockNodes.newNodeInfo(1, Resources.createResource(5120, 5),
3, "127.0.0.3"); 3, "127.0.0.3");
@ -4222,12 +4117,10 @@ public class TestFairScheduler extends FairSchedulerTestBase {
true); true);
Resource amResource1 = Resource.newInstance(1024, 1); Resource amResource1 = Resource.newInstance(1024, 1);
Resource amResource2 = Resource.newInstance(1024, 1); Resource amResource2 = Resource.newInstance(1024, 1);
Resource amResource3 = Resource amResource3 = Resource.newInstance(10240, 1);
Resource.newInstance(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, 1);
Resource amResource4 = Resource.newInstance(5120, 1); Resource amResource4 = Resource.newInstance(5120, 1);
Resource amResource5 = Resource.newInstance(1024, 1); Resource amResource5 = Resource.newInstance(1024, 1);
Resource amResource6 = Resource amResource6 = Resource.newInstance(10240, 1);
Resource.newInstance(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, 1);
Resource amResource7 = Resource.newInstance(1024, 1); Resource amResource7 = Resource.newInstance(1024, 1);
Resource amResource8 = Resource.newInstance(1024, 1); Resource amResource8 = Resource.newInstance(1024, 1);
int amPriority = RMAppAttemptImpl.AM_CONTAINER_PRIORITY.getPriority(); int amPriority = RMAppAttemptImpl.AM_CONTAINER_PRIORITY.getPriority();
@ -4261,8 +4154,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
ApplicationAttemptId attId3 = createAppAttemptId(3, 1); ApplicationAttemptId attId3 = createAppAttemptId(3, 1);
createApplicationWithAMResource(attId3, "queue1", "user1", amResource3); createApplicationWithAMResource(attId3, "queue1", "user1", amResource3);
createSchedulingRequestExistingApplication( createSchedulingRequestExistingApplication(10240, 1, amPriority, attId3);
RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, 1, amPriority, attId3);
FSAppAttempt app3 = scheduler.getSchedulerApp(attId3); FSAppAttempt app3 = scheduler.getSchedulerApp(attId3);
scheduler.update(); scheduler.update();
// app3 reserves a container on node1 because node1's available resource // app3 reserves a container on node1 because node1's available resource
@ -4336,8 +4228,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
ApplicationAttemptId attId6 = createAppAttemptId(6, 1); ApplicationAttemptId attId6 = createAppAttemptId(6, 1);
createApplicationWithAMResource(attId6, "queue1", "user1", amResource6); createApplicationWithAMResource(attId6, "queue1", "user1", amResource6);
createSchedulingRequestExistingApplication( createSchedulingRequestExistingApplication(10240, 1, amPriority, attId6);
RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, 1, amPriority, attId6);
FSAppAttempt app6 = scheduler.getSchedulerApp(attId6); FSAppAttempt app6 = scheduler.getSchedulerApp(attId6);
scheduler.update(); scheduler.update();
// app6 can't reserve a container on node1 because // app6 can't reserve a container on node1 because
@ -4426,8 +4317,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
// app6 turns the reservation into an allocation on node2. // app6 turns the reservation into an allocation on node2.
scheduler.handle(updateE2); scheduler.handle(updateE2);
assertEquals("Application6's AM requests 10240 MB memory", assertEquals("Application6's AM requests 10240 MB memory",
RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, 10240, app6.getAMResource().getMemorySize());
app6.getAMResource().getMemorySize());
assertEquals("Application6's AM should be running", assertEquals("Application6's AM should be running",
1, app6.getLiveContainers().size()); 1, app6.getLiveContainers().size());
assertEquals("Queue1's AM resource usage should be 11264 MB memory", assertEquals("Queue1's AM resource usage should be 11264 MB memory",

View File

@ -60,11 +60,9 @@ class AllocationFileQueue {
() -> AllocationFileWriter () -> AllocationFileWriter
.createNumberSupplier(properties.getFairSharePreemptionTimeout())); .createNumberSupplier(properties.getFairSharePreemptionTimeout()));
AllocationFileWriter.addIfPresent(pw, "fairSharePreemptionThreshold", AllocationFileWriter.addIfPresent(pw, "fairSharePreemptionThreshold",
() -> AllocationFileWriter.createNumberSupplier(
properties.getFairSharePreemptionThreshold()));
AllocationFileWriter.addIfPresent(pw, "maxContainerAllocation",
() -> AllocationFileWriter () -> AllocationFileWriter
.createNumberSupplier(properties.getMaxContainerAllocation())); .createNumberSupplier(
properties.getFairSharePreemptionThreshold()));
printEndTag(pw); printEndTag(pw);
pw.close(); pw.close();
return sw.toString(); return sw.toString();

View File

@ -94,12 +94,6 @@ public abstract class AllocationFileQueueBuilder {
return this; return this;
} }
public AllocationFileQueueBuilder maxContainerAllocation(
String maxContainerAllocation) {
this.queuePropertiesBuilder.maxContainerAllocation(maxContainerAllocation);
return this;
}
public AllocationFileQueueBuilder subQueue(String queueName) { public AllocationFileQueueBuilder subQueue(String queueName) {
if (this instanceof AllocationFileSimpleQueueBuilder) { if (this instanceof AllocationFileSimpleQueueBuilder) {
return new AllocationFileSubQueueBuilder( return new AllocationFileSubQueueBuilder(

View File

@ -33,7 +33,6 @@ public class AllocationFileQueueProperties {
private final String maxChildResources; private final String maxChildResources;
private final Integer fairSharePreemptionTimeout; private final Integer fairSharePreemptionTimeout;
private final Double fairSharePreemptionThreshold; private final Double fairSharePreemptionThreshold;
private final String maxContainerAllocation;
AllocationFileQueueProperties(Builder builder) { AllocationFileQueueProperties(Builder builder) {
this.queueName = builder.queueName; this.queueName = builder.queueName;
@ -49,7 +48,6 @@ public class AllocationFileQueueProperties {
this.maxChildResources = builder.maxChildResources; this.maxChildResources = builder.maxChildResources;
this.fairSharePreemptionTimeout = builder.fairSharePreemptionTimeout; this.fairSharePreemptionTimeout = builder.fairSharePreemptionTimeout;
this.fairSharePreemptionThreshold = builder.fairSharePreemptionThreshold; this.fairSharePreemptionThreshold = builder.fairSharePreemptionThreshold;
this.maxContainerAllocation = builder.maxContainerAllocation;
} }
public String getQueueName() { public String getQueueName() {
@ -104,10 +102,6 @@ public class AllocationFileQueueProperties {
return fairSharePreemptionThreshold; return fairSharePreemptionThreshold;
} }
public String getMaxContainerAllocation() {
return maxContainerAllocation;
}
/** /**
* Builder class for {@link AllocationFileQueueProperties}. * Builder class for {@link AllocationFileQueueProperties}.
*/ */
@ -125,7 +119,6 @@ public class AllocationFileQueueProperties {
private String maxChildResources; private String maxChildResources;
private Integer fairSharePreemptionTimeout; private Integer fairSharePreemptionTimeout;
private Double fairSharePreemptionThreshold; private Double fairSharePreemptionThreshold;
private String maxContainerAllocation;
Builder() { Builder() {
} }
@ -174,11 +167,6 @@ public class AllocationFileQueueProperties {
return this; return this;
} }
public Builder maxContainerAllocation(String maxContainerAllocation) {
this.maxContainerAllocation = maxContainerAllocation;
return this;
}
public Builder minSharePreemptionTimeout( public Builder minSharePreemptionTimeout(
Integer minSharePreemptionTimeout) { Integer minSharePreemptionTimeout) {
this.minSharePreemptionTimeout = minSharePreemptionTimeout; this.minSharePreemptionTimeout = minSharePreemptionTimeout;

View File

@ -92,8 +92,6 @@ The allocation file must be in XML format. The format contains five types of ele
* **maxChildResources**: maximum resources an ad hoc child queue is allocated, expressed either in absolute values (X mb, Y vcores) or as a percentage of the cluster resources (X% memory, Y% cpu). An ad hoc child queue will not be assigned a container that would put its aggregate usage over this limit. * **maxChildResources**: maximum resources an ad hoc child queue is allocated, expressed either in absolute values (X mb, Y vcores) or as a percentage of the cluster resources (X% memory, Y% cpu). An ad hoc child queue will not be assigned a container that would put its aggregate usage over this limit.
* **maxContainerAllocation**: maximum resources a queue can allocate for a single container, expressed in the form of "X mb, Y vcores" or "vcores=X, memory-mb=Y". The latter form is required when specifying resources other than memory and CPU. If the property is not set it's value is inherited from a parent queue. It's default value is **yarn.scheduler.maximum-allocation-mb**. Cannot be higher than **maxResources**. This property is invalid for root queue.
* **maxRunningApps**: limit the number of apps from the queue to run at once * **maxRunningApps**: limit the number of apps from the queue to run at once
* **maxAMShare**: limit the fraction of the queue's fair share that can be used to run application masters. This property can only be used for leaf queues. For example, if set to 1.0f, then AMs in the leaf queue can take up to 100% of both the memory and CPU fair share. The value of -1.0f will disable this feature and the amShare will not be checked. The default value is 0.5f. * **maxAMShare**: limit the fraction of the queue's fair share that can be used to run application masters. This property can only be used for leaf queues. For example, if set to 1.0f, then AMs in the leaf queue can take up to 100% of both the memory and CPU fair share. The value of -1.0f will disable this feature and the amShare will not be checked. The default value is 0.5f.