You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by jp...@apache.org on 2018/12/05 01:50:37 UTC

[mesos] 01/04: Moved the container root construction to the isolators.

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

jpeach pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit ede8155d1d043137e15007c48da36ac5fa0b5124
Author: James Peach <jp...@apache.org>
AuthorDate: Tue Dec 4 13:57:00 2018 -0800

    Moved the container root construction to the isolators.
    
    Previously, if the container was configured with a root filesytem,
    the root was populated by a combination of the `fs::chroot:prepare`
    API and the various isolators. The implementation details of some
    isolators had leaked into the chroot code, which had a special case
    for adding GPU devices.
    
    This change moves all the responsibility for defining the
    root filesystem from the `fs::chroot::prepare()` API to the
    `filesystem/linux` isolator. The `filesystem/linux` isolator is
    now the single place that captures how to mount the container
    pseudo-filesystems as well as how to construct a proper `/dev`
    directory.
    
    Since the `linux/filesystem` isolator is now entirely responsible
    for creating and mounting the container `/dev`, any other isolators
    that enable access to devices should populate device nodes in the
    container devices directory and add a corresponding bind mount.
    
    Review: https://reviews.apache.org/r/69086/
---
 src/common/protobuf_utils.cpp                      |  52 ++++
 src/common/protobuf_utils.hpp                      |  21 ++
 src/linux/fs.cpp                                   | 302 +--------------------
 src/linux/fs.hpp                                   |   9 +-
 .../mesos/isolators/filesystem/linux.cpp           | 237 +++++++++++++++-
 .../containerizer/mesos/isolators/gpu/isolator.cpp |  65 ++++-
 .../containerizer/mesos/isolators/gpu/isolator.hpp |   1 +
 .../mesos/isolators/linux/devices.cpp              |  38 +--
 src/slave/containerizer/mesos/launch.cpp           |  67 ++---
 9 files changed, 415 insertions(+), 377 deletions(-)

diff --git a/src/common/protobuf_utils.cpp b/src/common/protobuf_utils.cpp
index a45607e..a5a4ace 100644
--- a/src/common/protobuf_utils.cpp
+++ b/src/common/protobuf_utils.cpp
@@ -67,6 +67,7 @@ using google::protobuf::RepeatedPtrField;
 using mesos::authorization::VIEW_ROLE;
 
 using mesos::slave::ContainerLimitation;
+using mesos::slave::ContainerMountInfo;
 using mesos::slave::ContainerState;
 
 using process::Owned;
@@ -1122,6 +1123,57 @@ ContainerState createContainerState(
   return state;
 }
 
