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/27 02:04:21 UTC

[2/2] git commit: Updated the Reaper to use os::process instead of os::alive.

Updated the Reaper to use os::process instead of os::alive.

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


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

Branch: refs/heads/master
Commit: 60ce48818dc98c306f76438392112922a6812d69
Parents: 8ac7ecc
Author: Benjamin Mahler <bm...@twitter.com>
Authored: Wed Jun 26 17:03:43 2013 -0700
Committer: Benjamin Mahler <bm...@twitter.com>
Committed: Wed Jun 26 17:03:43 2013 -0700

----------------------------------------------------------------------
 src/linux/cgroups.cpp          |  9 ++---
 src/slave/process_isolator.cpp |  7 ++--
 src/slave/reaper.cpp           | 68 +++++++++++++------------------------
 src/slave/reaper.hpp           |  8 -----
 src/tests/reaper_tests.cpp     |  4 +--
 5 files changed, 34 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/60ce4881/src/linux/cgroups.cpp
----------------------------------------------------------------------
diff --git a/src/linux/cgroups.cpp b/src/linux/cgroups.cpp
index 5d63bde..5207ded 100644
--- a/src/linux/cgroups.cpp
+++ b/src/linux/cgroups.cpp
@@ -1250,11 +1250,12 @@ private:
       // is in the FREEZING state. We ignore such processes.
       // See: https://issues.apache.org/jira/browse/MESOS-461
       foreach (pid_t pid, pids.get()) {
-        Try<proc::ProcessStatus> status = proc::status(pid);
+        Result<proc::ProcessStatus> status = proc::status(pid);
 
-        if (status.isError()) {
-          LOG(WARNING) << "Failed to get process status for pid " << pid
-                       << ": " << status.error();
+        if (!status.isSome()) {
+          LOG(WARNING)
+            << "Failed to get process status for pid " << pid << ": "
+            << (status.isError() ? status.error() : "pid does not exist");
           continue;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/60ce4881/src/slave/process_isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/process_isolator.cpp b/src/slave/process_isolator.cpp
index b750b26..86616e4 100644
--- a/src/slave/process_isolator.cpp
+++ b/src/slave/process_isolator.cpp
@@ -380,10 +380,11 @@ Future<ResourceStatistics> ProcessIsolator::usage(
 
   CHECK_SOME(info->pid);
 
-  Try<os::Process> process = os::process(info->pid.get());
+  Result<os::Process> process = os::process(info->pid.get());
 
-  if (process.isError()) {
-    return Future<ResourceStatistics>::failed(process.error());
+  if (!process.isSome()) {
+    return Future<ResourceStatistics>::failed(
+        process.isError() ? process.error() : "Process does not exist");
   }
 
   result.set_timestamp(Clock::now().secs());

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/60ce4881/src/slave/reaper.cpp
----------------------------------------------------------------------
diff --git a/src/slave/reaper.cpp b/src/slave/reaper.cpp
index 8d3101d..4ce9413 100644
--- a/src/slave/reaper.cpp
+++ b/src/slave/reaper.cpp
@@ -47,45 +47,21 @@ ReaperProcess::ReaperProcess()
 
 Future<Option<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.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<Option<int> > > promise(new Promise<Option<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 None();
-    }
-  }
+  // Check to see if this pid exists.
+  const Result<os::Process>& process = os::process(pid);
 
-  // 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 Option<int>(status);
-  } else if (result == 0) {
-    // Child still active, add to the map.
+  if (process.isSome()) {
+    // The process exists, we add it to the promises map.
     Owned<Promise<Option<int> > > promise(new Promise<Option<int> >());
     promises.put(pid, promise);
     return promise->future();
+  } else if (process.isNone()) {
+    LOG(WARNING) << "Cannot monitor process " << pid
+                 << " because it doesn't exist";
+    return None();
   } else {
-    // Not a child nor do we have permission to for os::alive();
-    // we cannot monitor this pid.
     return Future<Option<int> >::failed(
-        "Failed to monitor process " + stringify(pid) + ": " + strerror(errno));
+        "Failed to monitor process " + stringify(pid) + ": " + process.error());
   }
 }
 
@@ -108,9 +84,10 @@ void ReaperProcess::notify(pid_t pid, Option<int> status)
 void ReaperProcess::reap()
 {
   // 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.
+  // 1) children: we can reap their exit status when they are
+  //    terminated.
+  // 2) non-children: we cannot reap their exit status.
+  // 3) nonexistent: already reaped elsewhere.
 
   // Reap all child processes first.
   pid_t pid;
@@ -128,23 +105,24 @@ void ReaperProcess::reap()
 
   // 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.
+  //    while loop, it won't be reaped until the next reap() cycle.
   // 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.)
+  //    e.g. by init, we notify the listeners.
   // 3) If a non-child process terminates and is not yet reaped,
-  //    alive() returns true and no notification is sent.
+  //    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);
+    const Result<os::Process>& process = os::process(pid);
 
-    if (alive.isSome() && !alive.get()) {
+    if (process.isError()) {
+      LOG(ERROR) << "Failed to get process information for " << pid
+                 <<": " << process.error();
+      notify(pid, None());
+    } else if (process.isNone()) {
       // 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);
+                   << " because it no longer exists";
       notify(pid, None());
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/60ce4881/src/slave/reaper.hpp
----------------------------------------------------------------------
diff --git a/src/slave/reaper.hpp b/src/slave/reaper.hpp
index 2d6db03..6eb9a3a 100644
--- a/src/slave/reaper.hpp
+++ b/src/slave/reaper.hpp
@@ -50,14 +50,6 @@ public:
   // 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 'None' is returned.

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/60ce4881/src/tests/reaper_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reaper_tests.cpp b/src/tests/reaper_tests.cpp
index d55ac24..c9818ac 100644
--- a/src/tests/reaper_tests.cpp
+++ b/src/tests/reaper_tests.cpp
@@ -199,11 +199,11 @@ TEST(ReaperTest, TerminatedChildProcess)
 
   Clock::pause();
 
-  ASSERT_SOME(os::alive(pid));
+  ASSERT_SOME(os::process(pid));
 
   // Because reaper reaps all child processes even if they aren't
   // registered, we advance time until that happens.
-  while (os::alive(pid).get()) {
+  while (os::process(pid).isSome()) {
     Clock::advance(Seconds(1));
     Clock::settle();
   }