You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by mp...@apache.org on 2016/04/22 07:30:31 UTC

[1/6] mesos git commit: Implemented `os::processes` on Windows in stout.

Repository: mesos
Updated Branches:
  refs/heads/master f72d2e200 -> fa7ca953c


Implemented `os::processes` on Windows in stout.

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


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

Branch: refs/heads/master
Commit: d534d64def535179ff7db06570283184278c8a4c
Parents: f72d2e2
Author: Alex Clemmer <cl...@gmail.com>
Authored: Thu Apr 21 11:51:25 2016 -0700
Committer: Michael Park <mp...@apache.org>
Committed: Thu Apr 21 22:25:23 2016 -0700

----------------------------------------------------------------------
 .../3rdparty/stout/include/stout/os.hpp         |  75 ++++++++++
 .../3rdparty/stout/include/stout/posix/os.hpp   |  76 +---------
 .../3rdparty/stout/include/stout/windows/os.hpp | 147 ++++++++++++++++++-
 3 files changed, 217 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d534d64d/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 a810868..c9576be 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
@@ -147,6 +147,81 @@ inline void appendPaths(const std::string& newPaths)
 }
 
 } // namespace libraries {
+
+
+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 Result<Process> process = os::process(pid);
+
+    // Ignore any processes that disappear between enumeration and now.
+    if (process.isSome()) {
+      result.push_back(process.get());
+    }
+  }
+  return result;
+}
+
+
+inline Option<Process> process(
+    pid_t pid,
+    const std::list<Process>& processes)
+{
+  foreach (const Process& process, processes) {
+    if (process.pid == pid) {
+      return process;
+    }
+  }
+  return None();
+}
+
+
+inline std::set<pid_t> children(
+    pid_t pid,
+    const std::list<Process>& processes,
+    bool recursive = true)
+{
+  // 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) {
+      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<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());
+  }
+
+  return children(pid, processes.get(), recursive);
+}
+
 } // namespace os {
 
 #endif // __STOUT_OS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/d534d64d/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
index f1584d0..e03be83 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
@@ -95,6 +95,7 @@ using ::hstrerror;
 
 // Forward declarations.
 inline Try<Nothing> utime(const std::string&);
+inline Try<std::list<Process>> processes();
 
 // Sets the value associated with the specified key in the set of
 // environment variables.
@@ -257,27 +258,6 @@ inline Try<UTSInfo> uname()
 }
 
 
-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 Result<Process> process = os::process(pid);
-
-    // Ignore any processes that disappear.
-    if (process.isSome()) {
-      result.push_back(process.get());
-    }
-  }
-  return result;
-}
-
-
 // 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.
@@ -410,60 +390,6 @@ inline Try<Version> release()
 }
 
 
-inline Option<Process> process(
-    pid_t pid,
-    const std::list<Process>& processes)
-{
-  foreach (const Process& process, processes) {
-    if (process.pid == pid) {
-      return process;
-    }
-  }
-  return None();
-}
-
-
-inline std::set<pid_t> children(
-    pid_t pid,
-    const std::list<Process>& processes,
-    bool recursive = true)
-{
-  // 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) {
-      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<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());
-  }
-
-  return children(pid, processes.get(), recursive);
-}
-
-
 inline Option<std::string> which(const std::string& command)
 {
   Option<std::string> path = getenv("PATH");

http://git-wip-us.apache.org/repos/asf/mesos/blob/d534d64d/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
index 37ef332..ad29093 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
@@ -15,6 +15,8 @@
 
 #include <direct.h>
 #include <io.h>
+#include <TlHelp32.h>
+#include <Psapi.h>
 
 #include <sys/utime.h>
 
@@ -63,7 +65,7 @@ inline Try<std::set<pid_t>> pids(Option<pid_t> group, Option<pid_t> session)
                                   &bytes_returned);
 
     if (!result) {
-      return WindowsError("`os::pids()`: Failed to call `EnumProcesses`");
+      return WindowsError("os::pids: Call to `EnumProcesses` failed");
     }
 
     max_items *= 2;
@@ -221,7 +223,7 @@ inline Try<Memory> memory()
   MEMORYSTATUSEX memory_status;
   memory_status.dwLength = sizeof(MEMORYSTATUSEX);
   if (!::GlobalMemoryStatusEx(&memory_status)) {
-    return WindowsError("memory(): Could not call GlobalMemoryStatusEx");
+    return WindowsError("os::memory: Call to `GlobalMemoryStatusEx` failed");
   }
 
   memory.total = Bytes(memory_status.ullTotalPhys);
@@ -237,9 +239,6 @@ inline Try<Memory> memory()
 inline Try<UTSInfo> uname() = delete;
 
 
