Ver Fonte

YARN-1855. Made Application-history server to be optional in MiniYARNCluster and thus avoid the failure of TestRMFailover#testRMWebAppRedirect. Contributed by Zhijie Shen.
svn merge --ignore-ancestry -c 1579838 ../../trunk/


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.4@1579840 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli há 11 anos atrás
pai
commit
309d0ea999

+ 4 - 0
hadoop-yarn-project/CHANGES.txt

@@ -473,6 +473,10 @@ Release 2.4.0 - UNRELEASED
     YARN-1640. Fixed manual failover of ResourceManagers to work correctly in
     secure clusters. (Xuan Gong via vinodkv)
 
+    YARN-1855. Made Application-history server to be optional in MiniYARNCluster
+    and thus avoid the failure of TestRMFailover#testRMWebAppRedirect. (Zhijie
+    Shen via vinodkv)
+
 Release 2.3.1 - UNRELEASED
 
   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

@@ -74,7 +74,7 @@ public class TestDistributedShell {
     conf.set("yarn.log.dir", "target");
     if (yarnCluster == null) {
       yarnCluster = new MiniYARNCluster(
-        TestDistributedShell.class.getSimpleName(), 1, 1, 1);
+        TestDistributedShell.class.getSimpleName(), 1, 1, 1, 1, true);
       yarnCluster.init(conf);
       yarnCluster.start();
       NodeManager  nm = yarnCluster.getNodeManager(0);

+ 29 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java

@@ -55,12 +55,15 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
 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.timeline.MemoryTimelineStore;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.timeline.TimelineStore;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.NodeHealthCheckerService;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
 import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.RMFatalEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
@@ -102,7 +105,6 @@ public class MiniYARNCluster extends CompositeService {
   private String[] rmIds;
 
   private ApplicationHistoryServer appHistoryServer;
-  private ApplicationHistoryServerWrapper appHistoryServerWrapper;
 
   private boolean useFixedPorts;
   private boolean useRpc = false;
@@ -117,6 +119,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
@@ -124,13 +127,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 {
@@ -180,6 +185,20 @@ 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
@@ -245,7 +264,9 @@ public class MiniYARNCluster extends CompositeService {
       addService(new NodeManagerWrapper(index));
     }
 
-    addService(new ApplicationHistoryServerWrapper());
+    if (enableAHS) {
+      addService(new ApplicationHistoryServerWrapper());
+    }
     
     super.serviceInit(
         conf instanceof YarnConfiguration ? conf : new YarnConfiguration(conf));
@@ -664,14 +685,11 @@ public class MiniYARNCluster extends CompositeService {
     @Override
     protected synchronized void serviceInit(Configuration conf)
         throws Exception {
-      if (!conf.getBoolean(YarnConfiguration.YARN_MINICLUSTER_FIXED_PORTS,
-          YarnConfiguration.DEFAULT_YARN_MINICLUSTER_FIXED_PORTS)) {
-        conf.set(YarnConfiguration.TIMELINE_SERVICE_ADDRESS,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ADDRESS);
-        conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS);
-      }
       appHistoryServer = new ApplicationHistoryServer();
+      conf.setClass(YarnConfiguration.APPLICATION_HISTORY_STORE,
+          MemoryApplicationHistoryStore.class, ApplicationHistoryStore.class);
+      conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
+          MemoryTimelineStore.class, TimelineStore.class);
       appHistoryServer.init(conf);
       super.serviceInit(conf);
     }