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/20 05:30:17 UTC

[1/6] git commit: Added generic process utilities to libstout.

Updated Branches:
  refs/heads/master b3c0a6551 -> 56c20b619


Added generic process utilities to libstout.

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


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

Branch: refs/heads/master
Commit: 376acc3425062e41d8cab1b093fe96e36bf15481
Parents: b3c0a65
Author: Benjamin Mahler <bm...@twitter.com>
Authored: Mon May 13 10:41:13 2013 -0700
Committer: Benjamin Mahler <bm...@twitter.com>
Committed: Wed Jun 19 20:29:26 2013 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/stout/Makefile.am  |   4 +
 .../3rdparty/stout/include/stout/fs.hpp         |   3 +-
 .../3rdparty/stout/include/stout/os.hpp         | 155 +++++++++---------
 .../3rdparty/stout/include/stout/os/linux.hpp   |  63 ++++++++
 .../3rdparty/stout/include/stout/os/ls.hpp      |  66 ++++++++
 .../3rdparty/stout/include/stout/os/osx.hpp     |  93 +++++++++++
 .../3rdparty/stout/include/stout/os/process.hpp |  53 +++++++
 .../3rdparty/stout/include/stout/os/sysctl.hpp  |  56 +++++--
 .../3rdparty/stout/include/stout/proc.hpp       |  96 +++---------
 .../3rdparty/stout/tests/os_tests.cpp           | 156 +++++++++++++++++++
 .../3rdparty/stout/tests/proc_tests.cpp         | 108 -------------
 11 files changed, 578 insertions(+), 275 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/376acc34/3rdparty/libprocess/3rdparty/stout/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/Makefile.am b/3rdparty/libprocess/3rdparty/stout/Makefile.am
index 2b7ee9c..5d9f35f 100644
--- a/3rdparty/libprocess/3rdparty/stout/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/stout/Makefile.am
@@ -34,6 +34,10 @@ EXTRA_DIST =					\
   include/stout/numify.hpp			\
   include/stout/option.hpp			\
   include/stout/os.hpp				\
+  include/stout/os/linux.hpp			\
+  include/stout/os/ls.hpp			\
+  include/stout/os/osx.hpp			\
+  include/stout/os/process.hpp			\
   include/stout/os/sysctl.hpp			\
   include/stout/owned.hpp			\
   include/stout/path.hpp			\

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/376acc34/3rdparty/libprocess/3rdparty/stout/include/stout/fs.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/fs.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/fs.hpp
index c1a05b5..e405f96 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/fs.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/fs.hpp
@@ -12,7 +12,8 @@
 #include "nothing.hpp"
 #include "try.hpp"
 
-// TODO(bmahler): Migrate the appropriate 'os' namespace funtions here.
+// TODO(bmahler): Merge available() and usage() into df() that returns
+// a struct, and move this back into os.hpp.
 namespace fs {
 
 // Returns the total available disk size in bytes.

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/376acc34/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
index 1b3fb47..f2c59ee 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
@@ -41,16 +41,24 @@
 #include <sstream>
 #include <string>
 
-#include "bytes.hpp"
-#include "duration.hpp"
-#include "error.hpp"
-#include "foreach.hpp"
-#include "none.hpp"
-#include "nothing.hpp"
-#include "path.hpp"
-#include "result.hpp"
-#include "strings.hpp"
-#include "try.hpp"
+#include <stout/bytes.hpp>
+#include <stout/duration.hpp>
+#include <stout/error.hpp>
+#include <stout/foreach.hpp>
+#include <stout/none.hpp>
+#include <stout/nothing.hpp>
+#include <stout/path.hpp>
+#include <stout/result.hpp>
+#include <stout/strings.hpp>
+#include <stout/try.hpp>
+
+#ifdef __linux__
+#include <stout/os/linux.hpp>
+#endif
+#include <stout/os/ls.hpp>
+#ifdef __APPLE__
+#include <stout/os/osx.hpp>
+#endif
 
 #ifdef __APPLE__
 // Assigning the result pointer to ret silences an unused var warning.
@@ -694,63 +702,6 @@ inline std::string getcwd()
 }
 
 
-// TODO(bmahler): Wrap with a Try.
-inline std::list<std::string> ls(const std::string& directory)
-{
-  std::list<std::string> result;
-
-  DIR* dir = opendir(directory.c_str());
-
-  if (dir == NULL) {
-    return std::list<std::string>();
-  }
-
-  // Calculate the size for a "directory entry".
-  long name_max = fpathconf(dirfd(dir), _PC_NAME_MAX);
-
-  // If we don't get a valid size, check NAME_MAX, but fall back on
-  // 255 in the worst case ... Danger, Will Robinson!
-  if (name_max == -1) {
-    name_max = (NAME_MAX > 255) ? NAME_MAX : 255;
-  }
-
-  size_t name_end =
-    (size_t) offsetof(dirent, d_name) + name_max + 1;
-
-  size_t size = (name_end > sizeof(dirent)
-    ? name_end
-    : sizeof(dirent));
-
-  dirent* temp = (dirent*) malloc(size);
-
-  if (temp == NULL) {
-    free(temp);
-    closedir(dir);
-    return std::list<std::string>();
-  }
-
-  struct dirent* entry;
-
-  int error;
-
-  while ((error = readdir_r(dir, temp, &entry)) == 0 && entry != NULL) {
-    if (strcmp(entry->d_name, ".") == 0 || strcmp(entry->d_name, "..") == 0) {
-      continue;
-    }
-    result.push_back(entry->d_name);
-  }
-
-  free(temp);
-  closedir(dir);
-
-  if (error != 0) {
-    return std::list<std::string>();
-  }
-
-  return result;
-}
-
-
 // Return the list of file paths that match the given pattern by recursively
 // searching the given directory. A match is successful if the pattern is a
 // substring of the file name.
@@ -971,18 +922,13 @@ inline Try<Bytes> memory()
   return Bytes(info.totalram);
 # endif
 #elif defined __APPLE__
-  const size_t NAME_SIZE = 2;
-  int name[NAME_SIZE];
-  name[0] = CTL_HW;
-  name[1] = HW_MEMSIZE;
-
-  int64_t memory;
-  size_t length = sizeof(memory);
+  const Try<int64_t>& memory =
+    os::sysctl(CTL_HW, HW_MEMSIZE).integer();
 
-  if (sysctl(name, NAME_SIZE, &memory, &length, NULL, 0) < 0) {
-    return ErrnoError("Failed to get sysctl of HW_MEMSIZE");
+  if (memory.isError()) {
+    return Error(memory.error());
   }
-  return Bytes(memory);
+  return Bytes(memory.get());
 #else
   return Error("Cannot determine the size of main memory");
 #endif
@@ -1062,6 +1008,61 @@ inline Try<Release> release()
 }
 
 