-inline Try<std::list<Process>> processes() = delete;
-
-
 // Looks in the environment variables for the specified key and
 // returns a string representation of its value. If no environment
 // variable matching key is found, None() is returned.
@@ -275,12 +274,148 @@ inline tm* gmtime_r(const time_t* timep, tm* result)
 inline Try<bool> access(const std::string& fileName, int how)
 {
   if (::_access(fileName.c_str(), how) != 0) {
-    return ErrnoError("access: Could not access path '" + fileName + "'");
+    return ErrnoError("os::access: Call to `_access` failed for path '" +
+                      fileName + "'");
   }
 
   return true;
 }
 
+inline Result<PROCESSENTRY32> process_entry(pid_t pid)
+{
+  // Get a snapshot of the processes in the system. NOTE: We should not check
+  // whether the handle is `NULL`, because this API will always return
+  // `INVALID_HANDLE_VALUE` on error.
+  HANDLE snapshot_handle = CreateToolhelp32Snapshot(TH32CS_SNAPPROCESS, pid);
+  if (snapshot_handle == INVALID_HANDLE_VALUE) {
+    return WindowsError(
+        "os::process_entry: Call to `CreateToolhelp32Snapshot` failed");
+  }
+
+  SharedHandle safe_snapshot_handle(snapshot_handle, ::CloseHandle);
+
+  // Initialize process entry.
+  PROCESSENTRY32 process_entry;
+  ZeroMemory(&process_entry, sizeof(PROCESSENTRY32));
+  process_entry.dwSize = sizeof(PROCESSENTRY32);
+
+  // Get first process so that we can loop through process entries until we
+  // find the one we care about.
+  SetLastError(ERROR_SUCCESS);
+  bool has_next = Process32First(safe_snapshot_handle.get(), &process_entry);
+  if (!has_next) {
+    // No first process was found. We should never be here; it is arguable we
+    // should return `None`, since we won't find the PID we're looking for, but
+    // we elect to return `Error` because something terrible has probably
+    // happened.
+    if (GetLastError() != ERROR_SUCCESS) {
+      return WindowsError("os::process_entry: Call to `Process32First` failed");
+    } else {
+      return Error("os::process_entry: Call to `Process32First` failed");
+    }
+  }
+
+  // Loop through processes until we find the one we're looking for.
+  while (has_next) {
+    if (process_entry.th32ProcessID == pid) {
+      // Process found.
+      return process_entry;
+    }
+
+    has_next = Process32Next(safe_snapshot_handle.get(), &process_entry);
+    if (!has_next) {
+      DWORD last_error = GetLastError();
+      if (last_error != ERROR_NO_MORE_FILES && last_error != ERROR_SUCCESS) {
+        return WindowsError(
+            "os::process_entry: Call to `Process32Next` failed");
+      }
+    }
+  }
+
+  return None();
+}
+
+
+// Generate a `Process` object for the process associated with `pid`. If
+// process is not found, we return `None`; error is reserved for the case where
+// something went wrong.
+inline Result<Process> process(pid_t pid)
+{
+  // Find process with pid.
+  Result<PROCESSENTRY32> entry = process_entry(pid);
+
+  if (entry.isError()) {
+    return WindowsError(entry.error());
+  } else if (entry.isNone()) {
+    return None();
+  }
+
+  HANDLE process_handle = ::OpenProcess(
+      PROCESS_QUERY_LIMITED_INFORMATION | PROCESS_VM_READ,
+      false,
+      pid);
+
+  if (process_handle == INVALID_HANDLE_VALUE) {
+    return WindowsError("os::process: Call to `OpenProcess` failed");
+  }
+
+  SharedHandle safe_process_handle(process_handle, ::CloseHandle);
+
+  // Get Windows Working set size (Resident set size in linux).
+  PROCESS_MEMORY_COUNTERS proc_mem_counters;
+  BOOL get_process_memory_info = ::GetProcessMemoryInfo(
+      safe_process_handle.get(),
+      &proc_mem_counters,
+      sizeof(proc_mem_counters));
+
+  if (!get_process_memory_info) {
+    return WindowsError("os::process: Call to `GetProcessMemoryInfo` failed");
+  }
+
+  // Get session Id.
+  pid_t session_id;
+  BOOL process_id_to_session_id = ::ProcessIdToSessionId(pid, &session_id);
+
+  if (!process_id_to_session_id) {
+    return WindowsError("os::process: Call to `ProcessIdToSessionId` failed");
+  }
+
+  // Get Process CPU time.
+  FILETIME create_filetime, exit_filetime, kernel_filetime, user_filetime;
+  BOOL get_process_times = ::GetProcessTimes(
+      safe_process_handle.get(),
+      &create_filetime,
+      &exit_filetime,
+      &kernel_filetime,
+      &user_filetime);
+
+  if (!get_process_times) {
+    return WindowsError("os::process: Call to `GetProcessTimes` failed");
+  }
+
+  // Get utime and stime.
+  ULARGE_INTEGER lKernelTime, lUserTime; // In 100 nanoseconds.
+  lKernelTime.HighPart = kernel_filetime.dwHighDateTime;
+  lKernelTime.LowPart = kernel_filetime.dwLowDateTime;
+  lUserTime.HighPart = user_filetime.dwHighDateTime;
+  lUserTime.LowPart = user_filetime.dwLowDateTime;
+
+  Try<Duration> utime = Nanoseconds(lKernelTime.QuadPart * 100);
+  Try<Duration> stime = Nanoseconds(lUserTime.QuadPart * 100);
+
+  return Process(
+      pid,
+      entry.get().th32ParentProcessID,         // Parent process id.
+      0,                                       // Group id.
+      session_id,
+      Bytes(proc_mem_counters.WorkingSetSize),
+      utime.isSome() ? utime.get() : Option<Duration>::none(),
+      stime.isSome() ? stime.get() : Option<Duration>::none(),
+      entry.get().szExeFile,                   // Executable filename.
+      false);                                  // Is not zombie process.
+}
+
+
 } // namespace os {
 
 #endif // __STOUT_WINDOWS_OS_HPP__


