|
@@ -16,7 +16,7 @@
|
|
* limitations under the License.
|
|
* limitations under the License.
|
|
*/
|
|
*/
|
|
|
|
|
|
-package org.apache.hadoop.util;
|
|
|
|
|
|
+package org.apache.hadoop.yarn.server.nodemanager.health;
|
|
|
|
|
|
import java.io.File;
|
|
import java.io.File;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
@@ -27,51 +27,95 @@ import java.util.TimerTask;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
-import org.apache.hadoop.service.AbstractService;
|
|
|
|
import org.apache.hadoop.util.Shell.ExitCodeException;
|
|
import org.apache.hadoop.util.Shell.ExitCodeException;
|
|
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
|
|
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
|
|
import org.apache.hadoop.util.Shell;
|
|
import org.apache.hadoop.util.Shell;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
|
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
/**
|
|
/**
|
|
- *
|
|
|
|
* The class which provides functionality of checking the health of the node
|
|
* The class which provides functionality of checking the health of the node
|
|
* using the configured node health script and reporting back to the service
|
|
* using the configured node health script and reporting back to the service
|
|
* for which the health checker has been asked to report.
|
|
* for which the health checker has been asked to report.
|
|
*/
|
|
*/
|
|
-public class NodeHealthScriptRunner extends AbstractService {
|
|
|
|
|
|
+public class NodeHealthScriptRunner extends TimedHealthReporterService {
|
|
|
|
|
|
private static final Logger LOG =
|
|
private static final Logger LOG =
|
|
LoggerFactory.getLogger(NodeHealthScriptRunner.class);
|
|
LoggerFactory.getLogger(NodeHealthScriptRunner.class);
|
|
|
|
|
|
/** Absolute path to the health script. */
|
|
/** Absolute path to the health script. */
|
|
private String nodeHealthScript;
|
|
private String nodeHealthScript;
|
|
- /** Delay after which node health script to be executed */
|
|
|
|
- private long intervalTime;
|
|
|
|
- /** Time after which the script should be timedout */
|
|
|
|
|
|
+ /** Time after which the script should be timed out. */
|
|
private long scriptTimeout;
|
|
private long scriptTimeout;
|
|
- /** Timer used to schedule node health monitoring script execution */
|
|
|
|
- private Timer nodeHealthScriptScheduler;
|
|
|
|
|
|
+ /** ShellCommandExecutor used to execute monitoring script. */
|
|
|
|
+ private ShellCommandExecutor commandExecutor = null;
|
|
|
|
|
|
- /** ShellCommandExecutor used to execute monitoring script */
|
|
|
|
- ShellCommandExecutor shexec = null;
|
|
|
|
|
|
+ /** Pattern used for searching in the output of the node health script. */
|
|
|
|
+ private static final String ERROR_PATTERN = "ERROR";
|
|
|
|
|
|
- /** 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";
|
|
|
|
|
|
- /** Time out error message */
|
|
|
|
- public static final String NODE_HEALTH_SCRIPT_TIMED_OUT_MSG = "Node health script timed out";
|
|
|
|
|
|
+ private NodeHealthScriptRunner(String scriptName, long checkInterval,
|
|
|
|
+ long timeout, String[] scriptArgs) {
|
|
|
|
+ super(NodeHealthScriptRunner.class.getName(), checkInterval);
|
|
|
|
+ this.nodeHealthScript = scriptName;
|
|
|
|
+ this.scriptTimeout = timeout;
|
|
|
|
+ setTimerTask(new NodeHealthMonitorExecutor(scriptArgs));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public static NodeHealthScriptRunner newInstance(String scriptName,
|
|
|
|
+ Configuration conf) {
|
|
|
|
+ String nodeHealthScriptsConfig = String.format(
|
|
|
|
+ YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH_TEMPLATE, scriptName);
|
|
|
|
+ String nodeHealthScript = conf.get(nodeHealthScriptsConfig);
|
|
|
|
+ if (!shouldRun(scriptName, nodeHealthScript)) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
|
|
- private boolean isHealthy;
|
|
|
|
|
|
+ // Determine check interval ms
|
|
|
|
+ String checkIntervalMsConfig = String.format(
|
|
|
|
+ YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_INTERVAL_MS_TEMPLATE,
|
|
|
|
+ scriptName);
|
|
|
|
+ long checkIntervalMs = conf.getLong(checkIntervalMsConfig, 0L);
|
|
|
|
+ if (checkIntervalMs == 0L) {
|
|
|
|
+ checkIntervalMs = conf.getLong(
|
|
|
|
+ YarnConfiguration.NM_HEALTH_CHECK_INTERVAL_MS,
|
|
|
|
+ YarnConfiguration.DEFAULT_NM_HEALTH_CHECK_INTERVAL_MS);
|
|
|
|
+ }
|
|
|
|
+ if (checkIntervalMs < 0) {
|
|
|
|
+ throw new IllegalArgumentException("The node health-checker's " +
|
|
|
|
+ "interval-ms can not be set to a negative number.");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Determine time out
|
|
|
|
+ String scriptTimeoutConfig = String.format(
|
|
|
|
+ YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_TIMEOUT_MS_TEMPLATE,
|
|
|
|
+ scriptName);
|
|
|
|
+ long scriptTimeout = conf.getLong(scriptTimeoutConfig, 0L);
|
|
|
|
+ if (scriptTimeout == 0L) {
|
|
|
|
+ scriptTimeout = conf.getLong(
|
|
|
|
+ YarnConfiguration.NM_HEALTH_CHECK_TIMEOUT_MS,
|
|
|
|
+ YarnConfiguration.DEFAULT_NM_HEALTH_CHECK_TIMEOUT_MS);
|
|
|
|
+ }
|
|
|
|
+ if (scriptTimeout <= 0) {
|
|
|
|
+ throw new IllegalArgumentException("The node health-checker's " +
|
|
|
|
+ "timeout can only be set to a positive number.");
|
|
|
|
+ }
|
|
|
|
|
|
- private String healthReport;
|
|
|
|
|
|
+ // Determine script arguments
|
|
|
|
+ String scriptArgsConfig = String.format(
|
|
|
|
+ YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_OPTS_TEMPLATE,
|
|
|
|
+ scriptName);
|
|
|
|
+ String[] scriptArgs = conf.getStrings(scriptArgsConfig, new String[]{});
|
|
|
|
|
|
- private long lastReportedTime;
|
|
|
|
|
|
+ return new NodeHealthScriptRunner(nodeHealthScript,
|
|
|
|
+ checkIntervalMs, scriptTimeout, scriptArgs);
|
|
|
|
+ }
|
|
|
|
|
|
- private TimerTask timer;
|
|
|
|
-
|
|
|
|
private enum HealthCheckerExitStatus {
|
|
private enum HealthCheckerExitStatus {
|
|
SUCCESS,
|
|
SUCCESS,
|
|
TIMED_OUT,
|
|
TIMED_OUT,
|
|
@@ -84,19 +128,17 @@ public class NodeHealthScriptRunner extends AbstractService {
|
|
/**
|
|
/**
|
|
* Class which is used by the {@link Timer} class to periodically execute the
|
|
* Class which is used by the {@link Timer} class to periodically execute the
|
|
* node health script.
|
|
* node health script.
|
|
- *
|
|
|
|
*/
|
|
*/
|
|
private class NodeHealthMonitorExecutor extends TimerTask {
|
|
private class NodeHealthMonitorExecutor extends TimerTask {
|
|
|
|
+ private String exceptionStackTrace = "";
|
|
|
|
|
|
- String exceptionStackTrace = "";
|
|
|
|
-
|
|
|
|
- public NodeHealthMonitorExecutor(String[] args) {
|
|
|
|
|
|
+ NodeHealthMonitorExecutor(String[] args) {
|
|
ArrayList<String> execScript = new ArrayList<String>();
|
|
ArrayList<String> execScript = new ArrayList<String>();
|
|
execScript.add(nodeHealthScript);
|
|
execScript.add(nodeHealthScript);
|
|
if (args != null) {
|
|
if (args != null) {
|
|
execScript.addAll(Arrays.asList(args));
|
|
execScript.addAll(Arrays.asList(args));
|
|
}
|
|
}
|
|
- shexec = new ShellCommandExecutor(execScript
|
|
|
|
|
|
+ commandExecutor = new ShellCommandExecutor(execScript
|
|
.toArray(new String[execScript.size()]), null, null, scriptTimeout);
|
|
.toArray(new String[execScript.size()]), null, null, scriptTimeout);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -104,18 +146,18 @@ public class NodeHealthScriptRunner extends AbstractService {
|
|
public void run() {
|
|
public void run() {
|
|
HealthCheckerExitStatus status = HealthCheckerExitStatus.SUCCESS;
|
|
HealthCheckerExitStatus status = HealthCheckerExitStatus.SUCCESS;
|
|
try {
|
|
try {
|
|
- shexec.execute();
|
|
|
|
|
|
+ commandExecutor.execute();
|
|
} catch (ExitCodeException e) {
|
|
} catch (ExitCodeException e) {
|
|
// ignore the exit code of the script
|
|
// ignore the exit code of the script
|
|
status = HealthCheckerExitStatus.FAILED_WITH_EXIT_CODE;
|
|
status = HealthCheckerExitStatus.FAILED_WITH_EXIT_CODE;
|
|
// On Windows, we will not hit the Stream closed IOException
|
|
// On Windows, we will not hit the Stream closed IOException
|
|
// thrown by stdout buffered reader for timeout event.
|
|
// thrown by stdout buffered reader for timeout event.
|
|
- if (Shell.WINDOWS && shexec.isTimedOut()) {
|
|
|
|
|
|
+ if (Shell.WINDOWS && commandExecutor.isTimedOut()) {
|
|
status = HealthCheckerExitStatus.TIMED_OUT;
|
|
status = HealthCheckerExitStatus.TIMED_OUT;
|
|
}
|
|
}
|
|
} catch (Exception e) {
|
|
} catch (Exception e) {
|
|
LOG.warn("Caught exception : " + e.getMessage());
|
|
LOG.warn("Caught exception : " + e.getMessage());
|
|
- if (!shexec.isTimedOut()) {
|
|
|
|
|
|
+ if (!commandExecutor.isTimedOut()) {
|
|
status = HealthCheckerExitStatus.FAILED_WITH_EXCEPTION;
|
|
status = HealthCheckerExitStatus.FAILED_WITH_EXCEPTION;
|
|
} else {
|
|
} else {
|
|
status = HealthCheckerExitStatus.TIMED_OUT;
|
|
status = HealthCheckerExitStatus.TIMED_OUT;
|
|
@@ -123,7 +165,7 @@ public class NodeHealthScriptRunner extends AbstractService {
|
|
exceptionStackTrace = StringUtils.stringifyException(e);
|
|
exceptionStackTrace = StringUtils.stringifyException(e);
|
|
} finally {
|
|
} finally {
|
|
if (status == HealthCheckerExitStatus.SUCCESS) {
|
|
if (status == HealthCheckerExitStatus.SUCCESS) {
|
|
- if (hasErrors(shexec.getOutput())) {
|
|
|
|
|
|
+ if (hasErrors(commandExecutor.getOutput())) {
|
|
status = HealthCheckerExitStatus.FAILED;
|
|
status = HealthCheckerExitStatus.FAILED;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -134,49 +176,49 @@ public class NodeHealthScriptRunner extends AbstractService {
|
|
/**
|
|
/**
|
|
* Method which is used to parse output from the node health monitor and
|
|
* Method which is used to parse output from the node health monitor and
|
|
* send to the report address.
|
|
* send to the report address.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* The timed out script or script which causes IOException output is
|
|
* The timed out script or script which causes IOException output is
|
|
* ignored.
|
|
* ignored.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* The node is marked unhealthy if
|
|
* The node is marked unhealthy if
|
|
* <ol>
|
|
* <ol>
|
|
* <li>The node health script times out</li>
|
|
* <li>The node health script times out</li>
|
|
- * <li>The node health scripts output has a line which begins with ERROR</li>
|
|
|
|
|
|
+ * <li>The node health scripts output has a line which begins
|
|
|
|
+ * with ERROR</li>
|
|
* <li>An exception is thrown while executing the script</li>
|
|
* <li>An exception is thrown while executing the script</li>
|
|
* </ol>
|
|
* </ol>
|
|
* If the script throws {@link IOException} or {@link ExitCodeException} the
|
|
* If the script throws {@link IOException} or {@link ExitCodeException} the
|
|
* output is ignored and node is left remaining healthy, as script might
|
|
* output is ignored and node is left remaining healthy, as script might
|
|
* have syntax error.
|
|
* have syntax error.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @param status
|
|
* @param status
|
|
*/
|
|
*/
|
|
void reportHealthStatus(HealthCheckerExitStatus status) {
|
|
void reportHealthStatus(HealthCheckerExitStatus status) {
|
|
- long now = System.currentTimeMillis();
|
|
|
|
switch (status) {
|
|
switch (status) {
|
|
case SUCCESS:
|
|
case SUCCESS:
|
|
- setHealthStatus(true, "", now);
|
|
|
|
|
|
+ case FAILED_WITH_EXIT_CODE:
|
|
|
|
+ // see Javadoc above - we don't report bad health intentionally
|
|
|
|
+ setHealthyWithoutReport();
|
|
break;
|
|
break;
|
|
case TIMED_OUT:
|
|
case TIMED_OUT:
|
|
- setHealthStatus(false, NODE_HEALTH_SCRIPT_TIMED_OUT_MSG);
|
|
|
|
|
|
+ setUnhealthyWithReport(NODE_HEALTH_SCRIPT_TIMED_OUT_MSG);
|
|
break;
|
|
break;
|
|
case FAILED_WITH_EXCEPTION:
|
|
case FAILED_WITH_EXCEPTION:
|
|
- setHealthStatus(false, exceptionStackTrace);
|
|
|
|
- break;
|
|
|
|
- case FAILED_WITH_EXIT_CODE:
|
|
|
|
- // see Javadoc above - we don't report bad health intentionally
|
|
|
|
- setHealthStatus(true, "", now);
|
|
|
|
|
|
+ setUnhealthyWithReport(exceptionStackTrace);
|
|
break;
|
|
break;
|
|
case FAILED:
|
|
case FAILED:
|
|
- setHealthStatus(false, shexec.getOutput());
|
|
|
|
|
|
+ setUnhealthyWithReport(commandExecutor.getOutput());
|
|
|
|
+ break;
|
|
|
|
+ default:
|
|
|
|
+ LOG.warn("Unknown HealthCheckerExitStatus - ignored.");
|
|
break;
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Method to check if the output string has line which begins with ERROR.
|
|
* Method to check if the output string has line which begins with ERROR.
|
|
- *
|
|
|
|
- * @param output
|
|
|
|
- * string
|
|
|
|
|
|
+ *
|
|
|
|
+ * @param output the output of the node health script to process
|
|
* @return true if output string has error pattern in it.
|
|
* @return true if output string has error pattern in it.
|
|
*/
|
|
*/
|
|
private boolean hasErrors(String output) {
|
|
private boolean hasErrors(String output) {
|
|
@@ -190,150 +232,46 @@ public class NodeHealthScriptRunner extends AbstractService {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- public NodeHealthScriptRunner(String scriptName, long chkInterval, long timeout,
|
|
|
|
- String[] scriptArgs) {
|
|
|
|
- super(NodeHealthScriptRunner.class.getName());
|
|
|
|
- this.lastReportedTime = System.currentTimeMillis();
|
|
|
|
- this.isHealthy = true;
|
|
|
|
- this.healthReport = "";
|
|
|
|
- this.nodeHealthScript = scriptName;
|
|
|
|
- this.intervalTime = chkInterval;
|
|
|
|
- this.scriptTimeout = timeout;
|
|
|
|
- this.timer = new NodeHealthMonitorExecutor(scriptArgs);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /*
|
|
|
|
- * Method which initializes the values for the script path and interval time.
|
|
|
|
- */
|
|
|
|
@Override
|
|
@Override
|
|
- protected void serviceInit(Configuration conf) throws Exception {
|
|
|
|
- super.serviceInit(conf);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Method used to start the Node health monitoring.
|
|
|
|
- *
|
|
|
|
- */
|
|
|
|
- @Override
|
|
|
|
- protected void serviceStart() throws Exception {
|
|
|
|
- nodeHealthScriptScheduler = new Timer("NodeHealthMonitor-Timer", true);
|
|
|
|
- // Start the timer task immediately and
|
|
|
|
- // then periodically at interval time.
|
|
|
|
- nodeHealthScriptScheduler.scheduleAtFixedRate(timer, 0, intervalTime);
|
|
|
|
- super.serviceStart();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Method used to terminate the node health monitoring service.
|
|
|
|
- *
|
|
|
|
- */
|
|
|
|
- @Override
|
|
|
|
- protected void serviceStop() {
|
|
|
|
- if (nodeHealthScriptScheduler != null) {
|
|
|
|
- nodeHealthScriptScheduler.cancel();
|
|
|
|
- }
|
|
|
|
- if (shexec != null) {
|
|
|
|
- Process p = shexec.getProcess();
|
|
|
|
|
|
+ public void serviceStop() throws Exception {
|
|
|
|
+ if (commandExecutor != null) {
|
|
|
|
+ Process p = commandExecutor.getProcess();
|
|
if (p != null) {
|
|
if (p != null) {
|
|
p.destroy();
|
|
p.destroy();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+ super.serviceStop();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Gets the if the node is healthy or not
|
|
|
|
- *
|
|
|
|
- * @return true if node is healthy
|
|
|
|
- */
|
|
|
|
- public boolean isHealthy() {
|
|
|
|
- return isHealthy;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Sets if the node is healthy or not considering disks' health also.
|
|
|
|
- *
|
|
|
|
- * @param isHealthy
|
|
|
|
- * if or not node is healthy
|
|
|
|
- */
|
|
|
|
- private synchronized void setHealthy(boolean isHealthy) {
|
|
|
|
- this.isHealthy = isHealthy;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Returns output from health script. if node is healthy then an empty string
|
|
|
|
- * is returned.
|
|
|
|
- *
|
|
|
|
- * @return output from health script
|
|
|
|
- */
|
|
|
|
- public String getHealthReport() {
|
|
|
|
- return healthReport;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Sets the health report from the node health script. Also set the disks'
|
|
|
|
- * health info obtained from DiskHealthCheckerService.
|
|
|
|
|
|
+ * Method used to determine whether the {@link NodeHealthScriptRunner}
|
|
|
|
+ * should be started or not.<p>
|
|
|
|
+ * Returns true if following conditions are met:
|
|
*
|
|
*
|
|
- * @param healthReport
|
|
|
|
- */
|
|
|
|
- private synchronized void setHealthReport(String healthReport) {
|
|
|
|
- this.healthReport = healthReport;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Returns time stamp when node health script was last run.
|
|
|
|
- *
|
|
|
|
- * @return timestamp when node health script was last run
|
|
|
|
- */
|
|
|
|
- public long getLastReportedTime() {
|
|
|
|
- return lastReportedTime;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Sets the last run time of the node health script.
|
|
|
|
- *
|
|
|
|
- * @param lastReportedTime
|
|
|
|
- */
|
|
|
|
- private synchronized void setLastReportedTime(long lastReportedTime) {
|
|
|
|
- this.lastReportedTime = lastReportedTime;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Method used to determine if or not node health monitoring service should be
|
|
|
|
- * started or not. Returns true if following conditions are met:
|
|
|
|
- *
|
|
|
|
* <ol>
|
|
* <ol>
|
|
* <li>Path to Node health check script is not empty</li>
|
|
* <li>Path to Node health check script is not empty</li>
|
|
* <li>Node health check script file exists</li>
|
|
* <li>Node health check script file exists</li>
|
|
* </ol>
|
|
* </ol>
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @return true if node health monitoring service can be started.
|
|
* @return true if node health monitoring service can be started.
|
|
*/
|
|
*/
|
|
- public static boolean shouldRun(String healthScript) {
|
|
|
|
|
|
+ static boolean shouldRun(String script, String healthScript) {
|
|
if (healthScript == null || healthScript.trim().isEmpty()) {
|
|
if (healthScript == null || healthScript.trim().isEmpty()) {
|
|
|
|
+ LOG.info("Missing location for the node health check script \"{}\".",
|
|
|
|
+ script);
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
File f = new File(healthScript);
|
|
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);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /**
|
|
|
|
- * Used only by tests to access the timer task directly
|
|
|
|
- * @return the timer task
|
|
|
|
- */
|
|
|
|
- public TimerTask getTimerTask() {
|
|
|
|
- return timer;
|
|
|
|
|
|
+ if (!f.exists()) {
|
|
|
|
+ LOG.warn("File {} for script \"{}\" does not exist.",
|
|
|
|
+ healthScript, script);
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+ if (!FileUtil.canExecute(f)) {
|
|
|
|
+ LOG.warn("File {} for script \"{}\" can not be executed.",
|
|
|
|
+ healthScript, script);
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+ return true;
|
|
}
|
|
}
|
|
}
|
|
}
|