You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by gi...@apache.org on 2018/10/10 23:53:30 UTC

[mesos] branch 1.6.x updated (332b9c3 -> 193a446)

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

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


    from 332b9c3  Added MESOS-7506 to the 1.6.0 CHANGELOG.
     new 3d122db  Whitelist inheritable file descriptors in libprocess.
     new 85a5feb  Whitelist inheritable file descriptors in the containerizer.
     new eb49bbc  Fixed break of `port_mapping_tests.cpp`.
     new 0de8e0a  Updated `os::pipe()` to always return O_CLOEXEC descriptors.
     new 03fe697  Made the containerizer launch be explicit about O_CLOEXEC.
     new 6b51a47  Fixed Windows build break.
     new 2df7fe3  Updated the ::pipe() system calls to pipe2 in posix subprocess.
     new 1c279f4  Updated the ::pipe() system calls to pipe2 in lib_logrotate.
     new ffe4928  Fixed the lib_logrotate inappropriate UNSET_CLOEXEC via ChildHook.
     new b582881  Added MESOS-8128 to 1.6.2 CHANGELOG.
     new 193a446  Added MESOS-9151 to 1.6.2 CHANGELOG.

The 11 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 | 15 ++++---
 3rdparty/libprocess/src/subprocess.cpp             |  6 ++-
 3rdparty/libprocess/src/subprocess_posix.cpp       | 22 ++++-----
 3rdparty/libprocess/src/subprocess_windows.hpp     |  3 +-
 3rdparty/stout/include/stout/os/posix/pipe.hpp     | 52 +++++++++++++++++++++-
 CHANGELOG                                          |  2 +
 src/slave/container_loggers/lib_logrotate.cpp      | 42 +++++------------
 src/slave/containerizer/mesos/containerizer.cpp    | 18 ++++++--
 src/slave/containerizer/mesos/launcher.cpp         | 27 +++++++----
 src/slave/containerizer/mesos/launcher.hpp         | 15 +++----
 src/slave/containerizer/mesos/linux_launcher.cpp   | 45 ++++++++++---------
 src/slave/containerizer/mesos/linux_launcher.hpp   |  7 ++-
 src/tests/containerizer/launcher.cpp               |  6 +--
 src/tests/containerizer/launcher.hpp               |  9 ++--
 .../containerizer/mesos_containerizer_tests.cpp    |  7 ++-
 src/tests/containerizer/port_mapping_tests.cpp     | 10 ++---
 16 files changed, 176 insertions(+), 110 deletions(-)


[mesos] 10/11: Added MESOS-8128 to 1.6.2 CHANGELOG.

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

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

commit b5828812d244e82a5196a46fc5ad1546bc877ea8
Author: Gilbert Song <so...@gmail.com>
AuthorDate: Tue Sep 18 14:49:29 2018 -0700

    Added MESOS-8128 to 1.6.2 CHANGELOG.
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index a81678f..c35ccb3 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -3,6 +3,7 @@ Release Notes - Mesos - Version 1.6.2 (WIP)
 * This is a bug fix release.
 
 ** Bug
+  * [MESOS-8128] - Make os::pipe file descriptors O_CLOEXEC.
   * [MESOS-8418] - mesos-agent high cpu usage because of numerous /proc/mounts reads.
   * [MESOS-8545] - AgentAPIStreamingTest.AttachInputToNestedContainerSession is flaky.
   * [MESOS-8568] - Command checks should always call `WAIT_NESTED_CONTAINER` before `REMOVE_NESTED_CONTAINER`


[mesos] 01/11: Whitelist inheritable file descriptors in libprocess.

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

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

commit 3d122db03dd515a13b013292d202b51f57aafbf4
Author: Radhika Jandhyala <ra...@microsoft.com>
AuthorDate: Wed Jun 13 15:56:20 2018 -0700

    Whitelist inheritable file descriptors in libprocess.
    
    This commit plumbs the list of whitelisted file descriptors through
    the libprocess APIs.
    
    Review: https://reviews.apache.org/r/67287/
    (cherry picked from commit 281cf5dd7239dce5103ee34b64df2b785672271f)
---
 3rdparty/libprocess/include/process/subprocess.hpp | 15 ++++++++++-----
 3rdparty/libprocess/src/subprocess.cpp             |  6 ++++--
 3rdparty/libprocess/src/subprocess_windows.hpp     |  3 ++-
 3 files changed, 16 insertions(+), 8 deletions(-)

diff --git a/3rdparty/libprocess/include/process/subprocess.hpp b/3rdparty/libprocess/include/process/subprocess.hpp
index 6a12623..135bf24 100644
--- a/3rdparty/libprocess/include/process/subprocess.hpp
+++ b/3rdparty/libprocess/include/process/subprocess.hpp
@@ -125,7 +125,8 @@ public:
         const Option<lambda::function<
             pid_t(const lambda::function<int()>&)>>& clone,
         const std::vector<Subprocess::ParentHook>& parent_hooks,
-        const std::vector<Subprocess::ChildHook>& child_hooks);
+        const std::vector<Subprocess::ChildHook>& child_hooks,
+        const std::vector<int_fd>& whitelist_fds);
 
     IO(const lambda::function<Try<InputFileDescriptors>()>& _input,
        const lambda::function<Try<OutputFileDescriptors>()>& _output)
@@ -305,7 +306,8 @@ private:
       const Option<lambda::function<
           pid_t(const lambda::function<int()>&)>>& clone,
       const std::vector<Subprocess::ParentHook>& parent_hooks,
