Browse Source

YARN-2980. Move health check script related functionality to hadoop-common (Varun Saxena via aw)

Allen Wittenauer 10 năm trước cách đây
mục cha
commit
d4ac6822e1
13 tập tin đã thay đổi với 259 bổ sung107 xóa
  1. 17 28
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NodeHealthScriptRunner.java
  2. 136 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNodeHealthScriptRunner.java
  3. 3 0
      hadoop-yarn-project/CHANGES.txt
  4. 8 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthCheckerService.java
  5. 24 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
  6. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
  7. 28 58
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeHealthService.java
  8. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
  9. 10 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java
  10. 10 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
  11. 5 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
  12. 5 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesApps.java
  13. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java

+ 17 - 28
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthScriptRunner.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NodeHealthScriptRunner.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.nodemanager;
+package org.apache.hadoop.util;
 
 import java.io.File;
 import java.io.IOException;
@@ -34,7 +34,6 @@ import org.apache.hadoop.util.Shell.ExitCodeException;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 /**
  * 
@@ -58,14 +57,11 @@ public class NodeHealthScriptRunner extends AbstractService {
   /** ShellCommandExecutor used to execute monitoring script */
   ShellCommandExecutor shexec = null;
 
-  /** Configuration used by the checker */
-  private Configuration conf;
-
   /** Pattern used for searching in the output of the node health script */
   static private final String ERROR_PATTERN = "ERROR";
 
   /** Time out error message */
-  static final String NODE_HEALTH_SCRIPT_TIMED_OUT_MSG = "Node health script timed out";
+  public static final String NODE_HEALTH_SCRIPT_TIMED_OUT_MSG = "Node health script timed out";
 
   private boolean isHealthy;
 
@@ -192,11 +188,16 @@ public class NodeHealthScriptRunner extends AbstractService {
     }
   }
 
-  public NodeHealthScriptRunner() {
+  public NodeHealthScriptRunner(String scriptName, long chkInterval, long timeout,
+      String[] scriptArgs) {
     super(NodeHealthScriptRunner.class.getName());
     this.lastReportedTime = System.currentTimeMillis();
     this.isHealthy = true;
-    this.healthReport = "";    
+    this.healthReport = "";
+    this.nodeHealthScript = scriptName;
+    this.intervalTime = chkInterval;
+    this.scriptTimeout = timeout;
+    this.timer = new NodeHealthMonitorExecutor(scriptArgs);
   }
 
   /*
@@ -204,17 +205,6 @@ public class NodeHealthScriptRunner extends AbstractService {
    */
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
-    this.conf = conf;
-    this.nodeHealthScript = 
-        conf.get(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH);
-    this.intervalTime = conf.getLong(YarnConfiguration.NM_HEALTH_CHECK_INTERVAL_MS,
-        YarnConfiguration.DEFAULT_NM_HEALTH_CHECK_INTERVAL_MS);
-    this.scriptTimeout = conf.getLong(
-        YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS,
-        YarnConfiguration.DEFAULT_NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS);
-    String[] args = conf.getStrings(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_OPTS,
-        new String[] {});
-    timer = new NodeHealthMonitorExecutor(args);
     super.serviceInit(conf);
   }
 
