浏览代码

YARN-10167. FS-CS Converter: Need to validate c-s.xml after converting. Contributed by Peter Bacsko

Szilard Nemeth 5 年之前
父节点
当前提交
004e955348
共有 14 个文件被更改,包括 495 次插入14 次删除
  1. 7 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/ConversionOptions.java
  2. 80 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/ConvertedConfigValidator.java
  3. 7 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/DryRunResultHolder.java
  4. 23 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSConfigToCSConfigArgumentHandler.java
  5. 3 3
      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
  6. 31 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/VerificationException.java
  7. 53 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestConvertedConfigValidator.java
  8. 60 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigArgumentHandler.java
  9. 22 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverterMain.java
  10. 14 1
      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
  11. 71 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/cs-validation-fail/capacity-scheduler.xml
  12. 32 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/cs-validation-fail/yarn-site.xml
  13. 60 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/cs-validation-pass/capacity-scheduler.xml
  14. 32 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/cs-validation-pass/yarn-site.xml

+ 7 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/ConversionOptions.java

@@ -73,6 +73,13 @@ public class ConversionOptions {
     }
   }
 
+  public void handleVerificationFailure(Throwable e, String msg) {
+    FSConfigToCSConfigArgumentHandler.logAndStdErr(e, msg);
+    if (dryRun) {
+      dryRunResultHolder.setVerificationFailed();
+    }
+  }
+
   public void handleParsingFinished() {
     if (dryRun) {
       dryRunResultHolder.printDryRunResults();

+ 80 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/ConvertedConfigValidator.java

@@ -0,0 +1,80 @@
+/*
+ * 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.converter;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider;
+import org.apache.hadoop.yarn.conf.ConfigurationProvider;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Validates the converted capacity-scheduler.xml by starting
+ * a Capacity Scheduler instance.
+ *
+ */
+public class ConvertedConfigValidator {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ConvertedConfigValidator.class);
+
+  public void validateConvertedConfig(String outputDir)
+      throws Exception {
+    QueueMetrics.clearQueueMetrics();
+    Path configPath = new Path(outputDir, "capacity-scheduler.xml");
+
+    CapacitySchedulerConfiguration csConfig =
+        new CapacitySchedulerConfiguration(
+            new Configuration(false), false);
+    csConfig.addResource(configPath);
+
+    Path convertedSiteConfigPath = new Path(outputDir, "yarn-site.xml");
+    Configuration siteConf = new YarnConfiguration(
+        new Configuration(false));
+    siteConf.addResource(convertedSiteConfigPath);
+
+    RMContextImpl rmContext = new RMContextImpl();
+    siteConf.set(YarnConfiguration.FS_BASED_RM_CONF_STORE, outputDir);
+    ConfigurationProvider provider = new FileSystemBasedConfigurationProvider();
+    provider.init(siteConf);
+    rmContext.setConfigurationProvider(provider);
+    RMNodeLabelsManager mgr = new RMNodeLabelsManager();
+    mgr.init(siteConf);
+    rmContext.setNodeLabelManager(mgr);
+
+    try (CapacityScheduler cs = new CapacityScheduler()) {
+      cs.setConf(siteConf);
+      cs.setRMContext(rmContext);
+      cs.serviceInit(csConfig);
+      cs.serviceStart();
+      LOG.info("Capacity scheduler was successfully started");
+      cs.serviceStop();
+    } catch (Exception e) {
+      LOG.error("Could not start Capacity Scheduler", e);
+      throw new VerificationException(
+          "Verification of converted configuration failed", e);
+    }
+  }
+}

+ 7 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/DryRunResultHolder.java

@@ -32,6 +32,7 @@ public class DryRunResultHolder {
 
   private Set<String> warnings;
   private Set<String> errors;
+  private boolean verificationFailed;
 
   public DryRunResultHolder() {
     this.warnings = new HashSet<>();
@@ -46,6 +47,10 @@ public class DryRunResultHolder {
     errors.add(message);
   }
 
+  public void setVerificationFailed() {
+    verificationFailed = true;
+  }
+
   public Set<String> getWarnings() {
     return ImmutableSet.copyOf(warnings);
   }
@@ -64,6 +69,8 @@ public class DryRunResultHolder {
 
     LOG.info("Number of errors: {}", noOfErrors);
     LOG.info("Number of warnings: {}", noOfWarnings);
+    LOG.info("Verification result: {}",
+        verificationFailed ? "FAILED" : "PASSED");
 
     if (noOfErrors > 0) {
       LOG.info("");

+ 23 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSConfigToCSConfigArgumentHandler.java

@@ -50,6 +50,7 @@ public class FSConfigToCSConfigArgumentHandler {
   private FSConfigToCSConfigRuleHandler ruleHandler;
   private FSConfigToCSConfigConverterParams converterParams;
   private ConversionOptions conversionOptions;
+  private ConvertedConfigValidator validator;
 
   private Supplier<FSConfigToCSConfigConverter>
       converterFunc = this::getConverter;
@@ -57,11 +58,14 @@ public class FSConfigToCSConfigArgumentHandler {
   public FSConfigToCSConfigArgumentHandler() {
     this.conversionOptions = new ConversionOptions(new DryRunResultHolder(),
         false);
+    this.validator = new ConvertedConfigValidator();
   }
 
   @VisibleForTesting
-  FSConfigToCSConfigArgumentHandler(ConversionOptions conversionOptions) {
+  FSConfigToCSConfigArgumentHandler(ConversionOptions conversionOptions,
+      ConvertedConfigValidator validator) {
     this.conversionOptions = conversionOptions;
+    this.validator = validator;
   }
 
   /**
@@ -102,6 +106,9 @@ public class FSConfigToCSConfigArgumentHandler {
         "m", "convert-placement-rules",
         "Convert Fair Scheduler placement rules to Capacity" +
         " Scheduler mapping rules", false),
+    SKIP_VERIFICATION("skip verification", "s",
+        "skip-verification",
+        "Skips the verification of the converted configuration", false),
     HELP("help", "h", "help", "Displays the list of options", false);
 
     private final String name;
@@ -147,6 +154,14 @@ public class FSConfigToCSConfigArgumentHandler {
           prepareAndGetConverter(cliParser);
 
       converter.convert(converterParams);
+
+      String outputDir = converterParams.getOutputDirectory();
+      boolean skipVerification =
+          cliParser.hasOption(CliOption.SKIP_VERIFICATION.shortSwitch);
+      if (outputDir != null && !skipVerification) {
+        validator.validateConvertedConfig(
+            converterParams.getOutputDirectory());
+      }
     } catch (ParseException e) {
       String msg = "Options parsing failed: " + e.getMessage();
       logAndStdErr(e, msg);
@@ -166,6 +181,11 @@ public class FSConfigToCSConfigArgumentHandler {
       String msg = "Fatal error during FS config conversion: " + e.getMessage();
       handleException(e, msg);
       retVal = -1;
+    } catch (VerificationException e) {
+      Throwable cause = e.getCause();
+      String msg = "Verification failed: " + e.getCause().getMessage();
+      conversionOptions.handleVerificationFailure(cause, msg);
+      retVal = -1;
     }
 
     conversionOptions.handleParsingFinished();
@@ -177,8 +197,8 @@ public class FSConfigToCSConfigArgumentHandler {
     conversionOptions.handleGenericException(e, msg);
   }
 
-  static void logAndStdErr(Exception e, String msg) {
-    LOG.debug("Stack trace", e);
+  static void logAndStdErr(Throwable t, String msg) {
+    LOG.debug("Stack trace", t);
     LOG.error(msg);
     System.err.println(msg);
   }

+ 3 - 3
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

@@ -94,7 +94,7 @@ public class FSQueueConverter {
 
     emitChildCapacity(queue);
     emitMaximumCapacity(queueName, queue);
-    emitAutoCreateChildQueue(queueName);
+    emitAutoCreateChildQueue(queueName, queue);
     emitSizeBasedWeight(queueName);
     emitOrderingPolicy(queueName, queue);
     checkMaxChildCapacitySetting(queue);
@@ -267,8 +267,8 @@ public class FSQueueConverter {
    * .auto-create-child-queue.enabled.
    * @param queueName
    */
-  private void emitAutoCreateChildQueue(String queueName) {
-    if (autoCreateChildQueues) {
+  private void emitAutoCreateChildQueue(String queueName, FSQueue queue) {
+    if (autoCreateChildQueues && !queue.getChildQueues().isEmpty()) {
       capacitySchedulerConfig.setBoolean(PREFIX + queueName +
           ".auto-create-child-queue.enabled", true);
     }

+ 31 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/VerificationException.java

@@ -0,0 +1,31 @@
+/*
+ * 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.converter;
+
+/**
+ * Thrown when Capacity Scheduler fails to start up with
+ * the converted configuration.
+ */
+public class VerificationException extends RuntimeException {
+  private static final long serialVersionUID = -7697926560416349141L;
+
+  public VerificationException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

+ 53 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestConvertedConfigValidator.java

@@ -0,0 +1,53 @@
+/*
+ * 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.converter;
+
+import java.io.File;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.junit.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestConvertedConfigValidator {
+  private static final String CONFIG_DIR_PASSES =
+      new File("src/test/resources/cs-validation-pass").getAbsolutePath();
+  private static final String CONFIG_DIR_FAIL =
+      new File("src/test/resources/cs-validation-fail").getAbsolutePath();
+
+  private ConvertedConfigValidator validator;
+
+  @Before
+  public void setup() {
+    validator = new ConvertedConfigValidator();
+  }
+
+  @Test
+  public void testValidationPassed() throws Exception {
+    validator.validateConvertedConfig(CONFIG_DIR_PASSES);
+
+    // expected: no exception
+  }
+
+  @Test(expected = VerificationException.class)
+  public void testValidationFails() throws Exception {
+    validator.validateConvertedConfig(CONFIG_DIR_FAIL);
+  }
+}

+ 60 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigArgumentHandler.java

@@ -19,7 +19,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyZeroInteractions;
 
 import java.io.File;
 import java.io.IOException;
@@ -28,6 +30,7 @@ import java.util.List;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -59,6 +62,9 @@ public class TestFSConfigToCSConfigArgumentHandler {
   @Mock
   private FSConfigToCSConfigConverter mockConverter;
 
+  @Mock
+  private ConvertedConfigValidator mockValidator;
+
   private DryRunResultHolder dryRunResultHolder;
   private ConversionOptions conversionOptions;
 
@@ -74,6 +80,7 @@ public class TestFSConfigToCSConfigArgumentHandler {
 
   @After
   public void tearDown() {
+    QueueMetrics.clearQueueMetrics();
     fsTestCommons.tearDown();
   }
 
@@ -290,7 +297,7 @@ public class TestFSConfigToCSConfigArgumentHandler {
     argumentHandler.parseAndConvert(args);
 
     // validate params
-    Mockito.verify(mockConverter).convert(conversionParams.capture());
+    verify(mockConverter).convert(conversionParams.capture());
     FSConfigToCSConfigConverterParams params = conversionParams.getValue();
     LOG.info("FS config converter parameters: " + params);
 
@@ -322,7 +329,7 @@ public class TestFSConfigToCSConfigArgumentHandler {
     argumentHandler.parseAndConvert(args);
 
     // validate params
-    Mockito.verify(mockConverter).convert(conversionParams.capture());
+    verify(mockConverter).convert(conversionParams.capture());
     FSConfigToCSConfigConverterParams params = conversionParams.getValue();
     LOG.info("FS config converter parameters: " + params);
 
@@ -355,7 +362,7 @@ public class TestFSConfigToCSConfigArgumentHandler {
     argumentHandler.parseAndConvert(args);
 
     // validate params
-    Mockito.verify(mockConverter).convert(conversionParams.capture());
+    verify(mockConverter).convert(conversionParams.capture());
     FSConfigToCSConfigConverterParams params = conversionParams.getValue();
     LOG.info("FS config converter parameters: " + params);
 
@@ -446,7 +453,7 @@ public class TestFSConfigToCSConfigArgumentHandler {
         "-r", FSConfigConverterTestCommons.CONVERSION_RULES_FILE,
         "-d");
     FSConfigToCSConfigArgumentHandler argumentHandler =
-        new FSConfigToCSConfigArgumentHandler(conversionOptions);
+        new FSConfigToCSConfigArgumentHandler(conversionOptions, mockValidator);
     argumentHandler.setConverterSupplier(this::getMockConverter);
 
     Mockito.doThrow(exception).when(mockConverter)
@@ -470,7 +477,7 @@ public class TestFSConfigToCSConfigArgumentHandler {
         "-t");
 
     FSConfigToCSConfigArgumentHandler argumentHandler =
-        new FSConfigToCSConfigArgumentHandler(conversionOptions);
+        new FSConfigToCSConfigArgumentHandler(conversionOptions, mockValidator);
     argumentHandler.setConverterSupplier(this::getMockConverter);
 
     argumentHandler.parseAndConvert(args);
@@ -488,7 +495,7 @@ public class TestFSConfigToCSConfigArgumentHandler {
         "-r", FSConfigConverterTestCommons.CONVERSION_RULES_FILE, "-p");
 
     FSConfigToCSConfigArgumentHandler argumentHandler =
-        new FSConfigToCSConfigArgumentHandler(conversionOptions);
+        new FSConfigToCSConfigArgumentHandler(conversionOptions, mockValidator);
     argumentHandler.setConverterSupplier(this::getMockConverter);
 
     argumentHandler.parseAndConvert(args);
@@ -581,7 +588,8 @@ public class TestFSConfigToCSConfigArgumentHandler {
           "-p");
     }
     FSConfigToCSConfigArgumentHandler argumentHandler =
-        new FSConfigToCSConfigArgumentHandler(conversionOptions);
+        new FSConfigToCSConfigArgumentHandler(conversionOptions,
+            mockValidator);
     argumentHandler.setConverterSupplier(this::getMockConverter);
 
     argumentHandler.parseAndConvert(args);
@@ -598,4 +606,49 @@ public class TestFSConfigToCSConfigArgumentHandler {
           params.isConvertPlacementRules());
     }
   }
+
+  public void testValidatorInvocation() throws Exception {
+    setupFSConfigConversionFiles(true);
+
+    FSConfigToCSConfigArgumentHandler argumentHandler =
+        new FSConfigToCSConfigArgumentHandler(conversionOptions,
+            mockValidator);
+
+    String[] args = getArgumentsAsArrayWithDefaults("-f",
+        FSConfigConverterTestCommons.FS_ALLOC_FILE);
+    argumentHandler.parseAndConvert(args);
+
+    verify(mockValidator).validateConvertedConfig(anyString());
+  }
+
+  @Test
+  public void testValidationSkippedWhenCmdLineSwitchIsDefined()
+      throws Exception {
+    setupFSConfigConversionFiles(true);
+
+    FSConfigToCSConfigArgumentHandler argumentHandler =
+        new FSConfigToCSConfigArgumentHandler(conversionOptions,
+            mockValidator);
+
+    String[] args = getArgumentsAsArrayWithDefaults("-f",
+        FSConfigConverterTestCommons.FS_ALLOC_FILE, "-s");
+    argumentHandler.parseAndConvert(args);
+
+    verifyZeroInteractions(mockValidator);
+  }
+
+  @Test
+  public void testValidationSkippedWhenOutputIsConsole() throws Exception {
+    setupFSConfigConversionFiles(true);
+
+    FSConfigToCSConfigArgumentHandler argumentHandler =
+        new FSConfigToCSConfigArgumentHandler(conversionOptions,
+            mockValidator);
+
+    String[] args = getArgumentsAsArrayWithDefaults("-f",
+        FSConfigConverterTestCommons.FS_ALLOC_FILE, "-s", "-p");
+    argumentHandler.parseAndConvert(args);
+
+    verifyZeroInteractions(mockValidator);
+  }
 }

+ 22 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverterMain.java

@@ -25,7 +25,9 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 import java.io.IOException;
+import java.security.Permission;
 
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -37,15 +39,20 @@ import org.junit.Test;
  */
 public class TestFSConfigToCSConfigConverterMain {
   private FSConfigConverterTestCommons converterTestCommons;
+  private SecurityManager originalSecurityManager;
 
   @Before
   public void setUp() throws Exception {
+    originalSecurityManager = System.getSecurityManager();
+    System.setSecurityManager(new ExitHandlerSecurityManager());
     converterTestCommons = new FSConfigConverterTestCommons();
     converterTestCommons.setUp();
   }
 
   @After
   public void tearDown() throws Exception {
+    QueueMetrics.clearQueueMetrics();
+    System.setSecurityManager(originalSecurityManager);
     converterTestCommons.tearDown();
   }
 
@@ -132,4 +139,19 @@ public class TestFSConfigToCSConfigConverterMain {
     assertTrue("Help was not displayed",
         stdout.contains("General options are:"));
   }
+
+  class ExitHandlerSecurityManager extends SecurityManager {
+    @Override
+    public void checkExit(int status) {
+      if (status != 0) {
+        throw new IllegalStateException(
+            "Exit code is not 0, it was " + status);
+      }
+    }
+
+    @Override
+    public void checkPermission(Permission perm) {
+      // allow all permissions
+    }
+  }
 }

+ 14 - 1
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

@@ -341,7 +341,20 @@ public class TestFSQueueConverter {
 
     converter.convertQueueHierarchy(rootQueue);
 
-    assertTrueForQueues(ALL_QUEUES, ".auto-create-child-queue.enabled",
+    Set<String> parentQueues = Sets.newHashSet("root",
+        "root.admins",
+        "root.users");
+
+    Set<String> leafQueues = Sets.newHashSet(
+        "root.default",
+        "root.admins.alice",
+        "root.admins.bob",
+        "root.users.joe",
+        "root.users.john");
+
+    assertTrueForQueues(parentQueues, ".auto-create-child-queue.enabled",
+        csConfig);
+    assertNoValueForQueues(leafQueues, ".auto-create-child-queue.enabled",
         csConfig);
   }
 

+ 71 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/cs-validation-fail/capacity-scheduler.xml

@@ -0,0 +1,71 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<configuration>
+    <property>
+        <name>yarn.scheduler.capacity.root.default.maximum-am-resource-percent
+        </name>
+        <value>1.0</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+    <property>
+        <name>yarn.scheduler.capacity.root.default.capacity</name>
+        <value>100.000</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+    <property>
+        <name>yarn.scheduler.capacity.root.default.ordering-policy</name>
+        <value>fair</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+
+    <!-- the next property is not accepted by CS -->
+    <property>
+        <name>
+            yarn.scheduler.capacity.root.default.auto-create-child-queue.enabled
+        </name>
+        <value>true</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+
+    <property>
+        <name>yarn.scheduler.capacity.root.maximum-am-resource-percent</name>
+        <value>1.0</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+    <property>
+        <name>yarn.scheduler.capacity.root.queues</name>
+        <value>default</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+    <property>
+        <name>yarn.scheduler.capacity.root.auto-create-child-queue.enabled
+        </name>
+        <value>true</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+    <property>
+        <name>yarn.scheduler.capacity.maximum-am-resource-percent</name>
+        <value>-1.0</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+</configuration>

+ 32 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/cs-validation-fail/yarn-site.xml

@@ -0,0 +1,32 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<configuration>
+    <property>
+        <name>
+            yarn.scheduler.capacity.per-node-heartbeat.multiple-assignments-enabled
+        </name>
+        <value>false</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+    <property>
+        <name>yarn.resourcemanager.scheduler.class</name>
+        <value>
+            org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler
+        </value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+</configuration>

+ 60 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/cs-validation-pass/capacity-scheduler.xml

@@ -0,0 +1,60 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<configuration>
+    <property>
+        <name>yarn.scheduler.capacity.root.default.maximum-am-resource-percent
+        </name>
+        <value>1.0</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+    <property>
+        <name>yarn.scheduler.capacity.root.default.capacity</name>
+        <value>100.000</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+    <property>
+        <name>yarn.scheduler.capacity.root.default.ordering-policy</name>
+        <value>fair</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+    <property>
+        <name>yarn.scheduler.capacity.root.maximum-am-resource-percent</name>
+        <value>1.0</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+    <property>
+        <name>yarn.scheduler.capacity.root.queues</name>
+        <value>default</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+    <property>
+        <name>yarn.scheduler.capacity.root.auto-create-child-queue.enabled
+        </name>
+        <value>true</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+    <property>
+        <name>yarn.scheduler.capacity.maximum-am-resource-percent</name>
+        <value>-1.0</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+</configuration>

+ 32 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/cs-validation-pass/yarn-site.xml

@@ -0,0 +1,32 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<configuration>
+    <property>
+        <name>
+            yarn.scheduler.capacity.per-node-heartbeat.multiple-assignments-enabled
+        </name>
+        <value>false</value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+    <property>
+        <name>yarn.resourcemanager.scheduler.class</name>
+        <value>
+            org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler
+        </value>
+        <final>false</final>
+        <source>programmatically</source>
+    </property>
+</configuration>