You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@mesos.apache.org by GitBox <gi...@apache.org> on 2018/10/10 23:54:16 UTC

[GitHub] asfgit closed pull request #311: BugFix backporting 1.4.x

asfgit closed pull request #311: BugFix backporting 1.4.x
URL: https://github.com/apache/mesos/pull/311
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/3rdparty/libprocess/include/process/subprocess.hpp b/3rdparty/libprocess/include/process/subprocess.hpp
index 6a1262340c..135bf243c7 100644
--- a/3rdparty/libprocess/include/process/subprocess.hpp
+++ b/3rdparty/libprocess/include/process/subprocess.hpp
@@ -125,7 +125,8 @@ class Subprocess
         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 @@ class Subprocess
       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 785e2e1083..fffb640769 100644
--- a/3rdparty/libprocess/src/subprocess.cpp
+++ b/3rdparty/libprocess/src/subprocess.cpp
@@ -324,7 +324,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.
@@ -423,7 +424,8 @@ Try<Subprocess> subprocess(
           parent_hooks,
           stdinfds,
           stdoutfds,
-          stderrfds);
+          stderrfds,
+          whitelist_fds);
 
     if (process_data.isError()) {
       process::internal::close(stdinfds, stdoutfds, stderrfds);
diff --git a/3rdparty/libprocess/src/subprocess_posix.cpp b/3rdparty/libprocess/src/subprocess_posix.cpp
index 01e3272fcc..caba704833 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;
       });
 }
diff --git a/3rdparty/libprocess/src/subprocess_windows.hpp b/3rdparty/libprocess/src/subprocess_windows.hpp
index 0183bb451f..fab57335ed 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 = {})
 {
   Try<::internal::windows::ProcessData> process_data =
     ::internal::windows::create_process(
diff --git a/3rdparty/stout/include/stout/os/posix/pipe.hpp b/3rdparty/stout/include/stout/os/posix/pipe.hpp
index ac76224ff8..9838d7b3ab 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;
 }
 
diff --git a/CHANGELOG b/CHANGELOG
index 7e32ac212d..f87df6795a 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -3,14 +3,18 @@ Release Notes - Mesos - Version 1.4.3 (WIP)
 * This is a bug fix release.
 
 ** Bug
+  * [MESOS-8128] - Make os::pipe file descriptors O_CLOEXEC.
   * [MESOS-8568] - Command checks should always call `WAIT_NESTED_CONTAINER` before `REMOVE_NESTED_CONTAINER`
   * [MESOS-8620] - Containers stuck in FETCHING possibly due to unresponsive server.
+  * [MESOS-8917] - Agent leaking file descriptors into forked processes.
   * [MESOS-8921] - Autotools don't work with newer OpenJDK versions
   * [MESOS-9144] - Master authentication handling leads to request amplification.
   * [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.
 
 
 Release Notes - Mesos - Version 1.4.2
diff --git a/src/slave/container_loggers/lib_logrotate.cpp b/src/slave/container_loggers/lib_logrotate.cpp
index bc13e6a524..8d003e2bae 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,14 @@ class LogrotateContainerLoggerProcess :
     // 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);
 
     // Spawn a process to handle stdout.
     mesos::internal::logger::rotate::Flags outFlags;
@@ -207,26 +199,16 @@ class LogrotateContainerLoggerProcess :
 
     // 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.get().pid(), SIGKILL);
-      return Failure(ErrnoError("Failed to create pipe").message);
+      os::killtree(outProcess->pid(), SIGKILL);
+      return Failure("Failed to create pipe: " + pipefd.error());
     }
 
     Subprocess::IO::InputFileDescriptors errfds;
-    errfds.read = pipefd[0];
-    errfds.write = pipefd[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.get().pid(), SIGKILL);
-      return Failure("Failed to cloexec: " + cloexec.error());
-    }
+    errfds.read = pipefd->at(0);
+    errfds.write = pipefd->at(1);
 
     // Spawn a process to handle stderr.
     mesos::internal::logger::rotate::Flags errFlags;
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 5f29fe183b..73334ffcee 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1666,6 +1666,8 @@ Future<bool> 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
@@ -1751,17 +1753,19 @@ Future<bool> 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()) {
+    os::close(pipes[0]);
+    os::close(pipes[1]);
+
     return Failure("Failed to fork: " + forked.error());
   }
 
