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:
parent
323b76bccf
commit
a0060cf8ee
|
@ -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.Priority;
|
||||
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.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
|
@ -87,7 +86,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
|
|||
private int maxCompletedAppsInMemory;
|
||||
private int maxCompletedAppsInStateStore;
|
||||
protected int completedAppsInStateStore = 0;
|
||||
protected LinkedList<ApplicationId> completedApps = new LinkedList<ApplicationId>();
|
||||
private LinkedList<ApplicationId> completedApps = new LinkedList<ApplicationId>();
|
||||
|
||||
private final RMContext rmContext;
|
||||
private final ApplicationMasterService masterService;
|
||||
|
@ -527,13 +526,13 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
|
|||
|
||||
// Normalize all requests
|
||||
String queue = submissionContext.getQueue();
|
||||
Resource maxAllocation = scheduler.getMaximumResourceCapability(queue);
|
||||
for (ResourceRequest amReq : amReqs) {
|
||||
SchedulerUtils.normalizeAndValidateRequest(amReq, maxAllocation,
|
||||
queue, scheduler, isRecovery, rmContext, null);
|
||||
SchedulerUtils.normalizeAndValidateRequest(amReq,
|
||||
scheduler.getMaximumResourceCapability(queue),
|
||||
queue, scheduler, isRecovery, rmContext);
|
||||
|
||||
amReq.setCapability(scheduler.getNormalizedResource(
|
||||
amReq.getCapability(), maxAllocation));
|
||||
amReq.setCapability(
|
||||
scheduler.getNormalizedResource(amReq.getCapability()));
|
||||
}
|
||||
return amReqs;
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
|
|
|
@ -97,7 +97,7 @@ public class RMServerUtils {
|
|||
"INCORRECT_CONTAINER_VERSION_ERROR";
|
||||
private static final String 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";
|
||||
|
||||
protected static final RecordFactory RECORD_FACTORY =
|
||||
|
@ -235,7 +235,7 @@ public class RMServerUtils {
|
|||
* requested memory/vcore is non-negative and not greater than max
|
||||
*/
|
||||
public static void normalizeAndValidateRequests(List<ResourceRequest> ask,
|
||||
Resource maximumAllocation, String queueName, YarnScheduler scheduler,
|
||||
Resource maximumResource, String queueName, YarnScheduler scheduler,
|
||||
RMContext rmContext) throws InvalidResourceRequestException {
|
||||
// Get queue from scheduler
|
||||
QueueInfo queueInfo = null;
|
||||
|
@ -247,7 +247,7 @@ public class RMServerUtils {
|
|||
}
|
||||
|
||||
for (ResourceRequest resReq : ask) {
|
||||
SchedulerUtils.normalizeAndValidateRequest(resReq, maximumAllocation,
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maximumResource,
|
||||
queueName, scheduler, rmContext, queueInfo);
|
||||
}
|
||||
}
|
||||
|
@ -338,8 +338,7 @@ public class RMServerUtils {
|
|||
return false;
|
||||
}
|
||||
ResourceScheduler scheduler = rmContext.getScheduler();
|
||||
request.setCapability(scheduler
|
||||
.getNormalizedResource(request.getCapability(), maximumAllocation));
|
||||
request.setCapability(scheduler.getNormalizedResource(request.getCapability()));
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
|
@ -1149,12 +1149,11 @@ public abstract class AbstractYarnScheduler
|
|||
}
|
||||
|
||||
@Override
|
||||
public Resource getNormalizedResource(Resource requestedResource,
|
||||
Resource maxResourceCapability) {
|
||||
public Resource getNormalizedResource(Resource requestedResource) {
|
||||
return SchedulerUtils.getNormalizedResource(requestedResource,
|
||||
getResourceCalculator(),
|
||||
getMinimumResourceCapability(),
|
||||
maxResourceCapability,
|
||||
getMaximumResourceCapability(),
|
||||
getMinimumResourceCapability());
|
||||
}
|
||||
|
||||
|
@ -1164,20 +1163,8 @@ public abstract class AbstractYarnScheduler
|
|||
* @param asks resource requests
|
||||
*/
|
||||
protected void normalizeResourceRequests(List<ResourceRequest> asks) {
|
||||
normalizeResourceRequests(asks, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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));
|
||||
ask.setCapability(getNormalizedResource(ask.getCapability()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -217,7 +217,16 @@ public class SchedulerUtils {
|
|||
}
|
||||
|
||||
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)
|
||||
throws InvalidResourceRequestException {
|
||||
Configuration conf = rmContext.getYarnConfiguration();
|
||||
|
@ -247,15 +256,22 @@ public class SchedulerUtils {
|
|||
SchedulerUtils.normalizeNodeLabelExpressionInRequest(resReq, queueInfo);
|
||||
|
||||
if (!isRecovery) {
|
||||
validateResourceRequest(resReq, maximumAllocation, queueInfo, rmContext);
|
||||
validateResourceRequest(resReq, maximumResource, queueInfo, rmContext);
|
||||
}
|
||||
}
|
||||
|
||||
public static void normalizeAndValidateRequest(ResourceRequest resReq,
|
||||
Resource maximumAllocation, String queueName, YarnScheduler scheduler,
|
||||
public static void normalizeAndvalidateRequest(ResourceRequest resReq,
|
||||
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)
|
||||
throws InvalidResourceRequestException {
|
||||
normalizeAndValidateRequest(resReq, maximumAllocation, queueName, scheduler,
|
||||
normalizeAndValidateRequest(resReq, maximumResource, queueName, scheduler,
|
||||
false, rmContext, queueInfo);
|
||||
}
|
||||
|
||||
|
@ -266,11 +282,11 @@ public class SchedulerUtils {
|
|||
* @throws InvalidResourceRequestException when there is invalid request
|
||||
*/
|
||||
private static void validateResourceRequest(ResourceRequest resReq,
|
||||
Resource maximumAllocation, QueueInfo queueInfo, RMContext rmContext)
|
||||
Resource maximumResource, QueueInfo queueInfo, RMContext rmContext)
|
||||
throws InvalidResourceRequestException {
|
||||
final Resource requestedResource = resReq.getCapability();
|
||||
checkResourceRequestAgainstAvailableResource(requestedResource,
|
||||
maximumAllocation);
|
||||
maximumResource);
|
||||
|
||||
String labelExp = resReq.getNodeLabelExpression();
|
||||
// we don't allow specify label expression other than resourceName=ANY now
|
||||
|
|
|
@ -380,17 +380,12 @@ public interface YarnScheduler extends EventHandler<SchedulerEvent> {
|
|||
SchedulerNode getSchedulerNode(NodeId nodeId);
|
||||
|
||||
/**
|
||||
* Normalize a resource request using scheduler level maximum resource or
|
||||
* queue based maximum resource.
|
||||
* Normalize a resource request.
|
||||
*
|
||||
* @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
|
||||
*/
|
||||
Resource getNormalizedResource(Resource requestedResource,
|
||||
Resource maxResourceCapability);
|
||||
Resource getNormalizedResource(Resource requestedResource);
|
||||
|
||||
/**
|
||||
* Verify whether a submitted application lifetime is valid as per configured
|
||||
|
|
|
@ -1107,12 +1107,10 @@ public class CapacityScheduler extends
|
|||
if (asks == null) {
|
||||
return;
|
||||
}
|
||||
Resource maxAllocation = getMaximumResourceCapability();
|
||||
for (SchedulingRequest ask: asks) {
|
||||
ResourceSizing sizing = ask.getResourceSizing();
|
||||
if (sizing != null && sizing.getResources() != null) {
|
||||
sizing.setResources(
|
||||
getNormalizedResource(sizing.getResources(), maxAllocation));
|
||||
sizing.setResources(getNormalizedResource(sizing.getResources()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2433,9 +2431,6 @@ public class CapacityScheduler extends
|
|||
|
||||
@Override
|
||||
public Resource getMaximumResourceCapability(String queueName) {
|
||||
if(queueName == null || queueName.isEmpty()) {
|
||||
return getMaximumResourceCapability();
|
||||
}
|
||||
CSQueue queue = getQueue(queueName);
|
||||
if (queue == null) {
|
||||
LOG.error("Unknown queue: " + queueName);
|
||||
|
|
|
@ -175,19 +175,11 @@ public class PlacementConstraintProcessor extends AbstractPlacementProcessor {
|
|||
private void dispatchRequestsForPlacement(ApplicationAttemptId appAttemptId,
|
||||
List<SchedulingRequest> schedulingRequests) {
|
||||
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
|
||||
schedulingRequests.forEach(req -> {
|
||||
Resource reqResource = req.getResourceSizing().getResources();
|
||||
req.getResourceSizing().setResources(
|
||||
this.scheduler.getNormalizedResource(reqResource, maxAllocation));
|
||||
req.getResourceSizing()
|
||||
.setResources(this.scheduler.getNormalizedResource(reqResource));
|
||||
});
|
||||
this.placementDispatcher.dispatch(new BatchedRequests(iteratorType,
|
||||
appAttemptId.getApplicationId(), schedulingRequests, 1));
|
||||
|
|
|
@ -93,9 +93,6 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
|
|||
|
||||
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
|
||||
@VisibleForTesting
|
||||
QueuePlacementPolicy placementPolicy;
|
||||
|
@ -143,8 +140,6 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
|
|||
this.placementPolicy = newPlacementPolicy;
|
||||
this.configuredQueues = queueProperties.getConfiguredQueues();
|
||||
this.nonPreemptableQueues = queueProperties.getNonPreemptableQueues();
|
||||
this.queueMaxContainerAllocationMap =
|
||||
queueProperties.getMaxContainerAllocation();
|
||||
}
|
||||
|
||||
public AllocationConfiguration(Configuration conf) {
|
||||
|
@ -174,7 +169,6 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
|
|||
placementPolicy =
|
||||
QueuePlacementPolicy.fromConfiguration(conf, configuredQueues);
|
||||
nonPreemptableQueues = new HashSet<>();
|
||||
queueMaxContainerAllocationMap = new HashMap<>();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -298,12 +292,6 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
|
|||
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.
|
||||
*
|
||||
|
@ -407,7 +395,6 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
|
|||
queue.setMaxRunningApps(getQueueMaxApps(name));
|
||||
queue.setMaxAMShare(getQueueMaxAMShare(name));
|
||||
queue.setMaxChildQueueResource(getMaxChildResources(name));
|
||||
queue.setMaxContainerAllocation(getQueueMaxContainerAllocation(name));
|
||||
|
||||
// Set queue metrics.
|
||||
queue.getMetrics().setMinShare(queue.getMinShare());
|
||||
|
|
|
@ -530,16 +530,6 @@ public class FSLeafQueue extends FSQueue {
|
|||
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.
|
||||
*
|
||||
|
|
|
@ -60,19 +60,6 @@ public class FSParentQueue extends FSQueue {
|
|||
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) {
|
||||
writeLock.lock();
|
||||
try {
|
||||
|
|
|
@ -83,7 +83,6 @@ public abstract class FSQueue implements Queue, Schedulable {
|
|||
private long minSharePreemptionTimeout = Long.MAX_VALUE;
|
||||
private float fairSharePreemptionThreshold = 0.5f;
|
||||
private boolean preemptable = true;
|
||||
protected Resource maxContainerAllocation;
|
||||
|
||||
public FSQueue(String name, FairScheduler scheduler, FSParentQueue parent) {
|
||||
this.name = name;
|
||||
|
@ -163,12 +162,6 @@ public abstract class FSQueue implements Queue, Schedulable {
|
|||
this.maxShare = maxShare;
|
||||
}
|
||||
|
||||
public void setMaxContainerAllocation(Resource maxContainerAllocation){
|
||||
this.maxContainerAllocation = maxContainerAllocation;
|
||||
}
|
||||
|
||||
public abstract Resource getMaximumContainerAllocation();
|
||||
|
||||
@Override
|
||||
public Resource getMaxShare() {
|
||||
Resource maxResource = maxShare.getResource(scheduler.getClusterResource());
|
||||
|
|
|
@ -186,7 +186,6 @@ public class FairScheduler extends
|
|||
protected long rackLocalityDelayMs; // Delay for rack locality
|
||||
protected boolean assignMultiple; // Allocate multiple containers per
|
||||
// heartbeat
|
||||
|
||||
@VisibleForTesting
|
||||
boolean maxAssignDynamic;
|
||||
protected int maxAssign; // Max containers to assign per heartbeat
|
||||
|
@ -222,11 +221,11 @@ public class FairScheduler extends
|
|||
|
||||
private void validateConf(FairSchedulerConfiguration config) {
|
||||
// validate scheduler memory allocation setting
|
||||
int minMem =
|
||||
config.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
|
||||
int minMem = config.getInt(
|
||||
YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
|
||||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
|
||||
int maxMem =
|
||||
config.getInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
||||
int maxMem = config.getInt(
|
||||
YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
||||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
|
||||
|
||||
if (minMem < 0 || minMem > maxMem) {
|
||||
|
@ -249,11 +248,11 @@ public class FairScheduler extends
|
|||
}
|
||||
|
||||
// validate scheduler vcores allocation setting
|
||||
int minVcores =
|
||||
config.getInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
|
||||
int minVcores = config.getInt(
|
||||
YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
|
||||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
|
||||
int maxVcores =
|
||||
config.getInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
|
||||
int maxVcores = config.getInt(
|
||||
YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
|
||||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
|
||||
|
||||
if (minVcores < 0 || minVcores > maxVcores) {
|
||||
|
@ -813,35 +812,14 @@ public class FairScheduler extends
|
|||
}
|
||||
|
||||
@Override
|
||||
public Resource getNormalizedResource(Resource requestedResource,
|
||||
Resource maxResourceCapability) {
|
||||
public Resource getNormalizedResource(Resource requestedResource) {
|
||||
return SchedulerUtils.getNormalizedResource(requestedResource,
|
||||
DOMINANT_RESOURCE_CALCULATOR,
|
||||
minimumAllocation,
|
||||
maxResourceCapability,
|
||||
getMaximumResourceCapability(),
|
||||
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
|
||||
@Override
|
||||
public void killContainer(RMContainer container) {
|
||||
|
@ -881,7 +859,7 @@ public class FairScheduler extends
|
|||
handleContainerUpdates(application, updateRequests);
|
||||
|
||||
// Sanity check
|
||||
normalizeResourceRequests(ask, application.getQueueName());
|
||||
normalizeResourceRequests(ask);
|
||||
|
||||
// TODO, normalize SchedulingRequest
|
||||
|
||||
|
|
|
@ -51,8 +51,6 @@ public class AllocationFileQueueParser {
|
|||
private static final String MAX_CHILD_RESOURCES = "maxChildResources";
|
||||
private static final String MAX_RUNNING_APPS = "maxRunningApps";
|
||||
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 MIN_SHARE_PREEMPTION_TIMEOUT =
|
||||
"minSharePreemptionTimeout";
|
||||
|
@ -157,11 +155,6 @@ public class AllocationFileQueueParser {
|
|||
float val = Float.parseFloat(text);
|
||||
val = Math.min(val, 1.0f);
|
||||
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())) {
|
||||
String text = getTrimmedTextData(field);
|
||||
double val = Double.parseDouble(text);
|
||||
|
|
|
@ -53,7 +53,6 @@ public class QueueProperties {
|
|||
private final Set<String> reservableQueues;
|
||||
private final Set<String> nonPreemptableQueues;
|
||||
private final Map<FSQueueType, Set<String>> configuredQueues;
|
||||
private final Map<String, Resource> queueMaxContainerAllocation;
|
||||
|
||||
QueueProperties(Builder builder) {
|
||||
this.reservableQueues = builder.reservableQueues;
|
||||
|
@ -71,7 +70,6 @@ public class QueueProperties {
|
|||
this.maxChildQueueResources = builder.maxChildQueueResources;
|
||||
this.reservationAcls = builder.reservationAcls;
|
||||
this.queueAcls = builder.queueAcls;
|
||||
this.queueMaxContainerAllocation = builder.queueMaxContainerAllocation;
|
||||
}
|
||||
|
||||
public Map<FSQueueType, Set<String>> getConfiguredQueues() {
|
||||
|
@ -135,10 +133,6 @@ public class QueueProperties {
|
|||
return nonPreemptableQueues;
|
||||
}
|
||||
|
||||
public Map<String, Resource> getMaxContainerAllocation() {
|
||||
return queueMaxContainerAllocation;
|
||||
}
|
||||
|
||||
/**
|
||||
* Builder class for {@link QueueProperties}.
|
||||
* All methods are adding queue properties to the maps of this builder
|
||||
|
@ -155,7 +149,6 @@ public class QueueProperties {
|
|||
new HashMap<>();
|
||||
private Map<String, Integer> queueMaxApps = 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, SchedulingPolicy> queuePolicies = new HashMap<>();
|
||||
private Map<String, Long> minSharePreemptionTimeouts = new HashMap<>();
|
||||
|
@ -260,12 +253,6 @@ public class QueueProperties {
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder queueMaxContainerAllocation(String queueName,
|
||||
Resource value) {
|
||||
queueMaxContainerAllocation.put(queueName, value);
|
||||
return this;
|
||||
}
|
||||
|
||||
public void configuredQueues(FSQueueType queueType, String queueName) {
|
||||
this.configuredQueues.get(queueType).add(queueName);
|
||||
}
|
||||
|
@ -288,5 +275,6 @@ public class QueueProperties {
|
|||
public QueueProperties build() {
|
||||
return new QueueProperties(this);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -78,8 +78,6 @@ public class FairSchedulerPage extends RmView {
|
|||
__("Num Pending Applications:", qinfo.getNumPendingApplications()).
|
||||
__("Min Resources:", qinfo.getMinResources().toString()).
|
||||
__("Max Resources:", qinfo.getMaxResources().toString()).
|
||||
__("Max Container Allocation:",
|
||||
qinfo.getMaxContainerAllocation().toString()).
|
||||
__("Reserved Resources:", qinfo.getReservedResources().toString());
|
||||
int maxApps = qinfo.getMaxApplications();
|
||||
if (maxApps < Integer.MAX_VALUE) {
|
||||
|
@ -109,8 +107,6 @@ public class FairSchedulerPage extends RmView {
|
|||
__("Used Resources:", qinfo.getUsedResources().toString()).
|
||||
__("Min Resources:", qinfo.getMinResources().toString()).
|
||||
__("Max Resources:", qinfo.getMaxResources().toString()).
|
||||
__("Max Container Allocation:",
|
||||
qinfo.getMaxContainerAllocation().toString()).
|
||||
__("Reserved Resources:", qinfo.getReservedResources().toString());
|
||||
int maxApps = qinfo.getMaxApplications();
|
||||
if (maxApps < Integer.MAX_VALUE) {
|
||||
|
|
|
@ -60,7 +60,6 @@ public class FairSchedulerQueueInfo {
|
|||
private ResourceInfo fairResources;
|
||||
private ResourceInfo clusterResources;
|
||||
private ResourceInfo reservedResources;
|
||||
private ResourceInfo maxContainerAllocation;
|
||||
|
||||
private long allocatedContainers;
|
||||
private long reservedContainers;
|
||||
|
@ -100,8 +99,6 @@ public class FairSchedulerQueueInfo {
|
|||
maxResources = new ResourceInfo(
|
||||
Resources.componentwiseMin(queue.getMaxShare(),
|
||||
scheduler.getClusterResource()));
|
||||
maxContainerAllocation =
|
||||
new ResourceInfo(scheduler.getMaximumResourceCapability(queueName));
|
||||
reservedResources = new ResourceInfo(queue.getReservedResource());
|
||||
|
||||
fractionMemSteadyFairShare =
|
||||
|
@ -190,10 +187,6 @@ public class FairSchedulerQueueInfo {
|
|||
return maxResources;
|
||||
}
|
||||
|
||||
public ResourceInfo getMaxContainerAllocation() {
|
||||
return maxContainerAllocation;
|
||||
}
|
||||
|
||||
public ResourceInfo getReservedResources() {
|
||||
return reservedResources;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -513,14 +513,6 @@ public class MockRM extends ResourceManager {
|
|||
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 {
|
||||
Resource resource = Resource.newInstance(masterMemory, 0);
|
||||
return submitApp(resource, "", UserGroupInformation.getCurrentUser()
|
||||
|
|
|
@ -105,7 +105,6 @@ import org.junit.After;
|
|||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import static org.mockito.Matchers.anyString;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
|
@ -1075,14 +1074,10 @@ public class TestAppManager{
|
|||
Resources.createResource(
|
||||
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);
|
||||
when(scheduler.getResourceCalculator()).thenReturn(rs);
|
||||
|
||||
when(scheduler.getNormalizedResource(any(), any()))
|
||||
when(scheduler.getNormalizedResource(any()))
|
||||
.thenAnswer(new Answer<Resource>() {
|
||||
@Override
|
||||
public Resource answer(InvocationOnMock invocationOnMock)
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -347,9 +347,9 @@ public class TestClientRMService {
|
|||
|
||||
@Test
|
||||
public void testGetApplicationReport() throws Exception {
|
||||
ResourceScheduler scheduler = mock(ResourceScheduler.class);
|
||||
YarnScheduler yarnScheduler = mock(YarnScheduler.class);
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
mockRMContext(scheduler, rmContext);
|
||||
mockRMContext(yarnScheduler, rmContext);
|
||||
|
||||
ApplicationId appId1 = getApplicationId(1);
|
||||
|
||||
|
@ -358,7 +358,7 @@ public class TestClientRMService {
|
|||
mockAclsManager.checkAccess(UserGroupInformation.getCurrentUser(),
|
||||
ApplicationAccessType.VIEW_APP, null, appId1)).thenReturn(true);
|
||||
|
||||
ClientRMService rmService = new ClientRMService(rmContext, scheduler,
|
||||
ClientRMService rmService = new ClientRMService(rmContext, yarnScheduler,
|
||||
null, mockAclsManager, null, null);
|
||||
try {
|
||||
RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
|
||||
|
@ -441,9 +441,9 @@ public class TestClientRMService {
|
|||
public void testGetApplicationResourceUsageReportDummy() throws YarnException,
|
||||
IOException {
|
||||
ApplicationAttemptId attemptId = getApplicationAttemptId(1);
|
||||
ResourceScheduler scheduler = mockResourceScheduler();
|
||||
YarnScheduler yarnScheduler = mockYarnScheduler();
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
mockRMContext(scheduler, rmContext);
|
||||
mockRMContext(yarnScheduler, rmContext);
|
||||
when(rmContext.getDispatcher().getEventHandler()).thenReturn(
|
||||
new EventHandler<Event>() {
|
||||
public void handle(Event event) {
|
||||
|
@ -453,7 +453,7 @@ public class TestClientRMService {
|
|||
mock(ApplicationSubmissionContext.class);
|
||||
YarnConfiguration config = new YarnConfiguration();
|
||||
RMAppAttemptImpl rmAppAttemptImpl = new RMAppAttemptImpl(attemptId,
|
||||
rmContext, scheduler, null, asContext, config, null, null);
|
||||
rmContext, yarnScheduler, null, asContext, config, null, null);
|
||||
ApplicationResourceUsageReport report = rmAppAttemptImpl
|
||||
.getApplicationResourceUsageReport();
|
||||
assertEquals(report, RMServerUtils.DUMMY_APPLICATION_RESOURCE_USAGE_REPORT);
|
||||
|
@ -522,14 +522,14 @@ public class TestClientRMService {
|
|||
}
|
||||
|
||||
public ClientRMService createRMService() throws IOException, YarnException {
|
||||
ResourceScheduler scheduler = mockResourceScheduler();
|
||||
YarnScheduler yarnScheduler = mockYarnScheduler();
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
mockRMContext(scheduler, rmContext);
|
||||
mockRMContext(yarnScheduler, rmContext);
|
||||
ConcurrentHashMap<ApplicationId, RMApp> apps = getRMApps(rmContext,
|
||||
scheduler);
|
||||
yarnScheduler);
|
||||
when(rmContext.getRMApps()).thenReturn(apps);
|
||||
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());
|
||||
when(rmContext.getDispatcher().getEventHandler()).thenReturn(
|
||||
new EventHandler<Event>() {
|
||||
|
@ -543,7 +543,7 @@ public class TestClientRMService {
|
|||
mockQueueACLsManager.checkAccess(any(UserGroupInformation.class),
|
||||
any(QueueACL.class), any(RMApp.class), any(String.class),
|
||||
any())).thenReturn(true);
|
||||
return new ClientRMService(rmContext, scheduler, appManager,
|
||||
return new ClientRMService(rmContext, yarnScheduler, appManager,
|
||||
mockAclsManager, mockQueueACLsManager, null);
|
||||
}
|
||||
|
||||
|
@ -892,9 +892,9 @@ public class TestClientRMService {
|
|||
|
||||
@Test
|
||||
public void testGetQueueInfo() throws Exception {
|
||||
ResourceScheduler scheduler = mock(ResourceScheduler.class);
|
||||
YarnScheduler yarnScheduler = mock(YarnScheduler.class);
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
mockRMContext(scheduler, rmContext);
|
||||
mockRMContext(yarnScheduler, rmContext);
|
||||
|
||||
ApplicationACLsManager mockAclsManager = mock(ApplicationACLsManager.class);
|
||||
QueueACLsManager mockQueueACLsManager = mock(QueueACLsManager.class);
|
||||
|
@ -906,7 +906,7 @@ public class TestClientRMService {
|
|||
any(ApplicationAccessType.class), anyString(),
|
||||
any(ApplicationId.class))).thenReturn(true);
|
||||
|
||||
ClientRMService rmService = new ClientRMService(rmContext, scheduler,
|
||||
ClientRMService rmService = new ClientRMService(rmContext, yarnScheduler,
|
||||
null, mockAclsManager, mockQueueACLsManager, null);
|
||||
GetQueueInfoRequest request = recordFactory
|
||||
.newRecordInstance(GetQueueInfoRequest.class);
|
||||
|
@ -945,7 +945,7 @@ public class TestClientRMService {
|
|||
any(ApplicationAccessType.class), anyString(),
|
||||
any(ApplicationId.class))).thenReturn(false);
|
||||
|
||||
ClientRMService rmService1 = new ClientRMService(rmContext, scheduler,
|
||||
ClientRMService rmService1 = new ClientRMService(rmContext, yarnScheduler,
|
||||
null, mockAclsManager1, mockQueueACLsManager1, null);
|
||||
request.setQueueName("testqueue");
|
||||
request.setIncludeApplications(true);
|
||||
|
@ -959,12 +959,12 @@ public class TestClientRMService {
|
|||
@Test (timeout = 30000)
|
||||
@SuppressWarnings ("rawtypes")
|
||||
public void testAppSubmit() throws Exception {
|
||||
ResourceScheduler scheduler = mockResourceScheduler();
|
||||
YarnScheduler yarnScheduler = mockYarnScheduler();
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
mockRMContext(scheduler, rmContext);
|
||||
mockRMContext(yarnScheduler, rmContext);
|
||||
RMStateStore stateStore = mock(RMStateStore.class);
|
||||
when(rmContext.getStateStore()).thenReturn(stateStore);
|
||||
RMAppManager appManager = new RMAppManager(rmContext, scheduler,
|
||||
RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler,
|
||||
null, mock(ApplicationACLsManager.class), new Configuration());
|
||||
when(rmContext.getDispatcher().getEventHandler()).thenReturn(
|
||||
new EventHandler<Event>() {
|
||||
|
@ -986,7 +986,7 @@ public class TestClientRMService {
|
|||
any()))
|
||||
.thenReturn(true);
|
||||
ClientRMService rmService =
|
||||
new ClientRMService(rmContext, scheduler, appManager,
|
||||
new ClientRMService(rmContext, yarnScheduler, appManager,
|
||||
mockAclsManager, mockQueueACLsManager, null);
|
||||
rmService.init(new Configuration());
|
||||
|
||||
|
@ -1070,15 +1070,15 @@ public class TestClientRMService {
|
|||
* 2. Test each of the filters
|
||||
*/
|
||||
// Basic setup
|
||||
ResourceScheduler scheduler = mockResourceScheduler();
|
||||
YarnScheduler yarnScheduler = mockYarnScheduler();
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
mockRMContext(scheduler, rmContext);
|
||||
mockRMContext(yarnScheduler, rmContext);
|
||||
RMStateStore stateStore = mock(RMStateStore.class);
|
||||
when(rmContext.getStateStore()).thenReturn(stateStore);
|
||||
doReturn(mock(RMTimelineCollectorManager.class)).when(rmContext)
|
||||
.getRMTimelineCollectorManager();
|
||||
|
||||
RMAppManager appManager = new RMAppManager(rmContext, scheduler,
|
||||
RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler,
|
||||
null, mock(ApplicationACLsManager.class), new Configuration());
|
||||
when(rmContext.getDispatcher().getEventHandler()).thenReturn(
|
||||
new EventHandler<Event>() {
|
||||
|
@ -1092,7 +1092,7 @@ public class TestClientRMService {
|
|||
any()))
|
||||
.thenReturn(true);
|
||||
ClientRMService rmService =
|
||||
new ClientRMService(rmContext, scheduler, appManager,
|
||||
new ClientRMService(rmContext, yarnScheduler, appManager,
|
||||
mockAclsManager, mockQueueACLsManager, null);
|
||||
rmService.init(new Configuration());
|
||||
|
||||
|
@ -1223,12 +1223,12 @@ public class TestClientRMService {
|
|||
public void testConcurrentAppSubmit()
|
||||
throws IOException, InterruptedException, BrokenBarrierException,
|
||||
YarnException {
|
||||
ResourceScheduler scheduler = mockResourceScheduler();
|
||||
YarnScheduler yarnScheduler = mockYarnScheduler();
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
mockRMContext(scheduler, rmContext);
|
||||
mockRMContext(yarnScheduler, rmContext);
|
||||
RMStateStore stateStore = mock(RMStateStore.class);
|
||||
when(rmContext.getStateStore()).thenReturn(stateStore);
|
||||
RMAppManager appManager = new RMAppManager(rmContext, scheduler,
|
||||
RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler,
|
||||
null, mock(ApplicationACLsManager.class), new Configuration());
|
||||
|
||||
final ApplicationId appId1 = getApplicationId(100);
|
||||
|
@ -1265,7 +1265,7 @@ public class TestClientRMService {
|
|||
.getRMTimelineCollectorManager();
|
||||
|
||||
final ClientRMService rmService =
|
||||
new ClientRMService(rmContext, scheduler, appManager, null, null,
|
||||
new ClientRMService(rmContext, yarnScheduler, appManager, null, null,
|
||||
null);
|
||||
rmService.init(new Configuration());
|
||||
|
||||
|
@ -1324,7 +1324,7 @@ public class TestClientRMService {
|
|||
return submitRequest;
|
||||
}
|
||||
|
||||
private void mockRMContext(ResourceScheduler scheduler, RMContext rmContext)
|
||||
private void mockRMContext(YarnScheduler yarnScheduler, RMContext rmContext)
|
||||
throws IOException {
|
||||
Dispatcher dispatcher = mock(Dispatcher.class);
|
||||
when(rmContext.getDispatcher()).thenReturn(dispatcher);
|
||||
|
@ -1346,21 +1346,22 @@ public class TestClientRMService {
|
|||
queueConfigsByPartition.put("*", queueConfigs);
|
||||
queInfo.setQueueConfigurations(queueConfigsByPartition);
|
||||
|
||||
when(scheduler.getQueueInfo(eq("testqueue"), anyBoolean(), anyBoolean()))
|
||||
when(yarnScheduler.getQueueInfo(eq("testqueue"), anyBoolean(), anyBoolean()))
|
||||
.thenReturn(queInfo);
|
||||
when(scheduler.getQueueInfo(eq("nonexistentqueue"), anyBoolean(),
|
||||
anyBoolean())).thenThrow(new IOException("queue does not exist"));
|
||||
when(yarnScheduler.getQueueInfo(eq("nonexistentqueue"), anyBoolean(), anyBoolean()))
|
||||
.thenThrow(new IOException("queue does not exist"));
|
||||
RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
|
||||
when(rmContext.getRMApplicationHistoryWriter()).thenReturn(writer);
|
||||
SystemMetricsPublisher publisher = mock(SystemMetricsPublisher.class);
|
||||
when(rmContext.getSystemMetricsPublisher()).thenReturn(publisher);
|
||||
when(rmContext.getYarnConfiguration()).thenReturn(new YarnConfiguration());
|
||||
ConcurrentHashMap<ApplicationId, RMApp> apps =
|
||||
getRMApps(rmContext, scheduler);
|
||||
ConcurrentHashMap<ApplicationId, RMApp> apps = getRMApps(rmContext,
|
||||
yarnScheduler);
|
||||
when(rmContext.getRMApps()).thenReturn(apps);
|
||||
when(scheduler.getAppsInQueue(eq("testqueue"))).thenReturn(
|
||||
when(yarnScheduler.getAppsInQueue(eq("testqueue"))).thenReturn(
|
||||
getSchedulerApps(apps));
|
||||
when(rmContext.getScheduler()).thenReturn(scheduler);
|
||||
ResourceScheduler rs = mock(ResourceScheduler.class);
|
||||
when(rmContext.getScheduler()).thenReturn(rs);
|
||||
}
|
||||
|
||||
private ConcurrentHashMap<ApplicationId, RMApp> getRMApps(
|
||||
|
@ -1464,32 +1465,31 @@ public class TestClientRMService {
|
|||
return app;
|
||||
}
|
||||
|
||||
private static ResourceScheduler mockResourceScheduler()
|
||||
throws YarnException {
|
||||
ResourceScheduler scheduler = mock(ResourceScheduler.class);
|
||||
when(scheduler.getMinimumResourceCapability()).thenReturn(
|
||||
private static YarnScheduler mockYarnScheduler() throws YarnException {
|
||||
YarnScheduler yarnScheduler = mock(YarnScheduler.class);
|
||||
when(yarnScheduler.getMinimumResourceCapability()).thenReturn(
|
||||
Resources.createResource(
|
||||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB));
|
||||
when(scheduler.getMaximumResourceCapability()).thenReturn(
|
||||
when(yarnScheduler.getMaximumResourceCapability()).thenReturn(
|
||||
Resources.createResource(
|
||||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB));
|
||||
when(scheduler.getMaximumResourceCapability(anyString())).thenReturn(
|
||||
Resources.createResource(
|
||||
when(yarnScheduler.getMaximumResourceCapability(any(String.class)))
|
||||
.thenReturn(Resources.createResource(
|
||||
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)));
|
||||
when(scheduler.getAppsInQueue(QUEUE_2)).thenReturn(
|
||||
when(yarnScheduler.getAppsInQueue(QUEUE_2)).thenReturn(
|
||||
Arrays.asList(getApplicationAttemptId(103)));
|
||||
ApplicationAttemptId attemptId = getApplicationAttemptId(1);
|
||||
when(scheduler.getAppResourceUsageReport(attemptId)).thenReturn(null);
|
||||
when(yarnScheduler.getAppResourceUsageReport(attemptId)).thenReturn(null);
|
||||
|
||||
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)))
|
||||
.thenReturn(Priority.newInstance(0));
|
||||
return scheduler;
|
||||
return yarnScheduler;
|
||||
}
|
||||
|
||||
private ResourceManager setupResourceManager() {
|
||||
|
@ -2186,15 +2186,15 @@ public class TestClientRMService {
|
|||
* Submit 3 applications alternately in two queues
|
||||
*/
|
||||
// Basic setup
|
||||
ResourceScheduler scheduler = mockResourceScheduler();
|
||||
YarnScheduler yarnScheduler = mockYarnScheduler();
|
||||
RMContext rmContext = mock(RMContext.class);
|
||||
mockRMContext(scheduler, rmContext);
|
||||
mockRMContext(yarnScheduler, rmContext);
|
||||
RMStateStore stateStore = mock(RMStateStore.class);
|
||||
when(rmContext.getStateStore()).thenReturn(stateStore);
|
||||
doReturn(mock(RMTimelineCollectorManager.class)).when(rmContext)
|
||||
.getRMTimelineCollectorManager();
|
||||
|
||||
RMAppManager appManager = new RMAppManager(rmContext, scheduler, null,
|
||||
RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler, null,
|
||||
mock(ApplicationACLsManager.class), new Configuration());
|
||||
when(rmContext.getDispatcher().getEventHandler())
|
||||
.thenReturn(new EventHandler<Event>() {
|
||||
|
@ -2213,7 +2213,7 @@ public class TestClientRMService {
|
|||
when(appAclsManager.checkAccess(eq(UserGroupInformation.getCurrentUser()),
|
||||
any(ApplicationAccessType.class), any(String.class),
|
||||
any(ApplicationId.class))).thenReturn(false);
|
||||
ClientRMService rmService = new ClientRMService(rmContext, scheduler,
|
||||
ClientRMService rmService = new ClientRMService(rmContext, yarnScheduler,
|
||||
appManager, appAclsManager, queueAclsManager, null);
|
||||
rmService.init(new Configuration());
|
||||
|
||||
|
|
|
@ -18,8 +18,16 @@
|
|||
|
||||
package org.apache.hadoop.yarn.server.resourcemanager;
|
||||
|
||||
import static org.apache.hadoop.yarn.api.records.ContainerUpdateType.INCREASE_RESOURCE;
|
||||
import static org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils.RESOURCE_OUTSIDE_ALLOWED_RANGE;
|
||||
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.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.Collections;
|
||||
|
@ -29,97 +37,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
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 {
|
||||
|
||||
@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
|
||||
public void testGetApplicableNodeCountForAMLocality() throws Exception {
|
||||
List<NodeId> rack1Nodes = new ArrayList<>();
|
||||
|
|
|
@ -99,7 +99,6 @@ import org.junit.Test;
|
|||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.junit.rules.ExpectedException;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
public class TestSchedulerUtils {
|
||||
|
||||
|
@ -265,7 +264,7 @@ public class TestSchedulerUtils {
|
|||
public void testValidateResourceRequestWithErrorLabelsPermission()
|
||||
throws IOException {
|
||||
// mock queue and scheduler
|
||||
ResourceScheduler scheduler = mock(ResourceScheduler.class);
|
||||
YarnScheduler scheduler = mock(YarnScheduler.class);
|
||||
Set<String> queueAccessibleNodeLabels = Sets.newHashSet();
|
||||
QueueInfo queueInfo = mock(QueueInfo.class);
|
||||
when(queueInfo.getQueueName()).thenReturn("queue");
|
||||
|
@ -274,8 +273,6 @@ public class TestSchedulerUtils {
|
|||
when(scheduler.getQueueInfo(any(String.class), anyBoolean(), anyBoolean()))
|
||||
.thenReturn(queueInfo);
|
||||
|
||||
when(rmContext.getScheduler()).thenReturn(scheduler);
|
||||
|
||||
Resource maxResource = Resources.createResource(
|
||||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
||||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
|
||||
|
@ -294,20 +291,20 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq = BuilderUtils.newResourceRequest(
|
||||
mock(Priority.class), ResourceRequest.ANY, resource, 1);
|
||||
resReq.setNodeLabelExpression("x");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
|
||||
resReq.setNodeLabelExpression("y");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
|
||||
resReq.setNodeLabelExpression("");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
|
||||
resReq.setNodeLabelExpression(" ");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
e.printStackTrace();
|
||||
fail("Should be valid when request labels is a subset of queue labels");
|
||||
|
@ -328,8 +325,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq = BuilderUtils.newResourceRequest(
|
||||
mock(Priority.class), ResourceRequest.ANY, resource, 1);
|
||||
resReq.setNodeLabelExpression("x");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
|
||||
fail("Should fail");
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
|
@ -350,8 +347,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq = BuilderUtils.newResourceRequest(
|
||||
mock(Priority.class), ResourceRequest.ANY, resource, 1);
|
||||
resReq.setNodeLabelExpression("z");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
fail("Should fail");
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
} finally {
|
||||
|
@ -375,8 +372,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq = BuilderUtils.newResourceRequest(
|
||||
mock(Priority.class), ResourceRequest.ANY, resource, 1);
|
||||
resReq.setNodeLabelExpression("x && y");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
fail("Should fail");
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
} finally {
|
||||
|
@ -395,16 +392,16 @@ public class TestSchedulerUtils {
|
|||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
|
||||
ResourceRequest resReq = BuilderUtils.newResourceRequest(
|
||||
mock(Priority.class), ResourceRequest.ANY, resource, 1);
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
|
||||
resReq.setNodeLabelExpression("");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
|
||||
resReq.setNodeLabelExpression(" ");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
e.printStackTrace();
|
||||
fail("Should be valid when request labels is empty");
|
||||
|
@ -424,8 +421,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq = BuilderUtils.newResourceRequest(
|
||||
mock(Priority.class), ResourceRequest.ANY, resource, 1);
|
||||
resReq.setNodeLabelExpression("x");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
fail("Should fail");
|
||||
} catch (InvalidLabelResourceRequestException e) {
|
||||
invalidlabelexception=true;
|
||||
|
@ -452,16 +449,16 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq = BuilderUtils.newResourceRequest(
|
||||
mock(Priority.class), ResourceRequest.ANY, resource, 1);
|
||||
resReq.setNodeLabelExpression("x");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
|
||||
resReq.setNodeLabelExpression("y");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
|
||||
resReq.setNodeLabelExpression("z");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
e.printStackTrace();
|
||||
fail("Should be valid when queue can access any labels");
|
||||
|
@ -482,8 +479,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq = BuilderUtils.newResourceRequest(
|
||||
mock(Priority.class), ResourceRequest.ANY, resource, 1);
|
||||
resReq.setNodeLabelExpression("x");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
fail("Should fail");
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
}
|
||||
|
@ -503,8 +500,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq = BuilderUtils.newResourceRequest(
|
||||
mock(Priority.class), "rack", resource, 1);
|
||||
resReq.setNodeLabelExpression("x");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
fail("Should fail");
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
} finally {
|
||||
|
@ -528,8 +525,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq = BuilderUtils.newResourceRequest(
|
||||
mock(Priority.class), "rack", resource, 1);
|
||||
resReq.setNodeLabelExpression("x");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
fail("Should fail");
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
} finally {
|
||||
|
@ -541,8 +538,8 @@ public class TestSchedulerUtils {
|
|||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
|
||||
ResourceRequest resReq1 = BuilderUtils
|
||||
.newResourceRequest(mock(Priority.class), "*", resource, 1, "x");
|
||||
normalizeAndvalidateRequest(resReq1, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq1, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
fail("Should fail");
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
assertEquals("Invalid label resource request, cluster do not contain , "
|
||||
|
@ -556,8 +553,8 @@ public class TestSchedulerUtils {
|
|||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
|
||||
ResourceRequest resReq1 = BuilderUtils
|
||||
.newResourceRequest(mock(Priority.class), "*", resource, 1, "x");
|
||||
normalizeAndvalidateRequest(resReq1, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq1, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
Assert.assertEquals(RMNodeLabelsManager.NO_LABEL,
|
||||
resReq1.getNodeLabelExpression());
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
|
@ -567,21 +564,14 @@ public class TestSchedulerUtils {
|
|||
}
|
||||
|
||||
@Test (timeout = 30000)
|
||||
public void testValidateResourceRequest() throws IOException {
|
||||
ResourceScheduler mockScheduler = mock(ResourceScheduler.class);
|
||||
|
||||
QueueInfo queueInfo = mock(QueueInfo.class);
|
||||
when(queueInfo.getQueueName()).thenReturn("queue");
|
||||
public void testValidateResourceRequest() {
|
||||
YarnScheduler mockScheduler = mock(YarnScheduler.class);
|
||||
|
||||
Resource maxResource =
|
||||
Resources.createResource(
|
||||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
||||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
|
||||
|
||||
when(rmContext.getScheduler()).thenReturn(mockScheduler);
|
||||
when(mockScheduler.getQueueInfo(Mockito.anyString(), Mockito.anyBoolean(),
|
||||
Mockito.anyBoolean())).thenReturn(queueInfo);
|
||||
|
||||
// zero memory
|
||||
try {
|
||||
Resource resource =
|
||||
|
@ -590,8 +580,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq =
|
||||
BuilderUtils.newResourceRequest(mock(Priority.class),
|
||||
ResourceRequest.ANY, resource, 1);
|
||||
normalizeAndvalidateRequest(resReq, null,
|
||||
mockScheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
|
||||
mockScheduler, rmContext);
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
fail("Zero memory should be accepted");
|
||||
}
|
||||
|
@ -604,8 +594,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq =
|
||||
BuilderUtils.newResourceRequest(mock(Priority.class),
|
||||
ResourceRequest.ANY, resource, 1);
|
||||
normalizeAndvalidateRequest(resReq, null,
|
||||
mockScheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
|
||||
mockScheduler, rmContext);
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
fail("Zero vcores should be accepted");
|
||||
}
|
||||
|
@ -619,8 +609,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq =
|
||||
BuilderUtils.newResourceRequest(mock(Priority.class),
|
||||
ResourceRequest.ANY, resource, 1);
|
||||
normalizeAndvalidateRequest(resReq, null,
|
||||
mockScheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
|
||||
mockScheduler, rmContext);
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
fail("Max memory should be accepted");
|
||||
}
|
||||
|
@ -634,8 +624,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq =
|
||||
BuilderUtils.newResourceRequest(mock(Priority.class),
|
||||
ResourceRequest.ANY, resource, 1);
|
||||
normalizeAndvalidateRequest(resReq, null,
|
||||
mockScheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
|
||||
mockScheduler, rmContext);
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
fail("Max vcores should not be accepted");
|
||||
}
|
||||
|
@ -648,8 +638,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq =
|
||||
BuilderUtils.newResourceRequest(mock(Priority.class),
|
||||
ResourceRequest.ANY, resource, 1);
|
||||
normalizeAndvalidateRequest(resReq, null,
|
||||
mockScheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
|
||||
mockScheduler, rmContext);
|
||||
fail("Negative memory should not be accepted");
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
// expected
|
||||
|
@ -663,8 +653,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq =
|
||||
BuilderUtils.newResourceRequest(mock(Priority.class),
|
||||
ResourceRequest.ANY, resource, 1);
|
||||
normalizeAndvalidateRequest(resReq, null,
|
||||
mockScheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
|
||||
mockScheduler, rmContext);
|
||||
fail("Negative vcores should not be accepted");
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
// expected
|
||||
|
@ -679,8 +669,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq =
|
||||
BuilderUtils.newResourceRequest(mock(Priority.class),
|
||||
ResourceRequest.ANY, resource, 1);
|
||||
normalizeAndvalidateRequest(resReq, null,
|
||||
mockScheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
|
||||
mockScheduler, rmContext);
|
||||
fail("More than max memory should not be accepted");
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
// expected
|
||||
|
@ -694,8 +684,8 @@ public class TestSchedulerUtils {
|
|||
ResourceRequest resReq =
|
||||
BuilderUtils.newResourceRequest(mock(Priority.class),
|
||||
ResourceRequest.ANY, resource, 1);
|
||||
normalizeAndvalidateRequest(resReq, null,
|
||||
mockScheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, null,
|
||||
mockScheduler, rmContext);
|
||||
fail("More than max vcores should not be accepted");
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
// expected
|
||||
|
@ -808,7 +798,7 @@ public class TestSchedulerUtils {
|
|||
public void testNormalizeNodeLabelExpression()
|
||||
throws IOException {
|
||||
// mock queue and scheduler
|
||||
ResourceScheduler scheduler = mock(ResourceScheduler.class);
|
||||
YarnScheduler scheduler = mock(YarnScheduler.class);
|
||||
Set<String> queueAccessibleNodeLabels = Sets.newHashSet();
|
||||
QueueInfo queueInfo = mock(QueueInfo.class);
|
||||
when(queueInfo.getQueueName()).thenReturn("queue");
|
||||
|
@ -821,8 +811,6 @@ public class TestSchedulerUtils {
|
|||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
||||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
|
||||
|
||||
when(rmContext.getScheduler()).thenReturn(scheduler);
|
||||
|
||||
// queue has labels, success cases
|
||||
try {
|
||||
// set queue accessible node labels to [x, y]
|
||||
|
@ -836,13 +824,13 @@ public class TestSchedulerUtils {
|
|||
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
|
||||
ResourceRequest resReq = BuilderUtils.newResourceRequest(
|
||||
mock(Priority.class), ResourceRequest.ANY, resource, 1);
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
Assert.assertEquals("x", resReq.getNodeLabelExpression());
|
||||
|
||||
resReq.setNodeLabelExpression(" y ");
|
||||
normalizeAndvalidateRequest(resReq, "queue",
|
||||
scheduler, rmContext, maxResource);
|
||||
SchedulerUtils.normalizeAndvalidateRequest(resReq, maxResource, "queue",
|
||||
scheduler, rmContext);
|
||||
Assert.assertEquals("y", resReq.getNodeLabelExpression());
|
||||
} catch (InvalidResourceRequestException e) {
|
||||
e.printStackTrace();
|
||||
|
@ -1031,14 +1019,6 @@ public class TestSchedulerUtils {
|
|||
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 StringBuilder sb;
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
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.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
|
@ -840,41 +838,6 @@ public class TestCapacityScheduler extends CapacitySchedulerTestBase {
|
|||
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
|
||||
public void testRefreshQueues() throws Exception {
|
||||
|
@ -4049,7 +4012,7 @@ public class TestCapacityScheduler extends CapacitySchedulerTestBase {
|
|||
private void setMaxAllocMb(CapacitySchedulerConfiguration conf,
|
||||
String queueName, int maxAllocMb) {
|
||||
String propName = CapacitySchedulerConfiguration.getQueuePrefix(queueName)
|
||||
+ MAXIMUM_ALLOCATION_MB;
|
||||
+ CapacitySchedulerConfiguration.MAXIMUM_ALLOCATION_MB;
|
||||
conf.setInt(propName, maxAllocMb);
|
||||
}
|
||||
|
||||
|
|
|
@ -75,7 +75,6 @@ public class FairSchedulerTestBase {
|
|||
public static final float TEST_RESERVATION_THRESHOLD = 0.09f;
|
||||
private static final int SLEEP_DURATION = 10;
|
||||
private static final int SLEEP_RETRIES = 1000;
|
||||
protected static final int RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE = 10240;
|
||||
final static ContainerUpdates NULL_UPDATE_REQUESTS =
|
||||
new ContainerUpdates();
|
||||
|
||||
|
@ -92,8 +91,7 @@ public class FairSchedulerTestBase {
|
|||
conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 0);
|
||||
conf.setInt(FairSchedulerConfiguration.RM_SCHEDULER_INCREMENT_ALLOCATION_MB,
|
||||
1024);
|
||||
conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
|
||||
RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE);
|
||||
conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 10240);
|
||||
conf.setBoolean(FairSchedulerConfiguration.ASSIGN_MULTIPLE, false);
|
||||
conf.setLong(FairSchedulerConfiguration.UPDATE_INTERVAL_MS, 10);
|
||||
conf.setFloat(FairSchedulerConfiguration.PREEMPTION_THRESHOLD, 0f);
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
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.server.resourcemanager.reservation.ReservationSchedulerConfiguration;
|
||||
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.FairSharePolicy;
|
||||
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.TestResourceUtils;
|
||||
import org.junit.Test;
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
|
@ -45,7 +42,6 @@ import java.io.PrintWriter;
|
|||
import java.net.URISyntaxException;
|
||||
import java.net.URL;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
|
@ -205,7 +201,7 @@ public class TestAllocationFileLoaderService {
|
|||
|
||||
@Test
|
||||
public void testAllocationFileParsing() throws Exception {
|
||||
Configuration conf = new YarnConfiguration();
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
||||
AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
|
||||
|
||||
|
@ -248,7 +244,6 @@ public class TestAllocationFileLoaderService {
|
|||
.fairSharePreemptionTimeout(120)
|
||||
.minSharePreemptionTimeout(50)
|
||||
.fairSharePreemptionThreshold(0.6)
|
||||
.maxContainerAllocation("512mb,16vcores")
|
||||
// Create hierarchical queues G,H, with different min/fair
|
||||
// share preemption timeouts and preemption thresholds.
|
||||
// Also add a child default to make sure it doesn't impact queue H.
|
||||
|
@ -256,7 +251,6 @@ public class TestAllocationFileLoaderService {
|
|||
.fairSharePreemptionTimeout(180)
|
||||
.minSharePreemptionTimeout(40)
|
||||
.fairSharePreemptionThreshold(0.7)
|
||||
.maxContainerAllocation("1024mb,8vcores")
|
||||
.buildSubQueue()
|
||||
.buildQueue()
|
||||
// Set default limit of apps per queue to 15
|
||||
|
@ -382,28 +376,6 @@ public class TestAllocationFileLoaderService {
|
|||
assertEquals("alice,bob admins", queueConf.getQueueAcl("root.queueC",
|
||||
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(-1, queueConf.getMinSharePreemptionTimeout("root." +
|
||||
YarnConfiguration.DEFAULT_QUEUE_NAME));
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -18,7 +18,6 @@
|
|||
|
||||
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.assertFalse;
|
||||
import static org.junit.Assert.assertNotEquals;
|
||||
|
@ -194,6 +193,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
}
|
||||
}
|
||||
|
||||
// TESTS
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
@Test(timeout=2000)
|
||||
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
|
||||
* the shares.
|
||||
|
@ -1415,9 +1311,8 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
// New node satisfies resource request
|
||||
scheduler.update();
|
||||
scheduler.handle(new NodeUpdateSchedulerEvent(node4));
|
||||
assertEquals(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE,
|
||||
scheduler.getQueueManager().getQueue("queue1").getResourceUsage()
|
||||
.getMemorySize());
|
||||
assertEquals(10240, scheduler.getQueueManager().getQueue("queue1").
|
||||
getResourceUsage().getMemorySize());
|
||||
|
||||
scheduler.handle(new NodeUpdateSchedulerEvent(node1));
|
||||
scheduler.handle(new NodeUpdateSchedulerEvent(node2));
|
||||
|
@ -4199,11 +4094,11 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
scheduler.start();
|
||||
scheduler.reinitialize(conf, resourceManager.getRMContext());
|
||||
|
||||
RMNode node1 = MockNodes.newNodeInfo(1,
|
||||
Resources.createResource(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, 10),
|
||||
RMNode node1 =
|
||||
MockNodes.newNodeInfo(1, Resources.createResource(10240, 10),
|
||||
1, "127.0.0.1");
|
||||
RMNode node2 = MockNodes.newNodeInfo(1,
|
||||
Resources.createResource(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, 10),
|
||||
RMNode node2 =
|
||||
MockNodes.newNodeInfo(1, Resources.createResource(10240, 10),
|
||||
2, "127.0.0.2");
|
||||
RMNode node3 =
|
||||
MockNodes.newNodeInfo(1, Resources.createResource(5120, 5),
|
||||
|
@ -4222,12 +4117,10 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
true);
|
||||
Resource amResource1 = Resource.newInstance(1024, 1);
|
||||
Resource amResource2 = Resource.newInstance(1024, 1);
|
||||
Resource amResource3 =
|
||||
Resource.newInstance(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, 1);
|
||||
Resource amResource3 = Resource.newInstance(10240, 1);
|
||||
Resource amResource4 = Resource.newInstance(5120, 1);
|
||||
Resource amResource5 = Resource.newInstance(1024, 1);
|
||||
Resource amResource6 =
|
||||
Resource.newInstance(RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, 1);
|
||||
Resource amResource6 = Resource.newInstance(10240, 1);
|
||||
Resource amResource7 = Resource.newInstance(1024, 1);
|
||||
Resource amResource8 = Resource.newInstance(1024, 1);
|
||||
int amPriority = RMAppAttemptImpl.AM_CONTAINER_PRIORITY.getPriority();
|
||||
|
@ -4261,8 +4154,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|||
|
||||
ApplicationAttemptId attId3 = createAppAttemptId(3, 1);
|
||||
createApplicationWithAMResource(attId3, "queue1", "user1", amResource3);
|
||||
createSchedulingRequestExistingApplication(
|
||||
RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, 1, amPriority, attId3);
|
||||
createSchedulingRequestExistingApplication(10240, 1, amPriority, attId3);
|
||||
FSAppAttempt app3 = scheduler.getSchedulerApp(attId3);
|
||||
scheduler.update();
|
||||
// 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);
|
||||
createApplicationWithAMResource(attId6, "queue1", "user1", amResource6);
|
||||
createSchedulingRequestExistingApplication(
|
||||
RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE, 1, amPriority, attId6);
|
||||
createSchedulingRequestExistingApplication(10240, 1, amPriority, attId6);
|
||||
FSAppAttempt app6 = scheduler.getSchedulerApp(attId6);
|
||||
scheduler.update();
|
||||
// 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.
|
||||
scheduler.handle(updateE2);
|
||||
assertEquals("Application6's AM requests 10240 MB memory",
|
||||
RM_SCHEDULER_MAXIMUM_ALLOCATION_MB_VALUE,
|
||||
app6.getAMResource().getMemorySize());
|
||||
10240, app6.getAMResource().getMemorySize());
|
||||
assertEquals("Application6's AM should be running",
|
||||
1, app6.getLiveContainers().size());
|
||||
assertEquals("Queue1's AM resource usage should be 11264 MB memory",
|
||||
|
|
|
@ -60,11 +60,9 @@ class AllocationFileQueue {
|
|||
() -> AllocationFileWriter
|
||||
.createNumberSupplier(properties.getFairSharePreemptionTimeout()));
|
||||
AllocationFileWriter.addIfPresent(pw, "fairSharePreemptionThreshold",
|
||||
() -> AllocationFileWriter.createNumberSupplier(
|
||||
properties.getFairSharePreemptionThreshold()));
|
||||
AllocationFileWriter.addIfPresent(pw, "maxContainerAllocation",
|
||||
() -> AllocationFileWriter
|
||||
.createNumberSupplier(properties.getMaxContainerAllocation()));
|
||||
.createNumberSupplier(
|
||||
properties.getFairSharePreemptionThreshold()));
|
||||
printEndTag(pw);
|
||||
pw.close();
|
||||
return sw.toString();
|
||||
|
|
|
@ -94,12 +94,6 @@ public abstract class AllocationFileQueueBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public AllocationFileQueueBuilder maxContainerAllocation(
|
||||
String maxContainerAllocation) {
|
||||
this.queuePropertiesBuilder.maxContainerAllocation(maxContainerAllocation);
|
||||
return this;
|
||||
}
|
||||
|
||||
public AllocationFileQueueBuilder subQueue(String queueName) {
|
||||
if (this instanceof AllocationFileSimpleQueueBuilder) {
|
||||
return new AllocationFileSubQueueBuilder(
|
||||
|
|
|
@ -33,7 +33,6 @@ public class AllocationFileQueueProperties {
|
|||
private final String maxChildResources;
|
||||
private final Integer fairSharePreemptionTimeout;
|
||||
private final Double fairSharePreemptionThreshold;
|
||||
private final String maxContainerAllocation;
|
||||
|
||||
AllocationFileQueueProperties(Builder builder) {
|
||||
this.queueName = builder.queueName;
|
||||
|
@ -49,7 +48,6 @@ public class AllocationFileQueueProperties {
|
|||
this.maxChildResources = builder.maxChildResources;
|
||||
this.fairSharePreemptionTimeout = builder.fairSharePreemptionTimeout;
|
||||
this.fairSharePreemptionThreshold = builder.fairSharePreemptionThreshold;
|
||||
this.maxContainerAllocation = builder.maxContainerAllocation;
|
||||
}
|
||||
|
||||
public String getQueueName() {
|
||||
|
@ -104,10 +102,6 @@ public class AllocationFileQueueProperties {
|
|||
return fairSharePreemptionThreshold;
|
||||
}
|
||||
|
||||
public String getMaxContainerAllocation() {
|
||||
return maxContainerAllocation;
|
||||
}
|
||||
|
||||
/**
|
||||
* Builder class for {@link AllocationFileQueueProperties}.
|
||||
*/
|
||||
|
@ -125,7 +119,6 @@ public class AllocationFileQueueProperties {
|
|||
private String maxChildResources;
|
||||
private Integer fairSharePreemptionTimeout;
|
||||
private Double fairSharePreemptionThreshold;
|
||||
private String maxContainerAllocation;
|
||||
|
||||
Builder() {
|
||||
}
|
||||
|
@ -174,11 +167,6 @@ public class AllocationFileQueueProperties {
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder maxContainerAllocation(String maxContainerAllocation) {
|
||||
this.maxContainerAllocation = maxContainerAllocation;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder minSharePreemptionTimeout(
|
||||
Integer minSharePreemptionTimeout) {
|
||||
this.minSharePreemptionTimeout = minSharePreemptionTimeout;
|
||||
|
|
|
@ -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.
|
||||
|
||||
* **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
|
||||
|
||||
* **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.
|
||||
|
|
Loading…
Reference in New Issue