You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by qi...@apache.org on 2018/11/08 08:19:14 UTC

[mesos] branch 1.7.x updated (a88f45c -> f87d669)

This is an automated email from the ASF dual-hosted git repository.

qianzhang pushed a change to branch 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git.


    from a88f45c  Used the tagged based image for Mesos mini.
     new 7c803db  Added `lsof()` into stout.
     new 5c61bb2  Added a test `FsTest.Lsof`.
     new d690dca  Updated `MesosContainerizerLaunch` to call `os::lsof()`.
     new 1b4836b  Closed all file descriptors except `whitelist_fds` in posix/subprocess.
     new b5ab1b9  Added a test `SubprocessTest.WhiteListFds`.
     new 44bccdc  Updated IO switchboard to use subprocess's `whitelist_fds` parameter.
     new 079febc  Updated launchers to use subprocess's `whitelist_fds` parameter.
     new 86378a0  Removed the child hook `UNSET_CLOEXEC`.
     new f87d669  Added MESOS-9152 and MESOS-9164 to the 1.7.1 CHANGELOG.

The 9 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 3rdparty/libprocess/include/process/subprocess.hpp |   6 --
 3rdparty/libprocess/src/posix/subprocess.hpp       | 104 ++++++++++++++++++++-
 3rdparty/libprocess/src/subprocess.cpp             |  11 +--
 3rdparty/libprocess/src/tests/subprocess_tests.cpp |  77 +++++++++++++++
 3rdparty/stout/include/Makefile.am                 |   3 +
 3rdparty/stout/include/stout/os.hpp                |   1 +
 .../stout/include/stout/os/{fork.hpp => lsof.hpp}  |  10 +-
 .../include/stout/os/posix/{ls.hpp => lsof.hpp}    |  49 ++++++----
 .../stout/os/{socket.hpp => windows/lsof.hpp}      |  19 ++--
 3rdparty/stout/tests/os/filesystem_tests.cpp       |  20 ++++
 CHANGELOG                                          |   2 +
 src/slave/containerizer/mesos/io/switchboard.cpp   |   6 +-
 src/slave/containerizer/mesos/launch.cpp           |  57 ++---------
 src/slave/containerizer/mesos/launcher.cpp         |   5 -
 src/slave/containerizer/mesos/linux_launcher.cpp   |   8 +-
 15 files changed, 269 insertions(+), 109 deletions(-)
 copy 3rdparty/stout/include/stout/os/{fork.hpp => lsof.hpp} (82%)
 copy 3rdparty/stout/include/stout/os/posix/{ls.hpp => lsof.hpp} (53%)
 copy 3rdparty/stout/include/stout/os/{socket.hpp => windows/lsof.hpp} (71%)


[mesos] 05/09: Added a test `SubprocessTest.WhiteListFds`.

Posted by qi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

qianzhang pushed a commit to branch 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit b5ab1b9cbedac112985a311a3e42e0540214bc45
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Fri Oct 12 22:04:02 2018 +0800

    Added a test `SubprocessTest.WhiteListFds`.
    
    Review: https://reviews.apache.org/r/69016
---
 3rdparty/libprocess/src/tests/subprocess_tests.cpp | 77 ++++++++++++++++++++++
 1 file changed, 77 insertions(+)

diff --git a/3rdparty/libprocess/src/tests/subprocess_tests.cpp b/3rdparty/libprocess/src/tests/subprocess_tests.cpp
index aa939a6..97a93c5 100644
--- a/3rdparty/libprocess/src/tests/subprocess_tests.cpp
+++ b/3rdparty/libprocess/src/tests/subprocess_tests.cpp
@@ -41,6 +41,7 @@
 namespace io = process::io;
 
 using process::Clock;
+using process::Future;
 using process::MAX_REAP_INTERVAL;
 using process::Subprocess;
 using process::subprocess;
@@ -1057,5 +1058,81 @@ TEST_F(SubprocessTest, EnvironmentOverride)
 }
 
 