[2/6] mesos git commit: Moved process tests to their own file to stout.

Posted by mp...@apache.org.
Moved process tests to their own file to stout.

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


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

Branch: refs/heads/master
Commit: f79b8bbdc38ab7afafbf37eb63b87497650e12d9
Parents: d534d64
Author: Alex Clemmer <cl...@gmail.com>
Authored: Thu Apr 21 11:51:28 2016 -0700
Committer: Michael Park <mp...@apache.org>
Committed: Thu Apr 21 22:26:12 2016 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/stout/Makefile.am  |   1 +
 .../3rdparty/stout/tests/CMakeLists.txt         |   1 +
 .../3rdparty/stout/tests/os/process_tests.cpp   | 183 +++++++++++++++++++
 .../3rdparty/stout/tests/os_tests.cpp           |  93 ----------
 4 files changed, 185 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f79b8bbd/3rdparty/libprocess/3rdparty/stout/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/Makefile.am b/3rdparty/libprocess/3rdparty/stout/Makefile.am
index 10be366..ad8731d 100644
--- a/3rdparty/libprocess/3rdparty/stout/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/stout/Makefile.am
@@ -47,6 +47,7 @@ EXTRA_DIST =					\
   tests/os_tests.cpp				\
   tests/os/env_tests.cpp			\
   tests/os/filesystem_tests.cpp			\
+  tests/os/process_tests.cpp			\
   tests/os/rmdir_tests.cpp			\
   tests/os/sendfile_tests.cpp			\
   tests/os/signals_tests.cpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/f79b8bbd/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt b/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt
