You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2014/06/23 20:41:38 UTC

git commit: Add task health check and integrate with command executor.

Repository: mesos
Updated Branches:
  refs/heads/master f943e2fc8 -> ff64e4ed6


Add task health check and integrate with command executor.

Added task health check process that is launched with the command
executor if health check is configured.

It runs configured health check command and return the status to the
executor to report the task health. The executor also reports the task
health status back to the scheduler.

The task health process keeps internal state based on health check
configuration, and determine when the task it is checking for should
be killed. Currently it's based on the number of consecutive failures
it observed. Once the condition meets it sends a task health status
update with a kill task flag turned on, and the executor will kill the
task.

Review: https://reviews.apache.org/r/22579


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

Branch: refs/heads/master
Commit: ff64e4ed6d5d2c7176c9051afc35996f18e633d7
Parents: f943e2f
Author: Timothy Chen <tn...@apache.org>
Authored: Mon Jun 23 11:40:31 2014 -0700
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Mon Jun 23 11:40:31 2014 -0700

----------------------------------------------------------------------
 include/mesos/mesos.proto                       |   6 +-
 src/Makefile.am                                 |   7 +
 src/common/status_utils.hpp                     |  48 +++
 src/common/type_utils.hpp                       |  12 +-
 src/health-check/main.cpp                       | 332 ++++++++++++++++
 src/launcher/executor.cpp                       | 164 ++++++--
 src/master/master.cpp                           |  10 +-
 src/messages/messages.proto                     |  18 +
 .../containerizer/external_containerizer.cpp    |  14 +-
 src/slave/slave.cpp                             |  12 +-
 src/tests/health_check_tests.cpp                | 382 +++++++++++++++++++
 src/tests/script.cpp                            |  13 +-
 12 files changed, 945 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ff64e4ed/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index 6968411..c02b8ec 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -131,8 +131,8 @@ message FrameworkInfo {
 /**
  * Describes a health check for a task or executor (or any arbitrary
  * process/command). A "strategy" is picked by specifying one of the
- * optional fields, currently only 'http' is supported. Specifying
- * more than one strategy is an error.
+ * optional fields, currently only 'http' and 'command' are
+ * supported. Specifying more than one strategy is an error.
  */
 message HealthCheck {
   // Describes an HTTP health check.
@@ -179,7 +179,7 @@ message HealthCheck {
   optional double timeout_seconds = 4 [default = 20.0];
 
   // Number of consecutive failures until considered unhealthy.
-  optional uint32 failures = 5 [default = 3];
+  optional uint32 consecutive_failures = 5 [default = 3];
 
   // Amount of time to allow failed health checks since launch.
   optional double grace_period_seconds = 6 [default = 10.0];

http://git-wip-us.apache.org/repos/asf/mesos/blob/ff64e4ed/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index a87eb21..861aad2 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -343,6 +343,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	authorizer/authorizer.hpp					\
 	common/attributes.hpp						\
 	common/build.hpp common/date_utils.hpp common/factory.hpp	\
+        common/status_utils.hpp                                         \
 	common/parse.hpp						\
 	common/protobuf_utils.hpp					\
 	common/http.hpp							\
@@ -563,6 +564,11 @@ mesos_executor_SOURCES = launcher/executor.cpp
 mesos_executor_CPPFLAGS = $(MESOS_CPPFLAGS)
 mesos_executor_LDADD = libmesos.la
 
+pkglibexec_PROGRAMS += mesos-health-check
+mesos_health_check_SOURCES = health-check/main.cpp
+mesos_health_check_CPPFLAGS = $(MESOS_CPPFLAGS)
+mesos_health_check_LDADD = libmesos.la
+
 pkglibexec_PROGRAMS += mesos-usage
 mesos_usage_SOURCES = usage/main.cpp
 mesos_usage_CPPFLAGS = $(MESOS_CPPFLAGS)
@@ -982,6 +988,7 @@ mesos_tests_SOURCES =				\
   tests/environment.cpp				\
   tests/examples_tests.cpp			\
   tests/exception_tests.cpp			\
+  tests/health_check_tests.cpp                  \
   tests/fault_tolerance_tests.cpp		\
   tests/files_tests.cpp				\
   tests/flags.cpp				\

http://git-wip-us.apache.org/repos/asf/mesos/blob/ff64e4ed/src/common/status_utils.hpp
----------------------------------------------------------------------
diff --git a/src/common/status_utils.hpp b/src/common/status_utils.hpp
new file mode 100644
index 0000000..da1f16a
--- /dev/null
+++ b/src/common/status_utils.hpp
@@ -0,0 +1,48 @@
+/**
+ * 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.
+ */
+
+#ifndef __STATUS_UTILS_HPP__
+#define __STATUS_UTILS_HPP__
+
+#include <string>
+
+#include <stout/option.hpp>
+#include "stout/stringify.hpp"
+
+namespace mesos {
+namespace internal {
+namespace status {
+
+inline std::string WSTRINGIFY(int status)
+{
+  std::string message;
+  if (WIFEXITED(status)) {
+    message += "exited with status ";
+    message += stringify(WEXITSTATUS(status));
+  } else {
+    message += "terminated with signal ";
+    message += strsignal(WTERMSIG(status));
+  }
+  return message;
+}
+
+} // namespace status
+} // namespace internal
+} // namespace mesos
+
+#endif // __STATUS_UTILS_HPP__
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/mesos/blob/ff64e4ed/src/common/type_utils.hpp
----------------------------------------------------------------------
diff --git a/src/common/type_utils.hpp b/src/common/type_utils.hpp
index bb357ac..ee946a3 100644
--- a/src/common/type_utils.hpp
+++ b/src/common/type_utils.hpp
@@ -430,10 +430,18 @@ inline std::ostream& operator << (
     std::ostream& stream,
     const StatusUpdate& update)
 {
-  return stream
+  stream
     << update.status().state()
     << " (UUID: " << UUID::fromBytes(update.uuid())
-    << ") for task " << update.status().task_id()
+    << ") for task " << update.status().task_id();
+
+  if (update.status().has_healthy()) {
+    stream
+      << " in health state "
+      << (update.status().healthy() ? "healthy" : "unhealthy");
+  }
+
+  return stream
     << " of framework " << update.framework_id();
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/ff64e4ed/src/health-check/main.cpp
----------------------------------------------------------------------
diff --git a/src/health-check/main.cpp b/src/health-check/main.cpp
new file mode 100644
index 0000000..05e2924
--- /dev/null
+++ b/src/health-check/main.cpp
@@ -0,0 +1,332 @@
+/**
+ * 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.
+ */
+
+#include <signal.h>
+#include <stdio.h>
+#include <iostream>
+#include <string>
+#include <string.h>
+#include <unistd.h>
+
+#include <mesos/mesos.hpp>
+
+#include <process/pid.hpp>
+#include <process/defer.hpp>
+#include <process/delay.hpp>
+#include <process/future.hpp>
+#include <process/io.hpp>
+#include <process/process.hpp>
+#include <process/protobuf.hpp>
+#include <process/subprocess.hpp>
+
+#include <stout/duration.hpp>
+#include <stout/os.hpp>
+#include <stout/option.hpp>
+#include <stout/flags.hpp>
+#include <stout/protobuf.hpp>
+#include <stout/json.hpp>
+
+#include "common/status_utils.hpp"
+
+#include "messages/messages.hpp"
+
+using namespace mesos;
+
+using std::cout;
+using std::cerr;
+using std::endl;
+using std::string;
+using std::map;
+
+using process::UPID;
+
+namespace mesos {
+namespace internal {
+
+using namespace process;
+
+using namespace mesos::internal::status;
+
+class HealthCheckerProcess : public ProtobufProcess<HealthCheckerProcess>
+{
+public:
+  HealthCheckerProcess(
+    const HealthCheck& _check,
+    const UPID& _executor,
+    const TaskID& _taskID)
+    : check(_check),
+      initializing(true),
+      executor(_executor),
+      taskID(_taskID),
+      consecutiveFailures(0) {}
+
+  virtual ~HealthCheckerProcess() {}
+
+  Future<Nothing> healthCheck()
+  {
+    VLOG(2) << "Health checks starting in "
+      << Seconds(check.delay_seconds()) << ", grace period "
+      << Seconds(check.grace_period_seconds());
+
+    startTime = Clock::now();
+
+    delay(Seconds(check.delay_seconds()), self(), &Self::_healthCheck);
+    return promise.future();
+  }
+
+private:
+  void failure(const string& message)
+  {
+    if (check.grace_period_seconds() > 0 &&
+        (Clock::now() - startTime).secs() <= check.grace_period_seconds()) {
+      LOG(INFO) << "Ignoring failure as health check still in grace period";
+      reschedule();
+      return;
+    }
+
+    consecutiveFailures++;
+    VLOG(1) << "#" << consecutiveFailures << " check failed: " << message;
+
+    bool killTask = consecutiveFailures >= check.consecutive_failures();
+
+    TaskHealthStatus taskHealthStatus;
+    taskHealthStatus.set_healthy(false);
+    taskHealthStatus.set_consecutive_failures(consecutiveFailures);
+    taskHealthStatus.set_kill_task(killTask);
+    taskHealthStatus.mutable_task_id()->CopyFrom(taskID);
+    send(executor, taskHealthStatus);
+
+    if (killTask) {
+      promise.fail(message);
+    } else {
+      reschedule();
+    }
+  }
+
+  void success()
+  {
+    VLOG(1) << "Check passed";
+    if (initializing) {
+      TaskHealthStatus taskHealthStatus;
+      taskHealthStatus.set_healthy(true);
+      taskHealthStatus.mutable_task_id()->CopyFrom(taskID);
+      send(executor, taskHealthStatus);
+      initializing = false;
+    }
+    consecutiveFailures = 0;
+    reschedule();
+  }
+
+  void _healthCheck()
+  {
+    if (check.has_http()) {
+      promise.fail("HTTP health check is not supported");
+    } else if (check.has_command()) {
+      const CommandInfo& command = check.command();
+
+      map<string, string> environment;
+
+      foreach (const Environment_Variable& variable,
+               command.environment().variables()) {
+        environment[variable.name()] = variable.value();
+      }
+
+      VLOG(2) << "Launching health command: " << command.value();
+
+      Try<Subprocess> external =
+        process::subprocess(
+          command.value(),
+          Subprocess::PIPE(),
+          Subprocess::FD(STDERR_FILENO),
+          Subprocess::FD(STDERR_FILENO),
+          None(),
+          environment);
+
+      if (external.isError()) {
+        promise.fail("Error creating subprocess for healthcheck");
+      } else {
+        Future<Option<int> > status = external.get().status();
+        status.await(Seconds(check.timeout_seconds()));
+
+        if (status.isFailed()) {
+          promise.fail("Shell command check failed with status: " +
+                        status.failure());
+          return;
+        }
+
+        int statusCode = status.get().get();
+        if (statusCode != 0) {
+          string message = "Health command check " + WSTRINGIFY(statusCode);
+          failure(message);
+        } else {
+          success();
+        }
+      }
+    } else {
+      promise.fail("No check found in health check");
+    }
+  }
+
+  void reschedule()
+  {
+    VLOG(1) << "Rescheduling health check in "
+      << Seconds(check.interval_seconds());
+
+    delay(Seconds(check.interval_seconds()), self(), &Self::_healthCheck);
+  }
+
+  Promise<Nothing> promise;
+  HealthCheck check;
+  bool initializing;
+  UPID executor;
+  TaskID taskID;
+  uint32_t consecutiveFailures;
+  process::Time startTime;
+};
+
+} // namespace internal {
+} // namespace mesos {
+
+
+class Flags : public virtual flags::FlagsBase
+{
+public:
+  Flags()
+  {
+    add(&Flags::health_check_json,
+        "health_check_json",
+        "JSON describing health check to perform");
+
+    add(&Flags::executor,
+        "executor",
+        "Executor UPID to send health check messages to");
+
+    add(&Flags::task_id,
+        "task_id",
+        "Task ID that this health check process is checking");
+  }
+
+  Option<std::string> health_check_json;
+  Option<UPID> executor;
+  Option<std::string> task_id;
+};
+
+
+void usage(const char* argv0, const flags::FlagsBase& flags)
+{
+  cerr << "Usage: " << os::basename(argv0).get() << " [...]" << endl
+       << endl
+       << "Supported options:" << endl
+       << flags.usage();
+}
+
+
+int main(int argc, char** argv)
+{
+  GOOGLE_PROTOBUF_VERIFY_VERSION;
+
+  Flags flags;
+
+  bool help;
+  flags.add(&help,
+            "help",
+            "Prints this help message",
+            false);
+
+  Try<Nothing> load = flags.load(None(), argc, argv);
+
+  if (load.isError()) {
+    LOG(WARNING) << load.error();
+    usage(argv[0], flags);
+    return -1;
+  }
+
+  if (help) {
+    usage(argv[0], flags);
+    return 0;
+  }
+
+  if (flags.health_check_json.isNone()) {
+    LOG(WARNING) << "Expected JSON with health check description";
+    usage(argv[0], flags);
+    return 0;
+  }
+
+  Try<JSON::Object> parse =
+    JSON::parse<JSON::Object>(flags.health_check_json.get());
+  if (parse.isError()) {
+    LOG(WARNING) << "JSON parse error: " << parse.error();
+    usage(argv[0], flags);
+    return 0;
+  }
+
+  if (flags.executor.isNone()) {
+    LOG(WARNING) << "Expected UPID for health check";
+    usage(argv[0], flags);
+    return 0;
+  }
+
+  Try<HealthCheck> check = protobuf::parse<HealthCheck>(parse.get());
+  if (check.isError()) {
+    LOG(WARNING) << "JSON error: " << check.error();
+    usage(argv[0], flags);
+    return 0;
+  }
+
+  if (check.get().has_http() && check.get().has_command()) {
+    LOG(WARNING) << "Both HTTP and Command check passed in";
+    return -1;
+  }
+
+  if (!check.get().has_http() && !check.get().has_command()) {
+    LOG(WARNING) << "No health check found";
+    return -1;
+  }
+
+  if (flags.task_id.isNone()) {
+    LOG(WARNING) << "TaskID error: " << check.error();
+    usage(argv[0], flags);
+    return 0;
+  }
+
+  TaskID taskID;
+  taskID.set_value(flags.task_id.get());
+
+  internal::HealthCheckerProcess process(
+    check.get(),
+    flags.executor.get(),
+    taskID);
+
+  process::spawn(&process);
+
+  process::Future<Nothing> checking =
+    process::dispatch(
+      process, &internal::HealthCheckerProcess::healthCheck);
+
+  checking.await();
+
+  process::terminate(process);
+  process::wait(process);
+
+  if (checking.isFailed()) {
+    LOG(WARNING) << "Health check failed " << checking.failure();
+    return 1;
+  }
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/ff64e4ed/src/launcher/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.cpp b/src/launcher/executor.cpp
index 3d55d93..9f48c88 100644
--- a/src/launcher/executor.cpp
+++ b/src/launcher/executor.cpp
@@ -30,21 +30,30 @@
 #include <process/defer.hpp>
 #include <process/delay.hpp>
 #include <process/future.hpp>
+#include <process/io.hpp>
 #include <process/process.hpp>
+#include <process/protobuf.hpp>
+#include <process/subprocess.hpp>
 #include <process/reap.hpp>
 #include <process/timer.hpp>
 
 #include <stout/duration.hpp>
 #include <stout/flags.hpp>
+#include <stout/path.hpp>
+#include <stout/protobuf.hpp>
 #include <stout/lambda.hpp>
 #include <stout/option.hpp>
 #include <stout/os.hpp>
 #include <stout/strings.hpp>
 
+#include "common/http.hpp"
 #include "common/type_utils.hpp"
+#include "common/status_utils.hpp"
 
 #include "logging/logging.hpp"
 
+#include "messages/messages.hpp"
+
 #include "slave/constants.hpp"
 
 using process::wait; // Necessary on some OS's to disambiguate.
@@ -59,26 +68,32 @@ namespace internal {
 
 using namespace process;
 
+using namespace mesos::internal::status;
+
 
-class CommandExecutorProcess : public Process<CommandExecutorProcess>
+class CommandExecutorProcess : public ProtobufProcess<CommandExecutorProcess>
 {
 public:
-  CommandExecutorProcess(Option<char**> override)
+  CommandExecutorProcess(Option<char**> override, const string& _healthCheckDir)
     : launched(false),
       killed(false),
       pid(-1),
+      healthPid(-1),
       escalationTimeout(slave::EXECUTOR_SIGNAL_ESCALATION_TIMEOUT),
+      driver(None()),
+      healthCheckDir(_healthCheckDir),
       override(override) {}
 
   virtual ~CommandExecutorProcess() {}
 
   void registered(
-      ExecutorDriver* driver,
+      ExecutorDriver* _driver,
       const ExecutorInfo& executorInfo,
       const FrameworkInfo& frameworkInfo,
       const SlaveInfo& slaveInfo)
   {
     cout << "Registered executor on " << slaveInfo.hostname() << endl;
+    driver = _driver;
   }
 
   void reregistered(
@@ -106,7 +121,7 @@ public:
     CHECK(task.has_command()) << "Expecting task " << task.task_id()
                               << " to have a command!";
 
-    std::cout << "Starting task " << task.task_id() << std::endl;
+    cout << "Starting task " << task.task_id() << endl;
 
     // TODO(benh): Clean this up with the new 'Fork' abstraction.
     // Use pipes to determine which child has successfully changed
@@ -121,21 +136,19 @@ public:
     // Set the FD_CLOEXEC flags on these pipes
     Try<Nothing> cloexec = os::cloexec(pipes[0]);
     if (cloexec.isError()) {
-      std::cerr << "Failed to cloexec(pipe[0]): " << cloexec.error()
-                << std::endl;
+      cerr << "Failed to cloexec(pipe[0]): " << cloexec.error() << endl;
       abort();
     }
 
     cloexec = os::cloexec(pipes[1]);
     if (cloexec.isError()) {
-      std::cerr << "Failed to cloexec(pipe[1]): " << cloexec.error()
-                << std::endl;
+      cerr << "Failed to cloexec(pipe[1]): " << cloexec.error() << endl;
       abort();
     }
 
     if ((pid = fork()) == -1) {
-      std::cerr << "Failed to fork to run '" << task.command().value() << "': "
-                << strerror(errno) << std::endl;
+      cerr << "Failed to fork to run '" << task.command().value() << "': "
+           << strerror(errno) << endl;
       abort();
     }
 
@@ -149,7 +162,7 @@ public:
       while ((pid = setsid()) == -1) {
         perror("Could not put command in its own session, setsid");
 
-        std::cout << "Forking another process and retrying" << std::endl;
+        cout << "Forking another process and retrying" << endl;
 
         if ((pid = fork()) == -1) {
           perror("Failed to fork to launch command");
@@ -173,7 +186,7 @@ public:
       // The child has successfully setsid, now run the command.
 
       if (override.isNone()) {
-        std::cout << "sh -c '" << task.command().value() << "'" << std::endl;
+        cout << "sh -c '" << task.command().value() << "'" << endl;
         execl("/bin/sh", "sh", "-c",
               task.command().value().c_str(), (char*) NULL);
       } else {
@@ -182,9 +195,9 @@ public:
         // argv is guaranteed to be NULL terminated and we rely on
         // that fact to print command to be executed.
         for (int i = 0; argv[i] != NULL; i++) {
-          std::cout << argv[i] << " ";
+          cout << argv[i] << " ";
         }
-        std::cout << std::endl;
+        cout << endl;
 
         execvp(argv[0], argv);
       }
@@ -198,14 +211,16 @@ public:
 
     // Get the child's pid via the pipe.
     if (read(pipes[0], &pid, sizeof(pid)) == -1) {
-      std::cerr << "Failed to get child PID from pipe, read: "
-                << strerror(errno) << std::endl;
+      cerr << "Failed to get child PID from pipe, read: " << strerror(errno)
+           << endl;
       abort();
     }
 
     os::close(pipes[0]);
 
-    std::cout << "Forked command at " << pid << std::endl;
+    cout << "Forked command at " << pid << endl;
+
+    launchHealthCheck(task);
 
     // Monitor this process.
     process::reap(pid)
@@ -227,31 +242,35 @@ public:
   void killTask(ExecutorDriver* driver, const TaskID& taskId)
   {
     shutdown(driver);
+    if (healthPid != -1) {
+      // Cleanup health check process
+      ::kill(healthPid, SIGKILL);
+    }
   }
 
   void frameworkMessage(ExecutorDriver* driver, const string& data) {}
 
   void shutdown(ExecutorDriver* driver)
   {
-    std::cout << "Shutting down" << std::endl;
+    cout << "Shutting down" << endl;
 
     if (pid > 0 && !killed) {
-      std::cout << "Sending SIGTERM to process tree at pid "
-                << pid << std::endl;
+      cout << "Sending SIGTERM to process tree at pid "
+           << pid << endl;
 
       Try<std::list<os::ProcessTree> > trees =
         os::killtree(pid, SIGTERM, true, true);
 
       if (trees.isError()) {
-        std::cerr << "Failed to kill the process tree rooted at pid "
-                  << pid << ": " << trees.error() << std::endl;
+        cerr << "Failed to kill the process tree rooted at pid "
+             << pid << ": " << trees.error() << endl;
 
         // Send SIGTERM directly to process 'pid' as it may not have
         // received signal before os::killtree() failed.
         ::kill(pid, SIGTERM);
       } else {
-        std::cout << "Killing the following process trees:\n"
-                  << stringify(trees.get()) << std::endl;
+        cout << "Killing the following process trees:\n"
+             << stringify(trees.get()) << endl;
       }
 
       // TODO(nnielsen): Make escalationTimeout configurable through
@@ -267,6 +286,40 @@ public:
 
   virtual void error(ExecutorDriver* driver, const string& message) {}
 
+protected:
+  virtual void initialize()
+  {
+    install<TaskHealthStatus>(
+        &CommandExecutorProcess::taskHealthUpdated,
+        &TaskHealthStatus::task_id,
+        &TaskHealthStatus::healthy,
+        &TaskHealthStatus::kill_task);
+  }
+
+  void taskHealthUpdated(
+      const TaskID& taskID,
+      const bool& healthy,
+      const bool& initiateTaskKill)
+  {
+    if (driver.isNone()) {
+      return;
+    }
+
+    cout << "Received task health update, healthy: "
+         << stringify(healthy) << endl;
+
+    TaskStatus status;
+    status.mutable_task_id()->CopyFrom(taskID);
+    status.set_healthy(healthy);
+    status.set_state(TASK_RUNNING);
+    driver.get()->sendStatusUpdate(status);
+
+    if (initiateTaskKill) {
+      killTask(driver.get(), taskID);
+    }
+  }
+
+
 private:
   void reaped(
       ExecutorDriver* driver,
@@ -301,13 +354,7 @@ private:
         state = TASK_FAILED;
       }
 
-      message = string("Command") +
-          (WIFEXITED(status)
-          ? " exited with status "
-          : " terminated with signal ") +
-          (WIFEXITED(status)
-          ? stringify(WEXITSTATUS(status))
-          : strsignal(WTERMSIG(status)));
+      message = "Command " + WSTRINGIFY(status);
     }
 
     cout << message << " (pid: " << pid << ")" << endl;
@@ -327,9 +374,9 @@ private:
 
   void escalated()
   {
-    std::cout << "Process " << pid << " did not terminate after "
-              << escalationTimeout << ", sending SIGKILL to "
-              << "process tree at " << pid << std::endl;
+    cout << "Process " << pid << " did not terminate after "
+         << escalationTimeout << ", sending SIGKILL to "
+         << "process tree at " << pid << endl;
 
     // TODO(nnielsen): Sending SIGTERM in the first stage of the
     // shutdown may leave orphan processes hanging off init. This
@@ -339,24 +386,59 @@ private:
       os::killtree(pid, SIGKILL, true, true);
 
     if (trees.isError()) {
-      std::cerr << "Failed to kill the process tree rooted at pid "
-                << pid << ": " << trees.error() << std::endl;
+      cerr << "Failed to kill the process tree rooted at pid "
+           << pid << ": " << trees.error() << endl;
 
       // Process 'pid' may not have received signal before
       // os::killtree() failed. To make sure process 'pid' is reaped
       // we send SIGKILL directly.
       ::kill(pid, SIGKILL);
     } else {
-      std::cout << "Killed the following process trees:\n"
-                << stringify(trees.get()) << std::endl;
+      cout << "Killed the following process trees:\n" << stringify(trees.get())
+           << endl;
+    }
+  }
+
+  void launchHealthCheck(const TaskInfo& task)
+  {
+    if (task.has_health_check()) {
+      const HealthCheck& healthCheck = task.health_check();
+      JSON::Object json = JSON::Protobuf(healthCheck);
+      // TODO(tnachen): Use flags when subprocess handle arguments
+      // with quotes.
+      const string& healthCommand =
+        path::join(healthCheckDir, "mesos-health-check") + " --executor=\"" +
+         stringify(self()) + "\" --health_check_json='" + stringify(json) +
+         "' --task_id=" + task.task_id().value();
+      cout << "Launching health check process: " << healthCommand << endl;
+      Try<Subprocess> healthProcess =
+        process::subprocess(
+          healthCommand,
+          Subprocess::PIPE(),
+          Subprocess::FD(STDOUT_FILENO),
+          Subprocess::FD(STDERR_FILENO),
+          None(),
+          None(),
+          None());
+      if (healthProcess.isError()) {
+        cerr << "Unable to launch health process: " << healthProcess.error();
+      } else {
+        const Subprocess& health = healthProcess.get();
+        healthPid = health.pid();
+        cout << "Health check process launched at pid: " << stringify(healthPid)
+             << endl;
+      }
     }
   }
 
   bool launched;
   bool killed;
   pid_t pid;
+  pid_t healthPid;
   Duration escalationTimeout;
   Timer escalationTimer;
+  Option<ExecutorDriver*> driver;
+  string healthCheckDir;
   Option<char**> override;
 };
 
@@ -364,9 +446,9 @@ private:
 class CommandExecutor: public Executor
 {
 public:
-  CommandExecutor(Option<char**> override)
+  CommandExecutor(Option<char**> override, string healthCheckDir)
   {
-    process = new CommandExecutorProcess(override);
+    process = new CommandExecutorProcess(override, healthCheckDir);
     spawn(process);
   }
 
@@ -505,7 +587,7 @@ int main(int argc, char** argv)
     }
   }
 
-  mesos::internal::CommandExecutor executor(override);
+  mesos::internal::CommandExecutor executor(override, dirname(argv[0]));
   mesos::MesosExecutorDriver driver(&executor);
   return driver.run() == mesos::DRIVER_STOPPED ? 0 : 1;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/ff64e4ed/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index bde0e57..21b07c7 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -57,6 +57,7 @@
 #include "common/build.hpp"
 #include "common/date_utils.hpp"
 #include "common/protobuf_utils.hpp"
+#include "common/status_utils.hpp"
 
 #include "credentials/credentials.hpp"
 
@@ -91,6 +92,8 @@ using process::metrics::Counter;
 
 using memory::shared_ptr;
 
+using namespace mesos::internal::status;
+
 namespace mesos {
 namespace internal {
 namespace master {
@@ -3179,11 +3182,8 @@ void Master::exitedExecutor(
 
     LOG(INFO) << "Executor " << executorId
               << " of framework " << frameworkId
-              << " on slave " << *slave
-              << (WIFEXITED(status) ? " has exited with status "
-                                     : " has terminated with signal ")
-              << (WIFEXITED(status) ? stringify(WEXITSTATUS(status))
-                                     : strsignal(WTERMSIG(status)));
+              << " on slave " << *slave << " "
+              << WSTRINGIFY(status);
 
     allocator->resourcesRecovered(
         frameworkId, slaveId, Resources(executor.resources()));

http://git-wip-us.apache.org/repos/asf/mesos/blob/ff64e4ed/src/messages/messages.proto
----------------------------------------------------------------------
diff --git a/src/messages/messages.proto b/src/messages/messages.proto
index 8aecc8b..0d63dc1 100644
--- a/src/messages/messages.proto
+++ b/src/messages/messages.proto
@@ -390,3 +390,21 @@ message Archive {
   repeated Framework frameworks = 1;
 }
 
+// Message describing task current health status that is sent by
+// the task health checker to the command executor.
+// The command executor reports the task status back to the
+// on each receive. If the health checker configured faiure
+// condition meets, then kill_task flag will be set to true which
+// the executor on message receive will kill the task.
+message TaskHealthStatus {
+  required TaskID task_id = 1;
+
+  required bool healthy = 2;
+
+  // Flag to initiate task kill.
+  optional bool kill_task = 3 [default = false];
+
+  // Number of consecutive counts in current status.
+  // This will not be populated if task is healthy.
+  optional int32 consecutive_failures = 4;
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/ff64e4ed/src/slave/containerizer/external_containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/external_containerizer.cpp b/src/slave/containerizer/external_containerizer.cpp
index 96c434b..bb3e5cc 100644
--- a/src/slave/containerizer/external_containerizer.cpp
+++ b/src/slave/containerizer/external_containerizer.cpp
@@ -43,6 +43,7 @@
 #include <stout/uuid.hpp>
 
 #include "common/type_utils.hpp"
+#include "common/status_utils.hpp"
 
 #include "slave/paths.hpp"
 
@@ -62,6 +63,9 @@ using tuples::tuple;
 
 using namespace process;
 
+using namespace mesos::internal::status;
+
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -87,14 +91,8 @@ static Option<Error> validate(
 
   // The status is a waitpid-result which has to be checked for SIGNAL
   // based termination before masking out the exit-code.
-  if (!WIFEXITED(status.get())) {
-    return Error(string("External containerizer terminated by signal ") +
-                 strsignal(WTERMSIG(status.get())));
-  }
-
-  if (WEXITSTATUS(status.get()) != 0) {
-    return Error("External containerizer failed (status: " +
-                 stringify(WEXITSTATUS(status.get())) + ")");
+  if (!WIFEXITED(status.get()) || WEXITSTATUS(status.get()) != 0) {
+    return Error("Externel containerizer " + WSTRINGIFY(status.get()));
   }
 
   return None();

http://git-wip-us.apache.org/repos/asf/mesos/blob/ff64e4ed/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index ed3483f..91a58b9 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -64,6 +64,7 @@
 #include "common/build.hpp"
 #include "common/protobuf_utils.hpp"
 #include "common/type_utils.hpp"
+#include "common/status_utils.hpp"
 
 #include "credentials/credentials.hpp"
 
@@ -98,6 +99,8 @@ namespace slave {
 
 using namespace state;
 
+using namespace mesos::internal::status;
+
 Slave::Slave(const slave::Flags& _flags,
              MasterDetector* _detector,
              Containerizer* _containerizer,
@@ -2505,13 +2508,8 @@ void Slave::executorTerminated(
   } else {
     status = termination.get().status();
     LOG(INFO) << "Executor '" << executorId
-              << "' of framework " << frameworkId
-              << (WIFEXITED(status)
-                  ? " has exited with status "
-                  : " has terminated with signal ")
-              << (WIFEXITED(status)
-                  ? stringify(WEXITSTATUS(status))
-                  : strsignal(WTERMSIG(status)));
+              << "' of framework " << frameworkId << " "
+              << WSTRINGIFY(status);
   }
 
   Framework* framework = getFramework(frameworkId);

http://git-wip-us.apache.org/repos/asf/mesos/blob/ff64e4ed/src/tests/health_check_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/health_check_tests.cpp b/src/tests/health_check_tests.cpp
new file mode 100644
index 0000000..bf51229
--- /dev/null
+++ b/src/tests/health_check_tests.cpp
@@ -0,0 +1,382 @@
+/**
+ * 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.
+ */
+
+#include <mesos/executor.hpp>
+#include <mesos/scheduler.hpp>
+
+#include <process/clock.hpp>
+#include <process/future.hpp>
+#include <process/pid.hpp>
+
+#include "slave/slave.hpp"
+
+#include "tests/containerizer.hpp"
+#include "tests/mesos.hpp"
+#include "tests/utils.hpp"
+
+using namespace mesos;
+using namespace mesos::internal;
+using namespace mesos::internal::tests;
+
+using mesos::internal::master::Master;
+
+using mesos::internal::slave::Slave;
+using mesos::internal::slave::Containerizer;
+using mesos::internal::slave::MesosContainerizer;
+using mesos::internal::slave::MesosContainerizerProcess;
+
+using process::Clock;
+using process::Future;
+using process::PID;
+
+using testing::_;
+using testing::AtMost;
+using testing::Eq;
+using testing::Return;
+
+using std::vector;
+using std::queue;
+using std::string;
+using std::map;
+
+class HealthCheckTest : public MesosTest
+{
+public:
+  vector<TaskInfo> populateTasks(
+    const string& cmd,
+    const string& healthCmd,
+    const Offer& offer,
+    const int gracePeriodSeconds = 0,
+    const Option<int>& consecutiveFailures = None(),
+    const Option<map<string, string> >& env = None())
+  {
+    TaskInfo task;
+    task.set_name("");
+    task.mutable_task_id()->set_value("1");
+    task.mutable_slave_id()->CopyFrom(offer.slave_id());
+    task.mutable_resources()->CopyFrom(offer.resources());
+
+    CommandInfo command;
+    command.set_value(cmd);
+
+    task.mutable_command()->CopyFrom(command);
+
+    HealthCheck healthCheck;
+
+    CommandInfo healthCommand;
+    healthCommand.set_value(healthCmd);
+
+    if (env.isSome()) {
+      foreachpair (const string& name, const string variable, env.get()) {
+        Environment_Variable* var =
+          healthCommand.mutable_environment()->mutable_variables()->Add();
+        var->set_name(name);
+        var->set_value(variable);
+      }
+    }
+
+    healthCheck.mutable_command()->CopyFrom(healthCommand);
+    healthCheck.set_delay_seconds(0);
+    healthCheck.set_interval_seconds(0);
+    healthCheck.set_grace_period_seconds(gracePeriodSeconds);
+
+    if (consecutiveFailures.isSome()) {
+      healthCheck.set_consecutive_failures(consecutiveFailures.get());
+    }
+
+    task.mutable_health_check()->CopyFrom(healthCheck);
+
+    vector<TaskInfo> tasks;
+    tasks.push_back(task);
+
+    return tasks;
+  }
+};
+
+
+// Testing a healthy task reporting one healthy status to scheduler.
+TEST_F(HealthCheckTest, HealthyTask)
+{
+  Try<PID<Master> > master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "posix/cpu,posix/mem";
+
+  Try<MesosContainerizer*> containerizer =
+    MesosContainerizer::create(flags, false);
+  CHECK_SOME(containerizer);
+
+  Try<PID<Slave> > slave = StartSlave(containerizer.get());
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+    &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .Times(1);
+
+  Future<vector<Offer> > offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  vector<TaskInfo> tasks = populateTasks("sleep 20", "exit 0", offers.get()[0]);
+
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusHealth;
+
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&statusHealth));
+
+  driver.launchTasks(offers.get()[0].id(), tasks);
+
+  AWAIT_READY(statusRunning);
+  EXPECT_EQ(TASK_RUNNING, statusRunning.get().state());
+
+  AWAIT_READY(statusHealth);
+  EXPECT_EQ(TASK_RUNNING, statusHealth.get().state());
+  EXPECT_TRUE(statusHealth.get().healthy());
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
+
+
+// Testing killing task after number of consecutive failures.
+TEST_F(HealthCheckTest, ConsecutiveFailures)
+{
+  Try<PID<Master> > master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "posix/cpu,posix/mem";
+
+  Try<MesosContainerizer*> containerizer =
+  MesosContainerizer::create(flags, false);
+  CHECK_SOME(containerizer);
+
+  Try<PID<Slave> > slave = StartSlave(containerizer.get());
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+    &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _))
+  .Times(1);
+
+  Future<vector<Offer> > offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  vector<TaskInfo> tasks = populateTasks(
+    "sleep 20", "exit 1", offers.get()[0], 0, 4);
+
+  // Expecting four unhealthy updates and one final kill update.
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> status1;
+  Future<TaskStatus> status2;
+  Future<TaskStatus> status3;
+  Future<TaskStatus> status4;
+  Future<TaskStatus> statusKilled;
+
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&status1))
+    .WillOnce(FutureArg<1>(&status2))
+    .WillOnce(FutureArg<1>(&status3))
+    .WillOnce(FutureArg<1>(&status4))
+    .WillOnce(FutureArg<1>(&statusKilled));
+
+  driver.launchTasks(offers.get()[0].id(), tasks);
+
+  AWAIT_READY(statusRunning);
+  EXPECT_EQ(TASK_RUNNING, statusRunning.get().state());
+
+  AWAIT_READY(status1);
+  EXPECT_EQ(TASK_RUNNING, status1.get().state());
+  EXPECT_FALSE(status1.get().healthy());
+
+  AWAIT_READY(status2);
+  EXPECT_EQ(TASK_RUNNING, status2.get().state());
+  EXPECT_FALSE(status2.get().healthy());
+
+  AWAIT_READY(status3);
+  EXPECT_EQ(TASK_RUNNING, status3.get().state());
+  EXPECT_FALSE(status3.get().healthy());
+
+  AWAIT_READY(status4);
+  EXPECT_EQ(TASK_RUNNING, status4.get().state());
+  EXPECT_FALSE(status4.get().healthy());
+
+  AWAIT_READY(statusKilled);
+  EXPECT_EQ(TASK_KILLED, statusKilled.get().state());
+  EXPECT_FALSE(statusKilled.get().healthy());
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
+
+
+// Testing command using environment variable.
+TEST_F(HealthCheckTest, EnvironmentSetup)
+{
+  Try<PID<Master> > master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "posix/cpu,posix/mem";
+
+  Try<MesosContainerizer*> containerizer =
+    MesosContainerizer::create(flags, false);
+
+  CHECK_SOME(containerizer);
+
+  Try<PID<Slave> > slave = StartSlave(containerizer.get());
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+    &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .Times(1);
+
+  Future<vector<Offer> > offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  map<string, string> env;
+  env["STATUS"] = "0";
+
+  vector<TaskInfo> tasks = populateTasks(
+    "sleep 20", "exit $STATUS", offers.get()[0], 0, None(), env);
+
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusHealth;
+
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+  .WillOnce(FutureArg<1>(&statusRunning))
+  .WillOnce(FutureArg<1>(&statusHealth));
+
+  driver.launchTasks(offers.get()[0].id(), tasks);
+
+  AWAIT_READY(statusRunning);
+  EXPECT_EQ(TASK_RUNNING, statusRunning.get().state());
+
+  AWAIT_READY(statusHealth);
+  EXPECT_EQ(TASK_RUNNING, statusHealth.get().state());
+  EXPECT_TRUE(statusHealth.get().healthy());
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
+
+
+// Testing grace period that ignores all failed task failures.
+TEST_F(HealthCheckTest, GracePeriod)
+{
+  Try<PID<Master> > master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "posix/cpu,posix/mem";
+
+  Try<MesosContainerizer*> containerizer =
+  MesosContainerizer::create(flags, false);
+  CHECK_SOME(containerizer);
+
+  Try<PID<Slave> > slave = StartSlave(containerizer.get());
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+    &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .Times(1);
+
+  Future<vector<Offer> > offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  vector<TaskInfo> tasks = populateTasks(
+    "sleep 20", "exit 1", offers.get()[0], 6);
+
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusHealth;
+
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&statusHealth))
+    .WillRepeatedly(Return());
+
+  driver.launchTasks(offers.get()[0].id(), tasks);
+
+  Clock::pause();
+  EXPECT_TRUE(statusHealth.isPending());
+
+  // No task unhealthy update should be called in grace period.
+  Clock::advance(Seconds(5));
+  EXPECT_TRUE(statusHealth.isPending());
+
+  Clock::advance(Seconds(1));
+  Clock::settle();
+  Clock::resume();
+
+  AWAIT_READY(statusHealth);
+  EXPECT_EQ(TASK_RUNNING, statusHealth.get().state());
+  EXPECT_FALSE(statusHealth.get().healthy());
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/ff64e4ed/src/tests/script.cpp
----------------------------------------------------------------------
diff --git a/src/tests/script.cpp b/src/tests/script.cpp
index d57fc7d..15a6542 100644
--- a/src/tests/script.cpp
+++ b/src/tests/script.cpp
@@ -31,6 +31,8 @@
 #include <stout/stringify.hpp>
 #include <stout/strings.hpp>
 
+#include "common/status_utils.hpp"
+
 #include "mesos/mesos.hpp"
 
 #include "tests/environment.hpp"
@@ -40,6 +42,8 @@
 
 using std::string;
 
+using namespace mesos::internal::status;
+
 namespace mesos {
 namespace internal {
 namespace tests {
@@ -77,13 +81,8 @@ void execute(const string& script)
     while (wait(&status) != pid || WIFSTOPPED(status));
     CHECK(WIFEXITED(status) || WIFSIGNALED(status));
 
-    if (WIFEXITED(status)) {
-      if (WEXITSTATUS(status) != 0) {
-        FAIL() << script << " exited with status " << WEXITSTATUS(status);
-      }
-    } else {
-      FAIL() << script << " terminated with signal '"
-             << strsignal(WTERMSIG(status)) << "'";
+    if (!WIFEXITED(status) || WEXITSTATUS(status) != 0) {
+      FAIL() << script << " " << WSTRINGIFY(status);
     }
   } else {
     // In child process. DO NOT USE GLOG!