-      const std::vector<Subprocess::ChildHook>& child_hooks);
+      const std::vector<Subprocess::ChildHook>& child_hooks,
+      const std::vector<int_fd>& whitelist_fds);
 
   struct Data
   {
@@ -377,7 +379,8 @@ Try<Subprocess> subprocess(
     const Option<lambda::function<
         pid_t(const lambda::function<int()>&)>>& clone = None(),
     const std::vector<Subprocess::ParentHook>& parent_hooks = {},
-    const std::vector<Subprocess::ChildHook>& child_hooks = {});
+    const std::vector<Subprocess::ChildHook>& child_hooks = {},
+    const std::vector<int_fd>& whitelist_fds = {});
 
 
 /**
@@ -413,7 +416,8 @@ inline Try<Subprocess> subprocess(
     const Option<lambda::function<
         pid_t(const lambda::function<int()>&)>>& clone = None(),
     const std::vector<Subprocess::ParentHook>& parent_hooks = {},
-    const std::vector<Subprocess::ChildHook>& child_hooks = {})
+    const std::vector<Subprocess::ChildHook>& child_hooks = {},
+    const std::vector<int_fd>& whitelist_fds = {})
 {
   std::vector<std::string> argv = {os::Shell::arg0, os::Shell::arg1, command};
 
@@ -427,7 +431,8 @@ inline Try<Subprocess> subprocess(
       environment,
       clone,
       parent_hooks,
-      child_hooks);
+      child_hooks,
+      whitelist_fds);
 }
 
 } // namespace process {
diff --git a/3rdparty/libprocess/src/subprocess.cpp b/3rdparty/libprocess/src/subprocess.cpp
index d7a7253..0b2c02a 100644
--- a/3rdparty/libprocess/src/subprocess.cpp
+++ b/3rdparty/libprocess/src/subprocess.cpp
@@ -331,7 +331,8 @@ Try<Subprocess> subprocess(
     const Option<lambda::function<
         pid_t(const lambda::function<int()>&)>>& _clone,
     const vector<Subprocess::ParentHook>& parent_hooks,
-    const vector<Subprocess::ChildHook>& child_hooks)
+    const vector<Subprocess::ChildHook>& child_hooks,
+    const vector<int_fd>& whitelist_fds)
 {
   // TODO(hausdorff): We should error out on Windows here if we are passing
   // parameters that aren't used.
@@ -430,7 +431,8 @@ Try<Subprocess> subprocess(
           parent_hooks,
           stdinfds,
           stdoutfds,
-          stderrfds);
+          stderrfds,
+          whitelist_fds);
 
     if (process_data.isError()) {
       // NOTE: `createChildProcess` either succeeds entirely or returns an
diff --git a/3rdparty/libprocess/src/subprocess_windows.hpp b/3rdparty/libprocess/src/subprocess_windows.hpp
index c7ed0ad..5afd34c 100644
--- a/3rdparty/libprocess/src/subprocess_windows.hpp
+++ b/3rdparty/libprocess/src/subprocess_windows.hpp
@@ -51,7 +51,8 @@ inline Try<::internal::windows::ProcessData> createChildProcess(
     const std::vector<Subprocess::ParentHook>& parent_hooks,
     const InputFileDescriptors& stdinfds,
     const OutputFileDescriptors& stdoutfds,
-    const OutputFileDescriptors& stderrfds)
+    const OutputFileDescriptors& stderrfds,
+    const std::vector<int_fd>& whitelist_fds = {})
 {
   const std::array<int_fd, 3> fds{
     stdinfds.read, stdoutfds.write, stderrfds.write};


[mesos] 04/11: Updated `os::pipe()` to always return O_CLOEXEC descriptors.

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

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

commit 0de8e0a7908a2925f805c74bcd555fe46e99ff51
Author: James Peach <jp...@apache.org>
AuthorDate: Fri Aug 17 11:45:52 2018 -0700

    Updated `os::pipe()` to always return O_CLOEXEC descriptors.
    
    Updated `os::pipe()` to always return O_CLOEXEC descriptors,
    atomically if we are on Linux or FreeBSD and the `pipe2(2)`
    system call is available.
    
    Review: https://reviews.apache.org/r/63270/
    (cherry picked from commit 2388ca4bd3be3ed5da266e74b518dd284de1be94)
---
 3rdparty/stout/include/stout/os/posix/pipe.hpp | 52 +++++++++++++++++++++++++-
 1 file changed, 51 insertions(+), 1 deletion(-)

diff --git a/3rdparty/stout/include/stout/os/posix/pipe.hpp b/3rdparty/stout/include/stout/os/posix/pipe.hpp
index ac76224..9838d7b 100644
--- a/3rdparty/stout/include/stout/os/posix/pipe.hpp
+++ b/3rdparty/stout/include/stout/os/posix/pipe.hpp
@@ -15,20 +15,70 @@
 
 #include <unistd.h>
 
+#include <sys/syscall.h>
+
 #include <array>
 
 #include <stout/error.hpp>
 #include <stout/try.hpp>
 
+#include <stout/os/posix/fcntl.hpp>
+
 namespace os {
 
-// Create pipes for interprocess communication.
+// Create pipes for interprocess communication. The pipe file descriptors
+// will be marked O_CLOEXEC (atomically if the platform supports it). To
+// pass the pipe to a child process, the caller should clear the CLOEXEC
+// flag after fork(2) but before exec(2).
 inline Try<std::array<int, 2>> pipe()
 {
   std::array<int, 2> result;
+
+  // The pipe2() function appeared in FreeBSD 10.0.
+#if defined(_FreeBSD__) && __FreeBSD_version >= 1000000
+
+  if (::pipe2(result.data(), O_CLOEXEC) < 0) {
+    return ErrnoError();
+  }
+
+#else
+
+  // pipe2() appeared in Linux 2.6.27 and glibc 2.9.
+#if defined(__linux__) && defined(SYS_pipe2)
+  if (::syscall(SYS_pipe2, result.data(), O_CLOEXEC) == 0) {
+    return result;
+  }
+
+  // Fall back if the kernel doesn't support pipe2().
+  if (errno != ENOSYS) {
+    return ErrnoError();
+  }
+#endif
+
   if (::pipe(result.data()) < 0) {
     return ErrnoError();
   }
+
+  Try<Nothing> cloexec = Nothing();
+
+  cloexec = os::cloexec(result[0]);
+  if (cloexec.isError()) {
+    Error error = Error("Failed to cloexec pipe: " + cloexec.error());
+    ::close(result[0]);
+    ::close(result[1]);
+    return error;
+  }
+
+  cloexec = os::cloexec(result[1]);
+  if (cloexec.isError()) {
+    Error error = Error("Failed to cloexec pipe: " + cloexec.error());
+    ::close(result[0]);
+    ::close(result[1]);
+    return error;
+  }
+
+#endif
+
   return result;
 }
 


[mesos] 02/11: Whitelist inheritable file descriptors in the containerizer.

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

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

commit 85a5febd5c44d18ff590c7ba57cbc0f38cdf5144
Author: Radhika Jandhyala <ra...@microsoft.com>
AuthorDate: Wed Jun 13 15:57:31 2018 -0700

    Whitelist inheritable file descriptors in the containerizer.
    
    This commit whitelists the read and write ends of the control pipe
    that are intended to be inherited by the containerizer. This is
    necessary because this pipe is passed to the child process
    implicitly (through environment variables), so previously the
    libprocess and stout APIs had no knowledge that these file descriptors
    needed to be inheritable.
    
    Adding the whitelist as yet another parameter caused us to exceed the
    mock methods of Google Mock, so we had to squash three other
    parameters into one, name the `containerIO` in/out/err fields are
    instead passed as one, unwrapped later.
    
    Also, a new `forkImpl` method had to be added to the tests because it
    is not possible to mock a function with default arguments in Google
    Mock, but this can be worked around by mocking in an implementation.
    
    Review: https://reviews.apache.org/r/67394/
    (cherry picked from commit f8c8c35af920518ecb6c56d873dd44160390c5c7)
---
 src/slave/containerizer/mesos/containerizer.cpp    |  9 +++---
 src/slave/containerizer/mesos/launcher.cpp         | 16 +++++-----
 src/slave/containerizer/mesos/launcher.hpp         | 15 +++++-----
 src/slave/containerizer/mesos/linux_launcher.cpp   | 34 ++++++++++------------
 src/slave/containerizer/mesos/linux_launcher.hpp   |  7 ++---
 src/tests/containerizer/launcher.cpp               |  6 ++--
 src/tests/containerizer/launcher.hpp               |  9 +++---
 .../containerizer/mesos_containerizer_tests.cpp    |  7 ++---
 8 files changed, 48 insertions(+), 55 deletions(-)

diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index d9550c0..2733235 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1873,6 +1873,8 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
   launchFlags.pipe_read = pipes[0];
   launchFlags.pipe_write = pipes[1];
 
+  const vector<int_fd> whitelistFds{pipes[0], pipes[1]};
+
 #ifndef __WINDOWS__
   // Set the `runtime_directory` launcher flag so that the launch
   // helper knows where to checkpoint the status of the container
@@ -1969,15 +1971,14 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
       containerId,
       argv[0],
       argv,
-      containerIO->in,
-      containerIO->out,
-      containerIO->err,
+      containerIO.get(),
       nullptr,
       launchEnvironment,
       // 'enterNamespaces' will be ignored by SubprocessLauncher.
       _enterNamespaces,
       // 'cloneNamespaces' will be ignored by SubprocessLauncher.
-      _cloneNamespaces);
+      _cloneNamespaces,
+      whitelistFds);
 
   if (forked.isError()) {
     return Failure("Failed to fork: " + forked.error());
diff --git a/src/slave/containerizer/mesos/launcher.cpp b/src/slave/containerizer/mesos/launcher.cpp
index 2fe47d3..42aa544 100644
--- a/src/slave/containerizer/mesos/launcher.cpp
+++ b/src/slave/containerizer/mesos/launcher.cpp
@@ -85,13 +85,12 @@ Try<pid_t> SubprocessLauncher::fork(
     const ContainerID& containerId,
     const string& path,
     const vector<string>& argv,
-    const Subprocess::IO& in,
-    const Subprocess::IO& out,
-    const Subprocess::IO& err,
+    const mesos::slave::ContainerIO& containerIO,
     const flags::FlagsBase* flags,
     const Option<map<string, string>>& environment,
     const Option<int>& enterNamespaces,
-    const Option<int>& cloneNamespaces)
+    const Option<int>& cloneNamespaces,
+    const vector<int_fd>& whitelistFds)
 {
   if (enterNamespaces.isSome() && enterNamespaces.get() != 0) {
     return Error("Subprocess launcher does not support entering namespaces");
@@ -124,14 +123,15 @@ Try<pid_t> SubprocessLauncher::fork(
   Try<Subprocess> child = subprocess(
       path,
       argv,
-      in,
-      out,
-      err,
+      containerIO.in,
+      containerIO.out,
+      containerIO.err,
       flags,
       environment,
       None(),
       parentHooks,
-      {Subprocess::ChildHook::SETSID()});
+      {Subprocess::ChildHook::SETSID()},
+      whitelistFds);
 
   if (child.isError()) {
     return Error("Failed to fork a child process: " + child.error());
diff --git a/src/slave/containerizer/mesos/launcher.hpp b/src/slave/containerizer/mesos/launcher.hpp
index f69d934..90a50b1 100644
--- a/src/slave/containerizer/mesos/launcher.hpp
+++ b/src/slave/containerizer/mesos/launcher.hpp
@@ -38,6 +38,7 @@
 #include <stout/try.hpp>
 
 #include "slave/flags.hpp"
+#include "slave/containerizer/containerizer.hpp"
 
 namespace mesos {
 namespace internal {
@@ -64,13 +65,12 @@ public:
       const ContainerID& containerId,
       const std::string& path,
       const std::vector<std::string>& argv,
-      const process::Subprocess::IO& in,
-      const process::Subprocess::IO& out,
-      const process::Subprocess::IO& err,
+      const mesos::slave::ContainerIO& containerIO,
       const flags::FlagsBase* flags,
       const Option<std::map<std::string, std::string>>& environment,
       const Option<int>& enterNamespaces,
-      const Option<int>& cloneNamespaces) = 0;
+      const Option<int>& cloneNamespaces,
+      const std::vector<int_fd>& whitelistFds) = 0;
 
   // Kill all processes in the containerized context.
   virtual process::Future<Nothing> destroy(const ContainerID& containerId) = 0;
@@ -102,13 +102,12 @@ public:
       const ContainerID& containerId,
       const std::string& path,
       const std::vector<std::string>& argv,
-      const process::Subprocess::IO& in,
-      const process::Subprocess::IO& out,
-      const process::Subprocess::IO& err,
+      const mesos::slave::ContainerIO& containerIO,
       const flags::FlagsBase* flags,
       const Option<std::map<std::string, std::string>>& environment,
       const Option<int>& enterNamespaces,
-      const Option<int>& cloneNamespaces);
+      const Option<int>& cloneNamespaces,
+      const std::vector<int_fd>& whitelistFds);
 
   virtual process::Future<Nothing> destroy(const ContainerID& containerId);
 
diff --git a/src/slave/containerizer/mesos/linux_launcher.cpp b/src/slave/containerizer/mesos/linux_launcher.cpp
index 80e4445..b176fd3 100644
--- a/src/slave/containerizer/mesos/linux_launcher.cpp
+++ b/src/slave/containerizer/mesos/linux_launcher.cpp
@@ -74,13 +74,12 @@ public:
       const ContainerID& containerId,
       const string& path,
       const vector<string>& argv,
-      const process::Subprocess::IO& in,
-      const process::Subprocess::IO& out,
-      const process::Subprocess::IO& err,
+      const mesos::slave::ContainerIO& containerIO,
       const flags::FlagsBase* flags,
       const Option<map<string, string>>& environment,
       const Option<int>& enterNamespaces,
-      const Option<int>& cloneNamespaces);
+      const Option<int>& cloneNamespaces,
+      const vector<int_fd>& whitelistFds);
 
   virtual process::Future<Nothing> destroy(const ContainerID& containerId);
 
@@ -250,13 +249,12 @@ Try<pid_t> LinuxLauncher::fork(
     const ContainerID& containerId,
     const string& path,
     const vector<string>& argv,
-    const process::Subprocess::IO& in,
-    const process::Subprocess::IO& out,
-    const process::Subprocess::IO& err,
+    const mesos::slave::ContainerIO& containerIO,
     const flags::FlagsBase* flags,
     const Option<map<string, string>>& environment,
     const Option<int>& enterNamespaces,
-    const Option<int>& cloneNamespaces)
+    const Option<int>& cloneNamespaces,
+    const vector<int_fd>& whitelistFds)
 {
   return dispatch(
       process.get(),
@@ -264,13 +262,12 @@ Try<pid_t> LinuxLauncher::fork(
       containerId,
       path,
       argv,
-      in,
-      out,
-      err,
+      containerIO,
       flags,
       environment,
       enterNamespaces,
-      cloneNamespaces).get();
+      cloneNamespaces,
+      whitelistFds).get();
 }
 
 
@@ -464,13 +461,12 @@ Try<pid_t> LinuxLauncherProcess::fork(
     const ContainerID& containerId,
     const string& path,
     const vector<string>& argv,
-    const process::Subprocess::IO& in,
-    const process::Subprocess::IO& out,
-    const process::Subprocess::IO& err,
+    const mesos::slave::ContainerIO& containerIO,
     const flags::FlagsBase* flags,
     const Option<map<string, string>>& environment,
     const Option<int>& enterNamespaces,
-    const Option<int>& cloneNamespaces)
+    const Option<int>& cloneNamespaces,
+    const vector<int_fd>& whitelistFds)
 {
   // Make sure this container (nested or not) is unique.
   if (containers.contains(containerId)) {
@@ -545,9 +541,9 @@ Try<pid_t> LinuxLauncherProcess::fork(
   Try<Subprocess> child = subprocess(
       path,
       argv,
-      in,
-      out,
-      err,
+      containerIO.in,
+      containerIO.out,
+      containerIO.err,
       flags,
       environment,
       [target, enterFlags, cloneFlags](const lambda::function<int()>& child) {
diff --git a/src/slave/containerizer/mesos/linux_launcher.hpp b/src/slave/containerizer/mesos/linux_launcher.hpp
index 0ea9b87..712c9f8 100644
--- a/src/slave/containerizer/mesos/linux_launcher.hpp
+++ b/src/slave/containerizer/mesos/linux_launcher.hpp
@@ -52,13 +52,12 @@ public:
       const ContainerID& containerId,
       const std::string& path,
       const std::vector<std::string>& argv,
-      const process::Subprocess::IO& in,
-      const process::Subprocess::IO& out,
-      const process::Subprocess::IO& err,
+      const mesos::slave::ContainerIO& containerIO,
       const flags::FlagsBase* flags,
       const Option<std::map<std::string, std::string>>& environment,
       const Option<int>& enterNamespaces,
-      const Option<int>& cloneNamespaces);
+      const Option<int>& cloneNamespaces,
+      const std::vector<int_fd>& whitelistFds);
 
   virtual process::Future<Nothing> destroy(const ContainerID& containerId);
 
diff --git a/src/tests/containerizer/launcher.cpp b/src/tests/containerizer/launcher.cpp
index a92d989..51ae4f9 100644
--- a/src/tests/containerizer/launcher.cpp
+++ b/src/tests/containerizer/launcher.cpp
@@ -30,7 +30,7 @@ ACTION_P(InvokeRecover, launcher)
 ACTION_P(InvokeFork, launcher)
 {
   return launcher->real->fork(
-      arg0, arg1, arg2, arg3, arg4, arg5, arg6, arg7, arg8, arg9);
+      arg0, arg1, arg2, arg3, arg4, arg5, arg6, arg7, arg8);
 }
 
 
@@ -51,9 +51,9 @@ TestLauncher::TestLauncher(const process::Owned<slave::Launcher>& _real)
   EXPECT_CALL(*this, recover(_))
     .WillRepeatedly(DoDefault());
 
-  ON_CALL(*this, fork(_, _, _, _, _, _, _, _, _, _))
+  ON_CALL(*this, fork(_, _, _, _, _, _, _, _, _))
     .WillByDefault(InvokeFork(this));
-  EXPECT_CALL(*this, fork(_, _, _, _, _, _, _, _, _, _))
+  EXPECT_CALL(*this, fork(_, _, _, _, _, _, _, _, _))
     .WillRepeatedly(DoDefault());
 
   ON_CALL(*this, destroy(_))
diff --git a/src/tests/containerizer/launcher.hpp b/src/tests/containerizer/launcher.hpp
index a8e436f..6057286 100644
--- a/src/tests/containerizer/launcher.hpp
+++ b/src/tests/containerizer/launcher.hpp
@@ -56,19 +56,18 @@ public:
       process::Future<hashset<ContainerID>>(
           const std::list<mesos::slave::ContainerState>& states));
 
-  MOCK_METHOD10(
+  MOCK_METHOD9(
       fork,
       Try<pid_t>(
           const ContainerID& containerId,
           const std::string& path,
           const std::vector<std::string>& argv,
-          const process::Subprocess::IO& in,
-          const process::Subprocess::IO& out,
-          const process::Subprocess::IO& err,
+          const mesos::slave::ContainerIO& containerIO,
           const flags::FlagsBase* flags,
           const Option<std::map<std::string, std::string>>& env,
           const Option<int>& enterNamespaces,
-          const Option<int>& cloneNamespaces));
+          const Option<int>& cloneNamespaces,
+          const std::vector<int_fd>& whitelistFds));
 
   MOCK_METHOD1(
       destroy,
diff --git a/src/tests/containerizer/mesos_containerizer_tests.cpp b/src/tests/containerizer/mesos_containerizer_tests.cpp
index 01f2b38..acaafdd 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -1339,13 +1339,12 @@ TEST_F(MesosLauncherStatusTest, ExecutorPIDTest)
       containerId,
       path::join(flags.launcher_dir, MESOS_CONTAINERIZER),
       vector<string>(),
-      Subprocess::FD(STDIN_FILENO),
-      Subprocess::FD(STDOUT_FILENO),
-      Subprocess::FD(STDERR_FILENO),
+      mesos::slave::ContainerIO(),
       nullptr,
       None(),
       None(),
-      None());
+      None(),
+      vector<int_fd>());
 
   ASSERT_SOME(forked);
 


[mesos] 06/11: Fixed Windows build break.

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

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

commit 6b51a471c866128d75fa385364e2d9151342d663
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
AuthorDate: Fri Aug 17 17:10:44 2018 -0700

    Fixed Windows build break.
    
    The `SETSID()` and `UNSET_CLOEXEC()` child hooks do not exist on
    Windows (in fact, no child hooks exist on Windows).
    
    (cherry picked from commit 0ac5af21d7d2408b91e877b163c8c70b6ef81438)
---
 src/slave/containerizer/mesos/launcher.cpp | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/src/slave/containerizer/mesos/launcher.cpp b/src/slave/containerizer/mesos/launcher.cpp
index 845180d..e44f273 100644
--- a/src/slave/containerizer/mesos/launcher.cpp
+++ b/src/slave/containerizer/mesos/launcher.cpp
@@ -122,12 +122,14 @@ Try<pid_t> SubprocessLauncher::fork(
 
   vector<Subprocess::ChildHook> childHooks;
 
+#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(
       path,


[mesos] 07/11: Updated the ::pipe() system calls to pipe2 in posix subprocess.

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

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

commit 2df7fe3b5a2cee176c56b5b370c2bfc17b93ae36
Author: Gilbert Song <so...@gmail.com>
AuthorDate: Thu Aug 16 11:03:52 2018 -0700

    Updated the ::pipe() system calls to pipe2 in posix subprocess.
    
    Review: https://reviews.apache.org/r/68396
    (cherry picked from commit 7a7a879310a5c01b69c8a56b3c1d85555888ec3e)
---
 3rdparty/libprocess/src/subprocess_posix.cpp | 22 ++++++++++++----------
 1 file changed, 12 insertions(+), 10 deletions(-)

diff --git a/3rdparty/libprocess/src/subprocess_posix.cpp b/3rdparty/libprocess/src/subprocess_posix.cpp
index 01e3272..caba704 100644
--- a/3rdparty/libprocess/src/subprocess_posix.cpp
+++ b/3rdparty/libprocess/src/subprocess_posix.cpp
@@ -28,10 +28,12 @@
 #include <stout/foreach.hpp>
 #include <stout/option.hpp>
 #include <stout/os.hpp>
+#include <stout/os/pipe.hpp>
 #include <stout/os/strerror.hpp>
 #include <stout/strings.hpp>
 #include <stout/try.hpp>
 
+using std::array;
 using std::map;
 using std::string;
 using std::vector;
@@ -46,25 +48,25 @@ Subprocess::IO Subprocess::PIPE()
 {
   return Subprocess::IO(
       []() -> Try<InputFileDescriptors> {
-        int pipefd[2];
-        if (::pipe(pipefd) == -1) {
-          return ErrnoError("Failed to create pipe");
+        Try<array<int, 2>> pipefd = os::pipe();
+        if (pipefd.isError()) {
+          return Error(pipefd.error());
         }
 
         InputFileDescriptors fds;
-        fds.read = pipefd[0];
-        fds.write = pipefd[1];
+        fds.read = pipefd->at(0);
+        fds.write = pipefd->at(1);
         return fds;
       },
       []() -> Try<OutputFileDescriptors> {
-        int pipefd[2];
-        if (::pipe(pipefd) == -1) {
-          return ErrnoError("Failed to create pipe");
+        Try<array<int, 2>> pipefd = os::pipe();
+        if (pipefd.isError()) {
+          return Error(pipefd.error());
         }
 
         OutputFileDescriptors fds;
-        fds.read = pipefd[0];
-        fds.write = pipefd[1];
+        fds.read = pipefd->at(0);
+        fds.write = pipefd->at(1);
         return fds;
       });
 }


[mesos] 03/11: Fixed break of `port_mapping_tests.cpp`.

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

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

commit eb49bbc662d74d4d751b205b442f775aef826696
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
AuthorDate: Thu Jun 14 09:45:34 2018 -0700

    Fixed break of `port_mapping_tests.cpp`.
    
    The last change applied to the file descriptor whitelist chain removed
    the default value from the new argument, and this function was not
    updated. The other change made was that the three FD arguments were
    turned into one `ContainerIO`. This was missed because this entire
    isolator and its associated tests are not in the CMake build.
    
    Review: https://reviews.apache.org/r/67600
    (cherry picked from commit 85adb1e8711325acb169239a2d9160e8f18f9579)
---
 src/tests/containerizer/port_mapping_tests.cpp | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)

diff --git a/src/tests/containerizer/port_mapping_tests.cpp b/src/tests/containerizer/port_mapping_tests.cpp
index 575ba46..553f530 100644
--- a/src/tests/containerizer/port_mapping_tests.cpp
+++ b/src/tests/containerizer/port_mapping_tests.cpp
@@ -35,8 +35,9 @@
 #include <stout/stopwatch.hpp>
 
 #include <stout/os/constants.hpp>
-#include <stout/os/stat.hpp>
 #include <stout/os/exists.hpp>
+#include <stout/os/int_fd.hpp>
+#include <stout/os/stat.hpp>
 
 #include "linux/fs.hpp"
 #include "linux/ns.hpp"
@@ -346,13 +347,12 @@ protected:
         containerId,
         path::join(flags.launcher_dir, MESOS_CONTAINERIZER),
         argv,
-        Subprocess::FD(STDIN_FILENO),
-        Subprocess::FD(STDOUT_FILENO),
-        Subprocess::FD(STDERR_FILENO),
+        mesos::slave::ContainerIO(),
         &launchFlags,
         None(),
         None(),
-        CLONE_NEWNET | CLONE_NEWNS);
+        CLONE_NEWNET | CLONE_NEWNS,
+        vector<int_fd>());
 
     return pid;
   }


[mesos] 08/11: Updated the ::pipe() system calls to pipe2 in lib_logrotate.

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

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

commit 1c279f4d3c5e117bb16fbffba6ebec23f5a0fbad
Author: Gilbert Song <so...@gmail.com>
AuthorDate: Thu Aug 16 11:05:41 2018 -0700

    Updated the ::pipe() system calls to pipe2 in lib_logrotate.
    
    Review: https://reviews.apache.org/r/68397
    (cherry picked from commit 3f49cf4351d200e4ba0ac7aa2cb69791b2786a23)
---
 src/slave/container_loggers/lib_logrotate.cpp | 64 ++++++++++++++-------------
 1 file changed, 33 insertions(+), 31 deletions(-)

diff --git a/src/slave/container_loggers/lib_logrotate.cpp b/src/slave/container_loggers/lib_logrotate.cpp
index 575ed6e..bdc8f27 100644
--- a/src/slave/container_loggers/lib_logrotate.cpp
+++ b/src/slave/container_loggers/lib_logrotate.cpp
@@ -43,6 +43,7 @@
 #include <stout/os/environment.hpp>
 #include <stout/os/fcntl.hpp>
 #include <stout/os/killtree.hpp>
+#include <stout/os/pipe.hpp>
 
 #ifdef __linux__
 #include "linux/systemd.hpp"
@@ -55,6 +56,7 @@
 using namespace mesos;
 using namespace process;
 
+using std::array;
 using std::map;
 using std::string;
 
@@ -151,24 +153,16 @@ public:
     // of the pipe and will be solely responsible for closing that end.
     // The ownership of the write-end will be passed to the caller
     // of this function.
-    int pipefd[2];
-    if (::pipe(pipefd) == -1) {
-      return Failure(ErrnoError("Failed to create pipe").message);
+    Try<array<int, 2>> pipefd = os::pipe();
+    if (pipefd.isError()) {
+      return Failure("Failed to create pipe: " + pipefd.error());
     }
 
     Subprocess::IO::InputFileDescriptors outfds;
-    outfds.read = pipefd[0];
-    outfds.write = pipefd[1];
-
-    // NOTE: We need to `cloexec` this FD so that it will be closed when
-    // the child subprocess is spawned and so that the FD will not be
-    // inherited by the second child for stderr.
-    Try<Nothing> cloexec = os::cloexec(outfds.write.get());
-    if (cloexec.isError()) {
-      os::close(outfds.read);
-      os::close(outfds.write.get());
-      return Failure("Failed to cloexec: " + cloexec.error());
-    }
+    outfds.read = pipefd->at(0);
+    outfds.write = pipefd->at(1);
+
+    const std::vector<int_fd> whitelistOutFds{pipefd->at(0), pipefd->at(1)};
 
     // Spawn a process to handle stdout.
     mesos::internal::logger::rotate::Flags outFlags;
@@ -189,6 +183,12 @@ public:
     }
 #endif // __linux__
 
+    // TODO(gilbert): libprocess should take care of this, see MESOS-9164.
+    std::vector<Subprocess::ChildHook> childHooks;
+    foreach (int_fd fd, whitelistOutFds) {
+      childHooks.push_back(Subprocess::ChildHook::UNSET_CLOEXEC(fd));
+    }
+
     Try<Subprocess> outProcess = subprocess(
         path::join(flags.launcher_dir, mesos::internal::logger::rotate::NAME),
         {mesos::internal::logger::rotate::NAME},
@@ -198,7 +198,9 @@ public:
         &outFlags,
         environment,
         None(),
-        parentHooks);
+        parentHooks,
+        childHooks,
+        whitelistOutFds);
 
     if (outProcess.isError()) {
       os::close(outfds.write.get());
@@ -207,26 +209,18 @@ public:
 
     // NOTE: We manually construct a pipe here to properly express
     // ownership of the FDs.  See the NOTE above.
-    if (::pipe(pipefd) == -1) {
+    pipefd = os::pipe();
+    if (pipefd.isError()) {
       os::close(outfds.write.get());
       os::killtree(outProcess->pid(), SIGKILL);
-      return Failure(ErrnoError("Failed to create pipe").message);
+      return Failure("Failed to create pipe: " + pipefd.error());
     }
 
     Subprocess::IO::InputFileDescriptors errfds;
-    errfds.read = pipefd[0];
-    errfds.write = pipefd[1];
+    errfds.read = pipefd->at(0);
+    errfds.write = pipefd->at(1);
 
-    // NOTE: We need to `cloexec` this FD so that it will be closed when
-    // the child subprocess is spawned.
-    cloexec = os::cloexec(errfds.write.get());
-    if (cloexec.isError()) {
-      os::close(outfds.write.get());
-      os::close(errfds.read);
-      os::close(errfds.write.get());
-      os::killtree(outProcess->pid(), SIGKILL);
-      return Failure("Failed to cloexec: " + cloexec.error());
-    }
+    const std::vector<int_fd> whitelistErrFds{pipefd->at(0), pipefd->at(1)};
 
     // Spawn a process to handle stderr.
     mesos::internal::logger::rotate::Flags errFlags;
@@ -236,6 +230,12 @@ public:
     errFlags.logrotate_path = flags.logrotate_path;
     errFlags.user = user;
 
+    // TODO(gilbert): libprocess should take care of this, see MESOS-9164.
+    childHooks.clear();
+    foreach (int_fd fd, whitelistErrFds) {
+      childHooks.push_back(Subprocess::ChildHook::UNSET_CLOEXEC(fd));
+    }
+
     Try<Subprocess> errProcess = subprocess(
         path::join(flags.launcher_dir, mesos::internal::logger::rotate::NAME),
         {mesos::internal::logger::rotate::NAME},
@@ -245,7 +245,9 @@ public:
         &errFlags,
         environment,
         None(),
-        parentHooks);
+        parentHooks,
+        childHooks,
+        whitelistErrFds);
 
     if (errProcess.isError()) {
       os::close(outfds.write.get());


[mesos] 05/11: Made the containerizer launch be explicit about O_CLOEXEC.

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

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

commit 03fe69797849335aa25e7e22b5c34f3fe622e8f5
Author: James Peach <jp...@apache.org>
AuthorDate: Fri Aug 17 11:45:56 2018 -0700

    Made the containerizer launch be explicit about O_CLOEXEC.
    
    Since the containerizer launch depends on the inherited pipe to signal
    the forked child, be explicit about manipulating O_CLOEXEC on the pipe
    file descriptors. Make sure to close the pipe on the error paths.
    
    Review: https://reviews.apache.org/r/63280/
    (cherry picked from commit 7569ffd05903be9b5284100f67eeffb35fcc7703)
---
 src/slave/containerizer/mesos/containerizer.cpp  |  9 +++++++++
 src/slave/containerizer/mesos/launcher.cpp       | 11 ++++++++++-
 src/slave/containerizer/mesos/linux_launcher.cpp | 11 ++++++++++-
 3 files changed, 29 insertions(+), 2 deletions(-)

diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 2733235..2b28e41 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1981,6 +1981,9 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
       whitelistFds);
 
   if (forked.isError()) {
+    os::close(pipes[0]);
+    os::close(pipes[1]);
+
     return Failure("Failed to fork: " + forked.error());
   }
 
@@ -2000,6 +2003,9 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
       LOG(ERROR) << "Failed to checkpoint container's forked pid to '"
                  << pidCheckpointPath.get() << "': " << checkpointed.error();
 
+      os::close(pipes[0]);
+      os::close(pipes[1]);
+
       return Failure("Could not checkpoint container's pid");
     }
   }
@@ -2023,6 +2029,9 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
   checkpointed = slave::state::checkpoint(pidPath, stringify(pid));
 
   if (checkpointed.isError()) {
+    os::close(pipes[0]);
+    os::close(pipes[1]);
+
     return Failure("Failed to checkpoint the container pid to"
                    " '" + pidPath + "': " + checkpointed.error());
   }
diff --git a/src/slave/containerizer/mesos/launcher.cpp b/src/slave/containerizer/mesos/launcher.cpp
index 42aa544..845180d 100644
--- a/src/slave/containerizer/mesos/launcher.cpp
+++ b/src/slave/containerizer/mesos/launcher.cpp
@@ -120,6 +120,15 @@ Try<pid_t> SubprocessLauncher::fork(
   parentHooks.emplace_back(Subprocess::ParentHook::CREATE_JOB());
 #endif // __linux__
 
+  vector<Subprocess::ChildHook> childHooks;
+
+  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,
@@ -130,7 +139,7 @@ Try<pid_t> SubprocessLauncher::fork(
       environment,
       None(),
       parentHooks,
-      {Subprocess::ChildHook::SETSID()},
+      childHooks,
       whitelistFds);
 
   if (child.isError()) {
diff --git a/src/slave/containerizer/mesos/linux_launcher.cpp b/src/slave/containerizer/mesos/linux_launcher.cpp
index b176fd3..d180690 100644
--- a/src/slave/containerizer/mesos/linux_launcher.cpp
+++ b/src/slave/containerizer/mesos/linux_launcher.cpp
@@ -538,6 +538,15 @@ Try<pid_t> LinuxLauncherProcess::fork(
     }));
   }
 
+  vector<Subprocess::ChildHook> childHooks;
+
+  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,
@@ -564,7 +573,7 @@ Try<pid_t> LinuxLauncherProcess::fork(
         }
       },
       parentHooks,
-      {Subprocess::ChildHook::SETSID()});
+      childHooks);
 
   if (child.isError()) {
     return Error("Failed to clone child process: " + child.error());


[mesos] 11/11: Added MESOS-9151 to 1.6.2 CHANGELOG.

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

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

commit 193a4467d5da3a5ec72b9e7af0d8b300953a309f
Author: Gilbert Song <so...@gmail.com>
AuthorDate: Tue Sep 18 14:52:36 2018 -0700

    Added MESOS-9151 to 1.6.2 CHANGELOG.
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index c35ccb3..30e10a7 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -18,6 +18,7 @@ Release Notes - Mesos - Version 1.6.2 (WIP)
   * [MESOS-9145] - Master has a fragile burned-in 5s authentication timeout.
   * [MESOS-9146] - Agent has a fragile burn-in 5s authentication timeout.
   * [MESOS-9147] - Agent and scheduler driver authentication retry backoff time could overflow.
+  * [MESOS-9151] - Container stuck at ISOLATING due to FD leak.
   * [MESOS-9170] - Zookeeper doesn't compile with newer gcc due to format error.
   * [MESOS-9196] - Removing rootfs mounts may fail with EBUSY.
   * [MESOS-9267] - Mesos agent crashes when CNI network is not configured but used.


[mesos] 09/11: Fixed the lib_logrotate inappropriate UNSET_CLOEXEC via ChildHook.

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

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

commit ffe4928447cdc72a0bbdea54dfc3cba20b51e326
Author: Gilbert Song <so...@gmail.com>
AuthorDate: Tue Aug 21 14:35:01 2018 -0700

    Fixed the lib_logrotate inappropriate UNSET_CLOEXEC via ChildHook.
    
    Previously, we call os::unsetCloexec() on write fd for both outfds and
    errfds. Basically, this is not needed because all stdout and stderr
    fds will be closed at childMain() method. As a result, no need to do
    unsetCloexec() either from the parent process side or UNSET_CLOEXEC
    via the ChildHook.
    
    Review: https://reviews.apache.org/r/68458
    (cherry picked from commit b9d19cd4e03f70014063341c64f96b5860db8b0f)
---
 src/slave/container_loggers/lib_logrotate.cpp | 24 ++----------------------
 1 file changed, 2 insertions(+), 22 deletions(-)

diff --git a/src/slave/container_loggers/lib_logrotate.cpp b/src/slave/container_loggers/lib_logrotate.cpp
index bdc8f27..9a344d9 100644
--- a/src/slave/container_loggers/lib_logrotate.cpp
+++ b/src/slave/container_loggers/lib_logrotate.cpp
@@ -162,8 +162,6 @@ public:
     outfds.read = pipefd->at(0);
     outfds.write = pipefd->at(1);
 
-    const std::vector<int_fd> whitelistOutFds{pipefd->at(0), pipefd->at(1)};
-
     // Spawn a process to handle stdout.
     mesos::internal::logger::rotate::Flags outFlags;
     outFlags.max_size = overriddenFlags.max_stdout_size;
@@ -183,12 +181,6 @@ public:
     }
 #endif // __linux__
 
-    // TODO(gilbert): libprocess should take care of this, see MESOS-9164.
-    std::vector<Subprocess::ChildHook> childHooks;
-    foreach (int_fd fd, whitelistOutFds) {
-      childHooks.push_back(Subprocess::ChildHook::UNSET_CLOEXEC(fd));
-    }
-
     Try<Subprocess> outProcess = subprocess(
         path::join(flags.launcher_dir, mesos::internal::logger::rotate::NAME),
         {mesos::internal::logger::rotate::NAME},
@@ -198,9 +190,7 @@ public:
         &outFlags,
         environment,
         None(),
-        parentHooks,
-        childHooks,
-        whitelistOutFds);
+        parentHooks);
 
     if (outProcess.isError()) {
       os::close(outfds.write.get());
@@ -220,8 +210,6 @@ public:
     errfds.read = pipefd->at(0);
     errfds.write = pipefd->at(1);
 
-    const std::vector<int_fd> whitelistErrFds{pipefd->at(0), pipefd->at(1)};
-
     // Spawn a process to handle stderr.
     mesos::internal::logger::rotate::Flags errFlags;
     errFlags.max_size = overriddenFlags.max_stderr_size;
@@ -230,12 +218,6 @@ public:
     errFlags.logrotate_path = flags.logrotate_path;
     errFlags.user = user;
 
-    // TODO(gilbert): libprocess should take care of this, see MESOS-9164.
-    childHooks.clear();
-    foreach (int_fd fd, whitelistErrFds) {
-      childHooks.push_back(Subprocess::ChildHook::UNSET_CLOEXEC(fd));
-    }
-
     Try<Subprocess> errProcess = subprocess(
         path::join(flags.launcher_dir, mesos::internal::logger::rotate::NAME),
         {mesos::internal::logger::rotate::NAME},
@@ -245,9 +227,7 @@ public:
         &errFlags,
         environment,
         None(),
-        parentHooks,
-        childHooks,
-        whitelistErrFds);
+        parentHooks);
 
     if (errProcess.isError()) {
       os::close(outfds.write.get());