index c9df907..b6de810 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt
+++ b/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt
@@ -40,6 +40,7 @@ set(STOUT_TESTS_SRC
   option_tests.cpp
   os/env_tests.cpp
   os/filesystem_tests.cpp
+  os/process_tests.cpp
   os/rmdir_tests.cpp
   os/strerror_tests.cpp
   protobuf_tests.pb.h

http://git-wip-us.apache.org/repos/asf/mesos/blob/f79b8bbd/3rdparty/libprocess/3rdparty/stout/tests/os/process_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/os/process_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/os/process_tests.cpp
new file mode 100644
index 0000000..5ba02ff
--- /dev/null
+++ b/3rdparty/libprocess/3rdparty/stout/tests/os/process_tests.cpp
@@ -0,0 +1,183 @@
+// Licensed 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
+
+#ifdef __WINDOWS__
+#include <process.h>
+#endif // __WINDOWS__
+
+#include <set>
+
+#include <gtest/gtest.h>
+
+#include <stout/os.hpp>
+
+#include <stout/tests/utils.hpp>
+
+
+class ProcessTest : public TemporaryDirectoryTest {};
+
+using os::Process;
+
+using std::list;
+using std::set;
+using std::string;
+
+
+const unsigned int init_pid =
+#ifdef __WINDOWS__
+    0;
+#else
+    1;
+#endif // __WINDOWS__
+
+
+#ifdef __WINDOWS__
+int getppid()
+{
+  const int pid = getpid();
+  HANDLE h = CreateToolhelp32Snapshot(TH32CS_SNAPPROCESS, 0);
+  std::shared_ptr<void> sh(h, CloseHandle);
+
+  PROCESSENTRY32 pe = { 0 };
+  pe.dwSize = sizeof(PROCESSENTRY32);
+
+  if (Process32First(h, &pe)) {
+    do {
+      if (pe.th32ProcessID == pid) {
+        return pe.th32ParentProcessID;
+      }
+    } while (Process32Next(h, &pe));
+  }
+}
+#endif // __WINDOWS__
+
+TEST_F(ProcessTest, Process)
+{
+  const Result<Process> process = os::process(getpid());
+
+  ASSERT_SOME(process);
+  EXPECT_EQ(getpid(), process.get().pid);
+  EXPECT_EQ(getppid(), process.get().parent);
+  ASSERT_SOME(process.get().session);
+
+#ifndef __WINDOWS__
+  // NOTE: `getsid` does not have a meaningful interpretation on Windows.
+  EXPECT_EQ(getsid(getpid()), process.get().session.get());
+#endif // __WINDOWS__
+
+  ASSERT_SOME(process.get().rss);
+  EXPECT_GT(process.get().rss.get(), 0);
+
+  // NOTE: On Linux /proc is a bit slow to update the CPU times,
+  // hence we allow 0 in this test.
+  ASSERT_SOME(process.get().utime);
+  EXPECT_GE(process.get().utime.get(), Nanoseconds(0));
+  ASSERT_SOME(process.get().stime);
+  EXPECT_GE(process.get().stime.get(), Nanoseconds(0));
+
+  EXPECT_FALSE(process.get().command.empty());
+
+  // Assert invalid PID returns `None`.
+  Result<Process> invalid_process = os::process(-1);
+  EXPECT_NONE(invalid_process);
+
+  // Assert init.
+  Result<Process> init_process = os::process(init_pid);
+#ifdef __WINDOWS__
+  // NOTE: On Windows, inspecting other processes usually requires privileges.
+  // So we expect it to error out instead of succeed, unlike the POSIX version.
+  EXPECT_ERROR(init_process);
+#else
+  EXPECT_SOME(init_process);
+#endif // __WINDOWS__
+}
+
+
+TEST_F(ProcessTest, Processes)
+{
+  const Try<list<Process>> processes = os::processes();
+
+  ASSERT_SOME(processes);
+  ASSERT_GT(processes.get().size(), 2);
+
+  // Look for ourselves in the table.
+  bool found = false;
+  foreach (const Process& process, processes.get()) {
+    if (process.pid == getpid()) {
+      found = true;
+      EXPECT_EQ(getpid(), process.pid);
+      EXPECT_EQ(getppid(), process.parent);
+      ASSERT_SOME(process.session);
+
+#ifndef __WINDOWS__
+      // NOTE: `getsid` does not have a meaningful interpretation on Windows.
+      EXPECT_EQ(getsid(getpid()), process.session.get());
+#endif // __WINDOWS__
+
+      ASSERT_SOME(process.rss);
+      EXPECT_GT(process.rss.get(), 0);
+
+      // NOTE: On linux /proc is a bit slow to update the cpu times,
+      // hence we allow 0 in this test.
+      ASSERT_SOME(process.utime);
+      EXPECT_GE(process.utime.get(), Nanoseconds(0));
+      ASSERT_SOME(process.stime);
+      EXPECT_GE(process.stime.get(), Nanoseconds(0));
+
+      EXPECT_FALSE(process.command.empty());
+
+      break;
+    }
+  }
+
+  EXPECT_TRUE(found);
+}
+
+
+TEST_F(ProcessTest, Pids)
+{
+  Try<set<pid_t> > pids = os::pids();
+  ASSERT_SOME(pids);
+  EXPECT_NE(0u, pids.get().size());
+  EXPECT_EQ(1u, pids.get().count(getpid()));
+
+  // In a FreeBSD jail, pid 1 may not exist.
+#ifdef __FreeBSD__
+  if (!isJailed()) {
+#endif
+    EXPECT_EQ(1u, pids.get().count(init_pid));
+#ifdef __FreeBSD__
+  }
+#endif
+
+#ifndef __WINDOWS__
+  // NOTE: `getpgid` does not have a meaningful interpretation on Windows.
+  pids = os::pids(getpgid(0), None());
+  EXPECT_SOME(pids);
+  EXPECT_GE(pids.get().size(), 1u);
+  EXPECT_EQ(1u, pids.get().count(getpid()));
+
+  // NOTE: This test is not meaningful on Windows because process IDs are
+  // expected to be non-negative.
+  EXPECT_ERROR(os::pids(-1, None()));
+
+  // NOTE: `getsid` does not have a meaningful interpretation on Windows.
+  pids = os::pids(None(), getsid(0));
+  EXPECT_SOME(pids);
+  EXPECT_GE(pids.get().size(), 1u);
+  EXPECT_EQ(1u, pids.get().count(getpid()));
+
+  // NOTE: This test is not meaningful on Windows because process IDs are
+  // expected to be non-negative.
+  EXPECT_ERROR(os::pids(None(), -1));
+#endif // __WINDOWS__
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/f79b8bbd/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 78cfd37..6c65ac1 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
+++ b/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
@@ -316,38 +316,6 @@ TEST_F(OsTest, Sysctl)
 #endif // __APPLE__ || __FreeBSD__
 
 
-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()));
-
-  // In a FreeBSD jail, pid 1 may not exist.
-#ifdef __FreeBSD__
-  if (!isJailed()) {
-#endif
-    EXPECT_EQ(1u, pids.get().count(1));
-#ifdef __FreeBSD__
-  }
-#endif
-
-  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));
-}
-
-
 TEST_F(OsTest, Children)
 {
   Try<set<pid_t> > children = os::children(getpid());
@@ -398,67 +366,6 @@ TEST_F(OsTest, Children)
 }
 
 
