You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sj...@apache.org on 2015/09/26 18:05:45 UTC

[38/50] [abbrv] hadoop git commit: Revert "YARN-2890. MiniYARNCluster should start the timeline server based on the configuration. Contributed by Mit Desai."

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

This reverts commit 8a47d1aa559c380869861c070af9d1e7a8732607.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9c4a6e12
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9c4a6e12
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9c4a6e12

Branch: refs/heads/branch-2.6
Commit: 9c4a6e127098bf0c337903d01d7ec902a7cc6084
Parents: d57c3f0
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Tue Sep 15 11:50:47 2015 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Tue Sep 15 17:30:15 2015 -0700

----------------------------------------------------------------------
 .../jobhistory/TestJobHistoryEventHandler.java  |  2 +-
 .../mapred/TestMRTimelineEventHandling.java     | 58 +-------------------
 .../hadoop/mapreduce/v2/MiniMRYarnCluster.java  |  6 +-
 hadoop-yarn-project/CHANGES.txt                 |  3 -
 .../distributedshell/TestDistributedShell.java  |  2 +-
 .../hadoop/yarn/client/ProtocolHATestBase.java  |  6 +-
 .../hadoop/yarn/server/MiniYARNCluster.java     | 24 +++++++-
 7 files changed, 34 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c4a6e12/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
index de260c9..43e3dbe 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
+++ b/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);
+            TestJobHistoryEventHandler.class.getSimpleName(), 1, 1, 1, 1, true);
       yarnCluster.init(conf);
       yarnCluster.start();
       jheh.start();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c4a6e12/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
index 4ffbad7..c2ef128 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
@@ -35,58 +35,6 @@ 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();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
@@ -94,7 +42,7 @@ public class TestMRTimelineEventHandling {
     MiniMRYarnCluster cluster = null;
     try {
       cluster = new MiniMRYarnCluster(
-          TestJobHistoryEventHandler.class.getSimpleName(), 1);
+              TestJobHistoryEventHandler.class.getSimpleName(), 1, true);
       cluster.init(conf);
       cluster.start();
       TimelineStore ts = cluster.getApplicationHistoryServer()
@@ -148,7 +96,7 @@ public class TestMRTimelineEventHandling {
     MiniMRYarnCluster cluster = null;
     try {
       cluster = new MiniMRYarnCluster(
-          TestJobHistoryEventHandler.class.getSimpleName(), 1);
+          TestJobHistoryEventHandler.class.getSimpleName(), 1, true);
       cluster.init(conf);
       cluster.start();
       TimelineStore ts = cluster.getApplicationHistoryServer()
@@ -185,7 +133,7 @@ public class TestMRTimelineEventHandling {
     cluster = null;
     try {
       cluster = new MiniMRYarnCluster(
-          TestJobHistoryEventHandler.class.getSimpleName(), 1);
+          TestJobHistoryEventHandler.class.getSimpleName(), 1, true);
       cluster.init(conf);
       cluster.start();
       TimelineStore ts = cluster.getApplicationHistoryServer()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c4a6e12/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
index 45b736e..47b38a1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
@@ -72,7 +72,11 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
   }
 
   public MiniMRYarnCluster(String testName, int noOfNMs) {
-    super(testName, 1, noOfNMs, 4, 4);
+    this(testName, noOfNMs, false);
+  }
+
+  public MiniMRYarnCluster(String testName, int noOfNMs, boolean enableAHS) {
+    super(testName, 1, noOfNMs, 4, 4, enableAHS);
     historyServerWrapper = new JobHistoryServerWrapper();
     addService(historyServerWrapper);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c4a6e12/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 17d888d..6ef7c62 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -49,9 +49,6 @@ Release 2.6.1 - 2015-09-09
     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)
-
     YARN-2894. Fixed a bug regarding application view acl when RM fails over.
     (Rohith Sharmaks via jianhe)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c4a6e12/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index 904ad58..17c29ec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/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);
+              numNodeManager, 1, 1, true);
       yarnCluster.init(conf);
       
       yarnCluster.start();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c4a6e12/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
index ea9e361..1cc43b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
@@ -294,7 +294,7 @@ public abstract class ProtocolHATestBase extends ClientBaseWithFixes {
     conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     cluster =
         new MiniYARNClusterForHATesting(TestRMFailover.class.getName(), 2,
-            numOfNMs, 1, 1, overrideClientRMService, overrideRTS,
+            numOfNMs, 1, 1, false, overrideClientRMService, overrideRTS,
             overrideApplicationMasterService);
     cluster.resetStartFailoverFlag(false);
     cluster.init(conf);
@@ -325,10 +325,10 @@ public abstract class ProtocolHATestBase extends ClientBaseWithFixes {
 
     public MiniYARNClusterForHATesting(String testName,
         int numResourceManagers, int numNodeManagers, int numLocalDirs,
-        int numLogDirs, boolean overrideClientRMService,
+        int numLogDirs, boolean enableAHS, boolean overrideClientRMService,
         boolean overrideRTS, boolean overrideApplicationMasterService) {
       super(testName, numResourceManagers, numNodeManagers, numLocalDirs,
-          numLogDirs);
+          numLogDirs, enableAHS);
       this.overrideClientRMService = overrideClientRMService;
       this.overrideRTS = overrideRTS;
       this.overrideApplicationMasterService = overrideApplicationMasterService;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c4a6e12/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
index 2eb5034..e83d601 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
@@ -57,6 +57,7 @@ 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;
@@ -119,6 +120,7 @@ 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
@@ -126,13 +128,15 @@ 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) {
+      int numLocalDirs, int numLogDirs, boolean enableAHS) {
     super(testName.replace("$", ""));
     this.numLocalDirs = numLocalDirs;
     this.numLogDirs = numLogDirs;
+    this.enableAHS = enableAHS;
     String testSubDir = testName.replace("$", "");
     File targetWorkDir = new File("target", testSubDir);
     try {
@@ -184,6 +188,20 @@ public class MiniYARNCluster extends CompositeService {
 
   /**
    * @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
    * @param numLocalDirs the number of nm-local-dirs per nodemanager
    * @param numLogDirs the number of nm-log-dirs per nodemanager
@@ -242,8 +260,8 @@ public class MiniYARNCluster extends CompositeService {
       addService(new NodeManagerWrapper(index));
     }
 
-    if(conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false)) {
-        addService(new ApplicationHistoryServerWrapper());
+    if (enableAHS) {
+      addService(new ApplicationHistoryServerWrapper());
     }
     
     super.serviceInit(