소스 검색

YARN-5007. Remove deprecated constructors of MiniYARNCluster and MiniMRYarnCluster. Contributed by Andras Bokor.

Akira Ajisaka 8 년 전
부모
커밋
34ab8e73d4

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java

@@ -203,7 +203,7 @@ public class TestMRTimelineEventHandling {
     MiniMRYarnCluster cluster = null;
     try {
       cluster = new MiniMRYarnCluster(
-          TestMRTimelineEventHandling.class.getSimpleName(), 1, true);
+          TestMRTimelineEventHandling.class.getSimpleName(), 1);
       cluster.init(conf);
       cluster.start();
       LOG.info("A MiniMRYarnCluster get start.");

+ 1 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java

@@ -74,11 +74,7 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
   }
 
   public MiniMRYarnCluster(String testName, int noOfNMs) {
-    this(testName, noOfNMs, false);
-  }
-  @Deprecated
-  public MiniMRYarnCluster(String testName, int noOfNMs, boolean enableAHS) {
-    super(testName, 1, noOfNMs, 4, 4, enableAHS);
+    super(testName, 1, noOfNMs, 4, 4);
     historyServerWrapper = new JobHistoryServerWrapper();
     addService(historyServerWrapper);
   }

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java

@@ -274,7 +274,7 @@ public abstract class ProtocolHATestBase extends ClientBaseWithFixes {
     conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     cluster =
         new MiniYARNClusterForHATesting(TestRMFailover.class.getName(), 2,
-            numOfNMs, 1, 1, false, overrideClientRMService, overrideRTS,
+            numOfNMs, 1, 1, overrideClientRMService, overrideRTS,
             overrideApplicationMasterService);
     cluster.resetStartFailoverFlag(false);
     cluster.init(conf);
@@ -304,10 +304,10 @@ public abstract class ProtocolHATestBase extends ClientBaseWithFixes {
 
     public MiniYARNClusterForHATesting(String testName,
         int numResourceManagers, int numNodeManagers, int numLocalDirs,
-        int numLogDirs, boolean enableAHS, boolean overrideClientRMService,
+        int numLogDirs, boolean overrideClientRMService,
         boolean overrideRTS, boolean overrideApplicationMasterService) {
       super(testName, numResourceManagers, numNodeManagers, numLocalDirs,
-          numLogDirs, enableAHS);
+          numLogDirs);
       this.overrideClientRMService = overrideClientRMService;
       this.overrideRTS = overrideRTS;
       this.overrideApplicationMasterService = overrideApplicationMasterService;

+ 2 - 20
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java

@@ -146,7 +146,6 @@ public class MiniYARNCluster extends CompositeService {
   private int numLocalDirs;
   // Number of nm-log-dirs per nodemanager
   private int numLogDirs;
-  private boolean enableAHS;
 
   /**
    * @param testName name of the test
@@ -154,16 +153,13 @@ public class MiniYARNCluster extends CompositeService {
    * @param numNodeManagers the number of node managers in the cluster
    * @param numLocalDirs the number of nm-local-dirs per nodemanager
    * @param numLogDirs the number of nm-log-dirs per nodemanager
-   * @param enableAHS enable ApplicationHistoryServer or not
    */
-  @Deprecated
   public MiniYARNCluster(
       String testName, int numResourceManagers, int numNodeManagers,
-      int numLocalDirs, int numLogDirs, boolean enableAHS) {
+      int numLocalDirs, int numLogDirs) {
     super(testName.replace("$", ""));
     this.numLocalDirs = numLocalDirs;
     this.numLogDirs = numLogDirs;
-    this.enableAHS = enableAHS;
     String testSubDir = testName.replace("$", "");
     File targetWorkDir = new File("target", testSubDir);
     try {
@@ -213,20 +209,6 @@ public class MiniYARNCluster extends CompositeService {
     nodeManagers = new NodeManager[numNodeManagers];
   }
 
-  /**
-   * @param testName name of the test
-   * @param numResourceManagers the number of resource managers in the cluster
-   * @param numNodeManagers the number of node managers in the cluster
-   * @param numLocalDirs the number of nm-local-dirs per nodemanager
-   * @param numLogDirs the number of nm-log-dirs per nodemanager
-   */
-  public MiniYARNCluster(
-      String testName, int numResourceManagers, int numNodeManagers,
-      int numLocalDirs, int numLogDirs) {
-    this(testName, numResourceManagers, numNodeManagers, numLocalDirs,
-        numLogDirs, false);
-  }
-
   /**
    * @param testName name of the test
    * @param numNodeManagers the number of node managers in the cluster
@@ -288,7 +270,7 @@ public class MiniYARNCluster extends CompositeService {
     }
 
     if(conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED) || enableAHS) {
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
         addService(new ApplicationHistoryServerWrapper());
     }
     

+ 2 - 32
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYarnCluster.java

@@ -34,18 +34,14 @@ public class TestMiniYarnCluster {
     int numNodeManagers = 1;
     int numLocalDirs = 1;
     int numLogDirs = 1;
-    boolean enableAHS;
 
     /*
      * Timeline service should not start if TIMELINE_SERVICE_ENABLED == false
-     * and enableAHS flag == false
      */
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false);
-    enableAHS = false;
     try (MiniYARNCluster cluster =
         new MiniYARNCluster(TestMiniYarnCluster.class.getSimpleName(),
-            numNodeManagers, numLocalDirs, numLogDirs, numLogDirs,
-                enableAHS)) {
+            numNodeManagers, numLocalDirs, numLogDirs, numLogDirs)) {
 
       cluster.init(conf);
       cluster.start();
@@ -57,14 +53,11 @@ public class TestMiniYarnCluster {
 
     /*
      * Timeline service should start if TIMELINE_SERVICE_ENABLED == true
-     * and enableAHS == false
      */
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
-    enableAHS = false;
     try (MiniYARNCluster cluster =
         new MiniYARNCluster(TestMiniYarnCluster.class.getSimpleName(),
-            numNodeManagers, numLocalDirs, numLogDirs, numLogDirs,
-                enableAHS)) {
+            numNodeManagers, numLocalDirs, numLogDirs, numLogDirs)) {
       cluster.init(conf);
 
       // Verify that the timeline-service starts on ephemeral ports by default
@@ -74,29 +67,6 @@ public class TestMiniYarnCluster {
 
       cluster.start();
 
-      //Timeline service may sometime take a while to get started
-      int wait = 0;
-      while(cluster.getApplicationHistoryServer() == null && wait < 20) {
-        Thread.sleep(500);
-        wait++;
-      }
-      //verify that the timeline service is started.
-      Assert.assertNotNull("Timeline Service should have been started",
-          cluster.getApplicationHistoryServer());
-    }
-    /*
-     * Timeline service should start if TIMELINE_SERVICE_ENABLED == false
-     * and enableAHS == true
-     */
-    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false);
-    enableAHS = true;
-    try (MiniYARNCluster cluster =
-        new MiniYARNCluster(TestMiniYarnCluster.class.getSimpleName(),
-            numNodeManagers, numLocalDirs, numLogDirs, numLogDirs,
-                enableAHS)) {
-      cluster.init(conf);
-      cluster.start();
-
       //Timeline service may sometime take a while to get started
       int wait = 0;
       while(cluster.getApplicationHistoryServer() == null && wait < 20) {