ソースを参照

YARN-2890. MiniYARNCluster should start the timeline server based on the configuration. Contributed by Mit Desai.

(cherry picked from commit 51af8d367de94689770f57c64bea3b244d7755f6)
Zhijie Shen 10 年 前
コミット
d21ef79707

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java

@@ -453,7 +453,7 @@ public class TestJobHistoryEventHandler {
     long currentTime = System.currentTimeMillis();
     try {
       yarnCluster = new MiniYARNCluster(
-            TestJobHistoryEventHandler.class.getSimpleName(), 1, 1, 1, 1, true);
+            TestJobHistoryEventHandler.class.getSimpleName(), 1, 1, 1, 1);
       yarnCluster.init(conf);
       yarnCluster.start();
       jheh.start();

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

@@ -34,6 +34,58 @@ import org.junit.Test;
 
 public class TestMRTimelineEventHandling {
 
+  @Test
+  public void testTimelineServiceStartInMiniCluster() throws Exception {
+    Configuration conf = new YarnConfiguration();
+
+    /*
+     * Timeline service should not start if the config is set to false
+     * Regardless to the value of MAPREDUCE_JOB_EMIT_TIMELINE_DATA
+     */
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false);
+    conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, true);
+    MiniMRYarnCluster cluster = null;
+    try {
+      cluster = new MiniMRYarnCluster(
+          TestJobHistoryEventHandler.class.getSimpleName(), 1);
+      cluster.init(conf);
+      cluster.start();
+
+      //verify that the timeline service is not started.
+      Assert.assertNull("Timeline Service should not have been started",
+          cluster.getApplicationHistoryServer());
+
+      //Run a MR job and verify it succeeds
+      Path inDir = new Path("input");
+      Path outDir = new Path("output");
+      RunningJob job =
+          UtilsForTests.runJobSucceed(new JobConf(conf), inDir, outDir);
+      Assert.assertEquals(JobStatus.SUCCEEDED,
+          job.getJobStatus().getState().getValue());
+    }
+    finally {
+      if(cluster != null) {
+        cluster.stop();
+      }
+    }
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false);
+    conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, false);
+    cluster = null;
+    try {
+        cluster = new MiniMRYarnCluster(
+            TestJobHistoryEventHandler.class.getSimpleName(), 1);
+        cluster.init(conf);
+        cluster.start();
+        Assert.assertNull("Timeline Service should not have been started",
+            cluster.getApplicationHistoryServer());
+      }
+      finally {
+        if(cluster != null) {
+          cluster.stop();
+        }
+      }
+  }
+
   @Test
   public void testMRTimelineEventHandling() throws Exception {
     Configuration conf = new YarnConfiguration();
@@ -42,7 +94,7 @@ public class TestMRTimelineEventHandling {
     MiniMRYarnCluster cluster = null;
     try {
       cluster = new MiniMRYarnCluster(
-              TestJobHistoryEventHandler.class.getSimpleName(), 1, true);
+          TestJobHistoryEventHandler.class.getSimpleName(), 1);
       cluster.init(conf);
       cluster.start();
       TimelineStore ts = cluster.getApplicationHistoryServer()
@@ -96,7 +148,7 @@ public class TestMRTimelineEventHandling {
     MiniMRYarnCluster cluster = null;
     try {
       cluster = new MiniMRYarnCluster(
-          TestJobHistoryEventHandler.class.getSimpleName(), 1, true);
+          TestJobHistoryEventHandler.class.getSimpleName(), 1);
       cluster.init(conf);
       cluster.start();
       TimelineStore ts = cluster.getApplicationHistoryServer()
@@ -133,7 +185,7 @@ public class TestMRTimelineEventHandling {
     cluster = null;
     try {
       cluster = new MiniMRYarnCluster(
-          TestJobHistoryEventHandler.class.getSimpleName(), 1, true);
+          TestJobHistoryEventHandler.class.getSimpleName(), 1);
       cluster.init(conf);
       cluster.start();
       TimelineStore ts = cluster.getApplicationHistoryServer()

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

@@ -72,11 +72,7 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
   }
 
   public MiniMRYarnCluster(String testName, int noOfNMs) {
-    this(testName, noOfNMs, false);
-  }
-
-  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 - 0
hadoop-yarn-project/CHANGES.txt

@@ -133,6 +133,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2905. AggregatedLogsBlock page can infinitely loop if the aggregated
     log file is corrupted (Varun Saxena via jlowe)
 
+    YARN-2890. MiniYARNCluster should start the timeline server based on the
+    configuration. (Mit Desai via zjshen)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java

@@ -84,7 +84,7 @@ public class TestDistributedShell {
     if (yarnCluster == null) {
       yarnCluster =
           new MiniYARNCluster(TestDistributedShell.class.getSimpleName(), 1,
-              numNodeManager, 1, 1, true);
+              numNodeManager, 1, 1);
       yarnCluster.init(conf);
       
       yarnCluster.start();

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

@@ -295,7 +295,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);
@@ -326,10 +326,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;

+ 3 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java

@@ -57,7 +57,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResp
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStore;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.MemoryApplicationHistoryStore;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.webapp.AHSWebApp;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.NodeHealthCheckerService;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
@@ -120,7 +119,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
@@ -128,15 +126,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
    */
   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 {
@@ -186,20 +182,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
@@ -260,8 +242,8 @@ public class MiniYARNCluster extends CompositeService {
       addService(new NodeManagerWrapper(index));
     }
 
-    if (enableAHS) {
-      addService(new ApplicationHistoryServerWrapper());
+    if(conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false)) {
+        addService(new ApplicationHistoryServerWrapper());
     }
     
     super.serviceInit(