+#ifdef __linux__
+// This test verifies:
+//   1. The subprocess will have the stdio file descriptors.
+//   2. The whitelisted file descriptors will be successfully
+//      inherited by the subprocess.
+//   3. The non-whitelisted file descriptors will be not be
+//      inherited by the subprocess.
+TEST_F(SubprocessTest, WhiteListFds)
+{
+  Try<int_fd> fd1 = os::open(
+      path::join(os::getcwd(), id::UUID::random().toString()),
+      O_CREAT | O_EXCL | O_RDONLY | O_CLOEXEC);
+
+  Try<int_fd> fd2 = os::open(
+      path::join(os::getcwd(), id::UUID::random().toString()),
+      O_CREAT | O_EXCL | O_RDONLY);
+
+  ASSERT_SOME(fd1);
+  ASSERT_SOME(fd2);
+
+  Try<Subprocess> s = subprocess(
+      "ls /dev/fd",
+      Subprocess::FD(STDIN_FILENO),
+      Subprocess::PIPE(),
+      Subprocess::FD(STDERR_FILENO),
+      None(),
+      None(),
+      {},
+      {},
+      {fd1.get()});
+
+  ASSERT_SOME(s);
+  ASSERT_SOME(s->out());
+
+  Future<string> output = io::read(s->out().get());
+  AWAIT_READY(output);
+
+  hashset<int_fd> fds;
+
+  vector<string> tokens = strings::tokenize(output.get(), "\n");
+  foreach (const string& fdString, tokens) {
+    Try<int_fd> fd = numify<int_fd>(fdString);
+    ASSERT_SOME(fd);
+
+    fds.insert(fd.get());
+  }
+
+  // The subprocess should always have the stdio file descriptors.
+  EXPECT_TRUE(fds.contains(STDIN_FILENO));
+  EXPECT_TRUE(fds.contains(STDOUT_FILENO));
+  EXPECT_TRUE(fds.contains(STDERR_FILENO));
+
+  // `fd1` should be inherited by the subprocess since it is whitelisted even
+  // it has `O_CLOEXEC` set initially.
+  EXPECT_TRUE(fds.contains(fd1.get()));
+
+  // `fd2` should not be inherited by the subprocess since it is not whitelisted
+  // even it has no `O_CLOEXEC` set initially.
+  EXPECT_FALSE(fds.contains(fd2.get()));
+
+  ASSERT_SOME(os::close(fd1.get()));
+  ASSERT_SOME(os::close(fd2.get()));
+
+  // Advance time until the internal reaper reaps the subprocess.
+  Clock::pause();
+  while (s->status().isPending()) {
+    Clock::advance(MAX_REAP_INTERVAL());
+    Clock::settle();
+  }
+  Clock::resume();
+
+  AWAIT_EXPECT_WEXITSTATUS_EQ(0, s->status());
+}
+#endif // __linux__
+
+
 // TODO(joerg84): Consider adding tests for setsid, working_directory,
 // and supervisor childHooks.


[mesos] 04/09: Closed all file descriptors except `whitelist_fds` in posix/subprocess.

Posted by qi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

qianzhang pushed a commit to branch 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 1b4836b209d2e53ddf28553fc0ac565896af3e89
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Wed Aug 29 10:17:05 2018 +0800

    Closed all file descriptors except `whitelist_fds` in posix/subprocess.
    
    Review: https://reviews.apache.org/r/68644
---
 3rdparty/libprocess/src/posix/subprocess.hpp | 104 ++++++++++++++++++++++++++-
 3rdparty/libprocess/src/subprocess.cpp       |   3 +-
 2 files changed, 105 insertions(+), 2 deletions(-)

diff --git a/3rdparty/libprocess/src/posix/subprocess.hpp b/3rdparty/libprocess/src/posix/subprocess.hpp
index 007058b..719bdf3 100644
--- a/3rdparty/libprocess/src/posix/subprocess.hpp
+++ b/3rdparty/libprocess/src/posix/subprocess.hpp
@@ -15,6 +15,7 @@
 
 #ifdef __linux__
 #include <sys/prctl.h>
+#include <sys/syscall.h>
 #endif // __linux__
 #include <sys/types.h>
 
@@ -55,6 +56,102 @@ static void close(
     const Subprocess::IO::OutputFileDescriptors& stderrfds);
 
 
