You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by bm...@apache.org on 2013/06/21 19:55:16 UTC

[3/4] git commit: Changed reaper to use a Future as the notification mechanism. The notification is now sent only if the pid is explicitly registered via the monitor() call.

Changed reaper to use a Future as the notification mechanism.
The notification is now sent only if the pid is explicitly registered
via the monitor() call.

From: Jiang Yan Xu <ya...@jxu.me>
Review: https://reviews.apache.org/r/11554


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

Branch: refs/heads/master
Commit: 4dd692f19973482e145d67fc429ab931d0ecb882
Parents: ad842c3
Author: Benjamin Mahler <bm...@twitter.com>
Authored: Fri Jun 21 10:46:33 2013 -0700
Committer: Benjamin Mahler <bm...@twitter.com>
Committed: Fri Jun 21 10:46:33 2013 -0700

----------------------------------------------------------------------
 src/slave/cgroups_isolator.cpp |  35 ++++++---
 src/slave/cgroups_isolator.hpp |   6 +-
 src/slave/process_isolator.cpp |  36 ++++++---
 src/slave/process_isolator.hpp |   5 +-
 src/slave/reaper.cpp           | 121 ++++++++++++++++++-----------
 src/slave/reaper.hpp           |  72 +++++++++--------
 src/tests/reaper_tests.cpp     | 149 +++++++++++++++++++++++++++++-------
 7 files changed, 291 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/4dd692f1/src/slave/cgroups_isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/cgroups_isolator.cpp b/src/slave/cgroups_isolator.cpp
index 9f5de60..6b87d08 100644
--- a/src/slave/cgroups_isolator.cpp
+++ b/src/slave/cgroups_isolator.cpp
@@ -195,13 +195,7 @@ std::ostream& operator << (std::ostream& out, const Cpuset& cpuset)
 CgroupsIsolator::CgroupsIsolator()
   : ProcessBase(ID::generate("cgroups-isolator")),
     initialized(false),