+inline Try<std::list<Process> > processes()
+{
+  const Try<std::set<pid_t> >& pids = os::pids();
+
+  if (pids.isError()) {
+    return Error(pids.error());
+  }
+
+  std::list<Process> result;
+  foreach (pid_t pid, pids.get()) {
+    const Try<Process>& process = os::process(pid);
+
+    // Ignore any processes that disappear.
+    if (process.isError()) {
+      continue;
+    }
+
+    result.push_back(process.get());
+  }
+  return result;
+}
+
+
+inline Try<std::set<pid_t> > children(pid_t pid, bool recursive = true)
+{
+  const Try<std::list<Process> >& processes = os::processes();
+
+  if (processes.isError()) {
+    return Error(processes.error());
+  }
+
+  // Perform a breadth first search for descendants.
+  std::set<pid_t> descendants;
+  std::queue<pid_t> parents;
+  parents.push(pid);
+
+  do {
+    pid_t parent = parents.front();
+    parents.pop();
+
+    // Search for children of parent.
+    foreach (const Process& process, processes.get()) {
+      if (process.parent == parent) {
+        // Have we seen this child yet?
+        if (descendants.insert(process.pid).second) {
+          parents.push(process.pid);
+        }
+      }
+    }
+  } while (recursive && !parents.empty());
+
+  return descendants;
+}
+
+
 inline Try<bool> alive(pid_t pid)
 {
   CHECK(pid > 0);

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/376acc34/3rdparty/libprocess/3rdparty/stout/include/stout/os/linux.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/linux.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/linux.hpp
new file mode 100644
index 0000000..6fb9256
--- /dev/null
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/linux.hpp
@@ -0,0 +1,63 @@
+#ifndef __STOUT_OS_LINUX_HPP__
+#define __STOUT_OS_LINUX_HPP__
+
+// This file contains Linux-only OS utilities.
+#ifndef __linux__
+#error "stout/os/linux.hpp is only available on Linux systems."
+#endif
+
+#include <sys/types.h> // For pid_t.
+
+#include <list>
+#include <queue>
+#include <set>
+
+#include <stout/error.hpp>
+#include <stout/foreach.hpp>
+#include <stout/proc.hpp>
+#include <stout/try.hpp>
+
+#include <stout/os/process.hpp>
+
+namespace os {
+
+inline Try<Process> process(pid_t pid)
+{
+  // Page size, used for memory accounting.
+  // NOTE: This is more portable than using getpagesize().
+  static const long pageSize = sysconf(_SC_PAGESIZE);
+  if (pageSize <= 0) {
+    return Error("Failed to get sysconf(_SC_PAGESIZE)");
+  }
+
+  // Number of clock ticks per second, used for cpu accounting.
+  static const long ticks = sysconf(_SC_CLK_TCK);
+  if (ticks <= 0) {
+    return Error("Failed to get sysconf(_SC_CLK_TCK)");
+  }
+
+  const Try<proc::ProcessStatus>& status = proc::status(pid);
+
+  if (status.isError()) {
+    return Error(status.error());
+  }
+
+  return Process(status.get().pid,
+                 status.get().ppid,
+                 status.get().pgrp,
+                 status.get().session,
+                 Bytes(status.get().rss * pageSize),
+                 Duration::create(status.get().utime / (double) ticks).get(),
+                 Duration::create(status.get().stime / (double) ticks).get(),
+                 status.get().comm);
+}
+
+
+inline Try<std::set<pid_t> > pids()
+{
+  return proc::pids();
+}
+
+} // namespace os {
+
+#endif // __STOUT_OS_LINUX_HPP__

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/376acc34/3rdparty/libprocess/3rdparty/stout/include/stout/os/ls.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/ls.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/ls.hpp
new file mode 100644
index 0000000..7637a0d
--- /dev/null
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/ls.hpp
@@ -0,0 +1,66 @@
+#ifndef __STOUT_OS_LS_HPP__
+#define __STOUT_OS_LS_HPP__
+
+#include <dirent.h>
+#include <stdlib.h>
+#include <unistd.h>
+
+#include <list>
+#include <string>
+
+namespace os {
+
+// TODO(bmahler): Wrap this with a Try.
+inline std::list<std::string> ls(const std::string& directory)
+{
+  DIR* dir = opendir(directory.c_str());
+
+  if (dir == NULL) {
+    return std::list<std::string>();
+  }
+
+  // Calculate the size for a "directory entry".
+  long name_max = fpathconf(dirfd(dir), _PC_NAME_MAX);
+
+  // If we don't get a valid size, check NAME_MAX, but fall back on
+  // 255 in the worst case ... Danger, Will Robinson!
+  if (name_max == -1) {
+    name_max = (NAME_MAX > 255) ? NAME_MAX : 255;
+  }
+
+  size_t name_end = (size_t) offsetof(dirent, d_name) + name_max + 1;
+
+  size_t size = (name_end > sizeof(dirent) ? name_end : sizeof(dirent));
+
+  dirent* temp = (dirent*) malloc(size);
+
+  if (temp == NULL) {
+    free(temp);
+    closedir(dir);
+    return std::list<std::string>();
+  }
+
+  std::list<std::string> result;
+  struct dirent* entry;
+  int error;
+
+  while ((error = readdir_r(dir, temp, &entry)) == 0 && entry != NULL) {
+    if (strcmp(entry->d_name, ".") == 0 || strcmp(entry->d_name, "..") == 0) {
+      continue;
+    }
+    result.push_back(entry->d_name);
+  }
+
+  free(temp);
+  closedir(dir);
+
+  if (error != 0) {
+    return std::list<std::string>();
+  }
+
+  return result;
+}
+
+} // namespace os {
+
+#endif // __STOUT_OS_LS_HPP__

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/376acc34/3rdparty/libprocess/3rdparty/stout/include/stout/os/osx.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/osx.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/osx.hpp
new file mode 100644
index 0000000..c27e7a2
--- /dev/null
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/osx.hpp
@@ -0,0 +1,93 @@
+#ifndef __STOUT_OS_OSX_HPP__
+#define __STOUT_OS_OSX_HPP__
+
+// This file contains OSX-only OS utilities.
+#ifndef __APPLE__
+#error "stout/os/osx.hpp is only available on OSX systems."
+#endif
+
+#include <libproc.h>
+
+#include <sys/sysctl.h>
+#include <sys/types.h> // For pid_t.
+
+#include <queue>
+#include <set>
+
+#include <stout/os/process.hpp>
+#include <stout/os/sysctl.hpp>
+
+namespace os {
+
+inline Try<Process> process(pid_t pid)
+{
+  const Try<std::vector<kinfo_proc> >& processes =
+    os::sysctl(CTL_KERN, KERN_PROC, KERN_PROC_PID, pid).table(1);
+
+  if (processes.isError()) {
+    return Error("Failed to get process via sysctl: " + processes.error());
+  } else if (processes.get().size() != 1) {
+    return Error("No process found with pid " + stringify(pid));
+  }
+
+  // We use proc_pidinfo() as it provides memory and CPU usage.
+  // NOTE: There are several pitfalls to using proc_pidinfo().
+  // In particular:
+  //   -This will not work for many root processes.
+  //   -This may not work for processes owned by other users.
+  //   -However, this always works for processes owned by the same user.
+  // This beats using task_for_pid(), which only works for the same pid.
+  // For further discussion around these issues,
+  // see: http://code.google.com/p/psutil/issues/detail?id=297
+  proc_taskinfo task;
+  int size = proc_pidinfo(pid, PROC_PIDTASKINFO, 0, &task, sizeof(task));
+
+  const kinfo_proc process = processes.get()[0];
+
+  if (size != sizeof(task)) {
+    return Process(process.kp_proc.p_pid,
+                   process.kp_eproc.e_ppid,
+                   process.kp_eproc.e_pgid,
+                   getsid(pid),
+                   Bytes(0),
+                   Nanoseconds(-1),
+                   Nanoseconds(-1),
+                   process.kp_proc.p_comm);
+  } else {
+    return Process(process.kp_proc.p_pid,
+                   process.kp_eproc.e_ppid,
+                   process.kp_eproc.e_pgid,
+                   getsid(pid),
+                   Bytes(task.pti_resident_size),
+                   Nanoseconds(task.pti_total_user),
+                   Nanoseconds(task.pti_total_system),
+                   process.kp_proc.p_comm);
+  }
+}
+
+
+inline Try<std::set<pid_t> > pids()
+{
+  const Try<int>& maxproc = os::sysctl(CTL_KERN, KERN_MAXPROC).integer();
+
+  if (maxproc.isError()) {
+    return Error(maxproc.error());
+  }
+
+  const Try<std::vector<kinfo_proc> >& processes =
+    os::sysctl(CTL_KERN, KERN_PROC, KERN_PROC_ALL).table(maxproc.get());
+
+  if (processes.isError()) {
+    return Error(processes.error());
+  }
+
+  std::set<pid_t> result;
+  foreach (const kinfo_proc& process, processes.get()) {
+    result.insert(process.kp_proc.p_pid);
+  }
+  return result;
+}
+
+} // namespace os {
+
+#endif // __STOUT_OS_OSX_HPP__

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/376acc34/3rdparty/libprocess/3rdparty/stout/include/stout/os/process.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/process.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/process.hpp
new file mode 100644
index 0000000..a88118a
--- /dev/null
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/process.hpp
@@ -0,0 +1,53 @@
+#ifndef __STOUT_OS_PROCESS_HPP__
+#define __STOUT_OS_PROCESS_HPP__
+
+#include <sys/types.h> // For pid_t.
+
+#include <string>
+
+#include <stout/bytes.hpp>
+#include <stout/duration.hpp>
+
+namespace os {
+
+struct Process
+{
+  Process(pid_t _pid,
+          pid_t _parent,
+          pid_t _group,
+          pid_t _session,
+          const Bytes& _rss,
+          const Duration& _utime,
+          const Duration& _stime,
+          const std::string& _command)
+    : pid(_pid),
+      parent(_parent),
+      group(_group),
+      session(_session),
+      rss(_rss),
+      utime(_utime),
+      stime(_stime),
+      command(_command) {}
+
+  const pid_t pid;
+  const pid_t parent;
+  const pid_t group;
+  const pid_t session;
+  const Bytes rss;
+  const Duration utime;
+  const Duration stime;
+  const std::string command;
+
+  // TODO(bmahler): Add additional data as needed.
+
+  bool operator <  (const Process& p) const { return pid <  p.pid; }
+  bool operator <= (const Process& p) const { return pid <= p.pid; }
+  bool operator >  (const Process& p) const { return pid >  p.pid; }
+  bool operator >= (const Process& p) const { return pid >= p.pid; }
+  bool operator == (const Process& p) const { return pid == p.pid; }
+  bool operator != (const Process& p) const { return pid != p.pid; }
+};
+
+} // namespace os {
+
+#endif // __STOUT_OS_PROCESS_HPP__

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/376acc34/3rdparty/libprocess/3rdparty/stout/include/stout/os/sysctl.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/sysctl.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/sysctl.hpp
index e36f3b5..a17a3bb 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/sysctl.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/sysctl.hpp
@@ -58,7 +58,9 @@ struct sysctl
   ~sysctl();
 
   // Get system information as an integer.
-  Try<int> integer() const;
+private: struct Integer; // Forward declaration.
+public:
+  Integer integer() const;
 
   // Get system information as a string.
   Try<std::string> string() const;
@@ -77,6 +79,17 @@ public:
   Table table(const Option<size_t>& length = None()) const;
 
 private:
+  struct Integer
+  {
+    Integer(int _levels, int* _name);
+
+    template <typename T>
+    operator Try<T> ();
+
+    const int levels;
+    int* name;
+  };
+
   struct Table
   {
     Table(int _levels, int* _name, const Option<size_t>& _length);
@@ -94,14 +107,14 @@ private:
 };
 
 
-sysctl::sysctl(int level1)
+inline sysctl::sysctl(int level1)
   : levels(1), name(new int[levels])
 {
   name[0] = level1;
 }
 
 
-sysctl::sysctl(int level1, int level2)
+inline sysctl::sysctl(int level1, int level2)
   : levels(2), name(new int[levels])
 {
   name[0] = level1;
@@ -109,7 +122,7 @@ sysctl::sysctl(int level1, int level2)
 }
 
 
-sysctl::sysctl(int level1, int level2, int level3)
+inline sysctl::sysctl(int level1, int level2, int level3)
   : levels(3), name(new int[levels])
 {
   name[0] = level1;
@@ -118,7 +131,7 @@ sysctl::sysctl(int level1, int level2, int level3)
 }
 
 
-sysctl::sysctl(int level1, int level2, int level3, int level4)
+inline sysctl::sysctl(int level1, int level2, int level3, int level4)
   : levels(4), name(new int[levels])
 {
   name[0] = level1;
@@ -128,7 +141,7 @@ sysctl::sysctl(int level1, int level2, int level3, int level4)
 }
 
 
-sysctl::sysctl(int level1, int level2, int level3, int level4, int level5)
+inline sysctl::sysctl(int level1, int level2, int level3, int level4, int level5)
   : levels(5), name(new int[levels])
 {
   name[0] = level1;
@@ -139,20 +152,15 @@ sysctl::sysctl(int level1, int level2, int level3, int level4, int level5)
 }
 
 
-sysctl::~sysctl()
+inline sysctl::~sysctl()
 {
   delete[] name;
 }
 
 
-inline Try<int> sysctl::integer() const
+inline sysctl::Integer sysctl::integer() const
 {
-  int i;
-  size_t size = sizeof(i);
-  if (::sysctl(name, levels, &i, &size, NULL, 0) == -1) {
-    return ErrnoError();
-  }
-  return i;
+  return Integer(levels, name);
 }
 
 
@@ -189,6 +197,26 @@ inline sysctl::Table sysctl::table(const Option<size_t>& length) const
 }
 
 