+// Convert a null-terminated string to an integer. This function
+// is async signal safe since it does not make any libc calls.
+static int convertStringToInt(const char *name)
+{
+  int num = 0;
+  while (*name >= '0' && *name <= '9') {
+    num = num * 10 + (*name - '0');
+    ++name;
+  }
+
+  if (*name) {
+    // Non digit found, not a number.
+    return -1;
+  }
+
+  return num;
+}
+
+
+// Close any file descriptors that are not stdio file descriptors and not
+// explicitly whitelisted to avoid leaking them into the forked process.
+// And unset the `close-on-exec` flag for the whitelist file descriptors
+// so that they can be inherited by the forked process.
+static void handleWhitelistFds(const std::vector<int_fd>& whitelist_fds)
+{
+  // We need to make a syscall (e.g., `SYS_getdents64` on Linux) to get each
+  // entry from `/dev/fd` since syscall function is async signal safe, but we
+  // cannot do that for macOS since Apple has decided to deprecate all syscall
+  // functions with OS 10.12 (see MESOS-8457).
+#if defined(__linux__) && defined(SYS_getdents64)
+  int fdDir = ::open("/dev/fd", O_RDONLY);
+  if (fdDir == -1) {
+    ABORT("Failed to open /dev/fd: " + os::strerror(errno));
+  }
+
+  struct linux_dirent64 {
+     ino64_t        d_ino;
+     off64_t        d_off;
+     unsigned short d_reclen;
+     unsigned char  d_type;
+     char           d_name[];
+  };
+
+  char buffer[1024];
+  int bytes;
+
+  while (true) {
+    bytes = ::syscall(SYS_getdents64, fdDir, buffer, sizeof(buffer));
+    if (bytes == -1) {
+      ABORT("Failed to call SYS_getdents64 on /dev/fd: " + os::strerror(errno));
+    }
+
+    if (bytes == 0) {
+      break;
+    }
+
+    struct linux_dirent64 *entry;
+    for (int offset = 0; offset < bytes; offset += entry->d_reclen) {
+      entry = reinterpret_cast<struct linux_dirent64 *>(buffer + offset);
+      int_fd fd = convertStringToInt(entry->d_name);
+      if (fd >= 0 &&
+          fd != fdDir &&
+          fd != STDIN_FILENO &&
+          fd != STDOUT_FILENO &&
+          fd != STDERR_FILENO) {
+        bool found = false;
+        foreach (int_fd whitelist_fd, whitelist_fds) {
+          if (whitelist_fd == fd) {
+            found = true;
+            break;
+          }
+        }
+
+        if (!found) {
+          ::close(fd);
+        }
+      }
+    }
+  }
+
+  ::close(fdDir);
+#endif
+
+  foreach (int_fd fd, whitelist_fds) {
+    int flags = ::fcntl(fd, F_GETFD);
+    if (flags == -1) {
+      ABORT("Failed to get file descriptor flags: " + os::strerror(errno));
+    }
+
+    if (::fcntl(fd, F_SETFD, flags & ~FD_CLOEXEC) == -1) {
+      ABORT("Failed to unset cloexec: " + os::strerror(errno));
+    }
+  }
+}
+
+
 inline pid_t defaultClone(const lambda::function<int()>& func)
 {
   pid_t pid = ::fork();
@@ -110,6 +207,7 @@ inline int childMain(
     const InputFileDescriptors& stdinfds,
     const OutputFileDescriptors& stdoutfds,
     const OutputFileDescriptors& stderrfds,
+    const std::vector<int_fd>& whitelist_fds,
     bool blocking,
     int pipes[2],
     const std::vector<Subprocess::ChildHook>& child_hooks)
@@ -190,6 +288,8 @@ inline int childMain(
     }
   }
 
+  handleWhitelistFds(whitelist_fds);
+
   os::execvpe(path.c_str(), argv, envp);
 
   SAFE_EXIT(
@@ -207,7 +307,8 @@ inline Try<pid_t> cloneChild(
     const std::vector<Subprocess::ChildHook>& child_hooks,
     const InputFileDescriptors stdinfds,
     const OutputFileDescriptors stdoutfds,
-    const OutputFileDescriptors stderrfds)
+    const OutputFileDescriptors stderrfds,
+    const std::vector<int_fd>& whitelist_fds)
 {
   // The real arguments that will be passed to 'os::execvpe'. We need
   // to construct them here before doing the clone as it might not be
@@ -268,6 +369,7 @@ inline Try<pid_t> cloneChild(
       stdinfds,
       stdoutfds,
       stderrfds,
+      whitelist_fds,
       blocking,
       pipes.data(),
       child_hooks));
