Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ public abstract class AbstractCSQueue implements CSQueue {
String defaultLabelExpression;
private String multiNodeSortingPolicyName = null;

Map<AccessType, AccessControlList> acls =
Map<AccessType, AccessControlList> acls =
new HashMap<AccessType, AccessControlList>();
volatile boolean reservationsContinueLooking;
private volatile boolean preemptionDisabled;
Expand All @@ -112,7 +112,7 @@ public abstract class AbstractCSQueue implements CSQueue {
volatile ResourceUsage queueUsage;

private final boolean fullPathQueueNamingPolicy = false;

// Track capacities like used-capcity/abs-used-capacity/capacity/abs-capacity,
// etc.
QueueCapacities queueCapacities;
Expand All @@ -134,7 +134,7 @@ protected enum CapacityConfigType {
protected CapacityConfigType capacityConfigType =
CapacityConfigType.NONE;

private final RecordFactory recordFactory =
private final RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);
protected CapacitySchedulerContext csContext;
protected YarnAuthorizationProvider authorizer = null;
Expand Down Expand Up @@ -250,12 +250,12 @@ public int getNumContainers() {
public QueueState getState() {
return state;
}

@Override
public CSQueueMetrics getMetrics() {
return metrics;
}

@Override
public String getQueueShortName() {
return queueName;
Expand Down Expand Up @@ -283,7 +283,7 @@ public CSQueue getParent() {
public void setParent(CSQueue newParentQueue) {
this.parent = newParentQueue;
}

public Set<String> getAccessibleNodeLabels() {
return accessibleLabels;
}
Expand Down Expand Up @@ -344,7 +344,7 @@ void setMaxCapacity(String nodeLabel, float maximumCapacity) {
public String getDefaultNodeLabelExpression() {
return defaultLabelExpression;
}

void setupQueueConfigs(Resource clusterResource)
throws IOException {
setupQueueConfigs(clusterResource, csContext.getConfiguration());
Expand Down Expand Up @@ -471,8 +471,14 @@ protected void setupQueueConfigs(Resource clusterResource,

private void setupMaximumAllocation(CapacitySchedulerConfiguration csConf) {
String myQueuePath = getQueuePath();
/* YARN-10869: When using AutoCreatedLeafQueues, the passed configuration
* object is a cloned one containing only the template configs
* (see ManagedParentQueue#getLeafQueueConfigs). To ensure that the actual
* cluster maximum allocation is fetched the original config object should
* be used.
*/
Resource clusterMax = ResourceUtils
.fetchMaximumAllocationFromConfig(csConf);
.fetchMaximumAllocationFromConfig(this.csContext.getConfiguration());
Resource queueMax = csConf.getQueueMaximumAllocation(myQueuePath);

maximumAllocation = Resources.clone(
Expand Down Expand Up @@ -740,7 +746,7 @@ public QueueStatistics getQueueStatistics() {
stats.setReservedContainers(getMetrics().getReservedContainers());
return stats;
}

public Map<String, QueueConfigurations> getQueueConfigurations() {
Map<String, QueueConfigurations> queueConfigurations = new HashMap<>();
Set<String> nodeLabels = getNodeLabelsForQueue();
Expand Down Expand Up @@ -776,12 +782,12 @@ public Map<String, QueueConfigurations> getQueueConfigurations() {
public Resource getMaximumAllocation() {
return maximumAllocation;
}

@Private
public Resource getMinimumAllocation() {
return minimumAllocation;
}

void allocateResource(Resource clusterResource,
Resource resource, String nodePartition) {
writeLock.lock();
Expand All @@ -796,7 +802,7 @@ void allocateResource(Resource clusterResource,
writeLock.unlock();
}
}

protected void releaseResource(Resource clusterResource,
Resource resource, String nodePartition) {
writeLock.lock();
Expand All @@ -811,12 +817,12 @@ protected void releaseResource(Resource clusterResource,
writeLock.unlock();
}
}

@Private
public boolean getReservationContinueLooking() {
return reservationsContinueLooking;
}

@Private
public Map<AccessType, AccessControlList> getACLs() {
readLock.lock();
Expand All @@ -841,12 +847,12 @@ public boolean getIntraQueuePreemptionDisabled() {
public boolean getIntraQueuePreemptionDisabledInHierarchy() {
return intraQueuePreemptionDisabledInHierarchy;
}

@Private
public QueueCapacities getQueueCapacities() {
return queueCapacities;
}

@Private
public ResourceUsage getQueueResourceUsage() {
return queueUsage;
Expand Down Expand Up @@ -1018,7 +1024,7 @@ private Resource getCurrentLimitResource(String nodePartition,
// all queues on this label equals to total resource with the label.
return labelManager.getResourceByLabel(nodePartition, clusterResource);
}

return Resources.none();
}

Expand Down Expand Up @@ -1159,7 +1165,7 @@ public void incPendingResource(String nodeLabel, Resource resourceToInc) {
parent.incPendingResource(nodeLabel, resourceToInc);
}
}

@Override
public void decPendingResource(String nodeLabel, Resource resourceToDec) {
if (nodeLabel == null) {
Expand All @@ -1171,7 +1177,7 @@ public void decPendingResource(String nodeLabel, Resource resourceToDec) {
parent.decPendingResource(nodeLabel, resourceToDec);
}
}

@Override
public void incUsedResource(String nodeLabel, Resource resourceToInc,
SchedulerApplicationAttempt application) {
Expand Down Expand Up @@ -1206,14 +1212,14 @@ public void decUsedResource(String nodeLabel, Resource resourceToDec,

/**
* Return if the queue has pending resource on given nodePartition and
* schedulingMode.
* schedulingMode.
*/
boolean hasPendingResourceRequest(String nodePartition,
boolean hasPendingResourceRequest(String nodePartition,
Resource cluster, SchedulingMode schedulingMode) {
return SchedulerUtils.hasPendingResourceRequest(resourceCalculator,
queueUsage, nodePartition, cluster, schedulingMode);
}

public boolean accessibleToPartition(String nodePartition) {
// if queue's label is *, it can access any node
if (accessibleLabels != null
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -953,6 +953,11 @@ public Resource getQueueMaximumAllocation(String queue) {
}
}

public void setQueueMaximumAllocation(String queue, String maximumAllocation) {
String queuePrefix = getQueuePrefix(queue);
set(queuePrefix + MAXIMUM_ALLOCATION, maximumAllocation);
}

public long getQueueMaximumAllocationMb(String queue) {
String queuePrefix = getQueuePrefix(queue);
return getInt(queuePrefix + MAXIMUM_ALLOCATION_MB, (int)UNDEFINED);
Expand Down Expand Up @@ -2061,6 +2066,15 @@ public void setAutoCreatedLeafQueueConfigDefaultNodeLabelExpression(String
setDefaultNodeLabelExpression(leafQueueConfPrefix, expression);
}

@Private
@VisibleForTesting
public void setAutoCreatedLeafQueueConfigMaximumAllocation(String
queuePath, String expression) {
String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
queuePath);
setQueueMaximumAllocation(leafQueueConfPrefix, expression);
}

public static String getUnits(String resourceValue) {
String units;
for (int i = 0; i < resourceValue.length(); i++) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -385,6 +385,9 @@ public static CapacitySchedulerConfiguration setupQueueConfiguration(
conf.setAutoCreatedLeafQueueConfigMaxCapacity(C, 100.0f);
conf.setAutoCreatedLeafQueueConfigUserLimit(C, 100);
conf.setAutoCreatedLeafQueueConfigUserLimitFactor(C, 3.0f);
conf.setAutoCreatedLeafQueueConfigUserLimitFactor(C, 3.0f);
conf.setAutoCreatedLeafQueueConfigMaximumAllocation(C,
"memory-mb=10240,vcores=6");

conf.setAutoCreatedLeafQueueTemplateCapacityByLabel(C, NODEL_LABEL_GPU,
NODE_LABEL_GPU_TEMPLATE_CAPACITY);
Expand Down Expand Up @@ -540,9 +543,31 @@ protected CapacitySchedulerConfiguration setupSchedulerConfiguration() {
schedConf.setInt(YarnConfiguration.RESOURCE_TYPES
+ ".memory-mb.maximum-allocation", 16384);


return new CapacitySchedulerConfiguration(schedConf);
}

protected void setSchedulerMinMaxAllocation(CapacitySchedulerConfiguration conf) {
unsetMinMaxAllocation(conf);

conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, 1);
conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, 8);
conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 1024);
conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 18384);

}

private void unsetMinMaxAllocation(CapacitySchedulerConfiguration conf) {
conf.unset(YarnConfiguration.RESOURCE_TYPES
+ ".vcores.minimum-allocation");
conf.unset(YarnConfiguration.RESOURCE_TYPES
+ ".vcores.maximum-allocation");
conf.unset(YarnConfiguration.RESOURCE_TYPES
+ ".memory-mb.minimum-allocation");
conf.unset(YarnConfiguration.RESOURCE_TYPES
+ ".memory-mb.maximum-allocation");
}

protected MockRM setupSchedulerInstance() throws Exception {

if (mockRM != null) {
Expand Down Expand Up @@ -640,10 +665,11 @@ protected void validateUserAndAppLimits(
}

protected void validateContainerLimits(
AutoCreatedLeafQueue autoCreatedLeafQueue) {
assertEquals(8,
AutoCreatedLeafQueue autoCreatedLeafQueue, int vCoreLimit,
long memorySize) {
assertEquals(vCoreLimit,
autoCreatedLeafQueue.getMaximumAllocation().getVirtualCores());
assertEquals(16384,
assertEquals(memorySize,
autoCreatedLeafQueue.getMaximumAllocation().getMemorySize());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ public void testAutoCreateLeafQueueCreation() throws Exception {
expectedChildQueueAbsCapacity, accessibleNodeLabelsOnC);

validateUserAndAppLimits(autoCreatedLeafQueue, 1000, 1000);
validateContainerLimits(autoCreatedLeafQueue);
validateContainerLimits(autoCreatedLeafQueue, 6, 10240);

assertTrue(autoCreatedLeafQueue
.getOrderingPolicy() instanceof FairOrderingPolicy);
Expand Down Expand Up @@ -166,6 +166,35 @@ public void testAutoCreateLeafQueueCreation() throws Exception {
}
}

@Test(timeout = 20000)
public void testAutoCreateLeafQueueCreationSchedulerMaximumAllocation()
throws Exception {
try {
// Check the minimum/maximum allocation settings via the
// yarn.scheduler.minimum/maximum-allocation-mb/vcore property
setSchedulerMinMaxAllocation(cs.getConfiguration());
cs.getConfiguration().setAutoCreatedLeafQueueConfigMaximumAllocation(C,
"memory-mb=18384,vcores=8");
cs.reinitialize(cs.getConfiguration(), mockRM.getRMContext());

// submit an app
submitApp(mockRM, cs.getQueue(PARENT_QUEUE), USER0, USER0, 1, 1);

// check preconditions
List<ApplicationAttemptId> appsInC = cs.getAppsInQueue(PARENT_QUEUE);
assertEquals(1, appsInC.size());
assertNotNull(cs.getQueue(USER0));

AutoCreatedLeafQueue autoCreatedLeafQueue =
(AutoCreatedLeafQueue) cs.getQueue(USER0);

validateContainerLimits(autoCreatedLeafQueue, 8, 18384);
} finally {
cleanupQueue(USER0);
cleanupQueue(TEST_GROUPUSER);
}
}

@Test(timeout = 20000)
public void testAutoCreateLeafQueueCreationUsingFullParentPath()
throws Exception {
Expand Down Expand Up @@ -825,7 +854,7 @@ public void testReinitializeQueuesWithAutoCreatedLeafQueues()
validateCapacities(user3Queue, 0.3f, 0.09f, 0.4f,0.2f);

validateUserAndAppLimits(user3Queue, 900, 900);
validateContainerLimits(user3Queue);
validateContainerLimits(user3Queue, 6, 10240);

GuaranteedOrZeroCapacityOverTimePolicy autoCreatedQueueManagementPolicy =
(GuaranteedOrZeroCapacityOverTimePolicy) ((ManagedParentQueue)
Expand Down