+
+ContainerMountInfo createContainerMount(
+    const string& source,
+    const string& target,
+    unsigned long flags)
+{
+  ContainerMountInfo mnt;
+
+  mnt.set_source(source);
+  mnt.set_target(target);
+  mnt.set_flags(flags);
+
+  return mnt;
+}
+
+
+ContainerMountInfo createContainerMount(
+    const string& source,
+    const string& target,
+    const string& type,
+    unsigned long flags)
+{
+  ContainerMountInfo mnt;
+
+  mnt.set_source(source);
+  mnt.set_target(target);
+  mnt.set_type(type);
+  mnt.set_flags(flags);
+
+  return mnt;
+}
+
+
+ContainerMountInfo createContainerMount(
+    const string& source,
+    const string& target,
+    const string& type,
+    const string& options,
+    unsigned long flags)
+{
+  ContainerMountInfo mnt;
+
+  mnt.set_source(source);
+  mnt.set_target(target);
+  mnt.set_type(type);
+  mnt.set_options(options);
+  mnt.set_flags(flags);
+
+  return mnt;
+}
+
 } // namespace slave {
 
 namespace maintenance {
diff --git a/src/common/protobuf_utils.hpp b/src/common/protobuf_utils.hpp
index 1662125..a7ddca4 100644
--- a/src/common/protobuf_utils.hpp
+++ b/src/common/protobuf_utils.hpp
@@ -339,6 +339,27 @@ mesos::slave::ContainerState createContainerState(
     pid_t pid,
     const std::string& directory);
 
+
+mesos::slave::ContainerMountInfo createContainerMount(
+    const std::string& source,
+    const std::string& target,
+    unsigned long flags);
+
+
+mesos::slave::ContainerMountInfo createContainerMount(
+    const std::string& source,
+    const std::string& target,
+    const std::string& type,
+    unsigned long flags);
+
+
+mesos::slave::ContainerMountInfo createContainerMount(
+    const std::string& source,
+    const std::string& target,
+    const std::string& type,
+    const std::string& options,
+    unsigned long flags);
+
 } // namespace slave {
 
 namespace maintenance {
diff --git a/src/linux/fs.cpp b/src/linux/fs.cpp
index 5cdffe1..6ff8b4d 100644
--- a/src/linux/fs.cpp
+++ b/src/linux/fs.cpp
@@ -38,7 +38,6 @@ extern "C" {
 #include <stout/adaptor.hpp>
 #include <stout/check.hpp>
 #include <stout/error.hpp>
-#include <stout/fs.hpp>
 #include <stout/hashmap.hpp>
 #include <stout/hashset.hpp>
 #include <stout/numify.hpp>
@@ -630,21 +629,13 @@ Try<Nothing> pivot_root(
 
 namespace chroot {
 
-namespace internal {
-
-// Make the source device node appear at the target path. We prefer to
-// `mknod` the device node since that avoids an otherwise unnecessary
-// mount table entry. The `mknod` can fail if we are in a user namespace
-// or if the devices cgroup is restricting that device. In that case, we
-// bind mount the device to the target path.
-Try<Nothing> importDeviceNode(const string& source, const string& target)
+Try<Nothing> copyDeviceNode(
+    const std::string& source,
+    const std::string& target)
 {
-  // We are likely to be operating in a multi-threaded environment so
-  // it's not safe to change the umask. Instead, we'll explicitly set
-  // permissions after we create the device node.
   Try<mode_t> mode = os::stat::mode(source);
   if (mode.isError()) {
-    return Error("Failed to source mode: " + mode.error());
+    return Error("Failed to get source mode: " + mode.error());
   }
 
   Try<dev_t> dev = os::stat::rdev(source);
@@ -652,284 +643,21 @@ Try<Nothing> importDeviceNode(const string& source, const string& target)
     return Error("Failed to get source dev: " + dev.error());
   }
 
-  Try<Nothing> mknod = os::mknod(target, mode.get(), dev.get());
-  if (mknod.isSome()) {
-    Try<Nothing> chmod = os::chmod(target, mode.get());
-    if (chmod.isError()) {
-      return Error("Failed to chmod device: " + chmod.error());
-    }
-
-    return Nothing();
-  }
-
-  Try<Nothing> touch = os::touch(target);
-  if (touch.isError()) {
-    return Error("Failed to create device mount point: " + touch.error());
-  }
-
-  Try<Nothing> mnt = fs::mount(source, target, None(), MS_BIND, None());
-  if (mnt.isError()) {
-    return Error("Failed to bind device: " + touch.error());
-  }
-
-  return Nothing();
-}
-
-
-// Some helpful types.
-struct Mount
-{
-  Option<string> source;
-  string target;
-  Option<string> type;
-  Option<string> options;
-  unsigned long flags;
-};
-
-struct SymLink
-{
-  string original;
-  string link;
-};
-
-
-Try<Nothing> mountSpecialFilesystems(const string& root)
-{
-  // List of special filesystems useful for a chroot environment.
-  // NOTE: This list is ordered, e.g., mount /proc before bind
-  // mounting /proc/sys and then making it read-only.
-  //
-  // TODO(jasonlai): These special filesystem mount points need to be
-  // bind-mounted prior to all other mount points specified in
-  // `ContainerLaunchInfo`.
-  //
-  // One example of the known issues caused by this behavior is:
-  // https://issues.apache.org/jira/browse/MESOS-6798
-  // There will be follow-up efforts on moving the logic below to
-  // proper isolators.
-  //
-  // TODO(jasonlai): Consider adding knobs to allow write access to
-  // those system files if configured by the operator.
-  static const vector<Mount> mounts = {
-    {
-      "proc",
-      "/proc",
-      "proc",
-      None(),
-      MS_NOSUID | MS_NOEXEC | MS_NODEV
-    },
-    {
-      "/proc/bus",
-      "/proc/bus",
-      None(),
-      None(),
-      MS_BIND | MS_RDONLY | MS_NOSUID | MS_NOEXEC | MS_NODEV
-    },
-    {
-      "/proc/fs",
-      "/proc/fs",
-      None(),
-      None(),
-      MS_BIND | MS_RDONLY | MS_NOSUID | MS_NOEXEC | MS_NODEV
-    },
-    {
-      "/proc/irq",
-      "/proc/irq",
-      None(),
-      None(),
-      MS_BIND | MS_RDONLY | MS_NOSUID | MS_NOEXEC | MS_NODEV
-    },
-    {
-      "/proc/sys",
-      "/proc/sys",
-      None(),
-      None(),
-      MS_BIND | MS_RDONLY | MS_NOSUID | MS_NOEXEC | MS_NODEV
-    },
-    {
-      "/proc/sysrq-trigger",
-      "/proc/sysrq-trigger",
-      None(),
-      None(),
-      MS_BIND | MS_RDONLY | MS_NOSUID | MS_NOEXEC | MS_NODEV
-    },
-    {
-      "sysfs",
-      "/sys",
-      "sysfs",
-      None(),
-      MS_RDONLY | MS_NOSUID | MS_NOEXEC | MS_NODEV
-    },
-    {
-      "tmpfs",
-      "/sys/fs/cgroup",
-      "tmpfs",
-      "mode=755",
-      MS_NOSUID | MS_NOEXEC | MS_NODEV
-    },
-    {
-      "tmpfs",
-      "/dev",
-      "tmpfs",
-      "mode=755",
-      MS_NOSUID | MS_STRICTATIME | MS_NOEXEC
-    },
-    // We mount devpts with the gid=5 option because the `tty` group is
-    // GID 5 on all standard Linux distributions. The glibc grantpt(3)
-    // API ensures that the terminal GID is that of the `tty` group, and
-    // invokes a privileged helper if necessary. Since the helper won't
-    // work in all container configurations (since it may not be possible
-    // to acquire the necessary privileges), mounting with the right `gid`
-    // option avoids any possible failure.
-    {
-      "devpts",
-      "/dev/pts",
-      "devpts",
-      "newinstance,ptmxmode=0666,mode=0620,gid=5",
-      MS_NOSUID | MS_NOEXEC
-    },
-    {
-      "tmpfs",
-      "/dev/shm",
-      "tmpfs",
-      "mode=1777",
-      MS_NOSUID | MS_NODEV | MS_STRICTATIME
-    },
-  };
-
-  foreach (const Mount& mount, mounts) {
-    // Target is always under the new root.
-    const string target = path::join(root, mount.target);
-
-    // Try to create the mount point, if it doesn't already exist.
-    if (!os::exists(target)) {
-      Try<Nothing> mkdir = os::mkdir(target);
-
-      if (mkdir.isError()) {
-        return Error("Failed to create mount point '" + target +
-                     "': " + mkdir.error());
-      }
-    }
-
-    // If source is a path, e.g,. for a bind mount, then it needs to
-    // be prefixed by the new root.
-    Option<string> source;
-    if (mount.source.isSome() && strings::startsWith(mount.source.get(), "/")) {
-      source = path::join(root, mount.source.get());
-    } else {
-      source = mount.source;
-    }
-
-    Try<Nothing> mnt = fs::mount(
-        source,
-        target,
-        mount.type,
-        mount.flags,
-        mount.options);
-
-    if (mnt.isError()) {
-      return Error("Failed to mount '" + target + "': " + mnt.error());
-    }
-  }
-
-  return Nothing();
-}
-
-
-Try<Nothing> createStandardDevices(const string& root)
-{
-  // List of standard devices useful for a chroot environment.
-  // TODO(idownes): Make this list configurable.
-  vector<string> devices = {
-    "full",
-    "null",
-    "random",
-    "tty",
-    "urandom",
-    "zero"
-  };
-
-  // Glob all Nvidia GPU devices on the system and add them to the
-  // list of devices injected into the chroot environment.
-  //
-  // TODO(klueska): Only inject these devices if the 'gpu/nvidia'
-  // isolator is enabled.
-  Try<list<string>> nvidia = os::glob("/dev/nvidia*");
-  if (nvidia.isError()) {
-    return Error("Failed to glob /dev/nvidia* on the host filesystem:"
-                 " " + nvidia.error());
-  }
-
-  foreach (const string& device, nvidia.get()) {
-    if (os::exists(device)) {
-      devices.push_back(Path(device).basename());
-    }
-  }
-
-  // Import each device into the chroot environment. Copy both the
-  // mode and the device itself from the corresponding host device.
-  foreach (const string& device, devices) {
-    Try<Nothing> import = importDeviceNode(
-        path::join("/",  "dev", device),
-        path::join(root, "dev", device));
-
-    if (import.isError()) {
-      return Error(
-          "Failed to import device '" + device + "': " + import.error());
-    }
-  }
-
-  const vector<SymLink> symlinks = {
-    {"/proc/self/fd",   path::join(root, "dev", "fd")},
-    {"/proc/self/fd/0", path::join(root, "dev", "stdin")},
-    {"/proc/self/fd/1", path::join(root, "dev", "stdout")},
-    {"/proc/self/fd/2", path::join(root, "dev", "stderr")},
-    {"pts/ptmx",        path::join(root, "dev", "ptmx")}
-  };
-
-  foreach (const SymLink& symlink, symlinks) {
-    Try<Nothing> link = ::fs::symlink(symlink.original, symlink.link);
-    if (link.isError()) {
-      return Error("Failed to symlink '" + symlink.original +
-                   "' to '" + symlink.link + "': " + link.error());
-    }
-  }
-
-  // TODO(idownes): Set up console device.
-  return Nothing();
-}
-
-} // namespace internal {
-
-
-// TODO(idownes): Add unit test.
-Try<Nothing> prepare(const string& root)
-{
-  // Recursively mark current mounts as slaves to prevent propagation.
-  Try<Nothing> mount =
-    fs::mount(None(), "/", None(), MS_REC | MS_SLAVE, nullptr);
-
-  if (mount.isError()) {
-    return Error("Failed to make slave mounts: " + mount.error());
-  }
-
-  // Bind mount 'root' itself. This is because pivot_root requires
-  // 'root' to be not on the same filesystem as process' current root.
-  mount = fs::mount(root, root, None(), MS_REC | MS_BIND, nullptr);
-  if (mount.isError()) {
-    return Error("Failed to bind mount root itself: " + mount.error());
+  Try<Nothing> mkdir = os::mkdir(Path(target).dirname());
+  if (mkdir.isError()) {
+    return Error(
+        "Failed to create parent directory for device '" +
+        target + "': " + mkdir.error());
   }
 
-  // Mount special filesystems.
-  mount = internal::mountSpecialFilesystems(root);
-  if (mount.isError()) {
-    return Error("Failed to mount: " + mount.error());
+  Try<Nothing> mknod = os::mknod(target, mode.get(), dev.get());
+  if (mknod.isError()) {
+    return Error("Failed to mknod device '" + target + "': " + mknod.error());
   }
 
-  // Create basic device nodes.
-  Try<Nothing> create = internal::createStandardDevices(root);
-  if (create.isError()) {
-    return Error("Failed to create devices: " + create.error());
+  Try<Nothing> chmod = os::chmod(target, mode.get());
+  if (chmod.isError()) {
+    return Error("Failed to chmod device '" + target + "': " + chmod.error());
   }
 
   return Nothing();
diff --git a/src/linux/fs.hpp b/src/linux/fs.hpp
index 31969f6..04bd706 100644
--- a/src/linux/fs.hpp
+++ b/src/linux/fs.hpp
@@ -387,11 +387,10 @@ Try<Nothing> pivot_root(const std::string& newRoot, const std::string& putOld);
 
 namespace chroot {
 
-// Enter a 'chroot' environment. The caller should be in a new mount
-// namespace. Basic configuration of special filesystems and device
-// nodes is performed.
-Try<Nothing> prepare(const std::string& root);
-
+// Clone a device node to a target directory. Intermediate directory paths
+// are created in the target.
+Try<Nothing> copyDeviceNode(
+    const std::string& device, const std::string& target);
 
 //  Enter a 'chroot' environment. The caller should be in a new mount
 //  unmounted. The root path must have already been provisioned by
diff --git a/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp b/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp
index c7d753a..c318fb9 100644
--- a/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp
+++ b/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp
@@ -19,6 +19,7 @@
 #include <sstream>
 #include <string>
 #include <vector>
+#include <utility>
 
 #include <glog/logging.h>
 
@@ -30,6 +31,7 @@
 #include <stout/adaptor.hpp>
 #include <stout/error.hpp>
 #include <stout/foreach.hpp>
+#include <stout/fs.hpp>
 #include <stout/os.hpp>
 #include <stout/path.hpp>
 #include <stout/stringify.hpp>
@@ -39,6 +41,8 @@
 #include <stout/os/strerror.hpp>
 #include <stout/os/realpath.hpp>
 
+#include "common/protobuf_utils.hpp"
+
 #include "linux/fs.hpp"
 #include "linux/ns.hpp"
 
@@ -52,21 +56,196 @@
 using namespace process;
 
 using std::ostringstream;
+using std::pair;
 using std::string;
 using std::vector;
 
+using mesos::internal::protobuf::slave::createContainerMount;
+
 using mesos::slave::ContainerClass;
 using mesos::slave::ContainerConfig;
-using mesos::slave::ContainerState;
 using mesos::slave::ContainerLaunchInfo;
 using mesos::slave::ContainerLimitation;
 using mesos::slave::ContainerMountInfo;
+using mesos::slave::ContainerState;
 using mesos::slave::Isolator;
 
 namespace mesos {
 namespace internal {
 namespace slave {
 
+// List of special filesystems useful for a chroot environment.
+// NOTE: This list is ordered, e.g., mount /proc before bind
+// mounting /proc/sys.
+//
+// TODO(jasonlai): These special filesystem mount points need to be
+// bind-mounted prior to all other mount points specified in
+// `ContainerLaunchInfo`.
+//
+// One example of the known issues caused by this behavior is:
+// https://issues.apache.org/jira/browse/MESOS-6798
+// There will be follow-up efforts on moving the logic below to
+// proper isolators.
+//
+// TODO(jasonlai): Consider adding knobs to allow write access to
+// those system files if configured by the operator.
+static const ContainerMountInfo ROOTFS_CONTAINER_MOUNTS[] = {
+  createContainerMount(
+      "proc",
+      "/proc",
+      "proc",
+      MS_NOSUID | MS_NOEXEC | MS_NODEV),
+  createContainerMount(
+      "/proc/bus",
+      "/proc/bus",
+      MS_BIND | MS_RDONLY | MS_NOSUID | MS_NOEXEC | MS_NODEV),
+  createContainerMount(
+      "/proc/fs",
+      "/proc/fs",
+      MS_BIND | MS_RDONLY | MS_NOSUID | MS_NOEXEC | MS_NODEV),
+  createContainerMount(
+      "/proc/irq",
+      "/proc/irq",
+      MS_BIND | MS_RDONLY | MS_NOSUID | MS_NOEXEC | MS_NODEV),
+  createContainerMount(
+      "/proc/sys",
+      "/proc/sys",
+      MS_BIND | MS_RDONLY | MS_NOSUID | MS_NOEXEC | MS_NODEV),
+  createContainerMount(
+      "/proc/sysrq-trigger",
+      "/proc/sysrq-trigger",
+      MS_BIND | MS_RDONLY | MS_NOSUID | MS_NOEXEC | MS_NODEV),
+  createContainerMount(
+      "sysfs",
+      "/sys",
+      "sysfs",
+      MS_RDONLY | MS_NOSUID | MS_NOEXEC | MS_NODEV),
+  createContainerMount(
+      "tmpfs",
+      "/sys/fs/cgroup",
+      "tmpfs",
+      "mode=755",
+      MS_NOSUID | MS_NOEXEC | MS_NODEV),
+  createContainerMount(
+      "tmpfs",
+      "/dev",
+      "tmpfs",
+      "mode=755",
+      MS_NOSUID | MS_NOEXEC | MS_STRICTATIME),
+  // We mount devpts with the gid=5 option because the `tty` group is
+  // GID 5 on all standard Linux distributions. The glibc grantpt(3)
+  // API ensures that the terminal GID is that of the `tty` group, and
+  // invokes a privileged helper if necessary. Since the helper won't
+  // work in all container configurations (since it may not be possible
+  // to acquire the necessary privileges), mounting with the right `gid`
+  // option avoids any possible failure.
+  createContainerMount(
+      "devpts",
+      "/dev/pts",
+      "devpts",
+      "newinstance,ptmxmode=0666,mode=0620,gid=5",
+      MS_NOSUID | MS_NOEXEC),
+  createContainerMount(
+      "tmpfs",
+      "/dev/shm",
+      "tmpfs",
+      "mode=1777",
+      MS_NOSUID | MS_NODEV | MS_STRICTATIME),
+};
+
+
+static Try<Nothing> makeStandardDevices(
+    const string& devicesDir,
+    const string& rootDir,
+    ContainerLaunchInfo& launchInfo)
+{
+  // List of standard devices useful for a chroot environment.
+  // TODO(idownes): Make this list configurable.
+  const vector<string> devices = {
+    "full",
+    "null",
+    "random",
+    "tty",
+    "urandom",
+    "zero"
+  };
+
+  // Import each device into the chroot environment. Copy both the
+  // mode and the device itself from the corresponding host device.
+  foreach (const string& device, devices) {
+    Try<Nothing> mknod = fs::chroot::copyDeviceNode(
+        path::join("/",  "dev", device),
+        path::join(devicesDir, device));
+
+    if (mknod.isError()) {
+      return Error(
+          "Failed to import device '" + device + "': " + mknod.error());
+    }
+
+    // Bind mount from the devices directory into the rootfs.
+    *launchInfo.add_mounts() = createContainerMount(
+        path::join(devicesDir, device),
+        path::join(rootDir, "dev", device),
+        MS_BIND);
+  }
+
+  const vector<pair<string, string>> symlinks = {
+    {"/proc/self/fd",   path::join(rootDir, "dev", "fd")},
+    {"/proc/self/fd/0", path::join(rootDir, "dev", "stdin")},
+    {"/proc/self/fd/1", path::join(rootDir, "dev", "stdout")},
+    {"/proc/self/fd/2", path::join(rootDir, "dev", "stderr")},
+    {"pts/ptmx",        path::join(rootDir, "dev", "ptmx")}
+  };
+
+  foreach (const auto& symlink, symlinks) {
+    CommandInfo* ln = launchInfo.add_pre_exec_commands();
+    ln->set_shell(false);
+    ln->set_value("ln");
+    ln->add_arguments("ln");
+    ln->add_arguments("-s");
+    ln->add_arguments(symlink.first);
+    ln->add_arguments(symlink.second);
+  }
+
+  // TODO(idownes): Set up console device.
+  return Nothing();
+}
+
+
+static Try<Nothing> makeDevicesDir(
+    const string& devicesDir,
+    const Option<string>& username)
+{
+  Try<Nothing> mkdir = os::mkdir(devicesDir);
+  if (mkdir.isError()) {
+    return Error(
+        "Failed to create container devices directory: " + mkdir.error());
+  }
+
+  Try<Nothing> chmod = os::chmod(devicesDir, 0700);
+  if (chmod.isError()) {
+    return Error(
+        "Failed to set container devices directory permissions: " +
+        chmod.error());
+  }
+
+  // We need to restrict access to the devices directory so that all
+  // processes on the system don't get access to devices that we make
+  // read-write. This means that we have to chown to ensure that the
+  // container user still has access.
+  if (username.isSome()) {
+    Try<Nothing> chown = os::chown(username.get(), devicesDir);
+    if (chown.isError()) {
+      return Error(
+          "Failed to set '" + username.get() + "' "
+          "as the container devices directory owner: " + chown.error());
+    }
+  }
+
+  return Nothing();
+}
+
+
 Try<Isolator*> LinuxFilesystemIsolatorProcess::create(const Flags& flags)
 {
   if (geteuid() != 0) {
@@ -375,26 +554,68 @@ Future<Option<ContainerLaunchInfo>> LinuxFilesystemIsolatorProcess::prepare(
   ContainerLaunchInfo launchInfo;
   launchInfo.add_clone_namespaces(CLONE_NEWNS);
 
-  // Bind mount the sandbox if the container specifies a rootfs.
   if (containerConfig.has_rootfs()) {
-    string sandbox = path::join(
+    // Set up the container devices directory.
+    const string devicesDir = containerizer::paths::getContainerDevicesPath(
+        flags.runtime_dir, containerId);
+
+    CHECK(!os::exists(devicesDir));
+
+    Try<Nothing> mkdir = makeDevicesDir(
+        devicesDir,
+        containerConfig.has_user() ? containerConfig.user()
+                                   : Option<string>::none());
+    if (mkdir.isError()) {
+      return Failure(
+          "Failed to create container devices directory: " + mkdir.error());
+    }
+
+    // Bind mount 'root' itself. This is because pivot_root requires
+    // 'root' to be not on the same filesystem as process' current root.
+    *launchInfo.add_mounts() = createContainerMount(
+        containerConfig.rootfs(),
+        containerConfig.rootfs(),
+        MS_REC | MS_BIND);
+
+    foreach (const ContainerMountInfo& mnt, ROOTFS_CONTAINER_MOUNTS) {
+      // The target for special mounts must always be an absolute path.
+      CHECK(path::absolute(mnt.target()));
+
+      ContainerMountInfo* info = launchInfo.add_mounts();
+
+      *info = mnt;
+      info->set_target(path::join(containerConfig.rootfs(), mnt.target()));
+
+      // Absolute path mounts are always relative to the container root.
+      if (mnt.has_source() && path::absolute(mnt.source())) {
+        info->set_source(path::join(containerConfig.rootfs(), info->source()));
+      }
+    }
+
+    Try<Nothing> makedev =
+      makeStandardDevices(devicesDir, containerConfig.rootfs(), launchInfo);
+    if (makedev.isError()) {
+      return Failure(
+          "Failed to prepare standard devices: " + makedev.error());
+    }
+
+    // Bind mount the sandbox if the container specifies a rootfs.
+    const string sandbox = path::join(
         containerConfig.rootfs(),
         flags.sandbox_directory);
 
     // If the rootfs is a read-only filesystem (e.g., using the bind
     // backend), the sandbox must be already exist. Please see the
     // comments in 'provisioner/backend.hpp' for details.
-    Try<Nothing> mkdir = os::mkdir(sandbox);
+    mkdir = os::mkdir(sandbox);
     if (mkdir.isError()) {
       return Failure(
           "Failed to create sandbox mount point at '" +
           sandbox + "': " + mkdir.error());
     }
 
-    ContainerMountInfo* mount = launchInfo.add_mounts();
-    mount->set_source(containerConfig.directory());
-    mount->set_target(sandbox);
-    mount->set_flags(MS_BIND | MS_REC);
+    *launchInfo.add_mounts() = createContainerMount(
+        containerConfig.directory(), sandbox, MS_BIND | MS_REC);
   }
 
   // Currently, we only need to update resources for top level containers.
diff --git a/src/slave/containerizer/mesos/isolators/gpu/isolator.cpp b/src/slave/containerizer/mesos/isolators/gpu/isolator.cpp
index 56d8357..1487128 100644
--- a/src/slave/containerizer/mesos/isolators/gpu/isolator.cpp
+++ b/src/slave/containerizer/mesos/isolators/gpu/isolator.cpp
@@ -27,6 +27,7 @@ extern "C" {
 }
 
 #include <algorithm>
+#include <list>
 #include <map>
 #include <set>
 #include <string>
@@ -44,7 +45,10 @@ extern "C" {
 #include <stout/os.hpp>
 #include <stout/try.hpp>
 
+#include "common/protobuf_utils.hpp"
+
 #include "linux/cgroups.hpp"
+#include "linux/fs.hpp"
 
 #include "slave/flags.hpp"
 
@@ -58,6 +62,8 @@ extern "C" {
 #include "slave/containerizer/mesos/isolators/gpu/isolator.hpp"
 #include "slave/containerizer/mesos/isolators/gpu/nvml.hpp"
 
+#include "slave/containerizer/mesos/paths.hpp"
+
 using cgroups::devices::Entry;
 
 using docker::spec::v1::ImageManifest;
@@ -75,6 +81,7 @@ using process::Failure;
 using process::Future;
 using process::PID;
 
+using std::list;
 using std::map;
 using std::set;
 using std::string;
@@ -326,7 +333,7 @@ Future<Option<ContainerLaunchInfo>> NvidiaGpuIsolatorProcess::prepare(
     // mount the necessary Nvidia libraries into the container (since
     // we live in a different mount namespace than our parent). We
     // directly call `_prepare()` to do this for us.
-    return _prepare(containerConfig);
+    return _prepare(containerId, containerConfig);
   }
 
   if (infos.contains(containerId)) {
@@ -356,6 +363,7 @@ Future<Option<ContainerLaunchInfo>> NvidiaGpuIsolatorProcess::prepare(
   return update(containerId, containerConfig.resources())
     .then(defer(PID<NvidiaGpuIsolatorProcess>(this),
                 &NvidiaGpuIsolatorProcess::_prepare,
+                containerId,
                 containerConfig));
 }
 
@@ -364,6 +372,7 @@ Future<Option<ContainerLaunchInfo>> NvidiaGpuIsolatorProcess::prepare(
 // host file system, then we need to prepare a script to inject our
 // `NvidiaVolume` into the container (if required).
 Future<Option<ContainerLaunchInfo>> NvidiaGpuIsolatorProcess::_prepare(
+    const ContainerID& containerId,
     const mesos::slave::ContainerConfig& containerConfig)
 {
   if (!containerConfig.has_rootfs()) {
@@ -380,10 +389,6 @@ Future<Option<ContainerLaunchInfo>> NvidiaGpuIsolatorProcess::_prepare(
   ContainerLaunchInfo launchInfo;
 
   // Inject the Nvidia volume into the container.
-  //
-  // TODO(klueska): Inject the Nvidia devices here as well once we
-  // have a way to pass them to `fs:enter()` instead of hardcoding
-  // them in `fs::createStandardDevices()`.
   if (!containerConfig.docker().has_manifest()) {
      return Failure("The 'ContainerConfig' for docker is missing a manifest");
   }
@@ -402,10 +407,52 @@ Future<Option<ContainerLaunchInfo>> NvidiaGpuIsolatorProcess::_prepare(
           " '" + target + "': " + mkdir.error());
     }
 
-    ContainerMountInfo* mount = launchInfo.add_mounts();
-    mount->set_source(volume.HOST_PATH());
-    mount->set_target(target);
-    mount->set_flags(MS_RDONLY | MS_BIND | MS_REC);
+    *launchInfo.add_mounts() = protobuf::slave::createContainerMount(
+        volume.HOST_PATH(), target, MS_RDONLY | MS_BIND | MS_REC);
+  }
+
+  const string devicesDir = containerizer::paths::getContainerDevicesPath(
+      flags.runtime_dir, containerId);
+
+  // The `filesystem/linux` isolator is responsible for creating the
+  // devices directory and ordered to run before we do. Here, we can
+  // just assert that the devices directory is still present.
+  if (!os::exists(devicesDir)) {
+    return Failure("Missing container devices directory '" + devicesDir + "'");
+  }
+
+  // Glob all Nvidia GPU devices on the system and add them to the
+  // list of devices injected into the chroot environment.
+  Try<list<string>> nvidia = os::glob("/dev/nvidia*");
+  if (nvidia.isError()) {
+    return Failure("Failed to glob /dev/nvidia*: " + nvidia.error());
+  }
+
+  foreach (const string& device, nvidia.get()) {
+    const string devicePath = path::join(
+        devicesDir, strings::remove(device, "/dev/", strings::PREFIX), device);
+
+    Try<Nothing> mknod =
+      fs::chroot::copyDeviceNode(device, devicePath);
+    if (mknod.isError()) {
+      return Failure(
+          "Failed to copy device '" + device + "': " + mknod.error());
+    }
+
+    // Since we are adding the GPU devices to the container, make
+    // them read/write to guarantee that they are accessible inside
+    // the container.
+    Try<Nothing> chmod = os::chmod(devicePath, 0666);
+    if (chmod.isError()) {
+      return Failure(
+          "Failed to set permissions on device '" + device + "': " +
+          chmod.error());
+    }
+
+    *launchInfo.add_mounts() = protobuf::slave::createContainerMount(
+        devicePath,
+        path::join(containerConfig.rootfs(), "dev", device),
+        MS_BIND);
   }
 
   return launchInfo;
diff --git a/src/slave/containerizer/mesos/isolators/gpu/isolator.hpp b/src/slave/containerizer/mesos/isolators/gpu/isolator.hpp
index 4645c62..cc3cab5 100644
--- a/src/slave/containerizer/mesos/isolators/gpu/isolator.hpp
+++ b/src/slave/containerizer/mesos/isolators/gpu/isolator.hpp
@@ -114,6 +114,7 @@ private:
       const std::map<Path, cgroups::devices::Entry>& _controlDeviceEntries);
 
   virtual process::Future<Option<mesos::slave::ContainerLaunchInfo>> _prepare(
+      const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig);
 
   process::Future<Nothing> _update(
diff --git a/src/slave/containerizer/mesos/isolators/linux/devices.cpp b/src/slave/containerizer/mesos/isolators/linux/devices.cpp
index 8f8ff95..bbc9c19 100644
--- a/src/slave/containerizer/mesos/isolators/linux/devices.cpp
+++ b/src/slave/containerizer/mesos/isolators/linux/devices.cpp
@@ -26,6 +26,8 @@
 
 #include <stout/os/posix/chown.hpp>
 
+#include "common/protobuf_utils.hpp"
+
 #include "slave/containerizer/mesos/paths.hpp"
 
 using process::Failure;
@@ -152,29 +154,11 @@ Future<Option<ContainerLaunchInfo>> LinuxDevicesIsolatorProcess::prepare(
   const string devicesDir = containerizer::paths::getContainerDevicesPath(
       runtimeDirectory, containerId);
 
-  Try<Nothing> mkdir = os::mkdir(devicesDir);
-  if (mkdir.isError()) {
-    return Failure(
-        "Failed to create container devices directory: " + mkdir.error());
-  }
-
-  Try<Nothing> chmod = os::chmod(devicesDir, 0700);
-  if (chmod.isError()) {
-    return Failure("Failed to set container devices directory permissions: " +
-                   chmod.error());
-  }
-
-  // We need to restrict access to the devices directory so that all
-  // processes on the system don't get access to devices that we make
-  // read-write. This means that we have to chown to ensure that the
-  // container user still has access.
-  if (containerConfig.has_user()) {
-    Try<Nothing> chown = os::chown(containerConfig.user(), devicesDir);
-    if (chown.isError()) {
-      return Failure(
-          "Failed to set '" + containerConfig.user() + "' "
-          "as the container devices directory owner: " + chown.error());
-    }
+  // The `filesystem/linux` isolator is responsible for creating the
+  // devices directory and ordered to run before we do. Here, we can
+  // just assert that the devices directory is still present.
+  if (!os::exists(devicesDir)) {
+    return Failure("Missing container devices directory '" + devicesDir + "'");
   }
 
   // Import the whitelisted devices to all containers.
@@ -202,10 +186,10 @@ Future<Option<ContainerLaunchInfo>> LinuxDevicesIsolatorProcess::prepare(
           "Failed to chmod device '" + devicePath + "': " + chmod.error());
     }
 
-    ContainerMountInfo* mount = launchInfo.add_mounts();
-    mount->set_source(devicePath);
-    mount->set_target(path::join(containerConfig.rootfs(), "dev", path));
-    mount->set_flags(MS_BIND);
+    *launchInfo.add_mounts() = protobuf::slave::createContainerMount(
+        devicePath,
+        path::join(containerConfig.rootfs(), "dev", path),
+        MS_BIND);
   }
 
   // TODO(jpeach) Define Task API to let schedulers specify the container
diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index 882bcdf..29f4b2e 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -410,9 +410,14 @@ static Try<Nothing> prepareMounts(const ContainerLaunchInfo& launchInfo)
         }
       }
 
-      Try<Nothing> target = os::stat::isdir(mount.source())
-        ? os::mkdir(mount.target())
-        : os::touch(mount.target());
+      // The mount source could be a file, a directory, or a Linux
+      // pseudo-filesystem. In the last case, the target must be a
+      // directory, so if the source doesn't exist, we default to
+      // mounting on a directory.
+      Try<Nothing> target =
+        (!os::exists(mount.source()) || os::stat::isdir(mount.source()))
+          ? os::mkdir(mount.target())
+          : os::touch(mount.target());
 
       if (target.isError()) {
         return Error(
@@ -462,37 +467,6 @@ static Try<Nothing> installResourceLimits(const RLimitInfo& limits)
 }
 
 
-static Try<Nothing> prepareChroot(const string& rootfs)
-{
-#ifdef __WINDOWS__
-  return Error("Changing rootfs is not supported on Windows");
-#else
-  // Verify that rootfs is an absolute path.
-  Result<string> realpath = os::realpath(rootfs);
-  if (realpath.isError()) {
-    return Error(
-        "Failed to determine if rootfs '" + rootfs +
-        "' is an absolute path: " + realpath.error());
-  } else if (realpath.isNone()) {
-    return Error("Rootfs path '" + rootfs + "' does not exist");
-  } else if (realpath.get() != rootfs) {
-    return Error("Rootfs path '" + rootfs + "' is not an absolute path");
-  }
-
-#ifdef __linux__
-  Try<Nothing> prepare = fs::chroot::prepare(rootfs);
-  if (prepare.isError()) {
-    return Error(
-        "Failed to prepare chroot '" + rootfs + "': " +
-        prepare.error());
-  }
-#endif // __linux__
-
-  return Nothing();
-#endif // __WINDOWS__
-}
-
-
 static Try<Nothing> enterChroot(const string& rootfs)
 {
 #ifdef __WINDOWS__
@@ -695,18 +669,29 @@ int MesosContainerizerLaunch::execute()
   }
 #endif // __linux__
 
-  // Prepare root to a new root, if provided. Make sure that we do this before
-  // processing the container mounts so that container mounts can be made on
-  // top of the rootfs template.
+  // Verify that the rootfs is an absolute path.
   if (launchInfo.has_rootfs()) {
-    cerr << "Preparing rootfs at " << launchInfo.rootfs() << endl;
+    const string& rootfs = launchInfo.rootfs();
 
-    Try<Nothing> prepare = prepareChroot(launchInfo.rootfs());
+    cerr << "Preparing rootfs at '" << rootfs << "'" << endl;
 
-    if (prepare.isError()) {
-      cerr << prepare.error() << endl;
+    Result<string> realpath = os::realpath(rootfs);
+    if (realpath.isError()) {
+      cerr << "Failed to determine if rootfs '" << rootfs
+           << "' is an absolute path: " << realpath.error() << endl;
+      exitWithStatus(EXIT_FAILURE);
+    } else if (realpath.isNone()) {
+      cerr << "Rootfs path '" << rootfs << "' does not exist" << endl;
+      exitWithStatus(EXIT_FAILURE);
+    } else if (realpath.get() != rootfs) {
+      cerr << "Rootfs path '" << rootfs << "' is not an absolute path" << endl;
       exitWithStatus(EXIT_FAILURE);
     }
+
+#ifdef __WINDOWS__
+    cerr << "Changing rootfs is not supported on Windows";
+    exitWithStatus(EXIT_FAILURE);
+#endif // __WINDOWS__
   }
 
   Try<Nothing> mount = prepareMounts(launchInfo);