diff --git a/3rdparty/libprocess/src/subprocess.cpp b/3rdparty/libprocess/src/subprocess.cpp
index c0640de..0bd7d54 100644
--- a/3rdparty/libprocess/src/subprocess.cpp
+++ b/3rdparty/libprocess/src/subprocess.cpp
@@ -414,7 +414,8 @@ Try<Subprocess> subprocess(
         child_hooks,
         stdinfds,
         stdoutfds,
-        stderrfds);
+        stderrfds,
+        whitelist_fds);
 
     if (pid.isError()) {
       return Error(pid.error());


[mesos] 06/09: Updated IO switchboard to use subprocess's `whitelist_fds` parameter.

Posted by qi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

qianzhang pushed a commit to branch 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 44bccdc8872b83acebf7749757505d416439c724
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Mon Sep 3 15:09:24 2018 +0800

    Updated IO switchboard to use subprocess's `whitelist_fds` parameter.
    
    Review: https://reviews.apache.org/r/68645
---
 src/slave/containerizer/mesos/io/switchboard.cpp | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)

diff --git a/src/slave/containerizer/mesos/io/switchboard.cpp b/src/slave/containerizer/mesos/io/switchboard.cpp
index e96504d..c445a8f 100644
--- a/src/slave/containerizer/mesos/io/switchboard.cpp
+++ b/src/slave/containerizer/mesos/io/switchboard.cpp
@@ -597,10 +597,8 @@ Future<Option<ContainerLaunchInfo>> IOSwitchboard::_prepare(
       environment,
       None(),
       parentHooks,
-      {Subprocess::ChildHook::SETSID(),
-       Subprocess::ChildHook::UNSET_CLOEXEC(stdinToFd),
-       Subprocess::ChildHook::UNSET_CLOEXEC(stdoutFromFd),
-       Subprocess::ChildHook::UNSET_CLOEXEC(stderrFromFd)});
+      {Subprocess::ChildHook::SETSID()},
+      {stdinToFd, stdoutFromFd, stderrFromFd});
 
   if (child.isError()) {
     close(openedFds);


[mesos] 02/09: Added a test `FsTest.Lsof`.

Posted by qi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

qianzhang pushed a commit to branch 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 5c61bb244b5e2d33e9853e5ce93b03b853869d85
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Wed Sep 26 16:58:48 2018 +0800

    Added a test `FsTest.Lsof`.
    
    Review: https://reviews.apache.org/r/68991
---
 3rdparty/stout/tests/os/filesystem_tests.cpp | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)

diff --git a/3rdparty/stout/tests/os/filesystem_tests.cpp b/3rdparty/stout/tests/os/filesystem_tests.cpp
index 09d0a40..5290309 100644
--- a/3rdparty/stout/tests/os/filesystem_tests.cpp
+++ b/3rdparty/stout/tests/os/filesystem_tests.cpp
@@ -655,6 +655,7 @@ TEST_F(FsTest, Xattr)
 }
 #endif // __linux__ || __APPLE__
 
+
 #ifdef __WINDOWS__
 // Check if the overlapped field is set properly on Windows.
 TEST_F(FsTest, Overlapped)
@@ -800,5 +801,24 @@ TEST_F(FsTest, ReadWriteAsyncLargeBuffer)
   EXPECT_SOME(os::close(pipes.get()[0]));
   EXPECT_SOME(os::close(pipes.get()[1]));
 }
+#endif // __WINDOWS__
+
+
+#ifndef __WINDOWS__
+// This test verifies that the file descriptors returned by `os::lsof()`
+// are all open file descriptors and contains stdin, stdout and stderr.
+TEST_F(FsTest, Lsof)
+{
+  Try<std::vector<int_fd>> fds = os::lsof();
+  ASSERT_SOME(fds);
+
+  // Verify each `fd` is an open file descriptor.
+  foreach (int_fd fd, fds.get()) {
+    EXPECT_NE(-1, ::fcntl(fd, F_GETFD));
+  }
 
+  EXPECT_NE(std::find(fds->begin(), fds->end(), 0), fds->end());
+  EXPECT_NE(std::find(fds->begin(), fds->end(), 1), fds->end());
+  EXPECT_NE(std::find(fds->begin(), fds->end(), 2), fds->end());
+}
 #endif // __WINDOWS__