-    lockFile(None())
-{
-  // Spawn the reaper, note that it might send us a message before we
-  // actually get spawned ourselves, but that's okay, the message will
-  // just get dropped.
-  reaper.addListener(this);
-}
+    lockFile(None()) {}
 
 
 void CgroupsIsolator::initialize(
@@ -540,6 +534,12 @@ void CgroupsIsolator::launchExecutor(
     // Store the pid of the leading process of the executor.
     info->pid = pid;
 
+    reaper.monitor(pid)
+      .onAny(defer(PID<CgroupsIsolator>(this),
+                   &CgroupsIsolator::reaped,
+                   pid,
+                   lambda::_1));
+
     // Tell the slave this executor has started.
     dispatch(slave,
              &Slave::executorStarted,
@@ -769,7 +769,11 @@ Future<Nothing> CgroupsIsolator::recover(
 
         // Add the pid to the reaper to monitor exit status.
         if (run.forkedPid.isSome()) {
-          reaper.monitor(run.forkedPid.get());
+          reaper.monitor(run.forkedPid.get())
+            .onAny(defer(PID<CgroupsIsolator>(this),
+                         &CgroupsIsolator::reaped,
+                         run.forkedPid.get(),
+                         lambda::_1));
         }
       }
     }
@@ -797,8 +801,17 @@ Future<Nothing> CgroupsIsolator::recover(
 }
 
 
-void CgroupsIsolator::processExited(pid_t pid, int status)
+void CgroupsIsolator::reaped(pid_t pid, const Future<int>& status)
 {
+  if (status.isDiscarded()) {
+    LOG(ERROR) << "The status was discarded";
+    return;
+  }
+  if (status.isFailed()) {
+    LOG(ERROR) << status.failure();
+    return;
+  }
+
   CgroupInfo* info = findCgroupInfo(pid);
   if (info != NULL) {
     FrameworkID frameworkId = info->frameworkId;
@@ -806,10 +819,10 @@ void CgroupsIsolator::processExited(pid_t pid, int status)
 
     LOG(INFO) << "Executor " << executorId
               << " of framework " << frameworkId
-              << " terminated with status " << status;
+              << " terminated with status " << status.get();
 
     // Set the exit status, so that '_killExecutor()' can send it to the slave.
-    info->status = status;
+    info->status = status.get();
 
     if (!info->killed) {
       killExecutor(frameworkId, executorId);

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/4dd692f1/src/slave/cgroups_isolator.hpp
----------------------------------------------------------------------
diff --git a/src/slave/cgroups_isolator.hpp b/src/slave/cgroups_isolator.hpp
index df6069e..06bac98 100644
--- a/src/slave/cgroups_isolator.hpp
+++ b/src/slave/cgroups_isolator.hpp
@@ -76,7 +76,7 @@ private:
 };
 
 
-class CgroupsIsolator : public Isolator, public ProcessExitedListener
+class CgroupsIsolator : public Isolator
 {
 public:
   CgroupsIsolator();
@@ -114,13 +114,13 @@ public:
   virtual process::Future<Nothing> recover(
       const Option<state::SlaveState>& state);
 
-  virtual void processExited(pid_t pid, int status);
-
 private:
   // No copying, no assigning.
   CgroupsIsolator(const CgroupsIsolator&);
   CgroupsIsolator& operator = (const CgroupsIsolator&);
 
+  void reaped(pid_t pid, const Future<int>& status);
+
   // The cgroup information for each live executor.
   struct CgroupInfo
   {

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/4dd692f1/src/slave/process_isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/process_isolator.cpp b/src/slave/process_isolator.cpp
index a3eace3..a3d31e5 100644
--- a/src/slave/process_isolator.cpp
+++ b/src/slave/process_isolator.cpp
@@ -25,12 +25,14 @@
 #include <set>
 
 #include <process/clock.hpp>
+#include <process/defer.hpp>
 #include <process/dispatch.hpp>
 #include <process/id.hpp>
 
 #include <stout/check.hpp>
 #include <stout/exit.hpp>
 #include <stout/foreach.hpp>
+#include <stout/lambda.hpp>
 #include <stout/nothing.hpp>
 #include <stout/option.hpp>
 #include <stout/os.hpp>
@@ -48,6 +50,7 @@ using std::map;
 using std::set;
 using std::string;
 
+using process::defer;
 using process::wait; // Necessary on some OS's to disambiguate.
 
 namespace mesos {
@@ -63,13 +66,7 @@ using state::RunState;
 
 ProcessIsolator::ProcessIsolator()
   : ProcessBase(ID::generate("process-isolator")),
-    initialized(false)
-{
-  // Spawn the reaper, note that it might send us a message before we
-  // actually get spawned ourselves, but that's okay, the message will
-  // just get dropped.
-  reaper.addListener(this);
-}
+    initialized(false) {}
 
 
 void ProcessIsolator::initialize(
@@ -166,6 +163,12 @@ void ProcessIsolator::launchExecutor(
     // Record the pid (should also be the pgid since we setsid below).
     infos[frameworkId][executorId]->pid = pid;
 
+    reaper.monitor(pid)
+      .onAny(defer(PID<ProcessIsolator>(this),
+                   &ProcessIsolator::reaped,
+                   pid,
+                   lambda::_1));
+
     // Tell the slave this executor has started.
     dispatch(slave, &Slave::executorStarted, frameworkId, executorId, pid);
   } else {
@@ -334,7 +337,11 @@ Future<Nothing> ProcessIsolator::recover(
 
       // Add the pid to the reaper to monitor exit status.
       if (run.forkedPid.isSome()) {
-        reaper.monitor(run.forkedPid.get());
+        reaper.monitor(run.forkedPid.get())
+          .onAny(defer(PID<ProcessIsolator>(this),
+                       &ProcessIsolator::reaped,
+                       run.forkedPid.get(),
+                       lambda::_1));
       }
     }
   }
@@ -416,8 +423,17 @@ Future<ResourceStatistics> ProcessIsolator::usage(
 }
 
 
-void ProcessIsolator::processExited(pid_t pid, int status)
+void ProcessIsolator::reaped(pid_t pid, const Future<int>& status)
 {
+  if (status.isDiscarded()) {
+    LOG(ERROR) << "The status was discarded";
+    return;
+  }
+  if (status.isFailed()) {
+    LOG(ERROR) << status.failure();
+    return;
+  }
+
   foreachkey (const FrameworkID& frameworkId, infos) {
     foreachkey (const ExecutorID& executorId, infos[frameworkId]) {
       ProcessInfo* info = infos[frameworkId][executorId];
@@ -430,7 +446,7 @@ void ProcessIsolator::processExited(pid_t pid, int status)
                  &Slave::executorTerminated,
                  frameworkId,
                  executorId,
-                 status,
+                 status.get(),
                  false,
                  "Executor terminated");
 

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/4dd692f1/src/slave/process_isolator.hpp
----------------------------------------------------------------------
diff --git a/src/slave/process_isolator.hpp b/src/slave/process_isolator.hpp
index c5abc6b..a3f6c68 100644
--- a/src/slave/process_isolator.hpp
+++ b/src/slave/process_isolator.hpp
@@ -41,7 +41,7 @@ namespace mesos {
 namespace internal {
 namespace slave {
 
-class ProcessIsolator : public Isolator, public ProcessExitedListener
+class ProcessIsolator : public Isolator
 {
 public:
   ProcessIsolator();
@@ -77,7 +77,6 @@ public:
   virtual process::Future<Nothing> recover(
       const Option<state::SlaveState>& state);
 
-  virtual void processExited(pid_t pid, int status);
 
 private:
   // No copying, no assigning.
@@ -109,6 +108,8 @@ private:
   bool initialized;
   Reaper reaper;
   hashmap<FrameworkID, hashmap<ExecutorID, ProcessInfo*> > infos;
+
+  void reaped(pid_t pid, const Future<int>& status);
 };
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/4dd692f1/src/slave/reaper.cpp
----------------------------------------------------------------------
diff --git a/src/slave/reaper.cpp b/src/slave/reaper.cpp
index c2e606e..1b19eb2 100644
--- a/src/slave/reaper.cpp
+++ b/src/slave/reaper.cpp
@@ -45,25 +45,48 @@ ReaperProcess::ReaperProcess()
   : ProcessBase(ID::generate("reaper")) {}
 
 
-void ReaperProcess::addListener(
-    const PID<ProcessExitedListener>& listener)
-{
-  listeners.push_back(listener);
-}
-
-
-Future<Nothing> ReaperProcess::monitor(pid_t pid)
+Future<int> ReaperProcess::monitor(pid_t pid)
 {
   // Check to see if the current process has sufficient privileges to
   // monitor the liveness of this pid.
   Try<bool> alive = os::alive(pid);
-  if (alive.isError()) {
-    return Future<Nothing>::failed("Failed to monitor process " +
-                                   stringify(pid) + ": " + alive.error());
+
+  if (alive.isSome()) {
+    if (alive.get()) {
+      // We have permissions to check the validity of the process
+      // and it's alive, so add it to the promises map.
+      Owned<Promise<int> > promise(new Promise<int>());
+      promises.put(pid, promise);
+      return promise->future();
+    } else {
+      // Process doesn't exist.
+      LOG(WARNING) << "Cannot monitor process " << pid
+                   << " because it doesn't exist";
+      return -1;
+    }
+  }
+
+  // Now we know we don't have permission for alive(), but we can
+  // still monitor it if it is our child.
+  int status;
+  pid_t result = waitpid(pid, &status, WNOHANG);
+
+  if (result > 0) {
+    // The process terminated and the status was reaped.
+    // Notify other listeners and return directly for this caller.
+    notify(pid, status);
+    return status;
+  } else if (result == 0) {
+    // Child still active, add to the map.
+    Owned<Promise<int> > promise(new Promise<int>());
+    promises.put(pid, promise);
+    return promise->future();
   } else {
-    pids.insert(pid);
+    // Not a child nor do we have permission to for os::alive();
+    // we cannot monitor this pid.
+    return Future<int>::failed("Failed to monitor process " +
+                               stringify(pid) + ": " + strerror(errno));
   }
-  return Nothing();
 }
 
 
@@ -75,45 +98,57 @@ void ReaperProcess::initialize()
 
 void ReaperProcess::notify(pid_t pid, int status)
 {
-  foreach (const PID<ProcessExitedListener>& listener, listeners) {
-    dispatch(listener, &ProcessExitedListener::processExited, pid, status);
+  foreach (const Owned<Promise<int> >& promise, promises.get(pid)) {
+    promise->set(status);
   }
+  promises.remove(pid);
 }
 
 
 void ReaperProcess::reap()
 {
-  // Check whether any monitored process has exited.
-  foreach (pid_t pid, utils::copy(pids)) {
-    Try<bool> alive = os::alive(pid);
-    CHECK_SOME(alive);
-
-    if (!alive.get()) { // The process has terminated.
-      // Attempt to reap the status.
-      // If pid is not a child process of the current process, this is a no-op.
-      int status = -1;
-      if (waitpid(pid, &status, WNOHANG) < 0) {
-        LOG(WARNING) << "Cannot get the exit status of process " << pid
-                     << " because it either does not exist or"
-                     << " is not a child of the calling process: "
-                     << strerror(errno);
-      }
-
-      notify(pid, status); // Notify the listeners.
-      pids.erase(pid);
-    }
-  }
+  // This method assumes that the registered PIDs are
+  // 1) children, or
+  // 2) non-children that we have permission to check liveness, or
+  // 3) nonexistent / reaped elsewhere.
 
-  // Check whether any child processes have exited.
+  // Reap all child processes first.
   pid_t pid;
   int status;
   while ((pid = waitpid(-1, &status, WNOHANG)) > 0) {
     // Ignore this if the process has only stopped.
-    if (!WIFSTOPPED(status)) {
-      notify(pid, status); // Notify the listeners.
-      pids.erase(pid);
+    // Notify the "listeners" only if they have requested to monitor
+    // this pid. Otherwise the status is discarded.
+    // This means if a child pid is registered via the monitor() call
+    // after it's reaped, an invalid status (-1) will be returned.
+    if (!WIFSTOPPED(status) && promises.contains(pid)) {
+      notify(pid, status);
+    }
+  }
+
+  // Check whether any monitored process has exited and been reaped.
+  // 1) If a child terminates before the foreach loop but after the
+  //    while loop, it won't be reaped until the next reap() cycle
+  //    and the alive() check below returns true.
+  // 2) If a non-child process terminates and is reaped elsewhere,
+  //    e.g. by init, we notify the listeners. (We know we have
+  //    permission to check its liveness in this case.)
+  // 3) If a non-child process terminates and is not yet reaped,
+  //    alive() returns true and no notification is sent.
+  // 4) If a child terminates before the while loop above, then we've
+  //    already reaped it and have the listeners notified!
+  foreach (pid_t pid, utils::copy(promises.keys())) {
+    Try<bool> alive = os::alive(pid);
+
+    if (alive.isSome() && !alive.get()) {
+      // The process has been reaped.
+      LOG(WARNING) << "Cannot get the exit status of process " << pid
+                   << " because it is not a child of the calling "
+                   << "process: " << strerror(errno);
+      notify(pid, -1);
     }
   }
+
   delay(Seconds(1), self(), &ReaperProcess::reap); // Reap forever!
 }
 
@@ -133,13 +168,7 @@ Reaper::~Reaper()
 }
 
 
-void Reaper::addListener(const process::PID<ProcessExitedListener>& listener)
-{
-  dispatch(process, &ReaperProcess::addListener, listener);
-}
-
-
-Future<Nothing> Reaper::monitor(pid_t pid)
+Future<int> Reaper::monitor(pid_t pid)
 {
   return dispatch(process, &ReaperProcess::monitor, pid);
 }

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/4dd692f1/src/slave/reaper.hpp
----------------------------------------------------------------------
diff --git a/src/slave/reaper.hpp b/src/slave/reaper.hpp
index d599e9c..15ebc0d 100644
--- a/src/slave/reaper.hpp
+++ b/src/slave/reaper.hpp
@@ -22,32 +22,59 @@
 #include <list>
 #include <set>
 
+#include <process/future.hpp>
 #include <process/process.hpp>
 
+#include <stout/multihashmap.hpp>
 #include <stout/nothing.hpp>
+#include <stout/owned.hpp>
 #include <stout/try.hpp>
 
-
 namespace mesos {
 namespace internal {
 namespace slave {
 
-class ProcessExitedListener : public process::Process<ProcessExitedListener>
+// Forward declaration.
+class ReaperProcess;
+
+
+// TODO(vinod): Refactor the Reaper into 2 components:
+// 1) Reaps the status of child processes.
+// 2) Checks the exit status of requested processes.
+class Reaper
 {
 public:
-  virtual void processExited(pid_t pid, int status) = 0;
+  Reaper();
+  virtual ~Reaper();
+
+  // Monitor the given process and notify the caller if it terminates
+  // via a Future of the exit status.
+  //
+  // NOTE: The termination of pid can only be monitored if the
+  // calling process:
+  //   1) has the same real or effective user ID as the real or saved
+  //      set-user-ID of 'pid', or
+  //   2) is run as a privileged user, or
+  //   3) pid is a child of the current process.
+  // Otherwise a failed Future is returned.
+  //
+  // The exit status of 'pid' can only be correctly captured if the
+  // calling process is the parent of 'pid' and the process hasn't
+  // been reaped yet, otherwise -1 is returned.
+  process::Future<int> monitor(pid_t pid);
+
+private:
+  ReaperProcess* process;
 };
 
 
-// Reaper implementation. See comments of the Reaper class.
+// Reaper implementation.
 class ReaperProcess : public process::Process<ReaperProcess>
 {
 public:
   ReaperProcess();
 
-  void addListener(const process::PID<ProcessExitedListener>&);
-
-  process::Future<Nothing> monitor(pid_t pid);
+  process::Future<int> monitor(pid_t pid);
 
 protected:
   virtual void initialize();
@@ -55,35 +82,14 @@ protected:
   void reap();
 
   // TODO(vinod): Make 'status' an option.
+  // The notification is sent only if the pid is explicitly registered
+  // via the monitor() call.
   void notify(pid_t pid, int status);
 
 private:
-  std::list<process::PID<ProcessExitedListener> > listeners;
-  std::set<pid_t> pids;
-};
-
-
-// TODO(vinod): Refactor the Reaper into 2 components:
-// 1) Reaps the status of child processes.
-// 2) Checks the exit status of requested processes.
-// Also, use Futures instead of callbacks to notify process exits.
-class Reaper
-{
-public:
-  Reaper();
-  virtual ~Reaper();
-
-  void addListener(const process::PID<ProcessExitedListener>&);
-
-  // Monitor the given process and notify the listener if it terminates.
-  // NOTE: A notification is only sent if the calling process:
-  // 1) is the parent of 'pid' or
-  // 2) has the same real/effective UID as that of 'pid' or
-  // 3) is run as a privileged user.
-  process::Future<Nothing> monitor(pid_t pid);
-
-private:
-  ReaperProcess* process;
+  // Mapping from the monitored pid to all promises the pid exit
+  // status should be sent to.
+  multihashmap<pid_t, Owned<process::Promise<int> > > promises;
 };
 
 

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/4dd692f1/src/tests/reaper_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reaper_tests.cpp b/src/tests/reaper_tests.cpp
index 6849419..cfda0f4 100644
--- a/src/tests/reaper_tests.cpp
+++ b/src/tests/reaper_tests.cpp
@@ -19,6 +19,8 @@
 #include <signal.h>
 #include <unistd.h>
 
+#include <sys/wait.h>
+
 #include <gtest/gtest.h>
 
 #include <process/clock.hpp>
@@ -26,6 +28,9 @@
 #include <process/gmock.hpp>
 #include <process/gtest.hpp>
 
+#include <stout/exit.hpp>
+#include <stout/os.hpp>
+
 #include "slave/reaper.hpp"
 
 using namespace mesos;
@@ -39,13 +44,6 @@ using testing::_;
 using testing::DoDefault;
 
 
-class MockProcessListener : public ProcessExitedListener
-{
-public:
-  MOCK_METHOD2(processExited, void(pid_t, int));
-};
-
-
 // This test checks that the Reaper can monitor a non-child process.
 TEST(ReaperTest, NonChildProcess)
 {
@@ -93,48 +91,143 @@ TEST(ReaperTest, NonChildProcess)
     }
   }
 
+  // In parent process.
   LOG(INFO) << "Grand child process " << pid;
 
-  MockProcessListener listener;
+  Reaper reaper;
 
-  // Spawn the listener.
-  spawn(listener);
+  Future<Nothing> monitor = FUTURE_DISPATCH(_, &ReaperProcess::monitor);
 
-  // Spawn the reaper.
-  Reaper reaper;
+  // Ask the reaper to monitor the grand child process.
+  Future<int> status = reaper.monitor(pid);
+
+  AWAIT_READY(monitor);
+
+  // Now kill the grand child.
+  // NOTE: We send a SIGKILL here because sometimes the grand child
+  // process seems to be in a hung state and not responding to
+  // SIGTERM/SIGINT.
+  EXPECT_EQ(0, kill(pid, SIGKILL));
+
+  Clock::pause();
+
+  // Now advance time until the reaper reaps the executor.
+  while (status.isPending()) {
+    Clock::advance(Seconds(1));
+    Clock::settle();
+  }
+
+  // Ensure the reaper notifies of the terminated process.
+  AWAIT_READY(status);
+
+  // Status is -1 because pid is not an immediate child.
+  ASSERT_EQ(-1, status.get());
+
+  Clock::resume();
+}
+
+
+// This test checks that the Reaper can monitor a child process with
+// accurate exit status returned.
+TEST(ReaperTest, ChildProcess)
+{
+  ASSERT_TRUE(GTEST_IS_THREADSAFE);
+
+  pid_t pid = fork();
+  ASSERT_NE(-1, pid);
+
+  if (pid == 0) {
+    // In child process. Keep waiting till we get a signal.
+    while (true);
+  }
+
+  // In parent process.
+  LOG(INFO) << "Child process " << pid;
 
-  // Ignore the exit of the child process.
-  EXPECT_CALL(listener, processExited(_,_))
-    .WillRepeatedly(DoDefault());
+  Reaper reaper;
 
-  reaper.addListener(listener.self());
+  Future<Nothing> monitor = FUTURE_DISPATCH(_, &ReaperProcess::monitor);
 
   // Ask the reaper to monitor the grand child process.
-  reaper.monitor(pid);
+  Future<int> status = reaper.monitor(pid);
 
-  // Catch the exit of the grand child process.
-  Future<Nothing> processExited;
-  EXPECT_CALL(listener, processExited(pid, _))
-    .WillOnce(FutureSatisfy(&processExited));
+  AWAIT_READY(monitor);
 
-  // Now kill the grand child.
-  // NOTE: We send a SIGKILL here because sometimes the grand child process
-  // seems to be in a hung state and not responding to SIGTERM/SIGINT.
+  // Now kill the child.
   EXPECT_EQ(0, kill(pid, SIGKILL));
 
   Clock::pause();
 
   // Now advance time until the reaper reaps the executor.
-  while (processExited.isPending()) {
+  while (status.isPending()) {
+    Clock::advance(Seconds(1));
+    Clock::settle();
+  }
+
+  // Ensure the reaper notifies of the terminated process.
+  AWAIT_READY(status);
+
+  // Check if the status is correct.
+  int stat = status.get();
+  ASSERT_TRUE(WIFSIGNALED(stat));
+  ASSERT_EQ(SIGKILL, WTERMSIG(stat));
+
+  Clock::resume();
+}
+
+
+// Check that the Reaper can monitor a child process that exits
+// before monitor() is called on it.
+TEST(ReaperTest, TerminatedChildProcess)
+{
+  ASSERT_TRUE(GTEST_IS_THREADSAFE);
+
+  pid_t pid = fork();
+  ASSERT_NE(-1, pid);
+
+  if (pid == 0) {
+    // In child process. Return directly
+    exit(EXIT_SUCCESS);
+  }
+
+  // In parent process.
+  LOG(INFO) << "Child process " << pid;
+
+  Reaper reaper;
+
+  Clock::pause();
+
+  ASSERT_SOME(os::alive(pid));
+
+  // Because reaper reaps all child processes even if they aren't
+  // registered, we advance time until that happens.
+  while (os::alive(pid).get()) {
+    Clock::advance(Seconds(1));
+    Clock::settle();
+  }
+
+  // Now we request to monitor the child process which is already
+  // reaped.
+
+  Future<Nothing> monitor = FUTURE_DISPATCH(_, &ReaperProcess::monitor);
+
+  // Ask the reaper to monitor the child process.
+  Future<int> status = reaper.monitor(pid);
+
+  AWAIT_READY(monitor);
+
+  // Now advance time until the reaper sends the notification.
+  while (status.isPending()) {
     Clock::advance(Seconds(1));
     Clock::settle();
   }
 
   // Ensure the reaper notifies of the terminated process.
-  AWAIT_READY(processExited);
+  AWAIT_READY(status);
 
-  terminate(listener);
-  wait(listener);
+  // Invalid status is returned because it is reaped before being
+  // monitored.
+  ASSERT_EQ(-1, status.get());
 
   Clock::resume();
 }