+inline sysctl::Integer::Integer(
+    int _levels,
+    int* _name)
+  : levels(_levels),
+    name(_name)
+{}
+
+
+template <typename T>
+sysctl::Integer::operator Try<T> ()
+{
+  T i;
+  size_t size = sizeof(i);
+  if (::sysctl(name, levels, &i, &size, NULL, 0) == -1) {
+    return ErrnoError();
+  }
+  return i;
+}
+
+
 inline sysctl::Table::Table(
     int _levels,
     int* _name,

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/376acc34/3rdparty/libprocess/3rdparty/stout/include/stout/proc.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/proc.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/proc.hpp
index b59735f..0bb1b1a 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/proc.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/proc.hpp
@@ -1,21 +1,3 @@
-/**
- * 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 __PROC_HPP__
 #define __PROC_HPP__
 
@@ -36,14 +18,15 @@
 #include <string>
 #include <vector>
 
-#include "error.hpp"
-#include "foreach.hpp"
-#include "none.hpp"
-#include "numify.hpp"
-#include "option.hpp"
-#include "os.hpp"
-#include "strings.hpp"
-#include "try.hpp"
+#include <stout/error.hpp>
+#include <stout/foreach.hpp>
+#include <stout/none.hpp>
+#include <stout/numify.hpp>
+#include <stout/option.hpp>
+#include <stout/strings.hpp>
+#include <stout/try.hpp>
+
+#include <stout/os/ls.hpp>
 
 namespace proc {
 
@@ -127,11 +110,11 @@ struct ProcessStatus
   const pid_t pid;
   const std::string comm;
   const char state;
-  const int ppid;
-  const int pgrp;
-  const int session;
+  const pid_t ppid;
+  const pid_t pgrp;
+  const pid_t session;
   const int tty_nr;
-  const int tpgid;
+  const pid_t tpgid;
   const unsigned int flags;
   const unsigned long minflt;
   const unsigned long cminflt;
@@ -264,48 +247,6 @@ inline Try<std::set<pid_t> > pids()
 }
 
 
-// Returns all child processes of the pid, including all descendants
-// if recursive.
-inline Try<std::set<pid_t> > children(pid_t pid, bool recursive = true)
-{
-  const Try<std::set<pid_t> >& pids = proc::pids();
-  if (pids.isError()) {
-    return Error(pids.error());
-  }
-
-  // Stat all the processes.
-  std::list<ProcessStatus> processes;
-  foreach (pid_t _pid, pids.get()) {
-    const Try<ProcessStatus>& process = status(_pid);
-    if (process.isSome()) {
-      processes.push_back(process.get());
-    }
-  }
-
-  // Perform a breadth first search for descendants.
-  std::set<pid_t> descendants;
-  std::queue<pid_t> parents;
-  parents.push(pid);
-
-  do {
-    pid_t parent = parents.front();
-    parents.pop();
-
-    // Search for children of parent.
-    foreach (const ProcessStatus& process, processes) {
-      if (process.ppid == parent) {
-        // Have we seen this child yet?
-        if (descendants.insert(process.pid).second) {
-          parents.push(process.pid);
-        }
-      }
-    }
-  } while (recursive && !parents.empty());
-
-  return descendants;
-}
-
-
 // Snapshot of a system (modeled after /proc/stat).
 struct SystemStatus
 {
@@ -422,7 +363,12 @@ inline Try<std::list<CPU> > cpus()
         line.find("core id") == 0) {
       // Get out and parse the value.
       std::vector<std::string> tokens = strings::tokenize(line, ": ");
-      CHECK(tokens.size() >= 2) << stringify(tokens);
+
+      if (tokens.size() < 2) {
+        return Error("Unexpected format in /proc/cpuinfo: " +
+                     stringify(tokens));
+      }
+
       Try<unsigned int> value = numify<unsigned int>(tokens.back());
       if (value.isError()) {
         return Error(value.error());
@@ -437,12 +383,12 @@ inline Try<std::list<CPU> > cpus()
         id = value.get();
       } else if (line.find("physical id") == 0) {
         if (socket.isSome()) {
-          return Error("Unexpected format of /proc/cpuinfo");
+          return Error("Unexpected format in /proc/cpuinfo");
         }
         socket = value.get();
       } else if (line.find("core id") == 0) {
         if (core.isSome()) {
-          return Error("Unexpected format of /proc/cpuinfo");
+          return Error("Unexpected format in /proc/cpuinfo");
         }
         core = value.get();
       }

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/376acc34/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
index 73b2336..041b4d1 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
+++ b/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
@@ -3,8 +3,11 @@
 #include <gmock/gmock.h>
 
 #include <cstdlib> // For rand.
+#include <list>
+#include <set>
 #include <string>
 
+#include <stout/duration.hpp>
 #include <stout/foreach.hpp>
 #include <stout/gtest.hpp>
 #include <stout/hashset.hpp>
@@ -17,6 +20,8 @@
 #include <stout/os/sysctl.hpp>
 #endif
 
+using std::list;
+using std::set;
 using std::string;
 
 
@@ -247,3 +252,154 @@ TEST_F(OsTest, sysctl)
   EXPECT_EQ(1, pids.count(getpid()));
 }
 #endif // __APPLE__
+
+
+TEST_F(OsTest, pids)
+{
+  Try<set<pid_t> > pids = os::pids();
+
+  ASSERT_SOME(pids);
+  EXPECT_NE(0u, pids.get().size());
+  EXPECT_EQ(1u, pids.get().count(getpid()));
+  EXPECT_EQ(1u, pids.get().count(1));
+}
+
+
+TEST_F(OsTest, children)
+{
+  Try<set<pid_t> > children = os::children(getpid());
+
+  ASSERT_SOME(children);
+  EXPECT_EQ(0u, children.get().size());
+
+  // Use pipes to determine the pids of the child and grandchild.
+  int childPipes[2];
+  int grandchildPipes[2];
+  ASSERT_NE(-1, pipe(childPipes));
+  ASSERT_NE(-1, pipe(grandchildPipes));
+
+  pid_t child;
+  pid_t grandchild;
+  pid_t pid = fork();
+  ASSERT_NE(-1, pid);
+
+  if (pid > 0) {
+    // In parent process.
+    close(childPipes[1]);
+    close(grandchildPipes[1]);
+
+    // Get the pids via the pipes.
+    ASSERT_NE(-1, read(childPipes[0], &child, sizeof(child)));
+    ASSERT_NE(-1, read(grandchildPipes[0], &grandchild, sizeof(grandchild)));
+
+    close(childPipes[0]);
+    close(grandchildPipes[0]);
+  } else {
+    // In child process.
+    close(childPipes[0]);
+    close(grandchildPipes[0]);
+
+    // Double fork!
+    if ((pid = fork()) == -1) {
+      perror("Failed to fork a grand child process");
+      abort();
+    }
+
+    if (pid > 0) {
+      // Still in child process.
+      pid = getpid();
+      if (write(childPipes[1], &pid, sizeof(pid)) != sizeof(pid)) {
+        perror("Failed to write PID on pipe");
+        abort();
+      }
+
+      close(childPipes[1]);
+
+      while (true); // Keep waiting until we get a signal.
+    } else {
+      // In grandchild process.
+      pid = getpid();
+      if (write(grandchildPipes[1], &pid, sizeof(pid)) != sizeof(pid)) {
+        perror("Failed to write PID on pipe");
+        abort();
+      }
+
+      close(grandchildPipes[1]);
+
+      while (true); // Keep waiting until we get a signal.
+    }
+  }
+
+  // Ensure the non-recursive children does not include the
+  // grandchild.
+  children = os::children(getpid(), false);
+
+  ASSERT_SOME(children);
+  EXPECT_EQ(1u, children.get().size());
+  EXPECT_EQ(1u, children.get().count(child));
+
+  children = os::children(getpid());
+
+  ASSERT_SOME(children);
+  EXPECT_EQ(2u, children.get().size());
+  EXPECT_EQ(1u, children.get().count(child));
+  EXPECT_EQ(1u, children.get().count(grandchild));
+
+  // Cleanup by killing the descendant processes.
+  EXPECT_EQ(0, kill(grandchild, SIGKILL)) << strerror(errno);
+  EXPECT_EQ(0, kill(child, SIGKILL)) << strerror(errno);
+
+  // We have to reap the child for running the tests in repetition.
+  ASSERT_EQ(child, waitpid(child, NULL, 0)) << strerror(errno);
+}
+
+
+TEST_F(OsTest, process)
+{
+  Try<os::Process> status = os::process(getpid());
+
+  ASSERT_SOME(status);
+  EXPECT_EQ(getpid(), status.get().pid);
+  EXPECT_EQ(getppid(), status.get().parent);
+  EXPECT_EQ(getsid(getpid()), status.get().session);
+  EXPECT_GT(status.get().rss, 0);
+
+  // NOTE: On Linux /proc is a bit slow to update the CPU times,
+  // hence we allow 0 in this test.
+  EXPECT_GE(status.get().utime, Nanoseconds(0));
+  EXPECT_GE(status.get().stime, Nanoseconds(0));
+
+  EXPECT_FALSE(status.get().command.empty());
+}
+
+
+TEST_F(OsTest, processes)
+{
+  Try<list<os::Process> > processes = os::processes();
+
+  ASSERT_SOME(processes);
+  ASSERT_GT(processes.get().size(), 2);
+
+  // Look for ourselves in the table.
+  bool found = false;
+  foreach (const os::Process& process, processes.get()) {
+    if (process.pid == getpid()) {
+      found = true;
+      EXPECT_EQ(getpid(), process.pid);
+      EXPECT_EQ(getppid(), process.parent);
+      EXPECT_EQ(getsid(getpid()), process.session);
+      EXPECT_GT(process.rss, 0);
+
+      // NOTE: On linux /proc is a bit slow to update the cpu times,
+      // hence we allow 0 in this test.
+      EXPECT_GE(process.utime, Nanoseconds(0));
+      EXPECT_GE(process.stime, Nanoseconds(0));
+
+      EXPECT_FALSE(process.command.empty());
+
+      break;
+    }
+  }
+
+  EXPECT_TRUE(found);
+}

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/376acc34/3rdparty/libprocess/3rdparty/stout/tests/proc_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/proc_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/proc_tests.cpp
index 2305ef5..56804ec 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/proc_tests.cpp
+++ b/3rdparty/libprocess/3rdparty/stout/tests/proc_tests.cpp
@@ -1,21 +1,3 @@
-/**
- * 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 <unistd.h> // For getpid, getppid.
 
 #include <gmock/gmock.h>
@@ -44,96 +26,6 @@ TEST(ProcTest, pids)
 }
 
 
-TEST(ProcTest, children)
-{
-  Try<set<pid_t> > children = proc::children(getpid());
-
-  ASSERT_SOME(children);
-  EXPECT_EQ(0u, children.get().size());
-
-  // Use pipes to determine the pids of the child and grandchild.
-  int childPipes[2];
-  int grandchildPipes[2];
-  ASSERT_NE(-1, pipe(childPipes));
-  ASSERT_NE(-1, pipe(grandchildPipes));
-
-  pid_t childPid;
-  pid_t grandchildPid;
-  pid_t pid = fork();
-  ASSERT_NE(-1, pid);
-
-  if (pid > 0) {
-    // In parent process.
-    close(childPipes[1]);
-    close(grandchildPipes[1]);
-
-    // Get the pids via the pipes.
-    ASSERT_NE(
-        -1,
-        read(childPipes[0], &childPid, sizeof(childPid)));
-    ASSERT_NE(
-        -1,
-        read(grandchildPipes[0], &grandchildPid, sizeof(grandchildPid)));
-
-    close(childPipes[0]);
-    close(grandchildPipes[0]);
-  } else {
-    // In child process.
-    close(childPipes[0]);
-    close(grandchildPipes[0]);
-
-    // Double fork!
-    if ((pid = fork()) == -1) {
-      perror("Failed to fork a grand child process");
-      abort();
-    }
-
-    if (pid > 0) {
-      // Still in child process.
-      pid = getpid();
-      if (write(childPipes[1], &pid, sizeof(pid)) != sizeof(pid)) {
-        perror("Failed to write PID on pipe");
-        abort();
-      }
-
-      close(childPipes[1]);
-
-      while (true); // Keep waiting until we get a signal.
-    } else {
-      // In grandchild process.
-      pid = getpid();
-      if (write(grandchildPipes[1], &pid, sizeof(pid)) != sizeof(pid)) {
-        perror("Failed to write PID on pipe");
-        abort();
-      }
-
-      close(grandchildPipes[1]);
-
-      while (true); // Keep waiting until we get a signal.
-    }
-  }
-
-  // Ensure the non-recursive children does not include the
-  // grandchild.
-  children = proc::children(getpid(), false);
-
-  ASSERT_SOME(children);
-  EXPECT_EQ(1u, children.get().size());
-  EXPECT_EQ(1u, children.get().count(childPid));
-
-  children = proc::children(getpid());
-
-  ASSERT_SOME(children);
-  EXPECT_EQ(2u, children.get().size());
-  EXPECT_EQ(1u, children.get().count(childPid));
-  EXPECT_EQ(1u, children.get().count(grandchildPid));
-
-  // Cleanup by killing the descendants.
-  EXPECT_EQ(0, kill(grandchildPid, SIGKILL));
-  EXPECT_EQ(0, kill(childPid, SIGKILL));
-}
-
-
 TEST(ProcTest, cpus)
 {
   Try<std::list<CPU> > cpus = proc::cpus();


[5/6] git commit: Implemented a killtree utility (inspired by the killtree.sh script in mesos).

Posted by bm...@apache.org.
Implemented a killtree utility (inspired by the killtree.sh script
in mesos).

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


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

Branch: refs/heads/master
Commit: 5738e5acf2080e797b0e8d39744080310b8d3101
Parents: a79526f
Author: Benjamin Mahler <bm...@twitter.com>
Authored: Mon May 13 10:46:46 2013 -0700
Committer: Benjamin Mahler <bm...@twitter.com>
Committed: Wed Jun 19 20:29:43 2013 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/stout/Makefile.am  |   1 +
 .../3rdparty/stout/include/stout/os.hpp         |   1 +
 .../stout/include/stout/os/killtree.hpp         | 147 +++++++++++++
 .../3rdparty/stout/tests/os_tests.cpp           | 215 +++++++++++++++++--
 4 files changed, 345 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/5738e5ac/3rdparty/libprocess/3rdparty/stout/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/Makefile.am b/3rdparty/libprocess/3rdparty/stout/Makefile.am
index 5d9f35f..99a3934 100644
--- a/3rdparty/libprocess/3rdparty/stout/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/stout/Makefile.am
@@ -34,6 +34,7 @@ EXTRA_DIST =					\
   include/stout/numify.hpp			\
   include/stout/option.hpp			\
   include/stout/os.hpp				\
+  include/stout/os/killtree.hpp			\
   include/stout/os/linux.hpp			\
   include/stout/os/ls.hpp			\
   include/stout/os/osx.hpp			\

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/5738e5ac/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
index af47e18..0e37ca0 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
@@ -53,6 +53,7 @@
 #include <stout/strings.hpp>
 #include <stout/try.hpp>
 
+#include <stout/os/killtree.hpp>
 #ifdef __linux__
 #include <stout/os/linux.hpp>
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/5738e5ac/3rdparty/libprocess/3rdparty/stout/include/stout/os/killtree.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/killtree.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/killtree.hpp
new file mode 100644
index 0000000..2a0f01a
--- /dev/null
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/killtree.hpp
@@ -0,0 +1,147 @@
+#ifndef __STOUT_OS_KILLTREE_HPP__
+#define __STOUT_OS_KILLTREE_HPP__
+
+#include <dirent.h>
+#include <stdlib.h>
+#include <unistd.h>
+
+#include <list>
+#include <ostream>
+#include <queue>
+#include <set>
+#include <sstream>
+#include <string>
+
+#include <stout/os.hpp>
+#include <stout/stringify.hpp>
+
+namespace os {
+
+// Forward declarations from os.hpp.
+inline Try<std::set<pid_t> > children(pid_t pid, bool recursive);
+inline Try<std::set<pid_t> > pids(Option<pid_t> group, Option<pid_t> session);
+
+
+// Sends a signal to a process tree rooted at the specified pid.
+// If groups is true, this also sends the signal to all encountered
+// process groups.
+// If sessions is true, this also sends the signal to all encountered
+// process sessions.
+// It is advised to only set groups / sessions when the process tree
+// is guaranteed to be isolated via a setsid call.
+// Optionally, an output stream can be provided for debugging output.
+inline Try<Nothing> killtree(
+    pid_t pid,
+    int signal,
+    bool groups = false,
+    bool sessions = false,
+    std::ostream* os = NULL)
+{
+  std::ostringstream output;
+  output << "Performing killtree operation on " << pid << std::endl;
+
+  // TODO(bmahler): Inspect parent session / group to ensure this
+  // doesn't kill up the tree?
+
+  // First we collect and stop the full process tree via a
+  // breadth-first-search.
+  std::set<pid_t> visited;
+  std::queue<pid_t> queue;
+  queue.push(pid);
+  visited.insert(pid);
+
+  while (!queue.empty()) {
+    pid_t pid = queue.front();
+    queue.pop();
+
+    // Stop the process to keep it from forking while we are killing
+    // it since a forked child might get re-parented by init and
+    // become impossible to find.
+    if (kill(pid, SIGSTOP) == -1) {
+      output << "Failed to stop " << pid << ": "
+             << strerror(errno) << std::endl;
+    } else {
+      output << "Stopped " << pid << std::endl;
+    }
+
+    // TODO(bmahler): Create and use sets::union here.
+    // Append all direct children to the queue.
+    const Try<std::set<pid_t> >& children = os::children(pid, false);
+    if (children.isSome()) {
+      output << "  Children of " << pid << ": "
+             << stringify(children.get()) << std::endl;
+      foreach (pid_t child, children.get()) {
+        if (visited.insert(child).second) {
+          queue.push(child);
+        }
+      }
+    }
+
+    if (groups) {
+      // Append all group members to the queue.
+      const Try<std::set<pid_t> >& pids = os::pids(getpgid(pid), None());
+      if (pids.isSome()) {
+        output << "  Members of group " << getpgid(pid) << ": "
+               << stringify(pids.get()) << std::endl;
+        foreach (pid_t pid, pids.get()) {
+          if (visited.insert(pid).second) {
+            queue.push(pid);
+          }
+        }
+      }
+    }
+
+    if (sessions) {
+      // Append all session members to the queue.
+      const Try<std::set<pid_t> >& pids = os::pids(None(), getsid(pid));
+      if (pids.isSome()) {
+        output << "  Members of session " << getsid(pid) << ": "
+               << stringify(pids.get()) << std::endl;
+        foreach (pid_t pid, pids.get()) {
+          if (visited.insert(pid).second) {
+            queue.push(pid);
+          }
+        }
+      }
+    }
+  }
+
+  // Now that all processes are stopped, we send the signal.
+  foreach (pid_t pid, visited) {
+    kill(pid, signal);
+    output << "Signaled " << pid << std::endl;
+  }
+
+  // There is a concern that even though some process is stopped,
+  // sending a signal to any of it's children may cause a SIGCLD to
+  // be delivered to it which wakes it up (or any other signal maybe
+  // delivered). However, from the Open Group standards on "Signal
+  // Concepts":
+  //
+  //   "While a process is stopped, any additional signals that are
+  //    sent to the process shall not be delivered until the process
+  //    is continued, except SIGKILL which always terminates the
+  //    receiving process."
+  //
+  // In practice, this is not what has been witnessed. Rather, a
+  // process that has been stopped will respond to SIGTERM, SIGINT,
+  // etc. That being said, we still continue the process below in the
+  // event that it doesn't terminate from the sending signal but it
+  // also doesn't get continued (as per the specifications above).
+
+  // Try and continue the processes in case the signal is
+  // non-terminating but doesn't continue the process.
+  foreach (pid_t pid, visited) {
+    kill(pid, SIGCONT);
+  }
+
+  if (os != NULL) {
+    *os << output.str();
+  }
+
+  return Nothing();
+}
+
+} // namespace os {
+
+#endif // __STOUT_OS_KILLTREE_HPP__

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/5738e5ac/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
index 99f69a2..2ac9030 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
+++ b/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
@@ -1,7 +1,7 @@
-#include <gtest/gtest.h>
-
 #include <gmock/gmock.h>
 
+#include <gtest/gtest.h>
+
 #include <cstdlib> // For rand.
 #include <list>
 #include <set>
@@ -285,18 +285,17 @@ TEST_F(OsTest, children)
   ASSERT_SOME(children);
   EXPECT_EQ(0u, children.get().size());
 
-  // Use pipes to determine the pids of the child and grandchild.
+  // Use pipes to determine the pids of the grandchild.
   int childPipes[2];
   int grandchildPipes[2];
   ASSERT_NE(-1, pipe(childPipes));
   ASSERT_NE(-1, pipe(grandchildPipes));
 
-  pid_t child;
   pid_t grandchild;
-  pid_t pid = fork();
-  ASSERT_NE(-1, pid);
+  pid_t child = fork();
+  ASSERT_NE(-1, child);
 
-  if (pid > 0) {
+  if (child > 0) {
     // In parent process.
     close(childPipes[1]);
     close(grandchildPipes[1]);
@@ -313,30 +312,31 @@ TEST_F(OsTest, children)
     close(grandchildPipes[0]);
 
     // Double fork!
-    if ((pid = fork()) == -1) {
-      perror("Failed to fork a grand child process");
+    if ((grandchild = fork()) == -1) {
+      perror("Failed to fork a grandchild process");
       abort();
     }
 
-    if (pid > 0) {
+    if (grandchild > 0) {
       // Still in child process.
-      pid = getpid();
-      if (write(childPipes[1], &pid, sizeof(pid)) != sizeof(pid)) {
+      close(grandchildPipes[1]);
+
+      child = getpid();
+      if (write(childPipes[1], &child, sizeof(child)) != sizeof(child)) {
         perror("Failed to write PID on pipe");
         abort();
       }
-
       close(childPipes[1]);
 
       while (true); // Keep waiting until we get a signal.
     } else {
       // In grandchild process.
-      pid = getpid();
-      if (write(grandchildPipes[1], &pid, sizeof(pid)) != sizeof(pid)) {
+      grandchild = getpid();
+      if (write(grandchildPipes[1], &grandchild, sizeof(grandchild)) !=
+          sizeof(grandchild)) {
         perror("Failed to write PID on pipe");
         abort();
       }
-
       close(grandchildPipes[1]);
 
       while (true); // Keep waiting until we get a signal.
@@ -359,11 +359,11 @@ TEST_F(OsTest, children)
   EXPECT_EQ(1u, children.get().count(grandchild));
 
   // Cleanup by killing the descendant processes.
-  EXPECT_EQ(0, kill(grandchild, SIGKILL)) << strerror(errno);
-  EXPECT_EQ(0, kill(child, SIGKILL)) << strerror(errno);
+  EXPECT_EQ(0, kill(grandchild, SIGKILL));
+  EXPECT_EQ(0, kill(child, SIGKILL));
 
   // We have to reap the child for running the tests in repetition.
-  ASSERT_EQ(child, waitpid(child, NULL, 0)) << strerror(errno);
+  ASSERT_EQ(child, waitpid(child, NULL, 0));
 }
 
 
@@ -416,3 +416,180 @@ TEST_F(OsTest, processes)
 
   EXPECT_TRUE(found);
 }
+
+
+TEST_F(OsTest, killtree) {
+  // Use pipes to determine the pids of the child and grandchild.
+  int childPipes[2];
+  int grandchildPipes[2];
+  int greatGrandchildPipes[2];
+  int greatGreatGrandchildPipes[2];
+
+  ASSERT_NE(-1, pipe(childPipes));
+  ASSERT_NE(-1, pipe(grandchildPipes));
+  ASSERT_NE(-1, pipe(greatGrandchildPipes));
+  ASSERT_NE(-1, pipe(greatGreatGrandchildPipes));
+
+  pid_t child;
+  pid_t grandchild;
+  pid_t greatGrandchild;
+  pid_t greatGreatGrandchild;
+
+  child = fork();
+  ASSERT_NE(-1, child);
+
+  // To test killtree, we create the following process chain:
+  // 1: This process.
+  // 2: Child process having called setsid().
+  // X: Grandchild process, terminates immediately after forking!
+  // 4: Great-grandchild process.
+  // 5: Great-great-grandchild process, calls setsid()!
+  // We expect killtree to kill 4 via its session or group.
+  // We also expect killtree to kill 5 via its parent process,
+  // despite having called setsid().
+  if (child > 0) {
+    // Parent.
+    close(childPipes[1]);
+    close(grandchildPipes[1]);
+    close(greatGrandchildPipes[1]);
+    close(greatGreatGrandchildPipes[1]);
+
+    // Get the pids via the pipes.
+    ASSERT_NE(-1, read(childPipes[0], &child, sizeof(child)));
+    ASSERT_NE(
+        -1,
+        read(grandchildPipes[0], &grandchild, sizeof(grandchild)));
+    ASSERT_NE(
+        -1,
+        read(greatGrandchildPipes[0],
+             &greatGrandchild,
+             sizeof(greatGrandchild)));
+    ASSERT_NE(
+        -1,
+        read(greatGreatGrandchildPipes[0],
+             &greatGreatGrandchild,
+             sizeof(greatGreatGrandchild)));
+
+    close(childPipes[0]);
+    close(grandchildPipes[0]);
+    close(greatGrandchildPipes[0]);
+    close(greatGreatGrandchildPipes[0]);
+  } else {
+    // --------------------------------------------------------------
+    // Child: setsid().
+    // --------------------------------------------------------------
+    close(childPipes[0]);
+    close(grandchildPipes[0]);
+    close(greatGrandchildPipes[0]);
+    close(greatGreatGrandchildPipes[0]);
+
+    if (setsid() == -1) {
+      perror("Failed to setsid in great-great-grandchild process");
+      abort();
+    }
+
+    child = getpid();
+    if (write(childPipes[1], &child, sizeof(child)) != sizeof(child)) {
+      perror("Failed to write child PID on pipe");
+      abort();
+    }
+    close(childPipes[1]);
+
+    if ((grandchild = fork()) == -1) {
+      perror("Failed to fork a grandchild process");
+      abort();
+    }
+
+    if (grandchild > 0) {
+      close(grandchildPipes[1]);
+      close(greatGrandchildPipes[1]);
+      close(greatGreatGrandchildPipes[1]);
+      while (true); // Await signal.
+    }
+
+    // --------------------------------------------------------------
+    // Grandchild: terminate.
+    // --------------------------------------------------------------
+    // Send the grandchild pid over the pipe.
+    grandchild = getpid();
+    if (write(grandchildPipes[1], &grandchild, sizeof(grandchild)) !=
+        sizeof(grandchild)) {
+      perror("Failed to write grandchild PID on pipe");
+      abort();
+    }
+    close(grandchildPipes[1]);
+
+    if ((greatGrandchild = fork()) == -1) {
+      perror("Failed to fork a great-grandchild process");
+      abort();
+    }
+
+    if (greatGrandchild > 0) {
+      // Terminate to break the parent link.
+      close(greatGrandchildPipes[1]);
+      close(greatGreatGrandchildPipes[1]);
+      exit(0);
+    }
+
+    // --------------------------------------------------------------
+    // Great-grandchild.
+    // --------------------------------------------------------------
+    // Send the Great-grandchild pid over the pipe.
+    greatGrandchild = getpid();
+    if (write(greatGrandchildPipes[1],
+              &greatGrandchild,
+              sizeof(greatGrandchild)) != sizeof(greatGrandchild)) {
+      perror("Failed to write great-grandchild PID on pipe");
+      abort();
+    }
+    close(greatGrandchildPipes[1]);
+
+    if ((greatGreatGrandchild = fork()) == -1) {
+      perror("Failed to fork a great-great-grandchild process");
+      abort();
+    }
+
+    if (greatGreatGrandchild > 0) {
+      // Great-grandchild.
+      close(greatGreatGrandchildPipes[1]);
+      while (true); // Await signal.
+    }
+
+    // --------------------------------------------------------------
+    // Great-great-grandchild: setsid().
+    // --------------------------------------------------------------
+    if (setsid() == -1) {
+      perror("Failed to setsid in great-great-grandchild process");
+      abort();
+    }
+
+    // Send the Great-great-grandchild pid over the pipe.
+    greatGreatGrandchild = getpid();
+    if (write(greatGreatGrandchildPipes[1],
+              &greatGreatGrandchild,
+              sizeof(greatGreatGrandchild)) != sizeof(greatGreatGrandchild)) {
+      perror("Failed to write great-great-grandchild PID on pipe");
+      abort();
+    }
+    close(greatGreatGrandchildPipes[1]);
+
+    while (true); // Await signal.
+  }
+
+  // Kill the child process tree, this is expected to
+  // cross the broken link to the grandchild
+  EXPECT_SOME(os::killtree(child, SIGKILL, true, true, &std::cout));
+
+  // There is a delay for the process to move into the zombie state.
+  os::sleep(Milliseconds(50));
+
+  // Expect the pids to be wiped!
+  EXPECT_SOME_EQ(false, os::alive(greatGreatGrandchild));
+  EXPECT_SOME_EQ(false, os::alive(greatGreatGrandchild));
+  EXPECT_SOME_EQ(false, os::alive(greatGrandchild));
+  EXPECT_SOME_EQ(false, os::alive(grandchild));
+  EXPECT_SOME_EQ(false, os::alive(child));
+
+  // We have to reap the child for running the tests in repetition.
+  ASSERT_EQ(child, waitpid(child, NULL, 0));
+}


[3/6] git commit: Added utilities for getting process group and session members.

Posted by bm...@apache.org.
Added utilities for getting process group and session members.

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


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

Branch: refs/heads/master
Commit: 5faf7001a7ef0b00dfd4d7611f6c7c0929e12e7a
Parents: f1dd735
Author: Benjamin Mahler <bm...@twitter.com>
Authored: Wed Jun 12 12:05:11 2013 -0700
Committer: Benjamin Mahler <bm...@twitter.com>
Committed: Wed Jun 19 20:29:34 2013 -0700

----------------------------------------------------------------------
 .../3rdparty/stout/include/stout/os.hpp         | 46 ++++++++++++++++++++
 .../3rdparty/stout/tests/os_tests.cpp           | 27 +++++++++---
 2 files changed, 66 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/5faf7001/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
index f2c59ee..47e877c 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
@@ -47,6 +47,7 @@
 #include <stout/foreach.hpp>
 #include <stout/none.hpp>
 #include <stout/nothing.hpp>
+#include <stout/option.hpp>
 #include <stout/path.hpp>
 #include <stout/result.hpp>
 #include <stout/strings.hpp>
@@ -1063,6 +1064,51 @@ inline Try<std::set<pid_t> > children(pid_t pid, bool recursive = true)
 }
 
 
+// Overload of os::pids for filtering by groups and sessions.
+// A group / session id of 0 will fitler on the group / session ID
+// of the calling process.
+inline Try<std::set<pid_t> > pids(Option<pid_t> group, Option<pid_t> session)
+{
+  if (group.isNone() && session.isNone()) {
+    return os::pids();
+  } else if (group.isSome() && group.get() < 0) {
+    return Error("Invalid group");
+  } else if (session.isSome() && session.get() < 0) {
+    return Error("Invalid session");
+  }
+
+  const Try<std::list<Process> >& processes = os::processes();
+
+  if (processes.isError()) {
+    return Error(processes.error());
+  }
+
+  // Obtain the calling process group / session ID when 0 is provided.
+  if (group.isSome() && group.get() == 0) {
+    group = getpgid(0);
+  }
+  if (session.isSome() && session.get() == 0) {
+    session = getsid(0);
+  }
+
+  std::set<pid_t> result;
+  foreach (const Process& process, processes.get()) {
+    // Group AND Session (intersection).
+    if (group.isSome() && session.isSome()) {
+      if (group.get() == process.group && session.get() == process.session) {
+        result.insert(process.pid);
+      }
+    } else if (group.isSome() && group.get() == process.group) {
+      result.insert(process.pid);
+    } else if (session.isSome() && session.get() == process.session) {
+      result.insert(process.pid);
+    }
+  }
+
+  return result;
+}
+
+
 inline Try<bool> alive(pid_t pid)
 {
   CHECK(pid > 0);

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/5faf7001/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
index 041b4d1..99f69a2 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
+++ b/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
@@ -40,7 +40,7 @@ class OsTest : public ::testing::Test
 protected:
   virtual void SetUp()
   {
-    Try<string> mkdtemp = os::mkdtemp();
+    const Try<string>& mkdtemp = os::mkdtemp();
     ASSERT_SOME(mkdtemp);
     tmpdir = mkdtemp.get();
   }
@@ -171,7 +171,7 @@ TEST_F(OsTest, find)
 
 TEST_F(OsTest, uname)
 {
-  Try<os::UTSInfo> info = os::uname();
+  const Try<os::UTSInfo>& info = os::uname();
 
   ASSERT_SOME(info);
 #ifdef __linux__
@@ -185,7 +185,7 @@ TEST_F(OsTest, uname)
 
 TEST_F(OsTest, sysname)
 {
-  Try<string> name = os::sysname();
+  const Try<string>& name = os::sysname();
 
   ASSERT_SOME(name);
 #ifdef __linux__
@@ -199,7 +199,7 @@ TEST_F(OsTest, sysname)
 
 TEST_F(OsTest, release)
 {
-  Try<os::Release> info = os::release();
+  const Try<os::Release>& info = os::release();
 
   ASSERT_SOME(info);
 }
@@ -257,11 +257,24 @@ TEST_F(OsTest, sysctl)
 TEST_F(OsTest, pids)
 {
   Try<set<pid_t> > pids = os::pids();
-
   ASSERT_SOME(pids);
   EXPECT_NE(0u, pids.get().size());
   EXPECT_EQ(1u, pids.get().count(getpid()));
   EXPECT_EQ(1u, pids.get().count(1));
+
+  pids = os::pids(getpgid(0), None());
+  EXPECT_SOME(pids);
+  EXPECT_GE(pids.get().size(), 1u);
+  EXPECT_EQ(1u, pids.get().count(getpid()));
+
+  EXPECT_ERROR(os::pids(-1, None()));
+
+  pids = os::pids(None(), getsid(0));
+  EXPECT_SOME(pids);
+  EXPECT_GE(pids.get().size(), 1u);
+  EXPECT_EQ(1u, pids.get().count(getpid()));
+
+  EXPECT_ERROR(os::pids(None(), -1));
 }
 
 
@@ -356,7 +369,7 @@ TEST_F(OsTest, children)
 
 TEST_F(OsTest, process)
 {
-  Try<os::Process> status = os::process(getpid());
+  const Try<os::Process>& status = os::process(getpid());
 
   ASSERT_SOME(status);
   EXPECT_EQ(getpid(), status.get().pid);
@@ -375,7 +388,7 @@ TEST_F(OsTest, process)
 
 TEST_F(OsTest, processes)
 {
-  Try<list<os::Process> > processes = os::processes();
+  const Try<list<os::Process> >& processes = os::processes();
 
   ASSERT_SOME(processes);
   ASSERT_GT(processes.get().size(), 2);


[2/6] git commit: Updated the ProcessIsolator to use the new libstout process utilities.

Posted by bm...@apache.org.
Updated the ProcessIsolator to use the new libstout process utilities.

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


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

Branch: refs/heads/master
Commit: f1dd7350d584a05a1ed3b76191531149b3960744
Parents: 376acc3
Author: Benjamin Mahler <bm...@twitter.com>
Authored: Mon May 13 10:43:44 2013 -0700
Committer: Benjamin Mahler <bm...@twitter.com>
Committed: Wed Jun 19 20:29:30 2013 -0700

----------------------------------------------------------------------
 src/slave/process_isolator.cpp | 83 ++++++++-----------------------------
 src/tests/environment.cpp      | 11 +++++
 src/tests/isolator_tests.cpp   |  7 ----
 3 files changed, 28 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/f1dd7350/src/slave/process_isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/process_isolator.cpp b/src/slave/process_isolator.cpp
index b54bf7e..83bd8da 100644
--- a/src/slave/process_isolator.cpp
+++ b/src/slave/process_isolator.cpp
@@ -21,10 +21,6 @@
 #include <stdio.h> // For perror.
 #include <string.h>
 
-#ifdef __APPLE__
-#include <libproc.h> // For proc_pidinfo.
-#endif
-
 #include <map>
 #include <set>
 
@@ -38,9 +34,6 @@
 #include <stout/nothing.hpp>
 #include <stout/option.hpp>
 #include <stout/os.hpp>
-#ifdef __linux__
-#include "stout/proc.hpp"
-#endif
 #include <stout/uuid.hpp>
 
 #include "common/type_utils.hpp"
@@ -373,49 +366,38 @@ Future<ResourceStatistics> ProcessIsolator::usage(
   }
 
   ProcessInfo* info = infos[frameworkId][executorId];
+  CHECK_NOTNULL(info);
 
   ResourceStatistics result;
 
   result.set_timestamp(Clock::now().secs());
 
   // Set the resource allocations.
-  Option<Bytes> mem = info->resources.mem();
+  const Option<Bytes>& mem = info->resources.mem();
   if (mem.isSome()) {
     result.set_mem_limit_bytes(mem.get().bytes());
   }
 
-  Option<double> cpus = info->resources.cpus();
+  const Option<double>& cpus = info->resources.cpus();
   if (cpus.isSome()) {
     result.set_cpus_limit(cpus.get());
   }
 
-#ifdef __linux__
-  // Get the page size, used for memory accounting.
-  // NOTE: This is more portable than using getpagesize().
-  long pageSize = sysconf(_SC_PAGESIZE);
-  PCHECK(pageSize > 0) << "Failed to get sysconf(_SC_PAGESIZE)";
-
-  // Get the number of clock ticks, used for cpu accounting.
-  long ticks = sysconf(_SC_CLK_TCK);
-  PCHECK(ticks > 0) << "Failed to get sysconf(_SC_CLK_TCK)";
-
   CHECK_SOME(info->pid);
 
-  // Get the parent process usage statistics.
-  Try<proc::ProcessStatus> status = proc::status(info->pid.get());
+  Try<os::Process> process = os::process(info->pid.get());
 
-  if (status.isError()) {
-    return Future<ResourceStatistics>::failed(status.error());
+  if (process.isError()) {
+    return Future<ResourceStatistics>::failed(process.error());
   }
 
-  result.set_mem_rss_bytes(status.get().rss * pageSize);
-  result.set_cpus_user_time_secs(
-      (double) status.get().utime / (double) ticks);
-  result.set_cpus_system_time_secs(
-      (double) status.get().stime / (double) ticks);
+  result.set_timestamp(Clock::now().secs());
+  result.set_mem_rss_bytes(process.get().rss.bytes());
+  result.set_cpus_user_time_secs(process.get().utime.secs());
+  result.set_cpus_system_time_secs(process.get().stime.secs());
 
   // Now aggregate all descendant process usage statistics.
-  Try<set<pid_t> > children = proc::children(info->pid.get(), true);
+  const Try<set<pid_t> >& children = os::children(info->pid.get(), true);
 
   if (children.isError()) {
     return Future<ResourceStatistics>::failed(
@@ -425,53 +407,22 @@ Future<ResourceStatistics> ProcessIsolator::usage(
 
   // Aggregate the usage of all child processes.
   foreach (pid_t child, children.get()) {
-    status = proc::status(child);
+    process = os::process(child);
 
-    if (status.isError()) {
+    if (process.isError()) {
       LOG(WARNING) << "Failed to get status of descendant process " << child
                    << " of parent " << info->pid.get() << ": "
-                   << status.error();
+                   << process.error();
       continue;
     }
 
     result.set_mem_rss_bytes(
-        result.mem_rss_bytes() + status.get().rss * pageSize);
-
+        result.mem_rss_bytes() + process.get().rss.bytes());
     result.set_cpus_user_time_secs(
-        result.cpus_user_time_secs() +
-        (double) status.get().utime / (double) ticks);
-
+        result.cpus_user_time_secs() + process.get().utime.secs());
     result.set_cpus_system_time_secs(
-        result.cpus_system_time_secs() +
-        (double) status.get().stime / (double) ticks);
+        result.cpus_system_time_secs() + process.get().stime.secs());
   }
-#elif defined __APPLE__
-  // TODO(bmahler): Aggregate the usage of all child processes.
-  // NOTE: There are several pitfalls to using proc_pidinfo().
-  // In particular:
-  //   -This will not work for many root processes.
-  //   -This may not work for processes owned by other users.
-  //   -However, this always works for processes owned by the same user.
-  // This beats using task_for_pid(), which only works for the same pid.
-  // For further discussion around these issues,
-  // see: http://code.google.com/p/psutil/issues/detail?id=297
-  CHECK_SOME(info->pid);
-
-  proc_taskinfo task;
-  int size =
-    proc_pidinfo(info->pid.get(), PROC_PIDTASKINFO, 0, &task, sizeof(task));
-
-  if (size != sizeof(task)) {
-    return Future<ResourceStatistics>::failed(
-        "Failed to get proc_pidinfo: " + stringify(size));
-  }
-
-  result.set_mem_rss_bytes(task.pti_resident_size);
-
-  // NOTE: CPU Times are in nanoseconds, but this is not documented!
-  result.set_cpus_user_time_secs(Nanoseconds(task.pti_total_user).secs());
-  result.set_cpus_system_time_secs(Nanoseconds(task.pti_total_system).secs());
-#endif
 
   return result;
 }

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/f1dd7350/src/tests/environment.cpp
----------------------------------------------------------------------
diff --git a/src/tests/environment.cpp b/src/tests/environment.cpp
index 24227c5..54649a5 100644
--- a/src/tests/environment.cpp
+++ b/src/tests/environment.cpp
@@ -108,6 +108,17 @@ static bool enable(const ::testing::TestInfo& test)
         (os::user() != "root" || !os::exists("/proc/cgroups"))) {
       return false;
     }
+#ifdef __APPLE__
+    if (strings::contains(test.test_case_name(), "IsolatorTest") &&
+        strings::contains(test.name(), "Usage") &&
+        strings::contains(type, "ProcessIsolator") &&
+        os::user() != "root") {
+      // We can't run the Isolator resource usage test when we're not
+      // the root user on OSX because proc_pidinfo() only returns
+      // memory and CPU usage reliably when running as root.
+      return false;
+    }
+#endif
   }
 
   return true;

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/f1dd7350/src/tests/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator_tests.cpp b/src/tests/isolator_tests.cpp
index 7013fa2..a37f006 100644
--- a/src/tests/isolator_tests.cpp
+++ b/src/tests/isolator_tests.cpp
@@ -75,14 +75,7 @@ typedef ::testing::Types<ProcessIsolator> IsolatorTypes;
 
 TYPED_TEST_CASE(IsolatorTest, IsolatorTypes);
 
-
-// TODO(bmahler): This test is disabled on OSX, until proc::children
-// is implemented for OSX.
-#ifdef __APPLE__
-TYPED_TEST(IsolatorTest, DISABLED_Usage)
-#else
 TYPED_TEST(IsolatorTest, Usage)
-#endif
 {
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);


[6/6] git commit: Removed killtree.sh in favor of os::killtree from libstout.

Posted by bm...@apache.org.
Removed killtree.sh in favor of os::killtree from libstout.

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


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

Branch: refs/heads/master
Commit: 56c20b619ad7aa915eaf4fff914f5e426b676761
Parents: 5738e5a
Author: Benjamin Mahler <bm...@twitter.com>
Authored: Wed Jun 12 14:41:03 2013 -0700
Committer: Benjamin Mahler <bm...@twitter.com>
Committed: Wed Jun 19 20:29:48 2013 -0700

----------------------------------------------------------------------
 bin/mesos-build-env.sh.in          |   2 -
 src/Makefile.am                    |  10 +-
 src/common/process_utils.hpp       |  72 ----------
 src/launcher/executor.cpp          |   5 +-
 src/scripts/killtree.sh            | 245 --------------------------------
 src/slave/process_isolator.cpp     |   3 +-
 src/tests/environment.cpp          |   9 --
 src/tests/environment.hpp          |   1 -
 src/tests/killtree_test.sh         |  10 --
 src/tests/killtree_tests.cpp       |  24 ----
 src/tests/mesos.cpp                |   9 --
 src/tests/mesos.hpp                |   1 -
 src/tests/slave_recovery_tests.cpp |   3 +-
 13 files changed, 6 insertions(+), 388 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/56c20b61/bin/mesos-build-env.sh.in
----------------------------------------------------------------------
diff --git a/bin/mesos-build-env.sh.in b/bin/mesos-build-env.sh.in
index 31bccad..f2bbe42 100644
--- a/bin/mesos-build-env.sh.in
+++ b/bin/mesos-build-env.sh.in
@@ -21,8 +21,6 @@
 
 MESOS_WEBUI_DIR=@abs_top_srcdir@/src/webui
 MESOS_LAUNCHER_DIR=@abs_top_builddir@/src
-MESOS_KILLTREE=@abs_top_srcdir@/src/scripts/killtree.sh
 
 export MESOS_WEBUI_DIR
 export MESOS_LAUNCHER_DIR
-export MESOS_KILLTREE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/56c20b61/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 6491f56..9337435 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -214,7 +214,7 @@ endif
 libmesos_no_3rdparty_la_SOURCES += common/attributes.hpp		\
 	common/build.hpp common/date_utils.hpp common/factory.hpp	\
 	common/protobuf_utils.hpp					\
-	common/lock.hpp common/resources.hpp common/process_utils.hpp	\
+	common/lock.hpp common/resources.hpp				\
 	common/type_utils.hpp common/thread.hpp common/units.hpp	\
 	common/values.hpp						\
 	detector/detector.hpp examples/utils.hpp files/files.hpp	\
@@ -458,10 +458,6 @@ nobase_dist_webui_DATA +=							  \
   webui/master/static/bootstrap/js/README.md
 
 
-# And need to install the killtree script.
-dist_pkglibexec_SCRIPTS += scripts/killtree.sh
-
-
 # And the deploy related stuff.
 nodist_sbin_SCRIPTS += deploy/mesos-daemon.sh			\
   deploy/mesos-start-cluster.sh deploy/mesos-start-masters.sh	\
@@ -804,7 +800,6 @@ mesos_tests_SOURCES = tests/main.cpp tests/utils.cpp			\
 	              tests/examples_tests.cpp				\
 	              tests/protobuf_io_tests.cpp			\
 	              tests/zookeeper_url_tests.cpp			\
-	              tests/killtree_tests.cpp				\
 	              tests/exception_tests.cpp				\
 	              tests/attributes_tests.cpp			\
 	              tests/master_detector_tests.cpp			\
@@ -869,8 +864,7 @@ dist_check_SCRIPTS +=				\
   tests/no_executor_framework_test.sh		\
   tests/java_exception_test.sh			\
   tests/java_framework_test.sh			\
-  tests/python_framework_test.sh		\
-  tests/killtree_test.sh
+  tests/python_framework_test.sh
 
 # We use a check-local target for now to avoid the parallel test
 # runner that ships with newer versions of autotools.

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/56c20b61/src/common/process_utils.hpp
----------------------------------------------------------------------
diff --git a/src/common/process_utils.hpp b/src/common/process_utils.hpp
deleted file mode 100644
index a04f4a2..0000000
--- a/src/common/process_utils.hpp
+++ /dev/null
@@ -1,72 +0,0 @@
-/**
- * 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 __PROCESS_UTILS_HPP__
-#define __PROCESS_UTILS_HPP__
-
-#include <iostream>
-#include <sstream>
-
-#include <stout/os.hpp>
-#include <stout/strings.hpp>
-
-namespace mesos {
-namespace internal {
-namespace utils {
-namespace process {
-
-inline Try<int> killtree(
-    pid_t pid,
-    int signal,
-    bool killgroups,
-    bool killsess,
-    bool verbose)
-{
-  std::string cmdline;
-
-  // TODO(Charles Reiss): Use a configuration option.
-  if (os::hasenv("MESOS_KILLTREE")) {
-    // Set by mesos-build-env.sh.
-    cmdline = os::getenv("MESOS_KILLTREE");
-  } else if (os::hasenv("MESOS_SOURCE_DIR")) {
-    // Set by test harness for external tests.
-    cmdline = os::getenv("MESOS_SOURCE_DIR") +
-      "/src/scripts/killtree.sh";
-  } else {
-    cmdline = MESOS_LIBEXECDIR "/killtree.sh";
-  }
-
-  // Add the arguments.
-  Try<std::string> args = strings::format(" -p %d -s %d", pid, signal);
-  CHECK(!args.isError()) << args.error();
-  cmdline += args.get();
-
-  // Also add flags to kill all encountered groups and sessions.
-  if (killgroups) cmdline += " -g";
-  if (killsess) cmdline += " -x";
-  if (verbose) cmdline += " -v";
-
-  return os::shell(verbose ? &LOG(INFO) : NULL, cmdline);
-}
-
-} // namespace mesos {
-} // namespace internal {
-} // namespace utils {
-} // namespace process {
-
-#endif // __PROCESS_UTILS_HPP__

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/56c20b61/src/launcher/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.cpp b/src/launcher/executor.cpp
index 1a6725a..ef42b60 100644
--- a/src/launcher/executor.cpp
+++ b/src/launcher/executor.cpp
@@ -30,7 +30,6 @@
 #include <stout/os.hpp>
 #include <stout/strings.hpp>
 
-#include "common/process_utils.hpp"
 #include "common/thread.hpp"
 
 #include "logging/logging.hpp"
@@ -223,7 +222,7 @@ public:
   {
     // TODO(benh): Do kill escalation (i.e., after n seconds, kill -9).
     if (pid > 0) {
-      utils::process::killtree(pid, SIGTERM, true, true, true);
+      os::killtree(pid, SIGTERM, true, true, &std::cerr);
     }
   }
 
@@ -233,7 +232,7 @@ public:
   {
     // TODO(benh): Do kill escalation (i.e., after n seconds, kill -9).
     if (pid > 0) {
-      utils::process::killtree(pid, SIGTERM, true, true, true);
+      os::killtree(pid, SIGTERM, true, true, &std::cerr);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/56c20b61/src/scripts/killtree.sh
----------------------------------------------------------------------
diff --git a/src/scripts/killtree.sh b/src/scripts/killtree.sh
deleted file mode 100755
index c426654..0000000
--- a/src/scripts/killtree.sh
+++ /dev/null
@@ -1,245 +0,0 @@
-#!/bin/bash
-
-# We unset the DYLD_LIBRARY_PATH environment variable because the
-# dynamic linker on OSX 10.8 complains if this path is set when
-# running setuid or setgid programs, e.g., /bin/ps used in this
-# script.
-unset DYLD_LIBRARY_PATH
-
-PID=
-SIGNAL="TERM"
-KILLGROUPS=0
-KILLSESS=0
-VERBOSE=0
-
-
-usage() {
-    cat <<EOF
-usage: $0 -p pid -s signal
-
-Sends a signal to a process tree rooted at the specified pid.
-
-Options:
--h      Display this message.
--p      Root pid of tree.
--s      Signal to send (number or symbolic name), default is TERM
--g      Recursively invoke on all processes that are members
-        of any encountered process group.
--x      Same as -g, but invoke on processes of encountered sessions.
--v      Be more verbose.
-EOF
-}
-
-set_union() {
-    A="$1"
-    B="$2"
-
-    result=$(printf "${A}\n${B}\n" | sort | uniq)
-    echo "${result}"
-}
-
-set_intersect() {
-    A="$1"
-    B="$2"
-
-    result=$(printf "${A}\n${B}" \
-        | sort | uniq -c | awk '{ if ($1 == 2) print $2 }')
-
-    echo "${result}"
-}
-
-set_diff() {
-    A="$1"
-    B="$2"
-
-    result=$(printf "${A}\n${B}" \
-        | sort | uniq -c | awk '{ if ($1 == 1) print $2 }')
-
-    echo "${result}"
-}
-
-while getopts "hp:s:gxv" OPTION
-do
-    HAVEOPT=1
-    case ${OPTION} in
-        h)
-            usage
-            exit 0
-            ;;
-        p)
-            PID="${OPTARG}"
-            test ${PID} -eq ${PID} >& /dev/null
-            if [[ ${?} != 0 ]]; then
-              echo "$(basename ${0}): pid should be a number"
-              exit 1
-            fi
-            if [[ ! ${PID} -eq ${PID} ]]; then
-                exit 1
-            fi
-            ;;
-        s)
-            SIGNAL="${OPTARG}"
-            if [[ -z ${SIGNAL} ]]; then
-              SIGNAL="TERM"
-            fi
-            ;;
-        g)
-            KILLGROUPS=1
-            ;;
-	x)
-	    KILLSESSIONS=1
-	    ;;
-        v)
-            VERBOSE=1
-            ;;
-        *)  usage
-            exit 1
-            ;;
-    esac
-done
-
-if [[ ! ${HAVEOPT} ]]; then
-    usage
-    exit 127
-fi
-
-if [[ -z ${PID} ]]; then
-    echo "$(basename ${0}): must specify pid"
-    exit 1
-fi
-
-# Confirm we get some output from ps.
-ps axo ppid,pid >/dev/null 2>&1
-if [[ ${?} -ne 0 ]]; then
-    echo "$(basename ${0}): failed to capture process tree using 'ps'"
-    exit 1
-fi
-
-# Confirm we have awk.
-ls | awk {'print $1'} >/dev/null 2>&1
-if [[ ${?} -ne 0 ]]; then
-    echo "$(basename ${0}): failed to detect awk on the system"
-    exit 1
-fi
-
-printpids() {
-  name="${1}"
-  tpids="${2}"
-  if [[ ! -z ${tpids} ]]; then
-      result=$(echo ${tpids} | xargs ps o pid,ppid,pgid,sess,command -p)
-      printf "\n${name}\n${result}\n"
-  fi
-}
-
-PIDS=${PID} # Already processed pids, global variable hack.
-
-killtree() {
-    MYPID=${1}
-
-    # Output pid if requested.
-    if [[ ${VERBOSE} -eq 1 ]]; then
-        echo "Stopping ... ${MYPID}"
-    fi
-
-    # Stop the process to keep it from forking while we are killing it
-    # since a forked child might get re-parented by init and become
-    # impossible to find.
-    kill -STOP ${MYPID} >/dev/null 2>&1
-
-    # There is a concern that even though some process is stopped,
-    # sending a signal to any of it's children may cause a SIGCLD to
-    # be delivered to it which wakes it up (or any other signal maybe
-    # delivered). However, from the Open Group standards on "Signal
-    # Concepts":
-    #
-    #   "While a process is stopped, any additional signals that are
-    #    sent to the process shall not be delivered until the process
-    #    is continued, except SIGKILL which always terminates the
-    #    receiving process."
-    #
-    # In practice, this is not what has been witnessed. Rather, a
-    # process that has been stopped will respond to SIGTERM, SIGINT,
-    # etc. That being said, we still continue the process below in the
-    # event that it doesn't terminate from the sending signal but it
-    # also doesn't get continued (as per the specifications above).
-
-    # Now collect all the children.
-    CHILDPIDS=$(ps axo ppid,pid | awk '{ if ($1 == '${MYPID}') print $2 }')
-
-    if [[ ${VERBOSE} -eq 1 ]]; then
-        printpids "Children:" "${CHILDPIDS}"
-    fi
-
-    # Optionally collect all processes that are part of the process group.
-    GROUPPIDS=""
-    if [[ ${KILLGROUPS} -eq 1 ]]; then
-
-	# First get the process group.
-        MYPGID=$(ps axo pid,pgid | awk '{ if ($1 == '${MYPID}') print $2 }')
-
-	if [[ ! -z ${MYPGID} ]]; then
-            # Now get all members.
-            GROUPPIDS=$(ps axo pgid,pid \
-                | awk '{ if ($1 == '${MYPGID}') print $2 }')
-
-	    if [[ ${VERBOSE} -eq 1 ]]; then
-	        printpids "Group members:" "${GROUPPIDS}"
-	    fi
-	fi
-    fi
-
-    # Optionally collect all processes that are part of the same session.
-    SESSPIDS=""
-    if [[ ${KILLSESSIONS} -eq 1 ]]; then
-
-	# First get the process session id.
-        MYPSID=$(ps axo pid,sess | awk '{ if ($1 == '${MYPID}') print $2 }')
-
-	# We check for session id not being "0", because on OSX 10.8
-	# the session id of all processes as reported by 'ps' is "0"!
-	if [[ ${MYPSID} -ne "0" && ! -z ${MYPSID} ]]; then
-            # Now get all members.
-            SESSPIDS=$(ps axo sess,pid \
-                | awk '{ if ($1 == "'${MYPSID}'") print $2 }')
-
-	    if [[ ${VERBOSE} -eq 1 ]]; then
-                printpids "Session members:" "${SESSPIDS}"
-	    fi
-	fi
-    fi
-
-    # Get out only the unseen pids.
-    # NEW = CHILDPIDS U GROUPPIDS U SESSPIDS
-    #NEW=$(printf "${CHILDPIDS}\n${GROUPPIDS}\n${SESSPIDS}" | sort | uniq)
-    NEW=$(set_union "$(set_union "${CHILDPIDS}" "${GROUPPIDS}")" "${SESSPIDS}")
-
-    # OLD = NEW ^ PIDS
-    #OLD=$(printf "${PIDS}\n${NEW}" \
-    #    | sort | uniq -c | awk '{ if ($1 == 2) print $2 }')
-    OLD=$(set_intersect "${PIDS}" "${NEW}")
-
-    # NEW = NEW - OLD
-    #NEW=$(printf "${NEW}\n${OLD}" \
-    #    | sort | uniq -c | awk '{ if ($1 == 1) print $2 }')
-    NEW=$(set_diff "${NEW}" "${OLD}")
-
-    # Add all the new pids.
-    PIDS=$(printf "${NEW}\n${PIDS}" | sort)
-
-    # Now send the signal.
-    kill -${SIGNAL} ${MYPID} >/dev/null 2>&1
-
-    # Try and continue the process in case ${SIGNAL} is
-    # non-terminating but doesn't continue the process.
-    kill -CONT ${MYPID} >/dev/null 2>&1
-
-    if [[ ${VERBOSE} -eq 1 ]]; then
-        echo "Sent signal to ${MYPID}"
-    fi
-
-    for pid in ${NEW}; do
-        killtree $pid
-    done
-}
-
-killtree ${PID}

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/56c20b61/src/slave/process_isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/process_isolator.cpp b/src/slave/process_isolator.cpp
index 83bd8da..97a017e 100644
--- a/src/slave/process_isolator.cpp
+++ b/src/slave/process_isolator.cpp
@@ -37,7 +37,6 @@
 #include <stout/uuid.hpp>
 
 #include "common/type_utils.hpp"
-#include "common/process_utils.hpp"
 
 #include "slave/flags.hpp"
 #include "slave/process_isolator.hpp"
@@ -265,7 +264,7 @@ void ProcessIsolator::killExecutor(
     // TODO(vinod): Call killtree on the pid of the actual executor process
     // that is running the tasks (stored in the local storage by the
     // executor module).
-    utils::process::killtree(pid.get(), SIGKILL, true, true, true);
+    os::killtree(pid.get(), SIGKILL, true, true, &LOG(INFO));
 
     // Also kill all processes that belong to the process group of the executor.
     // This is valuable in situations where the top level executor process

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/56c20b61/src/tests/environment.cpp
----------------------------------------------------------------------
diff --git a/src/tests/environment.cpp b/src/tests/environment.cpp
index 54649a5..2bdec1f 100644
--- a/src/tests/environment.cpp
+++ b/src/tests/environment.cpp
@@ -217,15 +217,6 @@ void Environment::SetUp()
   if (!GTEST_IS_THREADSAFE) {
     EXIT(1) << "Testing environment is not thread safe, bailing!";
   }
-
-  // For locating killtree.sh.
-  os::setenv("MESOS_SOURCE_DIR", tests::flags.source_dir);
-}
-
-
-void Environment::TearDown()
-{
-  os::unsetenv("MESOS_SOURCE_DIR");
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/56c20b61/src/tests/environment.hpp
----------------------------------------------------------------------
diff --git a/src/tests/environment.hpp b/src/tests/environment.hpp
index 691291f..9acbe2f 100644
--- a/src/tests/environment.hpp
+++ b/src/tests/environment.hpp
@@ -37,7 +37,6 @@ public:
   virtual ~Environment();
 
   virtual void SetUp();
-  virtual void TearDown();
 
   // Helper to create a temporary directory based on the current test
   // case name and test name (derived from TestInfo via

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/56c20b61/src/tests/killtree_test.sh
----------------------------------------------------------------------
diff --git a/src/tests/killtree_test.sh b/src/tests/killtree_test.sh
deleted file mode 100755
index 3962954..0000000
--- a/src/tests/killtree_test.sh
+++ /dev/null
@@ -1,10 +0,0 @@
-#!/bin/bash
-
-# Fork the proces that spawns processes in a chain
-$MESOS_HOME/tests/process-spawn &
-
-#Now get the pid of the above process
-
-pid=`ps -e | grep process-spawn | awk '{print $1}'`
-
-echo pid

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/56c20b61/src/tests/killtree_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/killtree_tests.cpp b/src/tests/killtree_tests.cpp
deleted file mode 100644
index 753c5c0..0000000
--- a/src/tests/killtree_tests.cpp
+++ /dev/null
@@ -1,24 +0,0 @@
-/**
- * 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 <gtest/gtest.h>
-
-#include "tests/script.hpp"
-
-
-TEST_SCRIPT(KillTreeTest, KillTree, "killtree_test.sh")

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/56c20b61/src/tests/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index 6dbf7f3..776cb0f 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -149,17 +149,8 @@ void MesosTest::ShutdownSlaves()
 }
 
 
-void MesosTest::SetUp()
-{
-  // For locating killtree.sh.
-  os::setenv("MESOS_SOURCE_DIR", tests::flags.source_dir);
-}
-
-
 void MesosTest::TearDown()
 {
-  os::unsetenv("MESOS_SOURCE_DIR");
-
   // TODO(benh): Fail the test if shutdown hasn't been called?
   Shutdown();
 }

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/56c20b61/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index fca41aa..891cc78 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -67,7 +67,6 @@ class MesosTest : public ::testing::Test
 protected:
   MesosTest(const Option<zookeeper::URL>& url = None());
 
-  virtual void SetUp();
   virtual void TearDown();
 
   // Returns the flags used to create masters.

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/56c20b61/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index 0315c52..0195e68 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -31,10 +31,10 @@
 #include <stout/none.hpp>
 #include <stout/numify.hpp>
 #include <stout/option.hpp>
+#include <stout/os.hpp>
 #include <stout/path.hpp>
 #include <stout/uuid.hpp>
 
-#include "common/process_utils.hpp"
 #include "common/protobuf_utils.hpp"
 #include "common/resources.hpp"
 
@@ -64,7 +64,6 @@ using namespace mesos;
 using namespace mesos::internal;
 using namespace mesos::internal::slave;
 using namespace mesos::internal::tests;
-using namespace mesos::internal::utils::process;
 
 using namespace process;
 


[4/6] git commit: Added a zombie bit to os::Process, improved os::alive to handle zombie processes.

Posted by bm...@apache.org.
Added a zombie bit to os::Process, improved os::alive to handle
zombie processes.

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


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

Branch: refs/heads/master
Commit: a79526fb3104610038c3f7abc78348d8d4516e4e
Parents: 5faf700
Author: Benjamin Mahler <bm...@twitter.com>
Authored: Wed Jun 12 12:04:00 2013 -0700
Committer: Benjamin Mahler <bm...@twitter.com>
Committed: Wed Jun 19 20:29:37 2013 -0700

----------------------------------------------------------------------
 .../3rdparty/stout/include/stout/os.hpp         | 24 ++++++++++++++------
 .../3rdparty/stout/include/stout/os/linux.hpp   |  3 ++-
 .../3rdparty/stout/include/stout/os/osx.hpp     |  6 +++--
 .../3rdparty/stout/include/stout/os/process.hpp |  7 ++++--
 4 files changed, 28 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/a79526fb/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
index 47e877c..af47e18 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
@@ -1111,17 +1111,27 @@ inline Try<std::set<pid_t> > pids(Option<pid_t> group, Option<pid_t> session)
 
 inline Try<bool> alive(pid_t pid)
 {
-  CHECK(pid > 0);
-
-  if (::kill(pid, 0) == 0) {
-    return true;
+  if (pid <= 0) {
+    return Error("Invalid pid");
   }
 
-  if (errno == ESRCH) {
-    return false;
+  const Try<os::Process>& process = os::process(pid);
+
+  // When we can't get the process information, fall back
+  // to using kill.
+  if (process.isError()) {
+    if (::kill(pid, 0) == 0) {
+      return true;
+    }
+
+    if (errno == ESRCH) {
+      return false;
+    }
+
+    return Try<bool>::error(strerror(errno));
   }
 
-  return Try<bool>::error(strerror(errno));
+  return !process.get().zombie;
 }
 
 } // namespace os {

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/a79526fb/3rdparty/libprocess/3rdparty/stout/include/stout/os/linux.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/linux.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/linux.hpp
index 6fb9256..632f95f 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/linux.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/linux.hpp
@@ -49,7 +49,8 @@ inline Try<Process> process(pid_t pid)
                  Bytes(status.get().rss * pageSize),
                  Duration::create(status.get().utime / (double) ticks).get(),
                  Duration::create(status.get().stime / (double) ticks).get(),
-                 status.get().comm);
+                 status.get().comm,
+                 status.get().state == 'Z');
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/a79526fb/3rdparty/libprocess/3rdparty/stout/include/stout/os/osx.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/osx.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/osx.hpp
index c27e7a2..f2d241a 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/osx.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/osx.hpp
@@ -52,7 +52,8 @@ inline Try<Process> process(pid_t pid)
                    Bytes(0),
                    Nanoseconds(-1),
                    Nanoseconds(-1),
-                   process.kp_proc.p_comm);
+                   process.kp_proc.p_comm,
+                   process.kp_proc.p_stat & SZOMB);
   } else {
     return Process(process.kp_proc.p_pid,
                    process.kp_eproc.e_ppid,
@@ -61,7 +62,8 @@ inline Try<Process> process(pid_t pid)
                    Bytes(task.pti_resident_size),
                    Nanoseconds(task.pti_total_user),
                    Nanoseconds(task.pti_total_system),
-                   process.kp_proc.p_comm);
+                   process.kp_proc.p_comm,
+                   process.kp_proc.p_stat & SZOMB);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-mesos/blob/a79526fb/3rdparty/libprocess/3rdparty/stout/include/stout/os/process.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/process.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/process.hpp
index a88118a..806949b 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/process.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/process.hpp
@@ -19,7 +19,8 @@ struct Process
           const Bytes& _rss,
           const Duration& _utime,
           const Duration& _stime,
-          const std::string& _command)
+          const std::string& _command,
+          bool _zombie)
     : pid(_pid),
       parent(_parent),
       group(_group),
@@ -27,7 +28,8 @@ struct Process
       rss(_rss),
       utime(_utime),
       stime(_stime),
-      command(_command) {}
+      command(_command),
+      zombie(_zombie) {}
 
   const pid_t pid;
   const pid_t parent;
@@ -37,6 +39,7 @@ struct Process
   const Duration utime;
   const Duration stime;
   const std::string command;
+  const bool zombie;
 
   // TODO(bmahler): Add additional data as needed.