@@ -225,7 +215,7 @@ public class NodeHealthScriptRunner extends AbstractService {
   @Override
   protected void serviceStart() throws Exception {
     // if health script path is not configured don't start the thread.
-    if (!shouldRun(conf)) {
+    if (!shouldRun(nodeHealthScript)) {
       LOG.info("Not starting node health monitor");
       return;
     }
@@ -242,7 +232,7 @@ public class NodeHealthScriptRunner extends AbstractService {
    */
   @Override
   protected void serviceStop() {
-    if (!shouldRun(conf)) {
+    if (!shouldRun(nodeHealthScript)) {
       return;
     }
     if (nodeHealthScriptScheduler != null) {
@@ -322,26 +312,25 @@ public class NodeHealthScriptRunner extends AbstractService {
    * <li>Node health check script file exists</li>
    * </ol>
    * 
-   * @param conf
    * @return true if node health monitoring service can be started.
    */
-  public static boolean shouldRun(Configuration conf) {
-    String nodeHealthScript = 
-      conf.get(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH);
-    if (nodeHealthScript == null || nodeHealthScript.trim().isEmpty()) {
+  public static boolean shouldRun(String healthScript) {
+    if (healthScript == null || healthScript.trim().isEmpty()) {
       return false;
     }
-    File f = new File(nodeHealthScript);
+    File f = new File(healthScript);
     return f.exists() && FileUtil.canExecute(f);
   }
 
   private synchronized void setHealthStatus(boolean isHealthy, String output) {
+		LOG.info("health status being set as " + output);
     this.setHealthy(isHealthy);
     this.setHealthReport(output);
   }
   
   private synchronized void setHealthStatus(boolean isHealthy, String output,
       long time) {
+	LOG.info("health status being set as " + output);
     this.setHealthStatus(isHealthy, output);
     this.setLastReportedTime(time);
   }
@@ -350,7 +339,7 @@ public class NodeHealthScriptRunner extends AbstractService {
    * Used only by tests to access the timer task directly
    * @return the timer task
    */
-  TimerTask getTimerTask() {
+  public TimerTask getTimerTask() {
     return timer;
   }
 }

+ 136 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestNodeHealthScriptRunner.java

@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.TimerTask;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestNodeHealthScriptRunner {
+
+  protected static File testRootDir = new File("target",
+      TestNodeHealthScriptRunner.class.getName() +
+      "-localDir").getAbsoluteFile();
+
+  private File nodeHealthscriptFile = new File(testRootDir,
+      Shell.appendScriptExtension("failingscript"));
+
+  @Before
+  public void setup() {
+    testRootDir.mkdirs();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (testRootDir.exists()) {
+      FileContext.getLocalFSFileContext().delete(
+          new Path(testRootDir.getAbsolutePath()), true);
+    }
+  }
+
+  private void writeNodeHealthScriptFile(String scriptStr, boolean setExecutable)
+          throws IOException {
+    PrintWriter pw = null;
+    try {
+      FileUtil.setWritable(nodeHealthscriptFile, true);
+      FileUtil.setReadable(nodeHealthscriptFile, true);
+      pw = new PrintWriter(new FileOutputStream(nodeHealthscriptFile));
+      pw.println(scriptStr);
+      pw.flush();
+    } finally {
+      pw.close();
+    }
+    FileUtil.setExecutable(nodeHealthscriptFile, setExecutable);
+  }
+
+  @Test
+  public void testNodeHealthScriptShouldRun() throws IOException {
+    Assert.assertFalse("Node health script should start",
+        NodeHealthScriptRunner.shouldRun(
+            nodeHealthscriptFile.getAbsolutePath()));
+    writeNodeHealthScriptFile("", false);
+    // Node health script should not start if the node health script is not
+    // executable.
+    Assert.assertFalse("Node health script should start",
+        NodeHealthScriptRunner.shouldRun(
+            nodeHealthscriptFile.getAbsolutePath()));
+    writeNodeHealthScriptFile("", true);
+    Assert.assertTrue("Node health script should start",
+        NodeHealthScriptRunner.shouldRun(
+            nodeHealthscriptFile.getAbsolutePath()));
+  }
+
+  @Test
+  public void testNodeHealthScript() throws Exception {
+    String errorScript = "echo ERROR\n echo \"Tracker not healthy\"";
+    String normalScript = "echo \"I am all fine\"";
+    String timeOutScript =
+      Shell.WINDOWS ? "@echo off\nping -n 4 127.0.0.1 >nul\necho \"I am fine\""
+      : "sleep 4\necho \"I am fine\"";
+    Configuration conf = new Configuration();
+    writeNodeHealthScriptFile(normalScript, true);
+    NodeHealthScriptRunner nodeHealthScriptRunner = new NodeHealthScriptRunner(
+            nodeHealthscriptFile.getAbsolutePath(),
+            500, 1000, new String[] {});
+    nodeHealthScriptRunner.init(conf);
+    TimerTask timerTask = nodeHealthScriptRunner.getTimerTask();
+
+    timerTask.run();
+    // Normal Script runs successfully
+    Assert.assertTrue("Node health status reported unhealthy",
+        nodeHealthScriptRunner.isHealthy());
+    Assert.assertEquals("", nodeHealthScriptRunner.getHealthReport());
+
+    // Error script.
+    writeNodeHealthScriptFile(errorScript, true);
+    // Run timer
+    timerTask.run();
+    Assert.assertFalse("Node health status reported healthy",
+        nodeHealthScriptRunner.isHealthy());
+    Assert.assertTrue(
+        nodeHealthScriptRunner.getHealthReport().contains("ERROR"));
+    
+    // Healthy script.
+    writeNodeHealthScriptFile(normalScript, true);
+    timerTask.run();
+    Assert.assertTrue("Node health status reported unhealthy",
+        nodeHealthScriptRunner.isHealthy());
+    Assert.assertEquals("", nodeHealthScriptRunner.getHealthReport());
+
+    // Timeout script.
+    writeNodeHealthScriptFile(timeOutScript, true);
+    timerTask.run();
+    Assert.assertFalse("Node health status reported healthy even after timeout",
+    nodeHealthScriptRunner.isHealthy());
+    Assert.assertEquals(
+            NodeHealthScriptRunner.NODE_HEALTH_SCRIPT_TIMED_OUT_MSG,
+            nodeHealthScriptRunner.getHealthReport());
+  }
+}

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

@@ -17,6 +17,9 @@ Trunk - Unreleased
 
     YARN-2796. deprecate sbin/yarn-daemon.sh (aw)
 
+    YARN-2980. Move health check script related functionality to hadoop-common
+    (Varun Saxena via aw)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 8 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthCheckerService.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.server.nodemanager;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.NodeHealthScriptRunner;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 /**
  * The class which provides functionality of checking the health of the node and
@@ -33,15 +35,17 @@ public class NodeHealthCheckerService extends CompositeService {
 
   static final String SEPARATOR = ";";
 
-  public NodeHealthCheckerService() {
+  public NodeHealthCheckerService(NodeHealthScriptRunner scriptRunner,
+      LocalDirsHandlerService dirHandlerService) {
     super(NodeHealthCheckerService.class.getName());
-    dirsHandler = new LocalDirsHandlerService();
+    nodeHealthScriptRunner = scriptRunner;
+    dirsHandler = dirHandlerService;
   }
 
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
-    if (NodeHealthScriptRunner.shouldRun(conf)) {
-      nodeHealthScriptRunner = new NodeHealthScriptRunner();
+    if (NodeHealthScriptRunner.shouldRun(
+        conf.get(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH))) {
       addService(nodeHealthScriptRunner);
     }
     addService(dirsHandler);

+ 24 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.NodeHealthScriptRunner;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
@@ -181,6 +182,25 @@ public class NodeManager extends CompositeService
     }
   }
 
+  public static NodeHealthScriptRunner getNodeHealthScriptRunner(Configuration conf) {
+    String nodeHealthScript = 
+        conf.get(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH);
+    if(!NodeHealthScriptRunner.shouldRun(nodeHealthScript)) {
+      LOG.info("Abey khali");
+      return null;
+    }
+    long nmCheckintervalTime = conf.getLong(
+        YarnConfiguration.NM_HEALTH_CHECK_INTERVAL_MS,
+        YarnConfiguration.DEFAULT_NM_HEALTH_CHECK_INTERVAL_MS);
+    long scriptTimeout = conf.getLong(
+        YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS,
+        YarnConfiguration.DEFAULT_NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS);
+    String[] scriptArgs = conf.getStrings(
+        YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_OPTS, new String[] {});
+    return new NodeHealthScriptRunner(nodeHealthScript,
+        nmCheckintervalTime, scriptTimeout, scriptArgs);
+  }
+
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
 
@@ -216,9 +236,11 @@ public class NodeManager extends CompositeService
     // NodeManager level dispatcher
     this.dispatcher = new AsyncDispatcher();
 
-    nodeHealthChecker = new NodeHealthCheckerService();
+    dirsHandler = new LocalDirsHandlerService();
+    nodeHealthChecker =
+        new NodeHealthCheckerService(
+            getNodeHealthScriptRunner(conf), dirsHandler);
     addService(nodeHealthChecker);
-    dirsHandler = nodeHealthChecker.getDiskHandler();
 
     this.context = createNMContext(containerTokenSecretManager,
         nmTokenSecretManager, nmStore);

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java

@@ -63,7 +63,7 @@ public class TestEventFlow {
   private static File remoteLogDir = new File("target",
       TestEventFlow.class.getName() + "-remoteLogDir").getAbsoluteFile();
   private static final long SIMULATED_RM_IDENTIFIER = 1234;
-  
+
   @Test
   public void testSuccessfulContainerLaunch() throws InterruptedException,
       IOException, YarnException {
@@ -98,9 +98,10 @@ public class TestEventFlow {
 
     DeletionService del = new DeletionService(exec);
     Dispatcher dispatcher = new AsyncDispatcher();
-    NodeHealthCheckerService healthChecker = new NodeHealthCheckerService();
+    LocalDirsHandlerService dirsHandler = new LocalDirsHandlerService();
+    NodeHealthCheckerService healthChecker = new NodeHealthCheckerService(
+        NodeManager.getNodeHealthScriptRunner(conf), dirsHandler);
     healthChecker.init(conf);
-    LocalDirsHandlerService dirsHandler = healthChecker.getDiskHandler();
     NodeManagerMetrics metrics = NodeManagerMetrics.create();
     NodeStatusUpdater nodeStatusUpdater =
         new NodeStatusUpdaterImpl(context, dispatcher, healthChecker, metrics) {

+ 28 - 58
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeHealthService.java

@@ -22,7 +22,6 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.util.TimerTask;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -30,6 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.NodeHealthScriptRunner;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -40,6 +40,9 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.spy;
+
 public class TestNodeHealthService {
 
   private static volatile Log LOG = LogFactory
@@ -66,17 +69,7 @@ public class TestNodeHealthService {
           new Path(testRootDir.getAbsolutePath()), true);
     }
   }
-
-  private Configuration getConfForNodeHealthScript() {
-    Configuration conf = new Configuration();
-    conf.set(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH,
-        nodeHealthscriptFile.getAbsolutePath());
-    conf.setLong(YarnConfiguration.NM_HEALTH_CHECK_INTERVAL_MS, 500);
-    conf.setLong(
-        YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS, 1000);
-    return conf;
-  }
-
+  
   private void writeNodeHealthScriptFile(String scriptStr, boolean setExecutable)
           throws IOException {
     PrintWriter pw = null;
@@ -92,28 +85,14 @@ public class TestNodeHealthService {
     FileUtil.setExecutable(nodeHealthscriptFile, setExecutable);
   }
 
-  @Test
-  public void testNodeHealthScriptShouldRun() throws IOException {
-    // Node health script should not start if there is no property called
-    // node health script path.
-    Assert.assertFalse("By default Health script should not have started",
-        NodeHealthScriptRunner.shouldRun(new Configuration()));
-    Configuration conf = getConfForNodeHealthScript();
-    // Node health script should not start if the node health script does not
-    // exists
-    Assert.assertFalse("Node health script should start",
-        NodeHealthScriptRunner.shouldRun(conf));
-    // Create script path.
-    conf.writeXml(new FileOutputStream(nodeHealthConfigFile));
-    conf.addResource(nodeHealthConfigFile.getName());
-    writeNodeHealthScriptFile("", false);
-    // Node health script should not start if the node health script is not
-    // executable.
-    Assert.assertFalse("Node health script should start",
-        NodeHealthScriptRunner.shouldRun(conf));
-    writeNodeHealthScriptFile("", true);
-    Assert.assertTrue("Node health script should start",
-        NodeHealthScriptRunner.shouldRun(conf));
+  private Configuration getConfForNodeHealthScript() {
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH,
+        nodeHealthscriptFile.getAbsolutePath());
+    conf.setLong(YarnConfiguration.NM_HEALTH_CHECK_INTERVAL_MS, 500);
+    conf.setLong(
+        YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS, 1000);
+    return conf;
   }
 
   private void setHealthStatus(NodeHealthStatus healthStatus, boolean isHealthy,
@@ -124,27 +103,24 @@ public class TestNodeHealthService {
   }
 
   @Test
-  public void testNodeHealthScript() throws Exception {
+  public void testNodeHealthService() throws Exception {
     RecordFactory factory = RecordFactoryProvider.getRecordFactory(null);
     NodeHealthStatus healthStatus =
         factory.newRecordInstance(NodeHealthStatus.class);
-    String errorScript = "echo ERROR\n echo \"Tracker not healthy\"";
-    String normalScript = "echo \"I am all fine\"";
-    String timeOutScript = Shell.WINDOWS ? "@echo off\nping -n 4 127.0.0.1 >nul\necho \"I am fine\""
-        : "sleep 4\necho \"I am fine\"";
     Configuration conf = getConfForNodeHealthScript();
     conf.writeXml(new FileOutputStream(nodeHealthConfigFile));
     conf.addResource(nodeHealthConfigFile.getName());
+    writeNodeHealthScriptFile("", true);
 
-    writeNodeHealthScriptFile(normalScript, true);
-    NodeHealthCheckerService nodeHealthChecker = new NodeHealthCheckerService();
-    nodeHealthChecker.init(conf);
+    LocalDirsHandlerService dirsHandler = new LocalDirsHandlerService();
     NodeHealthScriptRunner nodeHealthScriptRunner =
-        nodeHealthChecker.getNodeHealthScriptRunner();
-    TimerTask timerTask = nodeHealthScriptRunner.getTimerTask();
-
-    timerTask.run();
+        spy(NodeManager.getNodeHealthScriptRunner(conf));
+    NodeHealthCheckerService nodeHealthChecker = new NodeHealthCheckerService(
+    		nodeHealthScriptRunner, dirsHandler);
+    nodeHealthChecker.init(conf);
 
+    doReturn(true).when(nodeHealthScriptRunner).isHealthy();
+    doReturn("").when(nodeHealthScriptRunner).getHealthReport();
     setHealthStatus(healthStatus, nodeHealthChecker.isHealthy(),
         nodeHealthChecker.getHealthReport(),
         nodeHealthChecker.getLastHealthReportTime());
@@ -155,11 +131,7 @@ public class TestNodeHealthService {
     Assert.assertTrue("Node health status reported unhealthy", healthStatus
         .getHealthReport().equals(nodeHealthChecker.getHealthReport()));
 
-    // write out error file.
-    // Healthy to unhealthy transition
-    writeNodeHealthScriptFile(errorScript, true);
-    // Run timer
-    timerTask.run();
+    doReturn(false).when(nodeHealthScriptRunner).isHealthy();   
     // update health status
     setHealthStatus(healthStatus, nodeHealthChecker.isHealthy(),
         nodeHealthChecker.getHealthReport(),
@@ -169,10 +141,8 @@ public class TestNodeHealthService {
         .getIsNodeHealthy());
     Assert.assertTrue("Node health status reported healthy", healthStatus
         .getHealthReport().equals(nodeHealthChecker.getHealthReport()));
-    
-    // Check unhealthy to healthy transitions.
-    writeNodeHealthScriptFile(normalScript, true);
-    timerTask.run();
+
+    doReturn(true).when(nodeHealthScriptRunner).isHealthy();
     setHealthStatus(healthStatus, nodeHealthChecker.isHealthy(),
         nodeHealthChecker.getHealthReport(),
         nodeHealthChecker.getLastHealthReportTime());
@@ -184,8 +154,9 @@ public class TestNodeHealthService {
         .getHealthReport().equals(nodeHealthChecker.getHealthReport()));
 
     // Healthy to timeout transition.
-    writeNodeHealthScriptFile(timeOutScript, true);
-    timerTask.run();
+    doReturn(false).when(nodeHealthScriptRunner).isHealthy();
+    doReturn(NodeHealthScriptRunner.NODE_HEALTH_SCRIPT_TIMED_OUT_MSG)
+        .when(nodeHealthScriptRunner).getHealthReport();
     setHealthStatus(healthStatus, nodeHealthChecker.isHealthy(),
         nodeHealthChecker.getHealthReport(),
         nodeHealthChecker.getLastHealthReportTime());
@@ -198,5 +169,4 @@ public class TestNodeHealthService {
             + NodeHealthCheckerService.SEPARATOR
             + nodeHealthChecker.getDiskHandler().getDisksHealthReport(false)));
   }
-
 }

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.util.NodeHealthScriptRunner;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -57,6 +58,7 @@ import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
 import org.apache.hadoop.yarn.server.nodemanager.LocalRMInterface;
 import org.apache.hadoop.yarn.server.nodemanager.NodeHealthCheckerService;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
 import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
 import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl;
@@ -174,9 +176,10 @@ public abstract class BaseContainerManagerTest {
     delSrvc.init(conf);
 
     exec = createContainerExecutor();
-    nodeHealthChecker = new NodeHealthCheckerService();
+    dirsHandler = new LocalDirsHandlerService();
+    nodeHealthChecker = new NodeHealthCheckerService(
+        NodeManager.getNodeHealthScriptRunner(conf), dirsHandler);
     nodeHealthChecker.init(conf);
-    dirsHandler = nodeHealthChecker.getDiskHandler();
     containerManager = createContainerManager(delSrvc);
     ((NMContext)context).setContainerManager(containerManager);
     nodeStatusUpdater.init(conf);

+ 10 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.NodeHealthScriptRunner;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -74,6 +75,12 @@ import com.google.inject.Module;
 
 public class TestContainerLogsPage {
 
+  private NodeHealthCheckerService createNodeHealthCheckerService(Configuration conf) {
+    NodeHealthScriptRunner scriptRunner = NodeManager.getNodeHealthScriptRunner(conf);
+    LocalDirsHandlerService dirsHandler = new LocalDirsHandlerService();
+    return new NodeHealthCheckerService(scriptRunner, dirsHandler);
+  }
+
   @Test(timeout=30000)
   public void testContainerLogDirs() throws IOException, YarnException {
     File absLogDir = new File("target",
@@ -81,7 +88,7 @@ public class TestContainerLogsPage {
     String logdirwithFile = absLogDir.toURI().toString();
     Configuration conf = new Configuration();
     conf.set(YarnConfiguration.NM_LOG_DIRS, logdirwithFile);
-    NodeHealthCheckerService healthChecker = new NodeHealthCheckerService();
+    NodeHealthCheckerService healthChecker = createNodeHealthCheckerService(conf);
     healthChecker.init(conf);
     LocalDirsHandlerService dirsHandler = healthChecker.getDiskHandler();
     NMContext nmContext = new NodeManager.NMContext(null, null, dirsHandler,
@@ -116,7 +123,7 @@ public class TestContainerLogsPage {
     files = ContainerLogsUtils.getContainerLogDirs(container1, user, nmContext);
     Assert.assertTrue(!(files.get(0).toString().contains("file:")));
   }
-  
+
   @Test(timeout = 10000)
   public void testContainerLogPageAccess() throws IOException {
     // SecureIOUtils require Native IO to be enabled. This test will run
@@ -137,7 +144,7 @@ public class TestContainerLogsPage {
         "kerberos");
       UserGroupInformation.setConfiguration(conf);
 
-      NodeHealthCheckerService healthChecker = new NodeHealthCheckerService();
+      NodeHealthCheckerService healthChecker = createNodeHealthCheckerService(conf);
       healthChecker.init(conf);
       LocalDirsHandlerService dirsHandler = healthChecker.getDiskHandler();
       // Add an application and the corresponding containers

+ 10 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java

@@ -28,6 +28,7 @@ import java.io.Writer;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.NodeHealthScriptRunner;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -77,7 +78,13 @@ public class TestNMWebServer {
     FileUtil.fullyDelete(testRootDir);
     FileUtil.fullyDelete(testLogDir);
   }
-  
+
+  private NodeHealthCheckerService createNodeHealthCheckerService(Configuration conf) {
+    NodeHealthScriptRunner scriptRunner = NodeManager.getNodeHealthScriptRunner(conf);
+    LocalDirsHandlerService dirsHandler = new LocalDirsHandlerService();
+    return new NodeHealthCheckerService(scriptRunner, dirsHandler);
+  }
+
   private int startNMWebAppServer(String webAddr) {
     Context nmContext = new NodeManager.NMContext(null, null, null, null,
         null);
@@ -106,7 +113,7 @@ public class TestNMWebServer {
     Configuration conf = new Configuration();
     conf.set(YarnConfiguration.NM_LOCAL_DIRS, testRootDir.getAbsolutePath());
     conf.set(YarnConfiguration.NM_LOG_DIRS, testLogDir.getAbsolutePath());
-    NodeHealthCheckerService healthChecker = new NodeHealthCheckerService();
+    NodeHealthCheckerService healthChecker = createNodeHealthCheckerService(conf);
     healthChecker.init(conf);
     LocalDirsHandlerService dirsHandler = healthChecker.getDiskHandler();
     conf.set(YarnConfiguration.NM_WEBAPP_ADDRESS, webAddr);
@@ -169,7 +176,7 @@ public class TestNMWebServer {
     Configuration conf = new Configuration();
     conf.set(YarnConfiguration.NM_LOCAL_DIRS, testRootDir.getAbsolutePath());
     conf.set(YarnConfiguration.NM_LOG_DIRS, testLogDir.getAbsolutePath());
-    NodeHealthCheckerService healthChecker = new NodeHealthCheckerService();
+    NodeHealthCheckerService healthChecker = createNodeHealthCheckerService(conf);
     healthChecker.init(conf);
     LocalDirsHandlerService dirsHandler = healthChecker.getDiskHandler();
 

+ 5 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java

@@ -36,6 +36,7 @@ import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.NodeHealthScriptRunner;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -98,14 +99,16 @@ public class TestNMWebServices extends JerseyTestBase {
       TestNMWebServices.class.getSimpleName() + "LogDir");
 
   private Injector injector = Guice.createInjector(new ServletModule() {
+
     @Override
     protected void configureServlets() {
       Configuration conf = new Configuration();
       conf.set(YarnConfiguration.NM_LOCAL_DIRS, testRootDir.getAbsolutePath());
       conf.set(YarnConfiguration.NM_LOG_DIRS, testLogDir.getAbsolutePath());
-      NodeHealthCheckerService healthChecker = new NodeHealthCheckerService();
+      dirsHandler = new LocalDirsHandlerService();
+      NodeHealthCheckerService healthChecker = new NodeHealthCheckerService(
+          NodeManager.getNodeHealthScriptRunner(conf), dirsHandler);
       healthChecker.init(conf);
-      dirsHandler = healthChecker.getDiskHandler();
       aclsManager = new ApplicationACLsManager(conf);
       nmContext = new NodeManager.NMContext(null, null, dirsHandler,
           aclsManager, null);

+ 5 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesApps.java

@@ -33,6 +33,7 @@ import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.NodeHealthScriptRunner;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -91,11 +92,14 @@ public class TestNMWebServicesApps extends JerseyTestBase {
       TestNMWebServicesApps.class.getSimpleName() + "LogDir");
 
   private Injector injector = Guice.createInjector(new ServletModule() {
+
     @Override
     protected void configureServlets() {
       conf.set(YarnConfiguration.NM_LOCAL_DIRS, testRootDir.getAbsolutePath());
       conf.set(YarnConfiguration.NM_LOG_DIRS, testLogDir.getAbsolutePath());
-      NodeHealthCheckerService healthChecker = new NodeHealthCheckerService();
+      LocalDirsHandlerService dirsHandler = new LocalDirsHandlerService();
+      NodeHealthCheckerService healthChecker = new NodeHealthCheckerService(
+          NodeManager.getNodeHealthScriptRunner(conf), dirsHandler);
       healthChecker.init(conf);
       dirsHandler = healthChecker.getDiskHandler();
       aclsManager = new ApplicationACLsManager(conf);

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServicesContainers.java

@@ -34,6 +34,7 @@ import javax.xml.parsers.DocumentBuilderFactory;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.NodeHealthScriptRunner;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -123,7 +124,9 @@ public class TestNMWebServicesContainers extends JerseyTestBase {
       };
       conf.set(YarnConfiguration.NM_LOCAL_DIRS, testRootDir.getAbsolutePath());
       conf.set(YarnConfiguration.NM_LOG_DIRS, testLogDir.getAbsolutePath());
-      NodeHealthCheckerService healthChecker = new NodeHealthCheckerService();
+      LocalDirsHandlerService dirsHandler = new LocalDirsHandlerService();
+      NodeHealthCheckerService healthChecker = new NodeHealthCheckerService(
+          NodeManager.getNodeHealthScriptRunner(conf), dirsHandler);
       healthChecker.init(conf);
       dirsHandler = healthChecker.getDiskHandler();
       aclsManager = new ApplicationACLsManager(conf);