@@ -1780,6 +1784,9 @@ Future<bool> 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");
     }
   }
@@ -1803,6 +1810,9 @@ Future<bool> 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/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index 0affcf5962..2749b5df2b 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -29,8 +29,10 @@
 
 #include <stout/foreach.hpp>
 #include <stout/os.hpp>
-#include <stout/protobuf.hpp>
 #include <stout/path.hpp>
+#include <stout/protobuf.hpp>
+#include <stout/stringify.hpp>
+#include <stout/try.hpp>
 #include <stout/unreachable.hpp>
 
 #include <mesos/mesos.hpp>
@@ -57,6 +59,7 @@
 using std::cerr;
 using std::cout;
 using std::endl;
+using std::list;
 using std::set;
 using std::string;
 using std::vector;
@@ -230,6 +233,41 @@ static void exitWithStatus(int status)
 }
 
 
+// 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) {
@@ -746,6 +784,18 @@ int MesosContainerizerLaunch::execute()
   }
 
 #ifndef __WINDOWS__
+  // Construct a set of file descriptors to close before `exec`'ing.
+  Try<hashset<int_fd>> fds = getOpenFileDescriptors();
+  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
@@ -816,6 +866,13 @@ int MesosContainerizerLaunch::execute()
       os::close(containerStatusFd.get());
       ::_exit(EXIT_SUCCESS);
     }
+
+    // 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);
+    }
   }
 #endif // __WINDOWS__
 
diff --git a/src/slave/containerizer/mesos/launcher.cpp b/src/slave/containerizer/mesos/launcher.cpp
index ec31fa24c8..aa594ec9d9 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");
@@ -121,17 +120,29 @@ Try<pid_t> SubprocessLauncher::fork(
   parentHooks.emplace_back(Subprocess::ParentHook::CREATE_JOB());
 #endif // __linux__
 
+  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,
       argv,
-      in,
-      out,
-      err,
+      containerIO.in,
+      containerIO.out,
+      containerIO.err,
       flags,
       environment,
       None(),
       parentHooks,
-      {Subprocess::ChildHook::SETSID()});
+      childHooks,
+      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 f69d934d2e..90a50b1fd9 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 @@ class Launcher
       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 @@ class SubprocessLauncher : public Launcher
       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 1cea04edac..55602f6a71 100644
--- a/src/slave/containerizer/mesos/linux_launcher.cpp
+++ b/src/slave/containerizer/mesos/linux_launcher.cpp
@@ -74,13 +74,12 @@ class LinuxLauncherProcess : public Process<LinuxLauncherProcess>
       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);
 
@@ -206,13 +205,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(),
@@ -220,13 +218,12 @@ Try<pid_t> LinuxLauncher::fork(
       containerId,
       path,
       argv,
-      in,
-      out,
-      err,
+      containerIO,
       flags,
       environment,
       enterNamespaces,
-      cloneNamespaces).get();
+      cloneNamespaces,
+      whitelistFds).get();
 }
 
 
@@ -383,13 +380,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)) {
@@ -458,12 +454,21 @@ Try<pid_t> LinuxLauncherProcess::fork(
         child);
   }));
 
+  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,
-      in,
-      out,
-      err,
+      containerIO.in,
+      containerIO.out,
+      containerIO.err,
       flags,
       environment,
       [target, enterFlags, cloneFlags](const lambda::function<int()>& child) {
@@ -484,7 +489,7 @@ Try<pid_t> LinuxLauncherProcess::fork(
         }
       },
       parentHooks,
