瀏覽代碼

YARN-10257. FS-CS converter: skip increment properties for mem/vcores and fix DRF check. Contributed by Peter Bacsko

Szilard Nemeth 5 年之前
父節點
當前提交
adfb68cb22

+ 7 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml

@@ -243,6 +243,13 @@
       <artifactId>jersey-test-framework-grizzly2</artifactId>
       <scope>test</scope>
     </dependency>
+
+    <dependency>
+      <groupId>com.github.stefanbirkner</groupId>
+      <artifactId>system-rules</artifactId>
+      <scope>test</scope>
+    </dependency>
+
   </dependencies>
 
   <build>

+ 2 - 5
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

@@ -392,11 +392,8 @@ public class FSConfigToCSConfigConverter {
 
     String defaultPolicy = allocConf.getDefaultSchedulingPolicy().getName();
 
-    if (DominantResourceFairnessPolicy.NAME.equals(defaultPolicy)) {
-      return true;
-    } else {
-      return isDrfUsedOnQueueLevel(rootQueue);
-    }
+    return DominantResourceFairnessPolicy.NAME.equals(defaultPolicy) ||
+        isDrfUsedOnQueueLevel(rootQueue);
   }
 
   private boolean isDrfUsedOnQueueLevel(FSQueue queue) {

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

@@ -42,7 +42,6 @@ public class FSYarnSiteConverter {
     yarnSiteConfig.set(YarnConfiguration.RM_SCHEDULER,
         CapacityScheduler.class.getCanonicalName());
 
-    // TODO: deprecated property, check if necessary
     if (conf.getBoolean(
         FairSchedulerConfiguration.CONTINUOUS_SCHEDULING_ENABLED,
         FairSchedulerConfiguration.DEFAULT_CONTINUOUS_SCHEDULING_ENABLED)) {
@@ -55,20 +54,6 @@ public class FSYarnSiteConverter {
           "schedule-asynchronously.scheduling-interval-ms", interval);
     }
 
-    String mbIncrementAllocation =
-        conf.get("yarn.resource-types.memory-mb.increment-allocation");
-    if (mbIncrementAllocation != null) {
-      yarnSiteConfig.set("yarn.scheduler.minimum-allocation-mb",
-          mbIncrementAllocation);
-    }
-
-    String vcoreIncrementAllocation =
-        conf.get("yarn.resource-types.vcores.increment-allocation");
-    if (vcoreIncrementAllocation != null) {
-      yarnSiteConfig.set("yarn.scheduler.minimum-allocation-vcores",
-          vcoreIncrementAllocation);
-    }
-
     if (conf.getBoolean(FairSchedulerConfiguration.PREEMPTION,
         FairSchedulerConfiguration.DEFAULT_PREEMPTION)) {
       yarnSiteConfig.setBoolean(

+ 12 - 27
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

@@ -22,16 +22,16 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.conve
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.FSConfigConverterTestCommons.YARN_SITE_XML;
 import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter.FSConfigConverterTestCommons.setupFSConfigConversionFiles;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
 
 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.Rule;
 import org.junit.Test;
+import org.junit.contrib.java.lang.system.ExpectedSystemExit;
 
 
 /**
@@ -40,14 +40,12 @@ import org.junit.Test;
  */
 public class TestFSConfigToCSConfigConverterMain {
   private FSConfigConverterTestCommons converterTestCommons;
-  private SecurityManager originalSecurityManager;
-  private ExitHandlerSecurityManager exitHandlerSecurityManager;
+
+  @Rule
+  public final ExpectedSystemExit exit = ExpectedSystemExit.none();
 
   @Before
   public void setUp() throws Exception {
-    originalSecurityManager = System.getSecurityManager();
-    exitHandlerSecurityManager = new ExitHandlerSecurityManager();
-    System.setSecurityManager(exitHandlerSecurityManager);
     converterTestCommons = new FSConfigConverterTestCommons();
     converterTestCommons.setUp();
   }
@@ -55,7 +53,6 @@ public class TestFSConfigToCSConfigConverterMain {
   @After
   public void tearDown() throws Exception {
     QueueMetrics.clearQueueMetrics();
-    System.setSecurityManager(originalSecurityManager);
     converterTestCommons.tearDown();
   }
 
@@ -71,6 +68,7 @@ public class TestFSConfigToCSConfigConverterMain {
   public void testConvertFSConfigurationDefaults()
       throws Exception {
     setupFSConfigConversionFiles();
+    exit.expectSystemExitWithStatus(0);
 
     FSConfigToCSConfigConverterMain.main(new String[] {
         "-o", OUTPUT_DIR,
@@ -85,13 +83,13 @@ public class TestFSConfigToCSConfigConverterMain {
 
     assertTrue("capacity-scheduler.xml was not generated", csConfigExists);
     assertTrue("yarn-site.xml was not generated", yarnSiteConfigExists);
-    assertEquals("Exit code", 0, exitHandlerSecurityManager.exitCode);
   }
 
   @Test
   public void testConvertFSConfigurationWithConsoleParam()
       throws Exception {
     setupFSConfigConversionFiles();
+    exit.expectSystemExitWithStatus(0);
 
     FSConfigToCSConfigConverterMain.main(new String[] {
         "-p",
@@ -108,6 +106,8 @@ public class TestFSConfigToCSConfigConverterMain {
 
   @Test
   public void testShortHelpSwitch() {
+    exit.expectSystemExitWithStatus(0);
+
     FSConfigToCSConfigConverterMain.main(new String[] {"-h"});
 
     verifyHelpText();
@@ -115,6 +115,8 @@ public class TestFSConfigToCSConfigConverterMain {
 
   @Test
   public void testLongHelpSwitch() {
+    exit.expectSystemExitWithStatus(0);
+
     FSConfigToCSConfigConverterMain.main(new String[] {"--help"});
 
     verifyHelpText();
@@ -123,6 +125,7 @@ public class TestFSConfigToCSConfigConverterMain {
   @Test
   public void testConvertFSConfigurationWithLongSwitches()
       throws IOException {
+    exit.expectSystemExitWithStatus(0);
     setupFSConfigConversionFiles();
 
     FSConfigToCSConfigConverterMain.main(new String[] {
@@ -143,22 +146,4 @@ public class TestFSConfigToCSConfigConverterMain {
     assertTrue("Help was not displayed",
         stdout.contains("General options are:"));
   }
-
-  class ExitHandlerSecurityManager extends SecurityManager {
-    int exitCode = Integer.MIN_VALUE;
-
-    @Override
-    public void checkExit(int status) {
-      if (status != 0) {
-        throw new IllegalStateException(
-            "Exit code is not 0, it was " + status);
-      }
-      exitCode = status;
-    }
-
-    @Override
-    public void checkPermission(Permission perm) {
-      // allow all permissions
-    }
-  }
 }

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

@@ -62,21 +62,6 @@ public class TestFSYarnSiteConverter {
                 ".scheduling-interval-ms", -1));
   }
 
-  @Test
-  public void testSiteMinimumAllocationIncrementConversion() {
-    yarnConfig.setInt("yarn.resource-types.memory-mb.increment-allocation", 11);
-    yarnConfig.setInt("yarn.resource-types.vcores.increment-allocation", 5);
-
-    converter.convertSiteProperties(yarnConfig, yarnConvertedConfig, false);
-
-    assertEquals("Memory alloc increment", 11,
-        yarnConvertedConfig.getInt("yarn.scheduler.minimum-allocation-mb",
-            -1));
-    assertEquals("Vcore increment", 5,
-        yarnConvertedConfig.getInt("yarn.scheduler.minimum-allocation-vcores",
-            -1));
-  }
-
   @Test
   public void testSitePreemptionConversion() {
     yarnConfig.setBoolean(FairSchedulerConfiguration.PREEMPTION, true);