Skip to content

Commit 03f855e

Browse files
YARN-10316. FS-CS converter: convert maxAppsDefault, maxRunningApps settings. Contributed by Peter Bacsko
1 parent fa14e4b commit 03f855e

File tree

6 files changed

+76
-75
lines changed

6 files changed

+76
-75
lines changed

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSConfigToCSConfigConverter.java

Lines changed: 28 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -83,6 +83,9 @@ public class FSConfigToCSConfigConverter {
8383
private boolean preemptionEnabled = false;
8484
private int queueMaxAppsDefault;
8585
private float queueMaxAMShareDefault;
86+
private Map<String, Integer> userMaxApps;
87+
private int userMaxAppsDefault;
88+
8689
private boolean autoCreateChildQueues = false;
8790
private boolean sizeBasedWeight = false;
8891
private boolean userAsDefaultQueue = false;
@@ -99,6 +102,8 @@ public class FSConfigToCSConfigConverter {
99102
private boolean consoleMode = false;
100103
private boolean convertPlacementRules = false;
101104

105+
106+
102107
public FSConfigToCSConfigConverter(FSConfigToCSConfigRuleHandler
103108
ruleHandler, ConversionOptions conversionOptions) {
104109
this.ruleHandler = ruleHandler;
@@ -242,14 +247,13 @@ void convert(Configuration inputYarnSiteConfig) throws Exception {
242247

243248
AllocationConfiguration allocConf = fs.getAllocationConfiguration();
244249
queueMaxAppsDefault = allocConf.getQueueMaxAppsDefault();
250+
userMaxAppsDefault = allocConf.getUserMaxAppsDefault();
251+
userMaxApps = allocConf.getUserMaxApps();
245252
queueMaxAMShareDefault = allocConf.getQueueMaxAMShareDefault();
246253

247254
convertedYarnSiteConfig = new Configuration(false);
248255
capacitySchedulerConfig = new Configuration(false);
249256

250-
checkUserMaxApps(allocConf);
251-
checkUserMaxAppsDefault(allocConf);
252-
253257
convertYarnSiteXml(inputYarnSiteConfig, havePlacementPolicies);
254258
convertCapacitySchedulerXml(fs);
255259

@@ -287,7 +291,9 @@ private void convertYarnSiteXml(Configuration inputYarnSiteConfig,
287291

288292
private void convertCapacitySchedulerXml(FairScheduler fs) {
289293
FSParentQueue rootQueue = fs.getQueueManager().getRootQueue();
290-
emitDefaultMaxApplications();
294+
emitDefaultQueueMaxParallelApplications();
295+
emitDefaultUserMaxParallelApplications();
296+
emitUserMaxParallelApplications();
291297
emitDefaultMaxAMShare();
292298

293299
FSQueueConverter queueConverter = FSQueueConverterBuilder.create()
@@ -322,14 +328,30 @@ private void convertCapacitySchedulerXml(FairScheduler fs) {
322328
}
323329
}
324330

325-
private void emitDefaultMaxApplications() {
331+
private void emitDefaultQueueMaxParallelApplications() {
326332
if (queueMaxAppsDefault != Integer.MAX_VALUE) {
327333
capacitySchedulerConfig.set(
328-
CapacitySchedulerConfiguration.MAXIMUM_SYSTEM_APPLICATIONS,
334+
PREFIX + "max-parallel-apps",
329335
String.valueOf(queueMaxAppsDefault));
330336
}
331337
}
332338

339+
private void emitDefaultUserMaxParallelApplications() {
340+
if (userMaxAppsDefault != Integer.MAX_VALUE) {
341+
capacitySchedulerConfig.set(
342+
PREFIX + "user.max-parallel-apps",
343+
String.valueOf(userMaxAppsDefault));
344+
}
345+
}
346+
347+
private void emitUserMaxParallelApplications() {
348+
userMaxApps
349+
.forEach((user, apps) -> {
350+
capacitySchedulerConfig.setInt(
351+
PREFIX + "user." + user + ".max-parallel-apps", apps);
352+
});
353+
}
354+
333355
private void emitDefaultMaxAMShare() {
334356
if (queueMaxAMShareDefault == QUEUE_MAX_AM_SHARE_DISABLED) {
335357
capacitySchedulerConfig.setFloat(
@@ -374,19 +396,6 @@ private void checkReservationSystem(Configuration conf) {
374396
}
375397
}
376398

377-
private void checkUserMaxApps(AllocationConfiguration allocConf) {
378-
if (allocConf.getUserMaxApps() != null
379-
&& allocConf.getUserMaxApps().size() > 0) {
380-
ruleHandler.handleUserMaxApps();
381-
}
382-
}
383-
384-
private void checkUserMaxAppsDefault(AllocationConfiguration allocConf) {
385-
if (allocConf.getUserMaxAppsDefault() > 0) {
386-
ruleHandler.handleUserMaxAppsDefault();
387-
}
388-
}
389-
390399
private boolean isDrfUsed(FairScheduler fs) {
391400
FSQueue rootQueue = fs.getQueueManager().getRootQueue();
392401
AllocationConfiguration allocConf = fs.getAllocationConfiguration();

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSConfigToCSConfigRuleHandler.java

Lines changed: 0 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -170,14 +170,6 @@ public void handleChildQueueCount(String queue, int count) {
170170
}
171171
}
172172

173-
public void handleUserMaxApps() {
174-
handle(USER_MAX_RUNNING_APPS, "<maxRunningApps>", null);
175-
}
176-
177-
public void handleUserMaxAppsDefault() {
178-
handle(USER_MAX_APPS_DEFAULT, "<userMaxAppsDefault>", null);
179-
}
180-
181173
public void handleDynamicMaxAssign() {
182174
handle(DYNAMIC_MAX_ASSIGN,
183175
FairSchedulerConfiguration.DYNAMIC_MAX_ASSIGN, null);

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSQueueConverter.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@
4343
*/
4444
public class FSQueueConverter {
4545
public static final float QUEUE_MAX_AM_SHARE_DISABLED = -1.0f;
46-
private static final int MAX_RUNNING_APPS_UNSET = Integer.MIN_VALUE;
46+
private static final int MAX_RUNNING_APPS_UNSET = Integer.MAX_VALUE;
4747
private static final String FAIR_POLICY = "fair";
4848
private static final String FIFO_POLICY = "fifo";
4949

@@ -79,7 +79,7 @@ public void convertQueueHierarchy(FSQueue queue) {
7979

8080
emitChildQueues(queueName, children);
8181
emitMaxAMShare(queueName, queue);
82-
emitMaxRunningApps(queueName, queue);
82+
emitMaxParallelApps(queueName, queue);
8383
emitMaxAllocations(queueName, queue);
8484
emitPreemptionDisabled(queueName, queue);
8585

@@ -138,14 +138,14 @@ private void emitMaxAMShare(String queueName, FSQueue queue) {
138138

139139
/**
140140
* &lt;maxRunningApps&gt;
141-
* ==> yarn.scheduler.capacity.&lt;queue-name&gt;.maximum-applications.
141+
* ==> yarn.scheduler.capacity.&lt;queue-name&gt;.max-parallel-apps.
142142
* @param queueName
143143
* @param queue
144144
*/
145-
private void emitMaxRunningApps(String queueName, FSQueue queue) {
145+
private void emitMaxParallelApps(String queueName, FSQueue queue) {
146146
if (queue.getMaxRunningApps() != MAX_RUNNING_APPS_UNSET
147147
&& queue.getMaxRunningApps() != queueMaxAppsDefault) {
148-
capacitySchedulerConfig.set(PREFIX + queueName + ".maximum-applications",
148+
capacitySchedulerConfig.set(PREFIX + queueName + ".max-parallel-apps",
149149
String.valueOf(queue.getMaxRunningApps()));
150150
}
151151
}

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverter.java

Lines changed: 40 additions & 34 deletions
Original file line numberDiff line numberDiff line change
@@ -155,17 +155,7 @@ private void createConverter() {
155155
.withOutputDirectory(FSConfigConverterTestCommons.OUTPUT_DIR);
156156
}
157157

158-
@Test
159-
public void testDefaultMaxApplications() throws Exception {
160-
converter.convert(config);
161-
162-
Configuration conf = converter.getCapacitySchedulerConfig();
163-
int maxApps =
164-
conf.getInt(
165-
CapacitySchedulerConfiguration.MAXIMUM_SYSTEM_APPLICATIONS, -1);
166158

167-
assertEquals("Default max apps", 15, maxApps);
168-
}
169159

170160
@Test
171161
public void testDefaultMaxAMShare() throws Exception {
@@ -252,57 +242,73 @@ public void testConvertACLs() throws Exception {
252242
}
253243

254244
@Test
255-
public void testDefaultMaxRunningApps() throws Exception {
245+
public void testDefaultQueueMaxParallelApps() throws Exception {
256246
converter.convert(config);
257247

258248
Configuration conf = converter.getCapacitySchedulerConfig();
259249

260-
// default setting
261-
assertEquals("Default max apps", 15,
262-
conf.getInt(PREFIX + "maximum-applications", -1));
250+
assertEquals("Default max parallel apps", 15,
251+
conf.getInt(PREFIX + "max-parallel-apps", -1));
263252
}
264253

265254
@Test
266-
public void testQueueMaxChildCapacityNotSupported() throws Exception {
267-
expectedException.expect(UnsupportedPropertyException.class);
268-
expectedException.expectMessage("test");
255+
public void testSpecificQueueMaxParallelApps() throws Exception {
256+
converter.convert(config);
269257

270-
Mockito.doThrow(new UnsupportedPropertyException("test"))
271-
.when(ruleHandler).handleMaxChildCapacity();
258+
Configuration conf = converter.getCapacitySchedulerConfig();
272259

273-
converter.convert(config);
260+
assertEquals("root.admins.alice max parallel apps", 2,
261+
conf.getInt(PREFIX + "root.admins.alice.max-parallel-apps", -1));
274262
}
275263

276264
@Test
277-
public void testReservationSystemNotSupported() throws Exception {
278-
expectedException.expect(UnsupportedPropertyException.class);
279-
expectedException.expectMessage("maxCapacity");
265+
public void testDefaultUserMaxParallelApps() throws Exception {
266+
converter.convert(config);
280267

281-
Mockito.doThrow(new UnsupportedPropertyException("maxCapacity"))
282-
.when(ruleHandler).handleMaxChildCapacity();
283-
config.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE, true);
268+
Configuration conf = converter.getCapacitySchedulerConfig();
269+
int userMaxParallelApps =
270+
conf.getInt(
271+
PREFIX + "user.max-parallel-apps", -1);
272+
273+
assertEquals("Default user max parallel apps", 10,
274+
userMaxParallelApps);
275+
}
284276

277+
@Test
278+
public void testSpecificUserMaxParallelApps() throws Exception {
285279
converter.convert(config);
280+
281+
Configuration conf = converter.getCapacitySchedulerConfig();
282+
283+
assertEquals("Max parallel apps for alice", 30,
284+
conf.getInt(PREFIX + "user.alice.max-parallel-apps", -1));
285+
assertNull("Max parallel apps should be undefined for user bob",
286+
conf.get(PREFIX + "user.bob.max-parallel-apps"));
287+
assertNull("Max parallel apps should be undefined for user joe",
288+
conf.get(PREFIX + "user.joe.max-parallel-apps"));
289+
assertNull("Max parallel apps should be undefined for user john",
290+
conf.get(PREFIX + "user.john.max-parallel-apps"));
286291
}
287292

288293
@Test
289-
public void testUserMaxAppsNotSupported() throws Exception {
294+
public void testQueueMaxChildCapacityNotSupported() throws Exception {
290295
expectedException.expect(UnsupportedPropertyException.class);
291-
expectedException.expectMessage("userMaxApps");
296+
expectedException.expectMessage("test");
292297

293-
Mockito.doThrow(new UnsupportedPropertyException("userMaxApps"))
294-
.when(ruleHandler).handleUserMaxApps();
298+
Mockito.doThrow(new UnsupportedPropertyException("test"))
299+
.when(ruleHandler).handleMaxChildCapacity();
295300

296301
converter.convert(config);
297302
}
298303

299304
@Test
300-
public void testUserMaxAppsDefaultNotSupported() throws Exception {
305+
public void testReservationSystemNotSupported() throws Exception {
301306
expectedException.expect(UnsupportedPropertyException.class);
302-
expectedException.expectMessage("userMaxAppsDefault");
307+
expectedException.expectMessage("maxCapacity");
303308

304-
Mockito.doThrow(new UnsupportedPropertyException("userMaxAppsDefault"))
305-
.when(ruleHandler).handleUserMaxAppsDefault();
309+
Mockito.doThrow(new UnsupportedPropertyException("maxCapacity"))
310+
.when(ruleHandler).handleMaxChildCapacity();
311+
config.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE, true);
306312

307313
converter.convert(config);
308314
}

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigRuleHandler.java

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -76,8 +76,6 @@ public void testInitPropertyActionsToWarning() throws IOException {
7676
ruleHandler.handleQueueAutoCreate("test");
7777
ruleHandler.handleReservationSystem();
7878
ruleHandler.handleSpecifiedNotFirstRule();
79-
ruleHandler.handleUserMaxApps();
80-
ruleHandler.handleUserMaxAppsDefault();
8179
}
8280

8381
@Test
@@ -106,8 +104,6 @@ public void testAllRulesWarning() throws IOException {
106104
ruleHandler.handleQueueAutoCreate("test");
107105
ruleHandler.handleReservationSystem();
108106
ruleHandler.handleSpecifiedNotFirstRule();
109-
ruleHandler.handleUserMaxApps();
110-
ruleHandler.handleUserMaxAppsDefault();
111107
}
112108

113109
@Test
@@ -140,8 +136,6 @@ public void testAllRulesAbort() throws IOException {
140136
expectAbort(() -> ruleHandler.handleQueueAutoCreate("test"));
141137
expectAbort(() -> ruleHandler.handleReservationSystem());
142138
expectAbort(() -> ruleHandler.handleSpecifiedNotFirstRule());
143-
expectAbort(() -> ruleHandler.handleUserMaxApps());
144-
expectAbort(() -> ruleHandler.handleUserMaxAppsDefault());
145139
expectAbort(() -> ruleHandler.handleFairAsDrf("test"));
146140
}
147141

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSQueueConverter.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -195,18 +195,18 @@ public void testQueueMaxAMShare() {
195195
}
196196

197197
@Test
198-
public void testQueueMaxRunningApps() {
198+
public void testQueueMaxParallelApps() {
199199
converter = builder.build();
200200

201201
converter.convertQueueHierarchy(rootQueue);
202202

203203
assertEquals("root.admins.alice max apps", 2,
204-
csConfig.getInt(PREFIX + "root.admins.alice.maximum-applications",
204+
csConfig.getInt(PREFIX + "root.admins.alice.max-parallel-apps",
205205
-1));
206206

207207
Set<String> remaining = Sets.difference(ALL_QUEUES,
208208
Sets.newHashSet("root.admins.alice"));
209-
assertNoValueForQueues(remaining, ".maximum-applications", csConfig);
209+
assertNoValueForQueues(remaining, ".max-parallel-apps", csConfig);
210210
}
211211

212212
@Test

0 commit comments

Comments
 (0)