-TEST_F(OsTest, Process)
-{
-  const Result<Process> process = os::process(getpid());
-
-  ASSERT_SOME(process);
-  EXPECT_EQ(getpid(), process.get().pid);
-  EXPECT_EQ(getppid(), process.get().parent);
-  ASSERT_SOME(process.get().session);
-  EXPECT_EQ(getsid(getpid()), process.get().session.get());
-
-  ASSERT_SOME(process.get().rss);
-  EXPECT_GT(process.get().rss.get(), 0);
-
-  // NOTE: On Linux /proc is a bit slow to update the CPU times,
-  // hence we allow 0 in this test.
-  ASSERT_SOME(process.get().utime);
-  EXPECT_GE(process.get().utime.get(), Nanoseconds(0));
-  ASSERT_SOME(process.get().stime);
-  EXPECT_GE(process.get().stime.get(), Nanoseconds(0));
-
-  EXPECT_FALSE(process.get().command.empty());
-}
-
-
-TEST_F(OsTest, Processes)
-{
-  const Try<list<Process>> processes = os::processes();
-
-  ASSERT_SOME(processes);
-  ASSERT_GT(processes.get().size(), 2);
-
-  // Look for ourselves in the table.
-  bool found = false;
-  foreach (const Process& process, processes.get()) {
-    if (process.pid == getpid()) {
-      found = true;
-      EXPECT_EQ(getpid(), process.pid);
-      EXPECT_EQ(getppid(), process.parent);
-      ASSERT_SOME(process.session);
-      EXPECT_EQ(getsid(getpid()), process.session.get());
-
-      ASSERT_SOME(process.rss);
-      EXPECT_GT(process.rss.get(), 0);
-
-      // NOTE: On linux /proc is a bit slow to update the cpu times,
-      // hence we allow 0 in this test.
-      ASSERT_SOME(process.utime);
-      EXPECT_GE(process.utime.get(), Nanoseconds(0));
-      ASSERT_SOME(process.stime);
-      EXPECT_GE(process.stime.get(), Nanoseconds(0));
-
-      EXPECT_FALSE(process.command.empty());
-
-      break;
-    }
-  }
-
-  EXPECT_TRUE(found);
-}
-
-
 void dosetsid()
 {
   if (::setsid() == -1) {


[4/6] mesos git commit: Implemented `uname` on Windows in stout.

Posted by mp...@apache.org.
Implemented `uname` on Windows in stout.

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


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

Branch: refs/heads/master
Commit: 3287552fa9e90da7952d85245bcf7b410c8203f5
Parents: 3fd64f4
Author: Alex Clemmer <cl...@gmail.com>
Authored: Thu Apr 21 11:51:37 2016 -0700
Committer: Michael Park <mp...@apache.org>
Committed: Thu Apr 21 22:26:22 2016 -0700

----------------------------------------------------------------------
 .../3rdparty/stout/include/stout/os.hpp         |  12 ++
 .../3rdparty/stout/include/stout/posix/os.hpp   |  12 --
 .../3rdparty/stout/include/stout/windows/os.hpp | 127 ++++++++++++++++++-
 3 files changed, 138 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3287552f/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 c9576be..25e9320 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
@@ -149,6 +149,18 @@ inline void appendPaths(const std::string& newPaths)
 } // namespace libraries {
 
 
+// Return the operating system name (e.g. Linux).
+inline Try<std::string> sysname()
+{
+  Try<UTSInfo> info = uname();
+  if (info.isError()) {
+    return Error(info.error());
+  }
+
+  return info.get().sysname;
+}
+
+
 inline Try<std::list<Process>> processes()
 {
   const Try<std::set<pid_t>> pids = os::pids();

http://git-wip-us.apache.org/repos/asf/mesos/blob/3287552f/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
index e03be83..700d704 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/posix/os.hpp
@@ -348,18 +348,6 @@ inline Try<Nothing> tar(const std::string& path, const std::string& archive)
 }
 
 
-// Return the operating system name (e.g. Linux).
-inline Try<std::string> sysname()
-{
-  Try<UTSInfo> info = uname();
-  if (info.isError()) {
-    return Error(info.error());
-  }
-
-  return info.get().sysname;
-}
-
-
 // Return the OS release numbers.
 inline Try<Version> release()
 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/3287552f/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
index ad29093..5281a2e 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/windows/os.hpp
@@ -39,6 +39,96 @@
 #include <stout/os/raw/environment.hpp>
 
 namespace os {
+namespace internal {
+
+inline Try<OSVERSIONINFOEX> os_version()
+{
+  OSVERSIONINFOEX os_version;
+  os_version.dwOSVersionInfoSize = sizeof(OSVERSIONINFOEX);
+  if (!::GetVersionEx(reinterpret_cast<LPOSVERSIONINFO>(&os_version))) {
+    return WindowsError(
+        "os::internal::os_version: Call to `GetVersionEx` failed");
+  }
+
+  return os_version;
+}
+
+
+inline Try<std::string> nodename()
+{
+  // Get DNS name of the local computer. First, find the size of the output
+  // buffer.
+  DWORD size = 0;
+  if (!::GetComputerNameEx(ComputerNameDnsHostname, NULL, &size) &&
+      ::GetLastError() != ERROR_MORE_DATA) {
+    return WindowsError(
+        "os::internal::nodename: Call to `GetComputerNameEx` failed");
+  }
+
+  std::unique_ptr<char[]> name(new char[size + 1]);
+
+  if (!::GetComputerNameEx(ComputerNameDnsHostname, name.get(), &size)) {
+    return WindowsError(
+        "os::internal::nodename: Call to `GetComputerNameEx` failed");
+  }
+
+  return std::string(name.get());
+}
+
+
+inline std::string machine()
+{
+  SYSTEM_INFO system_info;
+  ::GetNativeSystemInfo(&system_info);
+
+  switch (system_info.wProcessorArchitecture) {
+    case PROCESSOR_ARCHITECTURE_AMD64:
+      return "AMD64";
+    case PROCESSOR_ARCHITECTURE_ARM:
+      return "ARM";
+    case PROCESSOR_ARCHITECTURE_IA64:
+      return "IA64";
+    case PROCESSOR_ARCHITECTURE_INTEL:
+      return "x86";
+    default:
+      return "Unknown";
+  }
+}
+
+
+inline std::string sysname(OSVERSIONINFOEX os_version)
+{
+  switch (os_version.wProductType) {
+    case VER_NT_DOMAIN_CONTROLLER:
+    case VER_NT_SERVER:
+      return "Windows Server";
+    default:
+      return "Windows";
+  }
+}
+
+
+inline std::string release(OSVERSIONINFOEX os_version)
+{
+  return stringify(
+      Version(os_version.dwMajorVersion, os_version.dwMinorVersion, 0));
+}
+
+
+inline std::string version(OSVERSIONINFOEX os_version)
+{
+  std::string version = std::to_string(os_version.dwBuildNumber);
+
+  if (os_version.szCSDVersion[0] != '\0') {
+    version.append(" ");
+    version.append(os_version.szCSDVersion);
+  }
+
+  return version;
+}
+
+} // namespace internal {
+
 
 // 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.
@@ -235,8 +325,43 @@ inline Try<Memory> memory()
 }
 
 
+inline Try<Version> release()
+{
+  OSVERSIONINFOEX os_version;
+  os_version.dwOSVersionInfoSize = sizeof(OSVERSIONINFOEX);
+  if (!::GetVersionEx(reinterpret_cast<LPOSVERSIONINFO>(&os_version))) {
+    return WindowsError("os::release: Call to `GetVersionEx` failed");
+  }
+
+  return Version(os_version.dwMajorVersion, os_version.dwMinorVersion, 0);
+}
+
+
 // Return the system information.
-inline Try<UTSInfo> uname() = delete;
+inline Try<UTSInfo> uname()
+{
+  Try<OSVERSIONINFOEX> os_version = internal::os_version();
+  if (os_version.isError()) {
+    return Error(os_version.error());
+  }
+
+  // Add nodename to `UTSInfo` object.
+  Try<std::string> nodename = internal::nodename();
+  if (nodename.isError()) {
+    return Error(nodename.error());
+  }
+
+  // Populate `UTSInfo`.
+  UTSInfo info;
+
+  info.sysname = internal::sysname(os_version.get());
+  info.release = internal::release(os_version.get());
+  info.version = internal::version(os_version.get());
+  info.nodename = nodename.get();
+  info.machine = internal::machine();
+
+  return info;
+}
 
 
 // Looks in the environment variables for the specified key and


[6/6] mesos git commit: Added `systems_tests.cpp` to libprocess.

Posted by mp...@apache.org.
Added `systems_tests.cpp` to libprocess.

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


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

Branch: refs/heads/master
Commit: fa7ca953cd03d610b51a8c8f8173a839202e6c32
Parents: 3c5712e
Author: Alex Clemmer <cl...@gmail.com>
Authored: Thu Apr 21 11:51:45 2016 -0700
Committer: Michael Park <mp...@apache.org>
Committed: Thu Apr 21 22:26:46 2016 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/Makefile.am | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fa7ca953/3rdparty/libprocess/3rdparty/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/Makefile.am b/3rdparty/libprocess/3rdparty/Makefile.am
index 3dd6028..895663b 100644
--- a/3rdparty/libprocess/3rdparty/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/Makefile.am
@@ -193,6 +193,7 @@ stout_tests_SOURCES =				\
   $(STOUT)/tests/os/sendfile_tests.cpp		\
   $(STOUT)/tests/os/signals_tests.cpp		\
   $(STOUT)/tests/os/strerror_tests.cpp		\
+  $(STOUT)/tests/os/systems_tests.cpp		\
   $(STOUT)/tests/path_tests.cpp			\
   $(STOUT)/tests/protobuf_tests.cpp		\
   $(STOUT)/tests/protobuf_tests.pb.cc		\


[5/6] mesos git commit: Added `systems_tests.cpp` to stout.

Posted by mp...@apache.org.
Added `systems_tests.cpp` to stout.

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


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

Branch: refs/heads/master
Commit: 3c5712e856ef2d910f5e9b3e595ea5a4201dce1a
Parents: 3287552
Author: Alex Clemmer <cl...@gmail.com>
Authored: Thu Apr 21 11:51:41 2016 -0700
Committer: Michael Park <mp...@apache.org>
Committed: Thu Apr 21 22:26:42 2016 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/stout/Makefile.am  |  1 +
 .../3rdparty/stout/tests/CMakeLists.txt         |  1 +
 .../3rdparty/stout/tests/os/systems_tests.cpp   | 78 ++++++++++++++++++++
 .../3rdparty/stout/tests/os_tests.cpp           | 36 ---------
 4 files changed, 80 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3c5712e8/3rdparty/libprocess/3rdparty/stout/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/Makefile.am b/3rdparty/libprocess/3rdparty/stout/Makefile.am
index ad8731d..08ede41 100644
--- a/3rdparty/libprocess/3rdparty/stout/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/stout/Makefile.am
@@ -52,6 +52,7 @@ EXTRA_DIST =					\
   tests/os/sendfile_tests.cpp			\
   tests/os/signals_tests.cpp			\
   tests/os/strerror_tests.cpp			\
+  tests/os/systems_tests.cpp			\
   tests/path_tests.cpp				\
   tests/proc_tests.cpp				\
   tests/protobuf_tests.cpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/3c5712e8/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt b/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt
index b6de810..7b3b2a5 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt
+++ b/3rdparty/libprocess/3rdparty/stout/tests/CMakeLists.txt
@@ -43,6 +43,7 @@ set(STOUT_TESTS_SRC
   os/process_tests.cpp
   os/rmdir_tests.cpp
   os/strerror_tests.cpp
+  os/systems_tests.cpp
   protobuf_tests.pb.h
   protobuf_tests.proto
   result_tests.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/3c5712e8/3rdparty/libprocess/3rdparty/stout/tests/os/systems_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/os/systems_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/os/systems_tests.cpp
new file mode 100644
index 0000000..110ba5b
--- /dev/null
+++ b/3rdparty/libprocess/3rdparty/stout/tests/os/systems_tests.cpp
@@ -0,0 +1,78 @@
+// Licensed 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 <stout/os.hpp>
+
+#include <stout/tests/utils.hpp>
+
+using std::string;
+
+
+class SystemsTests : public TemporaryDirectoryTest {};
+
+
+TEST_F(SystemsTests, Uname)
+{
+  const Try<os::UTSInfo> info = os::uname();
+
+  ASSERT_SOME(info);
+#ifdef __linux__
+  EXPECT_EQ(info.get().sysname, "Linux");
+
+  // Machine arch must be non-empty.
+  EXPECT_FALSE(info.get().machine.empty());
+#elif defined(__APPLE__)
+  EXPECT_EQ(info.get().sysname, "Darwin");
+
+  // Machine arch must be non-empty.
+  EXPECT_FALSE(info.get().machine.empty());
+#elif defined(__WINDOWS__)
+  // On Windows, `sysname` is one of 2 options.
+  hashset<string> server_types{"Windows", "Windows Server"};
+  EXPECT_TRUE(server_types.contains(info.get().sysname));
+
+  // On Windows, we `machine` takes one of 5 values.
+  hashset<string> arch_types{"AMD64", "ARM", "IA64", "x86", "Unknown"};
+  EXPECT_TRUE(arch_types.contains(info.get().machine));
+#endif // __linux__
+
+  // The `release`, `version`, and `nodename` properties should all be
+  // populated with a string of at least 1 character.
+  EXPECT_GT(info.get().release.size(), 0);
+  EXPECT_GT(info.get().version.size(), 0);
+  EXPECT_GT(info.get().nodename.size(), 0);
+}
+
+
+TEST_F(SystemsTests, Sysname)
+{
+  const Try<string> name = os::sysname();
+
+  ASSERT_SOME(name);
+#ifdef __linux__
+  EXPECT_EQ(name.get(), "Linux");
+#elif defined(__APPLE__)
+  EXPECT_EQ(name.get(), "Darwin");
+#elif defined(__WINDOWS__)
+  // On Windows, `sysname` is one of 2 options.
+  hashset<string> server_types{ "Windows", "Windows Server" };
+  EXPECT_TRUE(server_types.contains(name.get()));
+#endif // __linux__
+}
+
+
+TEST_F(SystemsTests, Release)
+{
+  const Try<Version> info = os::release();
+
+  ASSERT_SOME(info);
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/3c5712e8/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 6c65ac1..5f00f56 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
+++ b/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
@@ -216,42 +216,6 @@ TEST_F(OsTest, BootId)
 }
 
 
-TEST_F(OsTest, Uname)
-{
-  const Try<os::UTSInfo> info = os::uname();
-
-  ASSERT_SOME(info);
-#ifdef __linux__
-  EXPECT_EQ(info.get().sysname, "Linux");
-#endif
-#ifdef __APPLE__
-  EXPECT_EQ(info.get().sysname, "Darwin");
-#endif
-}
-
-
-TEST_F(OsTest, Sysname)
-{
-  const Try<string> name = os::sysname();
-
-  ASSERT_SOME(name);
-#ifdef __linux__
-  EXPECT_EQ(name.get(), "Linux");
-#endif
-#ifdef __APPLE__
-  EXPECT_EQ(name.get(), "Darwin");
-#endif
-}
-
-
-TEST_F(OsTest, Release)
-{
-  const Try<Version> info = os::release();
-
-  ASSERT_SOME(info);
-}
-
-
 TEST_F(OsTest, Sleep)
 {
   Duration duration = Milliseconds(10);


[3/6] mesos git commit: Moved process tests to their own file to libprocess.

Posted by mp...@apache.org.
Moved process tests to their own file to libprocess.

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


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

Branch: refs/heads/master
Commit: 3fd64f4d7a569199e184733c6c448c91ad70733a
Parents: f79b8bb
Author: Alex Clemmer <cl...@gmail.com>
Authored: Thu Apr 21 11:51:33 2016 -0700
Committer: Michael Park <mp...@apache.org>
Committed: Thu Apr 21 22:26:14 2016 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/Makefile.am | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3fd64f4d/3rdparty/libprocess/3rdparty/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/Makefile.am b/3rdparty/libprocess/3rdparty/Makefile.am
index f7aa579..3dd6028 100644
--- a/3rdparty/libprocess/3rdparty/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/Makefile.am
@@ -188,6 +188,7 @@ stout_tests_SOURCES =				\
   $(STOUT)/tests/os_tests.cpp			\
   $(STOUT)/tests/os/env_tests.cpp		\
   $(STOUT)/tests/os/filesystem_tests.cpp	\
+  $(STOUT)/tests/os/process_tests.cpp		\
   $(STOUT)/tests/os/rmdir_tests.cpp		\
   $(STOUT)/tests/os/sendfile_tests.cpp		\
   $(STOUT)/tests/os/signals_tests.cpp		\