Przeglądaj źródła

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

Allen Wittenauer 10 lat temu
rodzic
commit
d4ac6822e1
13 zmienionych plików z 259 dodań i 107 usunięć
  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);