[mesos] 03/09: Updated `MesosContainerizerLaunch` to call `os::lsof()`.

Posted by qi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

qianzhang pushed a commit to branch 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit d690dca1c3984ada0c3088918ed864d2f1651458
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Wed Aug 29 10:17:15 2018 +0800

    Updated `MesosContainerizerLaunch` to call `os::lsof()`.
    
    Review: https://reviews.apache.org/r/68643
---
 src/slave/containerizer/mesos/launch.cpp | 57 ++++++--------------------------
 1 file changed, 10 insertions(+), 47 deletions(-)

diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index 7193da0..882bcdf 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -74,7 +74,6 @@
 
 using std::cerr;
 using std::endl;
-using std::list;
 using std::set;
 using std::string;
 using std::vector;
@@ -517,40 +516,6 @@ static Try<Nothing> enterChroot(const string& rootfs)
 }
 
 
-// On Windows all new processes create by Mesos go through the
-// `create_process` wrapper which with the completion of MESOS-8926
-// will prevent inadvertent leaks making this code unnecessary there.
-//
-// TODO(bbannier): Consider moving this to stout as e.g., `os::lsof`.
-#ifndef __WINDOWS__
-static Try<hashset<int_fd>> getOpenFileDescriptors()
-{
-  Try<list<string>> fds =
-#if defined(__linux__)
-    os::ls("/proc/self/fd");
-#elif defined(__APPLE__)
-    os::ls("/dev/fd");
-#endif
-
-  if (fds.isError()) {
-    return Error(fds.error());
-  }
-
-  hashset<int_fd> result;
-  foreach (const string& fd, fds.get()) {
-    Try<int_fd> fd_ = numify<int_fd>(fd);
-
-    if (fd_.isError()) {
-      return Error("Could not interpret file descriptor: " + fd_.error());
-    }
-
-    result.insert(fd_.get());
-  }
-
-  return result;
-}
-#endif // __WINDOWS__
-
 int MesosContainerizerLaunch::execute()
 {
   if (flags.help) {
@@ -1097,17 +1062,13 @@ int MesosContainerizerLaunch::execute()
 
 #ifndef __WINDOWS__
   // Construct a set of file descriptors to close before `exec`'ing.
-  Try<hashset<int_fd>> fds = getOpenFileDescriptors();
+  //
+  // On Windows all new processes create by Mesos go through the
+  // `create_process` wrapper which with the completion of MESOS-8926
+  // will prevent inadvertent leaks making this code unnecessary there.
+  Try<vector<int_fd>> fds = os::lsof();
   CHECK_SOME(fds);
 
-  std::set<int_fd> whitelistedFds = {
-    STDIN_FILENO, STDOUT_FILENO, STDERR_FILENO};
-
-  // Exclude required file descriptors from closing.
-  foreach (int_fd fd, whitelistedFds) {
-    fds->erase(fd);
-  }
-
   // If we have `containerStatusFd` set, then we need to fork-exec the
   // command we are launching and checkpoint its status on exit. We
   // use fork-exec directly (as opposed to `process::subprocess()`) to
@@ -1181,9 +1142,11 @@ int MesosContainerizerLaunch::execute()
 
     // Avoid leaking not required file descriptors into the forked process.
     foreach (int_fd fd, fds.get()) {
-      // We use the unwrapped `::close` as opposed to `os::close`
-      // since the former is guaranteed to be async signal safe.
-      ::close(fd);
+      if (fd != STDIN_FILENO && fd != STDOUT_FILENO && fd != STDERR_FILENO) {
+        // We use the unwrapped `::close` as opposed to `os::close`
+        // since the former is guaranteed to be async signal safe.
+        ::close(fd);
+      }
     }
   }
 #endif // __WINDOWS__


[mesos] 01/09: Added `lsof()` into stout.

Posted by qi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

qianzhang pushed a commit to branch 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 7c803db018dcf6d1c16d6d88e9042a351c9f9f4c
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Wed Aug 29 10:16:43 2018 +0800

    Added `lsof()` into stout.
    
    Review: https://reviews.apache.org/r/68642
---
 3rdparty/stout/include/Makefile.am               |  3 +
 3rdparty/stout/include/stout/os.hpp              |  1 +
 3rdparty/stout/include/stout/os/lsof.hpp         | 26 ++++++++
 3rdparty/stout/include/stout/os/posix/lsof.hpp   | 82 ++++++++++++++++++++++++
 3rdparty/stout/include/stout/os/windows/lsof.hpp | 28 ++++++++
 5 files changed, 140 insertions(+)

diff --git a/3rdparty/stout/include/Makefile.am b/3rdparty/stout/include/Makefile.am
index 0a4ea7b..e55df8a 100644
--- a/3rdparty/stout/include/Makefile.am
+++ b/3rdparty/stout/include/Makefile.am
@@ -89,6 +89,7 @@ nobase_include_HEADERS =			\
   stout/os/linux.hpp				\
   stout/os/ls.hpp				\
   stout/os/lseek.hpp				\
+  stout/os/lsof.hpp				\
   stout/os/mkdir.hpp				\
   stout/os/mkdtemp.hpp				\
   stout/os/mktemp.hpp				\
@@ -139,6 +140,7 @@ nobase_include_HEADERS =			\
   stout/os/posix/killtree.hpp			\
   stout/os/posix/ls.hpp				\
   stout/os/posix/lseek.hpp			\
+  stout/os/posix/lsof.hpp			\
   stout/os/posix/mkdir.hpp			\
   stout/os/posix/mkdtemp.hpp			\
   stout/os/posix/mktemp.hpp			\
@@ -182,6 +184,7 @@ nobase_include_HEADERS =			\
   stout/os/windows/killtree.hpp			\
   stout/os/windows/ls.hpp			\
   stout/os/windows/lseek.hpp			\
+  stout/os/windows/lsof.hpp			\
   stout/os/windows/mkdir.hpp			\
   stout/os/windows/mktemp.hpp			\
   stout/os/windows/mkdtemp.hpp			\
diff --git a/3rdparty/stout/include/stout/os.hpp b/3rdparty/stout/include/stout/os.hpp
index aee0418..15cbb50 100644
--- a/3rdparty/stout/include/stout/os.hpp
+++ b/3rdparty/stout/include/stout/os.hpp
@@ -54,6 +54,7 @@
 #include <stout/os/kill.hpp>
 #include <stout/os/ls.hpp>
 #include <stout/os/lseek.hpp>
+#include <stout/os/lsof.hpp>
 #include <stout/os/mkdir.hpp>
 #include <stout/os/mkdtemp.hpp>
 #include <stout/os/mktemp.hpp>
diff --git a/3rdparty/stout/include/stout/os/lsof.hpp b/3rdparty/stout/include/stout/os/lsof.hpp
new file mode 100644
index 0000000..668c664
--- /dev/null
+++ b/3rdparty/stout/include/stout/os/lsof.hpp
@@ -0,0 +1,26 @@
+// 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.
+
+#ifndef __STOUT_OS_LSOF_HPP__
+#define __STOUT_OS_LSOF_HPP__
+
+
+// For readability, we minimize the number of #ifdef blocks in the code by
+// splitting platform specific system calls into separate directories.
+#ifdef __WINDOWS__
+#include <stout/os/windows/lsof.hpp>
+#else
+#include <stout/os/posix/lsof.hpp>
+#endif // __WINDOWS__
+
+
+#endif // __STOUT_OS_LSOF_HPP__
diff --git a/3rdparty/stout/include/stout/os/posix/lsof.hpp b/3rdparty/stout/include/stout/os/posix/lsof.hpp
new file mode 100644
index 0000000..12c522a
--- /dev/null
+++ b/3rdparty/stout/include/stout/os/posix/lsof.hpp
@@ -0,0 +1,82 @@
+// 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.
+
+#ifndef __STOUT_OS_POSIX_LSOF_HPP__
+#define __STOUT_OS_POSIX_LSOF_HPP__
+
+#include <string>
+#include <vector>
+
+#include <stout/numify.hpp>
+#include <stout/try.hpp>
+
+#include <stout/os/int_fd.hpp>
+#include <stout/os/ls.hpp>
+
+namespace os {
+
+// Get all the open file descriptors of the current process.
+inline Try<std::vector<int_fd>> lsof()
+{
+  int fdDir = ::open("/dev/fd", O_RDONLY | O_CLOEXEC);
+  if (fdDir == -1) {
+    return ErrnoError("Failed to open '/dev/fd'");
+  }
+
+  DIR* dir = ::fdopendir(fdDir);
+  if (dir == nullptr) {
+    Error error = ErrnoError("Failed to fdopendir '/dev/fd'");
+    ::close(fdDir);
+    return error;
+  }
+
+  struct dirent* entry;
+  std::vector<int_fd> result;
+
+  // Zero `errno` before starting to call `readdir`. This is necessary
+  // to allow us to determine when `readdir` returns an error.
+  errno = 0;
+
+  while ((entry = ::readdir(dir)) != nullptr) {
+    if (strcmp(entry->d_name, ".") == 0 || strcmp(entry->d_name, "..") == 0) {
+      continue;
+    }
+
+    Try<int_fd> fd = numify<int_fd>(entry->d_name);
+    if (fd.isError()) {
+      return Error(
+          "Could not interpret file descriptor '" +
+          std::string(entry->d_name) + "': " + fd.error());
+    }
+
+    if (fd.get() != fdDir) {
+      result.push_back(fd.get());
+    }
+  }
+
+  if (errno != 0) {
+    // Preserve `readdir` error.
+    Error error = ErrnoError("Failed to read directory");
+    ::closedir(dir);
+    return error;
+  }
+
+  if (::closedir(dir) == -1) {
+    return ErrnoError("Failed to close directory");
+  }
+
+  return result;
+}
+
+} // namespace os {
+
+#endif /* __STOUT_OS_POSIX_LSOF_HPP__  */
diff --git a/3rdparty/stout/include/stout/os/windows/lsof.hpp b/3rdparty/stout/include/stout/os/windows/lsof.hpp
new file mode 100644
index 0000000..d5a6397
--- /dev/null
+++ b/3rdparty/stout/include/stout/os/windows/lsof.hpp
@@ -0,0 +1,28 @@
+// 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.
+
+#ifndef __STOUT_OS_WINDOWS_LSOF_HPP__
+#define __STOUT_OS_WINDOWS_LSOF_HPP__
+
+#include <vector>
+
+#include <stout/try.hpp>
+
+#include <stout/os/int_fd.hpp>
+
+namespace os {
+
+inline Try<std::vector<int_fd>> lsof() = delete;
+
+} // namespace os {
+
+#endif /* __STOUT_OS_WINDOWS_LSOF_HPP__  */


[mesos] 07/09: Updated launchers to use subprocess's `whitelist_fds` parameter.

Posted by qi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

qianzhang pushed a commit to branch 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 079febcc1a63886abd694d3edeeafc97eca39d8f
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Mon Sep 3 15:11:51 2018 +0800

    Updated launchers to use subprocess's `whitelist_fds` parameter.
    
    Review: https://reviews.apache.org/r/68646
---
 src/slave/containerizer/mesos/launcher.cpp       | 5 -----
 src/slave/containerizer/mesos/linux_launcher.cpp | 8 ++------
 2 files changed, 2 insertions(+), 11 deletions(-)

diff --git a/src/slave/containerizer/mesos/launcher.cpp b/src/slave/containerizer/mesos/launcher.cpp
index a18bdff..b38f88d 100644
--- a/src/slave/containerizer/mesos/launcher.cpp
+++ b/src/slave/containerizer/mesos/launcher.cpp
@@ -123,11 +123,6 @@ Try<pid_t> SubprocessLauncher::fork(
 
 #ifndef __WINDOWS__
   childHooks.push_back(Subprocess::ChildHook::SETSID());
-
-  // TODO(jpeach) libprocess should take care of this, see MESOS-9164.
-  foreach (int_fd fd, whitelistFds) {
-    childHooks.push_back(Subprocess::ChildHook::UNSET_CLOEXEC(fd));
-  }
 #endif // __WINDOWS__
 
   Try<Subprocess> child = subprocess(
diff --git a/src/slave/containerizer/mesos/linux_launcher.cpp b/src/slave/containerizer/mesos/linux_launcher.cpp
index e9ab36a..c10092b 100644
--- a/src/slave/containerizer/mesos/linux_launcher.cpp
+++ b/src/slave/containerizer/mesos/linux_launcher.cpp
@@ -528,11 +528,6 @@ Try<pid_t> LinuxLauncherProcess::fork(
 
   childHooks.push_back(Subprocess::ChildHook::SETSID());
 
-  // TODO(jpeach) libprocess should take care of this, see MESOS-9164.
-  foreach (int_fd fd, whitelistFds) {
-    childHooks.push_back(Subprocess::ChildHook::UNSET_CLOEXEC(fd));
-  }
-
   Try<Subprocess> child = subprocess(
       path,
       argv,
@@ -559,7 +554,8 @@ Try<pid_t> LinuxLauncherProcess::fork(
         }
       },
       parentHooks,
-      childHooks);
+      childHooks,
+      whitelistFds);
 
   if (child.isError()) {
     return Error("Failed to clone child process: " + child.error());


[mesos] 08/09: Removed the child hook `UNSET_CLOEXEC`.

Posted by qi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

qianzhang pushed a commit to branch 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 86378a05a98f521ac9872aa643deb38e0e079f05
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Mon Oct 8 16:06:31 2018 +0800

    Removed the child hook `UNSET_CLOEXEC`.
    
    We do not need this child hook since any file descripters need
    to unset the close-on-exec flag can be put in the `whitelist_fds`
    parameter of the `subprocess` method.
    
    Review: https://reviews.apache.org/r/68995
---
 3rdparty/libprocess/include/process/subprocess.hpp | 6 ------
 3rdparty/libprocess/src/subprocess.cpp             | 8 --------
 2 files changed, 14 deletions(-)

diff --git a/3rdparty/libprocess/include/process/subprocess.hpp b/3rdparty/libprocess/include/process/subprocess.hpp
index 135bf24..3b2653d 100644
--- a/3rdparty/libprocess/include/process/subprocess.hpp
+++ b/3rdparty/libprocess/include/process/subprocess.hpp
@@ -205,12 +205,6 @@ public:
      * `ChildHook` for duplicating a file descriptor.
      */
     static ChildHook DUP2(int oldFd, int newFd);
-
-    /**
-     * `ChildHook` to unset CLOEXEC on a file descriptor. This is
-     * useful to explicitly pass an FD to a subprocess.
-     */
-    static ChildHook UNSET_CLOEXEC(int fd);
 #endif // __WINDOWS__
 
     /**
diff --git a/3rdparty/libprocess/src/subprocess.cpp b/3rdparty/libprocess/src/subprocess.cpp
index 0bd7d54..ea812c3 100644
--- a/3rdparty/libprocess/src/subprocess.cpp
+++ b/3rdparty/libprocess/src/subprocess.cpp
@@ -112,14 +112,6 @@ Subprocess::ChildHook Subprocess::ChildHook::DUP2(int oldFd, int newFd)
     return os::dup2(oldFd, newFd);
   });
 }
-
-
-Subprocess::ChildHook Subprocess::ChildHook::UNSET_CLOEXEC(int fd)
-{
-  return Subprocess::ChildHook([fd]() -> Try<Nothing> {
-    return os::unsetCloexec(fd);
-  });
-}
 #endif // __WINDOWS__
 
 


[mesos] 09/09: Added MESOS-9152 and MESOS-9164 to the 1.7.1 CHANGELOG.

Posted by qi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

qianzhang pushed a commit to branch 1.7.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit f87d6696ba220517f8e149f16f50ad2246c3d48b
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Thu Nov 8 15:59:20 2018 +0800

    Added MESOS-9152 and MESOS-9164 to the 1.7.1 CHANGELOG.
---
 CHANGELOG | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/CHANGELOG b/CHANGELOG
index 617a1aa..0353172 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -11,6 +11,8 @@ Release Notes - Mesos - Version 1.7.1 (WIP)
   * [MESOS-8907] - Docker image fetcher fails with HTTP/2.
   * [MESOS-8978] - Command executor calling setsid breaks the tty support.
   * [MESOS-9131] - Health checks launching nested containers while a container is being destroyed lead to unkillable tasks.
+  * [MESOS-9152] - Close all file descriptors except whitelist_fds in posix/subprocess.
+  * [MESOS-9164] - Subprocess should unset CLOEXEC on whitelisted file descriptors.
   * [MESOS-9228] - SLRP does not clean up plugin containers after it is removed.
   * [MESOS-9231] - `docker inspect` may return an unexpected result to Docker executor due to a race condition.
   * [MESOS-9267] - Mesos agent crashes when CNI network is not configured but used.