-      {Subprocess::ChildHook::SETSID()});
+      childHooks);
 
   if (child.isError()) {
     return Error("Failed to clone child process: " + child.error());
diff --git a/src/slave/containerizer/mesos/linux_launcher.hpp b/src/slave/containerizer/mesos/linux_launcher.hpp
index e1525231e0..7a8f6f0920 100644
--- a/src/slave/containerizer/mesos/linux_launcher.hpp
+++ b/src/slave/containerizer/mesos/linux_launcher.hpp
@@ -46,13 +46,12 @@ class LinuxLauncher : public Launcher
       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/provisioner/backends/aufs.cpp b/src/slave/containerizer/mesos/provisioner/backends/aufs.cpp
index 53ccbd1e08..2f082ac97d 100644
--- a/src/slave/containerizer/mesos/provisioner/backends/aufs.cpp
+++ b/src/slave/containerizer/mesos/provisioner/backends/aufs.cpp
@@ -251,8 +251,11 @@ Future<bool> AufsBackendProcess::destroy(
 
   foreach (const fs::MountInfoTable::Entry& entry, mountTable->entries) {
     if (entry.target == rootfs) {
-      // NOTE: This would fail if the rootfs is still in use.
-      Try<Nothing> unmount = fs::unmount(entry.target);
+      // NOTE: Use MNT_DETACH here so that if there are still
+      // processes holding files or directories in the rootfs, the
+      // unmount will still be successful. The kernel will cleanup the
+      // mount when the number of references reach zero.
+      Try<Nothing> unmount = fs::unmount(entry.target, MNT_DETACH);
       if (unmount.isError()) {
         return Failure(
             "Failed to destroy aufs-mounted rootfs '" + rootfs + "': " +
@@ -261,9 +264,16 @@ Future<bool> AufsBackendProcess::destroy(
 
       Try<Nothing> rmdir = os::rmdir(rootfs);
       if (rmdir.isError()) {
-        return Failure(
-            "Failed to remove rootfs mount point '" + rootfs + "': " +
-            rmdir.error());
+        // NOTE: Due to the use of MNT_DETACH above, it's possible
+        // that `rmdir` will fail with EBUSY if some other mounts in
+        // other mount namespaces are still on this mount point on
+        // some old kernel (https://lwn.net/Articles/570338/). No need
+        // to return a hard failure here because the directory will be
+        // removed later and re-attempted on agent recovery.
+        //
+        // TODO(jieyu): Consider only ignore EBUSY error.
+        LOG(ERROR) << "Failed to remove rootfs mount point "
+                   << "'" << rootfs << "': " << rmdir.error();
       }
 
       // Clean up tempDir used for image layer links.
diff --git a/src/slave/containerizer/mesos/provisioner/backends/bind.cpp b/src/slave/containerizer/mesos/provisioner/backends/bind.cpp
index d6cdc93128..9c312c1a8d 100644
--- a/src/slave/containerizer/mesos/provisioner/backends/bind.cpp
+++ b/src/slave/containerizer/mesos/provisioner/backends/bind.cpp
@@ -195,8 +195,11 @@ Future<bool> BindBackendProcess::destroy(const string& rootfs)
     // to check `strings::startsWith(entry.target, rootfs)` here to
     // unmount all nested mounts.
     if (entry.target == rootfs) {
-      // NOTE: This would fail if the rootfs is still in use.
-      Try<Nothing> unmount = fs::unmount(entry.target);
+      // NOTE: Use MNT_DETACH here so that if there are still
+      // processes holding files or directories in the rootfs, the
+      // unmount will still be successful. The kernel will cleanup the
+      // mount when the number of references reach zero.
+      Try<Nothing> unmount = fs::unmount(entry.target, MNT_DETACH);
       if (unmount.isError()) {
         return Failure(
             "Failed to destroy bind-mounted rootfs '" + rootfs + "': " +
diff --git a/src/slave/containerizer/mesos/provisioner/backends/copy.cpp b/src/slave/containerizer/mesos/provisioner/backends/copy.cpp
index 69faa03f30..1479bc921e 100644
--- a/src/slave/containerizer/mesos/provisioner/backends/copy.cpp
+++ b/src/slave/containerizer/mesos/provisioner/backends/copy.cpp
@@ -327,9 +327,15 @@ Future<bool> CopyBackendProcess::destroy(const string& rootfs)
     .then([](const Option<int>& status) -> Future<bool> {
       if (status.isNone()) {
         return Failure("Failed to reap subprocess to destroy rootfs");
-      } else if (status.get() != 0) {
-        return Failure("Failed to destroy rootfs, exit status: " +
-                       WSTRINGIFY(status.get()));
+      }
+
+      if (status.get() != 0) {
+        // It's possible that `rm -rf` will fail if some other
+        // programs are accessing the files.  No need to return a hard
+        // failure here because the directory will be removed later
+        // and re-attempted on agent recovery.
+        LOG(ERROR) << "Failed to destroy rootfs, exit status: "
+                   << WSTRINGIFY(status.get());
       }
 
       return true;
diff --git a/src/slave/containerizer/mesos/provisioner/backends/overlay.cpp b/src/slave/containerizer/mesos/provisioner/backends/overlay.cpp
index 3149220316..754ef1b992 100644
--- a/src/slave/containerizer/mesos/provisioner/backends/overlay.cpp
+++ b/src/slave/containerizer/mesos/provisioner/backends/overlay.cpp
@@ -250,8 +250,11 @@ Future<bool> OverlayBackendProcess::destroy(
 
   foreach (const fs::MountInfoTable::Entry& entry, mountTable->entries) {
     if (entry.target == rootfs) {
-      // NOTE: This would fail if the rootfs is still in use.
-      Try<Nothing> unmount = fs::unmount(entry.target);
+      // NOTE: Use MNT_DETACH here so that if there are still
+      // processes holding files or directories in the rootfs, the
+      // unmount will still be successful. The kernel will cleanup the
+      // mount when the number of references reach zero.
+      Try<Nothing> unmount = fs::unmount(entry.target, MNT_DETACH);
       if (unmount.isError()) {
         return Failure(
             "Failed to destroy overlay-mounted rootfs '" + rootfs + "': " +
@@ -260,9 +263,16 @@ Future<bool> OverlayBackendProcess::destroy(
 
       Try<Nothing> rmdir = os::rmdir(rootfs);
       if (rmdir.isError()) {
-        return Failure(
-            "Failed to remove rootfs mount point '" + rootfs + "': " +
-            rmdir.error());
+        // NOTE: Due to the use of MNT_DETACH above, it's possible
+        // that `rmdir` will fail with EBUSY if some other mounts in
+        // other mount namespaces are still on this mount point on
+        // some old kernel (https://lwn.net/Articles/570338/). No need
+        // to return a hard failure here because the directory will be
+        // removed later and re-attempted on agent recovery.
+        //
+        // TODO(jieyu): Consider only ignore EBUSY error.
+        LOG(ERROR) << "Failed to remove rootfs mount point "
+                   << "'" << rootfs << "': " << rmdir.error();
       }
 
       // Clean up tempDir used for image layer links.
diff --git a/src/tests/check_tests.cpp b/src/tests/check_tests.cpp
index 0810851cf1..bb0b3e8ace 100644
--- a/src/tests/check_tests.cpp
+++ b/src/tests/check_tests.cpp
@@ -1889,47 +1889,22 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
     .WillOnce(FutureArg<1>(&updateCheckResult))
     .WillRepeatedly(Return()); // Ignore subsequent updates.
 
-  // Default executor delegates launching both the task and its check to the
-  // agent. To avoid a race, we explicitly synchronize.
-  Try<std::array<int_fd, 2>> pipes_ = os::pipe();
-  ASSERT_SOME(pipes_);
-
-  const std::array<int_fd, 2>& pipes = pipes_.get();
-
   const string filename = "nested_inherits_work_dir";
 
-  // NOTE: We use a non-shell command here to use 'bash -c' to execute
-  // the 'echo', which deals with the file descriptor, because of a bug
-  // in ubuntu dash. Multi-digit file descriptor is not supported in
-  // ubuntu dash, which executes the shell command.
-  v1::CommandInfo command;
-  command.set_shell(false);
-  command.set_value("/bin/bash");
-  command.add_arguments("bash");
-  command.add_arguments("-c");
-  command.add_arguments(
-      "touch " + filename + ";echo running >&" +
-      stringify(pipes[1]) + ";sleep 1000");
-
-  v1::TaskInfo taskInfo = v1::createTask(agentId, resources, command);
+  v1::TaskInfo taskInfo = v1::createTask(
+      agentId,
+      resources,
+      v1::createCommandInfo(
+          strings::format("touch %s; sleep 1000", filename).get()));
 
   v1::CheckInfo* checkInfo = taskInfo.mutable_check();
   checkInfo->set_type(v1::CheckInfo::COMMAND);
   checkInfo->set_delay_seconds(0);
   checkInfo->set_interval_seconds(0);
 
-  // NOTE: We use a non-shell command here to use 'bash -c' to execute
-  // the 'read', which deals with the file descriptor, because of a bug
-  // in ubuntu dash. Multi-digit file descriptor is not supported in
-  // ubuntu dash, which executes the shell command.
-  v1::CommandInfo* checkCommand =
-    checkInfo->mutable_command()->mutable_command();
-  checkCommand->set_shell(false);
-  checkCommand->set_value("/bin/bash");
-  checkCommand->add_arguments("bash");
-  checkCommand->add_arguments("-c");
-  checkCommand->add_arguments(
-      "read INPUT <&" + stringify(pipes[0]) + ";ls " + filename);
+  // Wait in a busy loop until the file has been created.
+  checkInfo->mutable_command()->mutable_command()->CopyFrom(
+      v1::createCommandInfo("while [ -f " + filename + "]; do :; done"));
 
   v1::TaskGroupInfo taskGroup;
   taskGroup.add_tasks()->CopyFrom(taskInfo);
diff --git a/src/tests/containerizer/launcher.cpp b/src/tests/containerizer/launcher.cpp
index a92d9890f0..51ae4f90fd 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 a8e436f164..6057286ae0 100644
--- a/src/tests/containerizer/launcher.hpp
+++ b/src/tests/containerizer/launcher.hpp
@@ -56,19 +56,18 @@ class TestLauncher : public slave::Launcher
       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 1fc56c4a8c..59637c9457 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -1284,13 +1284,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);
 
diff --git a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
index f8b4423de8..c05c916e30 100644
--- a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
@@ -14,6 +14,7 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+#include <sys/stat.h>
 #include <sys/wait.h>
 
 #include <map>
@@ -449,23 +450,11 @@ TEST_F(NestedMesosContainerizerTest,
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
-  // Use a pipe to pass parent's MESOS_SANDBOX value to a child container.
-  Try<std::array<int_fd, 2>> pipes_ = os::pipe();
-  ASSERT_SOME(pipes_);
+  string pipe = path::join(sandbox.get(), "pipe");
+  ASSERT_EQ(0, ::mkfifo(pipe.c_str(), 0700));
 
-  const std::array<int_fd, 2>& pipes = pipes_.get();
-
-  // NOTE: We use a non-shell command here to use 'bash -c' to execute
-  // the 'echo', which deals with the file descriptor, because of a bug
-  // in ubuntu dash. Multi-digit file descriptor is not supported in
-  // ubuntu dash, which executes the shell command.
-  CommandInfo command;
-  command.set_shell(false);
-  command.set_value("/bin/bash");
-  command.add_arguments("bash");
-  command.add_arguments("-c");
-  command.add_arguments(
-      "echo $MESOS_SANDBOX >&" + stringify(pipes[1]) + ";" + "sleep 1000");
+  CommandInfo command =
+    createCommandInfo("echo ${MESOS_SANDBOX} > " + pipe + " ; sleep 1000");
 
   ExecutorInfo executor = createExecutorInfo("executor", command, "cpus:1");
 
@@ -483,30 +472,16 @@ TEST_F(NestedMesosContainerizerTest,
 
   AWAIT_ASSERT_TRUE(launch);
 
-  // Wait for the parent container to start running its task
-  // before launching a debug container inside it.
-  AWAIT_READY(process::io::poll(pipes[0], process::io::READ));
-  close(pipes[1]);
-
-  // Launch a nested debug container that compares MESOS_SANDBOX
+  // Launch a nested debug container that compares `MESOS_SANDBOX`
   // it sees with the one its parent sees.
   {
     ContainerID nestedContainerId;
     nestedContainerId.mutable_parent()->CopyFrom(containerId);
     nestedContainerId.set_value(UUID::random().toString());
 
-    // NOTE: We use a non-shell command here to use 'bash -c' to execute
-    // the 'read', which deals with the file descriptor, because of a bug
-    // in ubuntu dash. Multi-digit file descriptor is not supported in
-    // ubuntu dash, which executes the shell command.
-    CommandInfo nestedCommand;
-    nestedCommand.set_shell(false);
-    nestedCommand.set_value("/bin/bash");
-    nestedCommand.add_arguments("bash");
-    nestedCommand.add_arguments("-c");
-    nestedCommand.add_arguments(
-        "read PARENT_SANDBOX <&" + stringify(pipes[0]) + ";"
-        "[ ${PARENT_SANDBOX} == ${MESOS_SANDBOX} ] && exit 0 || exit 1;");
+    CommandInfo nestedCommand = createCommandInfo(
+        "read PARENT_SANDBOX < " + pipe + ";"
+        "[ ${PARENT_SANDBOX} = ${MESOS_SANDBOX} ] && exit 0 || exit 1;");
 
     Future<bool> launchNested = containerizer->launch(
         nestedContainerId,
@@ -526,8 +501,6 @@ TEST_F(NestedMesosContainerizerTest,
     ASSERT_SOME(waitNested.get());
     ASSERT_TRUE(waitNested.get()->has_status());
     EXPECT_WEXITSTATUS_EQ(0, waitNested.get()->status());
-
-    close(pipes[0]);
   }
 
   // Destroy the containerizer with all associated containers.
@@ -571,27 +544,15 @@ TEST_F(NestedMesosContainerizerTest,
   containerId.set_value(UUID::random().toString());
 
   // Use a pipe to synchronize with the top-level container.
-  Try<std::array<int_fd, 2>> pipes_ = os::pipe();
-  ASSERT_SOME(pipes_);
-
-  const std::array<int_fd, 2>& pipes = pipes_.get();
+  string pipe = path::join(sandbox.get(), "pipe");
+  ASSERT_EQ(0, ::mkfifo(pipe.c_str(), 0700));
 
   const string filename = "nested_inherits_work_dir";
 
-  // NOTE: We use a non-shell command here to use 'bash -c' to execute
-  // the 'echo', which deals with the file descriptor, because of a bug
-  // in ubuntu dash. Multi-digit file descriptor is not supported in
-  // ubuntu dash, which executes the shell command.
-  CommandInfo command;
-  command.set_shell(false);
-  command.set_value("/bin/bash");
-  command.add_arguments("bash");
-  command.add_arguments("-c");
-  command.add_arguments(
-      "touch " + filename + ";echo running >&" +
-      stringify(pipes[1]) + ";sleep 1000");
-
-  ExecutorInfo executor = createExecutorInfo("executor", command, "cpus:1");
+  ExecutorInfo executor = createExecutorInfo(
+      "executor",
+      "touch " + filename + "; echo running > " + pipe + "; sleep 1000",
+      "cpus:1");
 
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
@@ -614,9 +575,9 @@ TEST_F(NestedMesosContainerizerTest,
 
   // Wait for the parent container to start running its task
   // before launching a debug container inside it.
-  AWAIT_READY(process::io::poll(pipes[0], process::io::READ));
-  close(pipes[1]);
-  close(pipes[0]);
+  Result<string> read = os::read(pipe);
+  ASSERT_SOME(read);
+  ASSERT_EQ("running\n", read.get());
 
   Future<ContainerStatus> status = containerizer->status(containerId);
   AWAIT_READY(status);
@@ -1016,19 +977,17 @@ TEST_F(NestedMesosContainerizerTest,
   ASSERT_EQ(1u, offers->size());
 
   // Use a pipe to synchronize with the top-level container.
-  Try<std::array<int_fd, 2>> pipes_ = os::pipe();
-  ASSERT_SOME(pipes_);
-
-  const std::array<int_fd, 2>& pipes = pipes_.get();
+  string pipe = path::join(sandbox.get(), "pipe");
+  ASSERT_EQ(0, ::mkfifo(pipe.c_str(), 0700));
 
-  // Launch a command task within the `alpine` docker image and
-  // synchronize its launch with the launch of a debug container below.
+  // Launch a command task within the `alpine` docker image.
   TaskInfo task = createTask(
       offers->front().slave_id(),
       offers->front().resources(),
-      "echo running >&" + stringify(pipes[1]) + ";" + "sleep 1000");
+      "echo running > /tmp/pipe; sleep 1000");
 
-  task.mutable_container()->CopyFrom(createContainerInfo("alpine"));
+  task.mutable_container()->CopyFrom(createContainerInfo(
+      "alpine", {createVolumeFromHostPath("/tmp", sandbox.get(), Volume::RW)}));
 
   Future<TaskStatus> statusRunning;
   EXPECT_CALL(sched, statusUpdate(_, _))
@@ -1041,12 +1000,11 @@ TEST_F(NestedMesosContainerizerTest,
   AWAIT_READY_FOR(statusRunning, Seconds(120));
   ASSERT_EQ(TASK_RUNNING, statusRunning->state());
 
-  close(pipes[1]);
-
   // Wait for the parent container to start running its task
   // before launching a debug container inside it.
-  AWAIT_READY(process::io::poll(pipes[0], process::io::READ));
-  close(pipes[0]);
+  Result<string> read = os::read(pipe);
+  ASSERT_SOME(read);
+  ASSERT_EQ("running\n", read.get());
 
   ASSERT_TRUE(statusRunning->has_slave_id());
   ASSERT_TRUE(statusRunning->has_container_status());
@@ -1385,23 +1343,14 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentExit)
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
-  Try<std::array<int_fd, 2>> pipes_ = os::pipe();
-  ASSERT_SOME(pipes_);
+  string pipe = path::join(sandbox.get(), "pipe");
+  ASSERT_EQ(0, ::mkfifo(pipe.c_str(), 0700));
 
-  const std::array<int_fd, 2>& pipes = pipes_.get();
-
-  // NOTE: We use a non-shell command here to use 'bash -c' to execute
-  // the 'read', which deals with the file descriptor, because of a bug
-  // in ubuntu dash. Multi-digit file descriptor is not supported in
-  // ubuntu dash, which executes the shell command.
-  CommandInfo command;
-  command.set_shell(false);
-  command.set_value("/bin/bash");
-  command.add_arguments("bash");
-  command.add_arguments("-c");
-  command.add_arguments("read key <&" + stringify(pipes[0]));
-
-  ExecutorInfo executor = createExecutorInfo("executor", command, "cpus:1");
+  // We launch a blocking `read` after which we return with a non-success code.
+  ExecutorInfo executor = createExecutorInfo(
+      "executor",
+      createCommandInfo("read < " + pipe + " && exit 1"),
+      "cpus:1");
 
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
@@ -1412,8 +1361,6 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentExit)
       map<string, string>(),
       None());
 
-  close(pipes[0]); // We're never going to read.
-
   AWAIT_ASSERT_TRUE(launch);
 
   // Now launch nested container.
@@ -1434,7 +1381,8 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentExit)
   Future<Option<ContainerTermination>> nestedWait = containerizer->wait(
       nestedContainerId);
 
-  close(pipes[1]); // Force the 'read key' to exit!
+  // Write to the fifo to unblock the `read` in the parent container.
+  os::write(pipe, "");
 
   AWAIT_READY(wait);
   ASSERT_SOME(wait.get());
@@ -1479,25 +1427,14 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentSigterm)
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
-  // Use a pipe to synchronize with the top-level container.
-  Try<std::array<int_fd, 2>> pipes_ = os::pipe();
-  ASSERT_SOME(pipes_);
-
-  const std::array<int_fd, 2>& pipes = pipes_.get();
-
-  // NOTE: We use a non-shell command here to use 'bash -c' to execute
-  // the 'echo', which deals with the file descriptor, because of a bug
-  // in ubuntu dash. Multi-digit file descriptor is not supported in
-  // ubuntu dash, which executes the shell command.
-  CommandInfo command;
-  command.set_shell(false);
-  command.set_value("/bin/bash");
-  command.add_arguments("bash");
-  command.add_arguments("-c");
-  command.add_arguments(
-      "echo running >&" + stringify(pipes[1]) + ";" + "sleep 1000");
+  // Use a fifo to synchronize with the top-level container.
+  string pipe = path::join(sandbox.get(), "pipe");
+  ASSERT_EQ(0, ::mkfifo(pipe.c_str(), 0700));
 
-  ExecutorInfo executor = createExecutorInfo("executor", command, "cpus:1");
+  ExecutorInfo executor = createExecutorInfo(
+      "executor",
+      createCommandInfo("echo running > " + pipe + "; sleep 1000"),
+      "cpus:1");
 
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
@@ -1510,8 +1447,6 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentSigterm)
 
   AWAIT_ASSERT_TRUE(launch);
 
-  close(pipes[1]);
-
   // Now launch nested container.
   ContainerID nestedContainerId;
   nestedContainerId.mutable_parent()->CopyFrom(containerId);
@@ -1536,8 +1471,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentSigterm)
 
   // Wait for the parent container to start running its executor
   // process before sending it a signal.
-  AWAIT_READY(process::io::poll(pipes[0], process::io::READ));
-  close(pipes[0]);
+  Result<string> read = os::read(pipe);
+  ASSERT_SOME(read);
+  ASSERT_EQ("running\n", read.get());
 
   ASSERT_EQ(0u, os::kill(status->executor_pid(), SIGTERM));
 
diff --git a/src/tests/containerizer/port_mapping_tests.cpp b/src/tests/containerizer/port_mapping_tests.cpp
index 84b39b1588..06dcce9402 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 @@ class PortMappingIsolatorTest : public TemporaryDirectoryTest
         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;
   }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services