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/05/25 21:12:50 UTC

[1/6] mesos git commit: Split linux chroot into prepare and enter phases.

Repository: mesos
Updated Branches:
  refs/heads/master fcc8ca168 -> ee6c6cfcb


Split linux chroot into prepare and enter phases.

Since we will need to perform additional work to configure
the chroot before entering it, split the Linux chroot API
into `fs::chroot::prepare()` and `fs::chroot::enter()`.

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


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

Branch: refs/heads/master
Commit: 096908812c1e7481536390de45ff78d5d31516e0
Parents: fcc8ca1
Author: James Peach <jp...@apache.org>
Authored: Fri May 25 13:37:42 2018 -0700
Committer: James Peach <jp...@apache.org>
Committed: Fri May 25 13:37:42 2018 -0700

----------------------------------------------------------------------
 src/linux/fs.cpp                         | 11 +++++++++--
 src/linux/fs.hpp                         |  9 +++++++--
 src/slave/containerizer/mesos/launch.cpp |  9 +++++++++
 3 files changed, 25 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/09690881/src/linux/fs.cpp
----------------------------------------------------------------------
diff --git a/src/linux/fs.cpp b/src/linux/fs.cpp
index fbd03b1..6b38b4a 100644
--- a/src/linux/fs.cpp
+++ b/src/linux/fs.cpp
@@ -908,7 +908,7 @@ Try<Nothing> createStandardDevices(const string& root)
 
 
 // TODO(idownes): Add unit test.
-Try<Nothing> enter(const string& root)
+Try<Nothing> prepare(const string& root)
 {
   // Recursively mark current mounts as slaves to prevent propagation.
   Try<Nothing> mount =
@@ -937,6 +937,13 @@ Try<Nothing> enter(const string& root)
     return Error("Failed to create devices: " + create.error());
   }
 
+  return Nothing();
+}
+
+
+// TODO(idownes): Add unit test.
+Try<Nothing> enter(const string& root)
+{
   // Prepare /tmp in the new root. Note that we cannot assume that the
   // new root is writable (i.e., it could be a read only filesystem).
   // Therefore, we always mount a tmpfs on /tmp in the new root so
@@ -953,7 +960,7 @@ Try<Nothing> enter(const string& root)
   }
 
   // TODO(jieyu): Consider limiting the size of the tmpfs.
-  mount = fs::mount(
+  Try<Nothing> mount = fs::mount(
       "tmpfs",
       path::join(root, "tmp"),
       "tmpfs",

http://git-wip-us.apache.org/repos/asf/mesos/blob/09690881/src/linux/fs.hpp
----------------------------------------------------------------------
diff --git a/src/linux/fs.hpp b/src/linux/fs.hpp
index 76dc09c..502f85c 100644
--- a/src/linux/fs.hpp
+++ b/src/linux/fs.hpp
@@ -385,8 +385,13 @@ 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. Any mounts to the current root will be
-// unmounted.
+// nodes is performed.
+Try<Nothing> prepare(const std::string& root);
+
+
+//  Enter a 'chroot' environment. The caller should be in a new mount
+//  unmounted. The root path must have already been provisioned by
+//  calling `prepare`()`.
 Try<Nothing> enter(const std::string& root);
 
 } // namespace chroot {

http://git-wip-us.apache.org/repos/asf/mesos/blob/09690881/src/slave/containerizer/mesos/launch.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index f25d906..b8ca608 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -455,6 +455,15 @@ static Try<Nothing> enterChroot(const string& rootfs)
   }
 
 #ifdef __linux__
+  Try<Nothing> prepare = fs::chroot::prepare(rootfs);
+  if (prepare.isError()) {
+    return Error(
+        "Failed to prepare chroot '" + rootfs + "': " +
+        prepare.error());
+  }
+
+  // TODO(jpeach): apply container mounts here.
+
   Try<Nothing> chroot = fs::chroot::enter(rootfs);
 #else
   // For any other platform we'll just use POSIX chroot.


[3/6] mesos git commit: Added a `linux/devices` isolator skeleton.

Posted by jp...@apache.org.
Added a `linux/devices` isolator skeleton.

Added the skeleton of a `linux/devices` isolator and wired it into
the build and the Mesos containerizer.

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


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

Branch: refs/heads/master
Commit: 377478d5967535a4f71abc6ae3351fabd422452b
Parents: 0e9a414
Author: James Peach <jp...@apache.org>
Authored: Fri May 25 13:38:00 2018 -0700
Committer: James Peach <jp...@apache.org>
Committed: Fri May 25 13:38:00 2018 -0700

----------------------------------------------------------------------
 src/CMakeLists.txt                              |  1 +
 src/Makefile.am                                 |  2 +
 src/slave/containerizer/mesos/containerizer.cpp |  2 +
 .../mesos/isolators/linux/devices.cpp           | 64 ++++++++++++++++++++
 .../mesos/isolators/linux/devices.hpp           | 50 +++++++++++++++
 5 files changed, 119 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/377478d5/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index d488131..f86884d 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -302,6 +302,7 @@ set(LINUX_SRC
   slave/containerizer/mesos/isolators/gpu/nvml.cpp
   slave/containerizer/mesos/isolators/gpu/volume.cpp
   slave/containerizer/mesos/isolators/linux/capabilities.cpp
+  slave/containerizer/mesos/isolators/linux/devices.cpp
   slave/containerizer/mesos/isolators/namespaces/ipc.cpp
   slave/containerizer/mesos/isolators/namespaces/pid.cpp
   slave/containerizer/mesos/isolators/network/cni/cni.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/377478d5/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index c08ac6e..da0d683 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1335,6 +1335,7 @@ MESOS_LINUX_FILES =									\
   slave/containerizer/mesos/isolators/gpu/nvml.cpp					\
   slave/containerizer/mesos/isolators/gpu/volume.cpp					\
   slave/containerizer/mesos/isolators/linux/capabilities.cpp				\
+  slave/containerizer/mesos/isolators/linux/devices.cpp					\
   slave/containerizer/mesos/isolators/namespaces/ipc.cpp				\
   slave/containerizer/mesos/isolators/namespaces/pid.cpp				\
   slave/containerizer/mesos/isolators/network/cni/cni.cpp				\
@@ -1383,6 +1384,7 @@ MESOS_LINUX_FILES +=									\
   slave/containerizer/mesos/isolators/gpu/nvml.hpp					\
   slave/containerizer/mesos/isolators/gpu/volume.hpp					\
   slave/containerizer/mesos/isolators/linux/capabilities.hpp				\
+  slave/containerizer/mesos/isolators/linux/devices.hpp					\
   slave/containerizer/mesos/isolators/namespaces/ipc.hpp				\
   slave/containerizer/mesos/isolators/namespaces/pid.hpp				\
   slave/containerizer/mesos/isolators/network/cni/cni.hpp				\

http://git-wip-us.apache.org/repos/asf/mesos/blob/377478d5/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 6620e94..93197b0 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -98,6 +98,7 @@
 #include "slave/containerizer/mesos/isolators/filesystem/shared.hpp"
 #include "slave/containerizer/mesos/isolators/gpu/nvidia.hpp"
 #include "slave/containerizer/mesos/isolators/linux/capabilities.hpp"
+#include "slave/containerizer/mesos/isolators/linux/devices.hpp"
 #include "slave/containerizer/mesos/isolators/namespaces/ipc.hpp"
 #include "slave/containerizer/mesos/isolators/namespaces/pid.hpp"
 #include "slave/containerizer/mesos/isolators/network/cni/cni.hpp"
@@ -396,6 +397,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     {"appc/runtime", &AppcRuntimeIsolatorProcess::create},
     {"docker/runtime", &DockerRuntimeIsolatorProcess::create},
 
+    {"linux/devices", &LinuxDevicesIsolatorProcess::create},
     {"linux/capabilities", &LinuxCapabilitiesIsolatorProcess::create},
 
     {"namespaces/ipc", &NamespacesIPCIsolatorProcess::create},

http://git-wip-us.apache.org/repos/asf/mesos/blob/377478d5/src/slave/containerizer/mesos/isolators/linux/devices.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/linux/devices.cpp b/src/slave/containerizer/mesos/isolators/linux/devices.cpp
new file mode 100644
index 0000000..35ed008
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/linux/devices.cpp
@@ -0,0 +1,64 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+
+#include "slave/containerizer/mesos/isolators/linux/devices.hpp"
+
+using process::Failure;
+using process::Future;
+using process::Owned;
+
+using mesos::slave::ContainerConfig;
+using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::Isolator;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+Try<Isolator*> LinuxDevicesIsolatorProcess::create(const Flags& flags)
+{
+  if (geteuid() != 0) {
+    return Error("Linux devices isolator requires root permissions");
+  }
+
+  return new MesosIsolator(
+      Owned<MesosIsolatorProcess>(new LinuxDevicesIsolatorProcess(flags)));
+}
+
+
+bool LinuxDevicesIsolatorProcess::supportsNesting()
+{
+  return true;
+}
+
+
+bool LinuxDevicesIsolatorProcess::supportsStandalone()
+{
+  return true;
+}
+
+
+Future<Option<ContainerLaunchInfo>> LinuxDevicesIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ContainerConfig& containerConfig)
+{
+    return None();
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/377478d5/src/slave/containerizer/mesos/isolators/linux/devices.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/linux/devices.hpp b/src/slave/containerizer/mesos/isolators/linux/devices.hpp
new file mode 100644
index 0000000..e731ef3
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/linux/devices.hpp
@@ -0,0 +1,50 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef __LINUX_DEVICES_ISOLATOR_HPP__
+#define __LINUX_DEVICES_ISOLATOR_HPP__
+
+#include <stout/try.hpp>
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolator.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class LinuxDevicesIsolatorProcess : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(const Flags& flags);
+
+  virtual bool supportsNesting();
+  virtual bool supportsStandalone();
+
+  virtual process::Future<Option<mesos::slave::ContainerLaunchInfo>> prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig);
+
+private:
+  LinuxDevicesIsolatorProcess(const Flags& _flags);
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif  // __LINUX_DEVICES_ISOLATOR_HPP__


[2/6] mesos git commit: Added a containerizer devices path helper.

Posted by jp...@apache.org.
Added a containerizer devices path helper.

Added a helper to define a per-container directory for storing
container device nodes.

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


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

Branch: refs/heads/master
Commit: 0e9a4149edb777a66818572134b08bc1dea88eea
Parents: 0969088
Author: James Peach <jp...@apache.org>
Authored: Fri May 25 13:37:56 2018 -0700
Committer: James Peach <jp...@apache.org>
Committed: Fri May 25 13:37:56 2018 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/paths.cpp | 10 ++++++++++
 src/slave/containerizer/mesos/paths.hpp |  8 ++++++++
 2 files changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0e9a4149/src/slave/containerizer/mesos/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/paths.cpp b/src/slave/containerizer/mesos/paths.cpp
index cf7d47b..94ab921 100644
--- a/src/slave/containerizer/mesos/paths.cpp
+++ b/src/slave/containerizer/mesos/paths.cpp
@@ -78,6 +78,16 @@ string getRuntimePath(
 }
 
 
+string getContainerDevicesPath(
+    const string& runtimeDir,
+    const ContainerID& containerId)
+{
+  return path::join(
+      getRuntimePath(runtimeDir, containerId),
+      CONTAINER_DEVICES_DIRECTORY);
+}
+
+
 Result<pid_t> getContainerPid(
     const string& runtimeDir,
     const ContainerID& containerId)

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e9a4149/src/slave/containerizer/mesos/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/paths.hpp b/src/slave/containerizer/mesos/paths.hpp
index b9f0f45..de3981d 100644
--- a/src/slave/containerizer/mesos/paths.hpp
+++ b/src/slave/containerizer/mesos/paths.hpp
@@ -73,6 +73,7 @@ constexpr char SOCKET_FILE[] = "socket";
 constexpr char FORCE_DESTROY_ON_RECOVERY_FILE[] = "force_destroy_on_recovery";
 constexpr char IO_SWITCHBOARD_DIRECTORY[] = "io_switchboard";
 constexpr char CONTAINER_DIRECTORY[] = "containers";
+constexpr char CONTAINER_DEVICES_DIRECTORY[] = "devices";
 constexpr char CONTAINER_LAUNCH_INFO_FILE[] = "launch_info";
 constexpr char STANDALONE_MARKER_FILE[] = "standalone.marker";
 
@@ -107,6 +108,13 @@ std::string getRuntimePath(
     const ContainerID& containerId);
 
 
+// Given a `runtimeDir`, construct a unique directory to store
+// per-container device nodes.
+std::string getContainerDevicesPath(
+    const std::string& runtimeDir,
+    const ContainerID& containerId);
+
+
 // The helper method to read the pid file.
 Result<pid_t> getContainerPid(
     const std::string& runtimeDir,


[5/6] mesos git commit: Updated the container launcher mount sequence.

Posted by jp...@apache.org.
Updated the container launcher mount sequence.

The `linux/devices` isolator needs to make bind mounts into
the `/dev` directory of the container. However, the container
mounts are made before the container `/dev` is mounted as part
of the chroot preparation. We need to prepare the chroot,
then make any necessary container mounts, and finally enter
the chroot. This sequence of operations also requires us to
touch the target mount point, since we can't do it from the
isolator because the '/dev' directory doesn't exist yet.

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


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

Branch: refs/heads/master
Commit: 68db3f988d557e326f88bf32ef32368f0b919bcf
Parents: ae413c9
Author: James Peach <jp...@apache.org>
Authored: Fri May 25 13:38:10 2018 -0700
Committer: James Peach <jp...@apache.org>
Committed: Fri May 25 13:38:10 2018 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/launch.cpp | 112 ++++++++++++++++++++------
 1 file changed, 87 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/68db3f98/src/slave/containerizer/mesos/launch.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index b8ca608..70b7805 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -396,6 +396,32 @@ static Try<Nothing> prepareMounts(const ContainerLaunchInfo& launchInfo)
       }
     }
 
+    // If the mount target doesn't exist yet, create it. The isolator
+    // is responsible for ensuring the target path is safe.
+    if (mount.has_source() && !os::exists(mount.target())) {
+      const string dirname = Path(mount.target()).dirname();
+
+      if (!os::exists(dirname)) {
+        Try<Nothing> mkdir = os::mkdir(Path(mount.target()).dirname());
+
+        if (mkdir.isError()) {
+          return Error(
+              "Failed to create mount target directory '" + dirname + "': " +
+              mkdir.error());
+        }
+      }
+
+      Try<Nothing> target = os::stat::isdir(mount.source())
+        ? os::mkdir(mount.target())
+        : os::touch(mount.target());
+
+      if (target.isError()) {
+        return Error(
+            "Failed to create mount target '" + mount.target() + "': " +
+            target.error());
+      }
+    }
+
     Try<Nothing> mnt = fs::mount(
         (mount.has_source() ? Option<string>(mount.source()) : None()),
         mount.target(),
@@ -437,7 +463,7 @@ static Try<Nothing> installResourceLimits(const RLimitInfo& limits)
 }
 
 
-static Try<Nothing> enterChroot(const string& rootfs)
+static Try<Nothing> prepareChroot(const string& rootfs)
 {
 #ifdef __WINDOWS__
   return Error("Changing rootfs is not supported on Windows");
@@ -461,9 +487,19 @@ static Try<Nothing> enterChroot(const string& rootfs)
         "Failed to prepare chroot '" + rootfs + "': " +
         prepare.error());
   }
+#endif // __linux__
+
+  return Nothing();
+#endif // __WINDOWS__
+}
 
-  // TODO(jpeach): apply container mounts here.
 
+static Try<Nothing> enterChroot(const string& rootfs)
+{
+#ifdef __WINDOWS__
+  return Error("Changing rootfs is not supported on Windows");
+#else
+#ifdef __linux__
   Try<Nothing> chroot = fs::chroot::enter(rootfs);
 #else
   // For any other platform we'll just use POSIX chroot.
@@ -634,6 +670,52 @@ int MesosContainerizerLaunch::execute()
   }
 #endif // __WINDOWS__
 
+#ifdef __linux__
+  // If we need a new mount namespace, we have to do it before
+  // we make the mounts needed to prepare the rootfs template.
+  if (flags.unshare_namespace_mnt) {
+    if (!launchInfo.mounts().empty()) {
+      cerr << "Mounts are not supported if "
+           << "'unshare_namespace_mnt' is set" << endl;
+      exitWithStatus(EXIT_FAILURE);
+    }
+
+    if (unshare(CLONE_NEWNS) != 0) {
+      cerr << "Failed to unshare mount namespace: "
+           << os::strerror(errno) << endl;
+      exitWithStatus(EXIT_FAILURE);
+    }
+  }
+
+  if (flags.namespace_mnt_target.isSome()) {
+    if (!launchInfo.mounts().empty()) {
+      cerr << "Mounts are not supported if "
+           << "'namespace_mnt_target' is set" << endl;
+      exitWithStatus(EXIT_FAILURE);
+    }
+
+    if (launchInfo.has_rootfs()) {
+      cerr << "Container rootfs is not supported if "
+           << "'namespace_mnt_target' is set" << endl;
+      exitWithStatus(EXIT_FAILURE);
+    }
+  }
+#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.
+  if (launchInfo.has_rootfs()) {
+    cout << "Preparing rootfs at " << launchInfo.rootfs() << endl;
+
+    Try<Nothing> prepare = prepareChroot(launchInfo.rootfs());
+
+    if (prepare.isError()) {
+      cerr << prepare.error() << endl;
+      exitWithStatus(EXIT_FAILURE);
+    }
+  }
+
   Try<Nothing> mount = prepareMounts(launchInfo);
   if (mount.isError()) {
     cerr << "Failed to prepare mounts: " << mount.error() << endl;
@@ -767,12 +849,6 @@ int MesosContainerizerLaunch::execute()
 
 #ifdef __linux__
   if (flags.namespace_mnt_target.isSome()) {
-    if (!launchInfo.mounts().empty()) {
-      cerr << "Mounts are not supported if "
-           << "'namespace_mnt_target' is set" << endl;
-      exitWithStatus(EXIT_FAILURE);
-    }
-
     string path = path::join(
         "/proc",
         stringify(flags.namespace_mnt_target.get()),
@@ -786,30 +862,16 @@ int MesosContainerizerLaunch::execute()
       exitWithStatus(EXIT_FAILURE);
     }
   }
-
-  if (flags.unshare_namespace_mnt) {
-    if (!launchInfo.mounts().empty()) {
-      cerr << "Mounts are not supported if "
-           << "'unshare_namespace_mnt' is set" << endl;
-      exitWithStatus(EXIT_FAILURE);
-    }
-
-    if (unshare(CLONE_NEWNS) != 0) {
-      cerr << "Failed to unshare mount namespace: "
-           << os::strerror(errno) << endl;
-      exitWithStatus(EXIT_FAILURE);
-    }
-  }
 #endif // __linux__
 
   // Change root to a new root, if provided.
   if (launchInfo.has_rootfs()) {
     cout << "Changing root to " << launchInfo.rootfs() << endl;
 
-    Try<Nothing> chroot = enterChroot(launchInfo.rootfs());
+    Try<Nothing> enter = enterChroot(launchInfo.rootfs());
 
-    if (chroot.isError()) {
-      cerr << chroot.error() << endl;
+    if (enter.isError()) {
+      cerr << enter.error() << endl;
       exitWithStatus(EXIT_FAILURE);
     }
   }


[4/6] mesos git commit: Added `linux/devices` isolator whitelist support.

Posted by jp...@apache.org.
Added `linux/devices` isolator whitelist support.

Added `linux/devices` isolator support for populating the container
devices.  This introduces a general mechanism for populating devices
into a specific container but currently only implements devices for all
containers based on the devices specified by the `--allowed_devices`
agent flag.

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


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

Branch: refs/heads/master
Commit: ae413c9f0496b24845bd868f740dfbb63ee6db4b
Parents: 377478d
Author: James Peach <jp...@apache.org>
Authored: Fri May 25 13:38:06 2018 -0700
Committer: James Peach <jp...@apache.org>
Committed: Fri May 25 13:38:06 2018 -0700

----------------------------------------------------------------------
 .../mesos/isolators/linux/devices.cpp           | 167 ++++++++++++++++++-
 .../mesos/isolators/linux/devices.hpp           |  23 ++-
 2 files changed, 182 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ae413c9f/src/slave/containerizer/mesos/isolators/linux/devices.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/linux/devices.cpp b/src/slave/containerizer/mesos/isolators/linux/devices.cpp
index 35ed008..cc2fd90 100644
--- a/src/slave/containerizer/mesos/isolators/linux/devices.cpp
+++ b/src/slave/containerizer/mesos/isolators/linux/devices.cpp
@@ -14,31 +14,116 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-
 #include "slave/containerizer/mesos/isolators/linux/devices.hpp"
 
+#include <sys/mount.h>
+
+#include <process/id.hpp>
+
+#include <stout/os.hpp>
+#include <stout/path.hpp>
+#include <stout/strings.hpp>
+
+#include <stout/os/posix/chown.hpp>
+
+#include "slave/containerizer/mesos/paths.hpp"
+
 using process::Failure;
 using process::Future;
 using process::Owned;
 
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::ContainerMountInfo;
 using mesos::slave::Isolator;
 
+using std::string;
+
 namespace mesos {
 namespace internal {
 namespace slave {
 
 Try<Isolator*> LinuxDevicesIsolatorProcess::create(const Flags& flags)
 {
-  if (geteuid() != 0) {
+  if (::geteuid() != 0) {
     return Error("Linux devices isolator requires root permissions");
   }
 
-  return new MesosIsolator(
-      Owned<MesosIsolatorProcess>(new LinuxDevicesIsolatorProcess(flags)));
+  if (flags.launcher != "linux") {
+    return Error("'linux' launcher must be used");
+  }
+
+  if (!strings::contains(flags.isolation, "filesystem/linux")) {
+    return Error("'filesystem/linux' isolator must be used");
+  }
+
+  hashmap<string, Device> whitelistedDevices;
+
+  if (flags.allowed_devices.isSome()) {
+    foreach (const DeviceAccess& deviceAccess,
+             flags.allowed_devices->allowed_devices()) {
+      // TODO(jpeach) The `cgroups/devices` isolator silently ignores devices
+      // that are whitelisted with no path. We should do the same if we want
+      // to be consistent, but silently ignoring configuration is not the
+      // right policy, so perhaps we should fix `cgroups/devices` instead.
+      if (!deviceAccess.device().has_path()) {
+        return Error("Whitelisted device has no device path provided");
+      }
+
+      const string& path = deviceAccess.device().path();
+
+      Try<dev_t> rdev = os::stat::rdev(path);
+      if (rdev.isError()) {
+        return Error("Failed to obtain device ID for '" + path +
+                     "': " + rdev.error());
+      }
+
+      Try<mode_t> mode = os::stat::mode(path);
+      if (mode.isError()) {
+        return Error("Failed to obtain device mode for '" + path +
+                     "': " + mode.error());
+      }
+
+      Device dev = {rdev.get(), S_IRUSR | S_IWUSR };
+
+      if (S_ISBLK(mode.get())) {
+        dev.mode |= S_IFBLK;
+      } else if (S_ISCHR(mode.get())) {
+        dev.mode |= S_IFCHR;
+      } else {
+        return Error("'" + path + "' is not a block or character device");
+      }
+
+      // Set the desired access for the device. Access is controlled at
+      // container granularity, which is consistent with the devices cgroup
+      // policy. This means that if we populate a read-write device into a
+      // container, then every process in that container should have access,
+      // regardless of the credential of that process.
+
+      if (deviceAccess.access().read()) {
+        dev.mode |= (S_IRGRP | S_IROTH);
+      }
+
+      if (deviceAccess.access().write()) {
+        dev.mode |= (S_IWGRP | S_IWOTH);
+      }
+
+      whitelistedDevices.put(
+          strings::remove(path, "/dev/", strings::PREFIX), dev);
+    }
+  }
+
+  return new MesosIsolator(Owned<MesosIsolatorProcess>(
+      new LinuxDevicesIsolatorProcess(flags.runtime_dir, whitelistedDevices)));
 }
 
+LinuxDevicesIsolatorProcess::LinuxDevicesIsolatorProcess(
+    const string& _runtimeDirectory,
+    const hashmap<string, Device>& _whitelistedDevices)
+  : ProcessBase(process::ID::generate("linux-devices-isolator")),
+    runtimeDirectory(_runtimeDirectory),
+    whitelistedDevices(_whitelistedDevices) {}
+
 
 bool LinuxDevicesIsolatorProcess::supportsNesting()
 {
@@ -56,7 +141,81 @@ Future<Option<ContainerLaunchInfo>> LinuxDevicesIsolatorProcess::prepare(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig)
 {
+  // If there's no rootfs, we won't be building a private `/dev`
+  // so there's nothing to do.
+  if (!containerConfig.has_rootfs()) {
+    return None();
+  }
+
+  if (whitelistedDevices.empty()) {
     return None();
+  }
+
+  ContainerLaunchInfo launchInfo;
+
+  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());
+    }
+  }
+
+  // Import the whitelisted devices to all containers.
+  foreachpair (const string& path, const Device& dev, whitelistedDevices) {
+    const string devicePath = path::join(devicesDir, path);
+
+    Try<Nothing> mkdir = os::mkdir(Path(devicePath).dirname());
+    if (mkdir.isError()) {
+      return Failure(
+          "Failed to create parent directory for device '" +
+          devicePath + "': " + mkdir.error());
+    }
+
+    Try<Nothing> mknod = os::mknod(devicePath, dev.mode, dev.dev);
+    if (mknod.isError()) {
+      return Failure(
+          "Failed to create device '" + devicePath + "': " + mknod.error());
+    }
+
+    // We have to chmod the device to make sure that the umask doesn't filter
+    // the permissions defined by the whitelist.
+    Try<Nothing> chmod = os::chmod(devicePath, dev.mode & ~S_IFMT);
+    if (chmod.isError()) {
+      return Failure(
+          "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);
+  }
+
+  // TODO(jpeach) Define Task API to let schedulers specify the container
+  // devices and automatically populate the right devices cgroup entries.
+
+  return launchInfo;
 }
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/ae413c9f/src/slave/containerizer/mesos/isolators/linux/devices.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/linux/devices.hpp b/src/slave/containerizer/mesos/isolators/linux/devices.hpp
index e731ef3..58128f0 100644
--- a/src/slave/containerizer/mesos/isolators/linux/devices.hpp
+++ b/src/slave/containerizer/mesos/isolators/linux/devices.hpp
@@ -17,6 +17,11 @@
 #ifndef __LINUX_DEVICES_ISOLATOR_HPP__
 #define __LINUX_DEVICES_ISOLATOR_HPP__
 
+#include <sys/types.h>
+
+#include <string>
+
+#include <stout/hashmap.hpp>
 #include <stout/try.hpp>
 
 #include "slave/flags.hpp"
@@ -32,15 +37,25 @@ class LinuxDevicesIsolatorProcess : public MesosIsolatorProcess
 public:
   static Try<mesos::slave::Isolator*> create(const Flags& flags);
 
-  virtual bool supportsNesting();
-  virtual bool supportsStandalone();
+  virtual bool supportsNesting() override;
+  virtual bool supportsStandalone() override;
 
   virtual process::Future<Option<mesos::slave::ContainerLaunchInfo>> prepare(
       const ContainerID& containerId,
-      const mesos::slave::ContainerConfig& containerConfig);
+      const mesos::slave::ContainerConfig& containerConfig) override;
 
 private:
-  LinuxDevicesIsolatorProcess(const Flags& _flags);
+  struct Device {
+    dev_t dev;
+    mode_t mode;
+  };
+
+  const std::string runtimeDirectory;
+  const hashmap<std::string, Device> whitelistedDevices;
+
+  LinuxDevicesIsolatorProcess(
+      const std::string& runtimeDirectory,
+      const hashmap<std::string, Device>& whitelistedDevices);
 };
 
 } // namespace slave {


Re: [6/6] mesos git commit: Added `linux/devices` isolator whitelist tests.

Posted by James Peach <ja...@me.com>.

> On May 28, 2018, at 6:05 AM, Alex R <al...@apache.org> wrote:
> 
> This commit breaks the build on Ubuntu 14.04 with `gcc (Ubuntu 4.8.4-2ubuntu1~14.04.4) 4.8.4` due to what seems to me a compiler bug, likely this one [1]. Ubuntu 14.04 is officially supported until mid-2019, hence not sure we can ignore this issue.
> 
> James, can you commit a workaround?

Oh it’s the raw strings that break it. I’ll try to come up with something....


> 
> [1] https://gcc.gnu.org/bugzilla/show_bug.cgi?id=57824
> 
>> On 25 May 2018 at 23:12, <jp...@apache.org> wrote:
>> Added `linux/devices` isolator whitelist tests.
>> 
>> Added test to verify that the `linux/devices` isolator supports
>> populating devices that are whitelisted by the `allowed_devices`
>> agent flag.
>> 
>> Review: https://reviews.apache.org/r/67145/
>> 
>> 
>> Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
>> Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/ee6c6cfc
>> Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/ee6c6cfc
>> Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/ee6c6cfc
>> 
>> Branch: refs/heads/master
>> Commit: ee6c6cfcbe2b91eaf540afa38bab4521d23b747f
>> Parents: 68db3f9
>> Author: James Peach <jp...@apache.org>
>> Authored: Fri May 25 13:38:14 2018 -0700
>> Committer: James Peach <jp...@apache.org>
>> Committed: Fri May 25 13:38:14 2018 -0700
>> 
>> ----------------------------------------------------------------------
>>  src/Makefile.am                                 |   1 +
>>  src/tests/CMakeLists.txt                        |   1 +
>>  .../linux_devices_isolator_tests.cpp            | 231 +++++++++++++++++++
>>  3 files changed, 233 insertions(+)
>> ----------------------------------------------------------------------
>> 
>> 
>> http://git-wip-us.apache.org/repos/asf/mesos/blob/ee6c6cfc/src/Makefile.am
>> ----------------------------------------------------------------------
>> diff --git a/src/Makefile.am b/src/Makefile.am
>> index da0d683..b7184ce 100644
>> --- a/src/Makefile.am
>> +++ b/src/Makefile.am
>> @@ -2666,6 +2666,7 @@ mesos_tests_SOURCES +=                                            \
>>    tests/containerizer/cgroups_tests.cpp                                \
>>    tests/containerizer/cni_isolator_tests.cpp                   \
>>    tests/containerizer/docker_volume_isolator_tests.cpp         \
>> +  tests/containerizer/linux_devices_isolator_tests.cpp         \
>>    tests/containerizer/linux_filesystem_isolator_tests.cpp      \
>>    tests/containerizer/fs_tests.cpp                             \
>>    tests/containerizer/memory_pressure_tests.cpp                        \
>> 
>> http://git-wip-us.apache.org/repos/asf/mesos/blob/ee6c6cfc/src/tests/CMakeLists.txt
>> ----------------------------------------------------------------------
>> diff --git a/src/tests/CMakeLists.txt b/src/tests/CMakeLists.txt
>> index 1fef060..b9c906d 100644
>> --- a/src/tests/CMakeLists.txt
>> +++ b/src/tests/CMakeLists.txt
>> @@ -224,6 +224,7 @@ if (LINUX)
>>      containerizer/docker_volume_isolator_tests.cpp
>>      containerizer/fs_tests.cpp
>>      containerizer/linux_capabilities_isolator_tests.cpp
>> +    containerizer/linux_devices_isolator_tests.cpp
>>      containerizer/linux_filesystem_isolator_tests.cpp
>>      containerizer/memory_pressure_tests.cpp
>>      containerizer/nested_mesos_containerizer_tests.cpp
>> 
>> http://git-wip-us.apache.org/repos/asf/mesos/blob/ee6c6cfc/src/tests/containerizer/linux_devices_isolator_tests.cpp
>> ----------------------------------------------------------------------
>> diff --git a/src/tests/containerizer/linux_devices_isolator_tests.cpp b/src/tests/containerizer/linux_devices_isolator_tests.cpp
>> new file mode 100644
>> index 0000000..efaa43b
>> --- /dev/null
>> +++ b/src/tests/containerizer/linux_devices_isolator_tests.cpp
>> @@ -0,0 +1,231 @@
>> +// Licensed to the Apache Software Foundation (ASF) under one
>> +// or more contributor license agreements.  See the NOTICE file
>> +// distributed with this work for additional information
>> +// regarding copyright ownership.  The ASF licenses this file
>> +// to you under the Apache License, Version 2.0 (the
>> +// "License"); you may not use this file except in compliance
>> +// with the License.  You may obtain a copy of the License at
>> +//
>> +//     http://www.apache.org/licenses/LICENSE-2.0
>> +//
>> +// Unless required by applicable law or agreed to in writing, software
>> +// distributed under the License is distributed on an "AS IS" BASIS,
>> +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
>> +// See the License for the specific language governing permissions and
>> +// limitations under the License.
>> +
>> +#include <map>
>> +#include <ostream>
>> +#include <string>
>> +
>> +#include <process/future.hpp>
>> +#include <process/gtest.hpp>
>> +#include <process/owned.hpp>
>> +
>> +#include <stout/gtest.hpp>
>> +#include <stout/option.hpp>
>> +#include <stout/path.hpp>
>> +#include <stout/strings.hpp>
>> +
>> +#include "common/parse.hpp"
>> +
>> +#include "tests/cluster.hpp"
>> +#include "tests/mesos.hpp"
>> +
>> +#include "slave/containerizer/mesos/containerizer.hpp"
>> +
>> +#include "tests/containerizer/docker_archive.hpp"
>> +
>> +using process::Future;
>> +using process::Owned;
>> +
>> +using std::map;
>> +using std::ostream;
>> +using std::string;
>> +
>> +using mesos::internal::slave::Containerizer;
>> +using mesos::internal::slave::Fetcher;
>> +using mesos::internal::slave::MesosContainerizer;
>> +
>> +using mesos::slave::ContainerTermination;
>> +
>> +namespace mesos {
>> +namespace internal {
>> +namespace tests {
>> +
>> +struct DevicesTestParam
>> +{
>> +  DevicesTestParam(
>> +      const string& _containerCheck,
>> +      const string& _allowedDevices)
>> +    : containerCheck(_containerCheck),
>> +      allowedDevices(_allowedDevices) {}
>> +
>> +  const string containerCheck;
>> +  const string allowedDevices;
>> +};
>> +
>> +
>> +ostream& operator<<(ostream& stream, const DevicesTestParam& param)
>> +{
>> +  return stream << param.containerCheck;
>> +}
>> +
>> +
>> +class LinuxDevicesIsolatorTest
>> +  : public MesosTest,
>> +    public ::testing::WithParamInterface<DevicesTestParam>
>> +{
>> +public:
>> +  LinuxDevicesIsolatorTest()
>> +    : param(GetParam()) {}
>> +
>> +  DevicesTestParam param;
>> +};
>> +
>> +
>> +TEST_P(LinuxDevicesIsolatorTest, ROOT_PopulateWhitelistedDevices)
>> +{
>> +  // Verify that all the necessary devices are present on the host.
>> +  // All reasonable Linux configuration should have these devices.
>> +  ASSERT_TRUE(os::exists("/dev/kmsg"));
>> +  ASSERT_TRUE(os::exists("/dev/loop-control"));
>> +
>> +  slave::Flags flags = CreateSlaveFlags();
>> +
>> +  flags.isolation = "linux/devices,filesystem/linux,docker/runtime";
>> +  flags.docker_registry =  path::join(sandbox.get(), "registry");
>> +  flags.docker_store_dir = path::join(sandbox.get(), "store");
>> +  flags.image_providers = "docker";
>> +
>> +  flags.allowed_devices =
>> +    flags::parse<DeviceWhitelist>(param.allowedDevices).get();
>> +
>> +  AWAIT_READY(DockerArchive::create(flags.docker_registry, "test_image"));
>> +
>> +  Fetcher fetcher(flags);
>> +
>> +  Try<MesosContainerizer*> create =
>> +    MesosContainerizer::create(flags, true, &fetcher);
>> +
>> +  ASSERT_SOME(create);
>> +
>> +  Owned<Containerizer> containerizer(create.get());
>> +
>> +  ContainerID containerId;
>> +  containerId.set_value(id::UUID::random().toString());
>> +
>> +  ExecutorInfo executor = createExecutorInfo(
>> +      "test_executor",
>> +      strings::join(";", "ls -l /dev", param.containerCheck));
>> +
>> +  executor.mutable_container()->CopyFrom(createContainerInfo("test_image"));
>> +
>> +  string directory = path::join(flags.work_dir, "sandbox");
>> +  ASSERT_SOME(os::mkdir(directory));
>> +
>> +  // Launch the container check command as the non-root user. All the
>> +  // check commands are testing for device file access, but root will
>> +  // always have access.
>> +  Future<Containerizer::LaunchResult> launch = containerizer->launch(
>> +      containerId,
>> +      createContainerConfig(
>> +          None(), executor, directory, os::getenv("SUDO_USER").get()),
>> +      map<string, string>(),
>> +      None());
>> +
>> +  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
>> +
>> +  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
>> +
>> +  AWAIT_READY(wait);
>> +  ASSERT_SOME(wait.get());
>> +  ASSERT_TRUE(wait->get().has_status());
>> +  EXPECT_WEXITSTATUS_EQ(0, wait->get().status());
>> +}
>> +
>> +
>> +INSTANTIATE_TEST_CASE_P(
>> +  DevicesTestParam,
>> +  LinuxDevicesIsolatorTest,
>> +  ::testing::Values(
>> +    // Test that /dev/loop-control is a character device and that
>> +    // /dev/kmsg doesn't exist. The latter test ensures that we
>> +    // won't succeed by accidentally running on the host.
>> +    DevicesTestParam(
>> +      "test -c /dev/loop-control && test ! -e /dev/kmsg",
>> +      R"~({
>> +        "allowed_devices": [
>> +          {
>> +            "device": {
>> +              "path": "/dev/loop-control"
>> +            },
>> +            "access": {}
>> +          }
>> +        ]
>> +      })~"),
>> +    // Test that a device in a subdirectory is populated.
>> +    DevicesTestParam(
>> +      "test -r /dev/cpu/0/cpuid",
>> +      R"~({
>> +        "allowed_devices": [
>> +          {
>> +            "device": {
>> +              "path": "/dev/cpu/0/cpuid"
>> +            },
>> +            "access": {
>> +              "read": true
>> +            }
>> +          }
>> +        ]
>> +      })~"),
>> +    // Test that read-only devices are populated in read-only mode.
>> +    DevicesTestParam(
>> +      "test -r /dev/loop-control && test ! -w /dev/loop-control",
>> +      R"~({
>> +        "allowed_devices": [
>> +          {
>> +            "device": {
>> +              "path": "/dev/loop-control"
>> +            },
>> +            "access": {
>> +              "read": true
>> +            }
>> +          }
>> +        ]
>> +      })~"),
>> +    // Test that write-only devices are populated in write-only mode.
>> +    DevicesTestParam(
>> +      "test -w /dev/loop-control && test ! -r /dev/loop-control",
>> +      R"~({
>> +        "allowed_devices": [
>> +          {
>> +            "device": {
>> +              "path": "/dev/loop-control"
>> +            },
>> +            "access": {
>> +              "write": true
>> +            }
>> +          }
>> +        ]
>> +      })~"),
>> +    // Test that read-write devices are populated in read-write mode.
>> +    DevicesTestParam(
>> +      "test -w /dev/loop-control && test -r /dev/loop-control",
>> +      R"~({
>> +        "allowed_devices": [
>> +          {
>> +            "device": {
>> +              "path": "/dev/loop-control"
>> +            },
>> +            "access": {
>> +              "read": true,
>> +              "write": true
>> +            }
>> +          }
>> +        ]
>> +      })~")));
>> +
>> +} // namespace tests {
>> +} // namespace internal {
>> +} // namespace mesos {
>> 
> 

Re: [6/6] mesos git commit: Added `linux/devices` isolator whitelist tests.

Posted by Alex R <al...@apache.org>.
This commit breaks the build on Ubuntu 14.04 with `gcc (Ubuntu
4.8.4-2ubuntu1~14.04.4) 4.8.4` due to what seems to me a compiler bug,
likely this one [1]. Ubuntu 14.04 is officially supported until mid-2019,
hence not sure we can ignore this issue.

James, can you commit a workaround?

[1] https://gcc.gnu.org/bugzilla/show_bug.cgi?id=57824

On 25 May 2018 at 23:12, <jp...@apache.org> wrote:

> Added `linux/devices` isolator whitelist tests.
>
> Added test to verify that the `linux/devices` isolator supports
> populating devices that are whitelisted by the `allowed_devices`
> agent flag.
>
> Review: https://reviews.apache.org/r/67145/
>
>
> Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
> Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/ee6c6cfc
> Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/ee6c6cfc
> Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/ee6c6cfc
>
> Branch: refs/heads/master
> Commit: ee6c6cfcbe2b91eaf540afa38bab4521d23b747f
> Parents: 68db3f9
> Author: James Peach <jp...@apache.org>
> Authored: Fri May 25 13:38:14 2018 -0700
> Committer: James Peach <jp...@apache.org>
> Committed: Fri May 25 13:38:14 2018 -0700
>
> ----------------------------------------------------------------------
>  src/Makefile.am                                 |   1 +
>  src/tests/CMakeLists.txt                        |   1 +
>  .../linux_devices_isolator_tests.cpp            | 231 +++++++++++++++++++
>  3 files changed, 233 insertions(+)
> ----------------------------------------------------------------------
>
>
> http://git-wip-us.apache.org/repos/asf/mesos/blob/ee6c6cfc/src/Makefile.am
> ----------------------------------------------------------------------
> diff --git a/src/Makefile.am b/src/Makefile.am
> index da0d683..b7184ce 100644
> --- a/src/Makefile.am
> +++ b/src/Makefile.am
> @@ -2666,6 +2666,7 @@ mesos_tests_SOURCES +=
>               \
>    tests/containerizer/cgroups_tests.cpp                                \
>    tests/containerizer/cni_isolator_tests.cpp                   \
>    tests/containerizer/docker_volume_isolator_tests.cpp         \
> +  tests/containerizer/linux_devices_isolator_tests.cpp         \
>    tests/containerizer/linux_filesystem_isolator_tests.cpp      \
>    tests/containerizer/fs_tests.cpp                             \
>    tests/containerizer/memory_pressure_tests.cpp                        \
>
> http://git-wip-us.apache.org/repos/asf/mesos/blob/ee6c6cfc/
> src/tests/CMakeLists.txt
> ----------------------------------------------------------------------
> diff --git a/src/tests/CMakeLists.txt b/src/tests/CMakeLists.txt
> index 1fef060..b9c906d 100644
> --- a/src/tests/CMakeLists.txt
> +++ b/src/tests/CMakeLists.txt
> @@ -224,6 +224,7 @@ if (LINUX)
>      containerizer/docker_volume_isolator_tests.cpp
>      containerizer/fs_tests.cpp
>      containerizer/linux_capabilities_isolator_tests.cpp
> +    containerizer/linux_devices_isolator_tests.cpp
>      containerizer/linux_filesystem_isolator_tests.cpp
>      containerizer/memory_pressure_tests.cpp
>      containerizer/nested_mesos_containerizer_tests.cpp
>
> http://git-wip-us.apache.org/repos/asf/mesos/blob/ee6c6cfc/
> src/tests/containerizer/linux_devices_isolator_tests.cpp
> ----------------------------------------------------------------------
> diff --git a/src/tests/containerizer/linux_devices_isolator_tests.cpp
> b/src/tests/containerizer/linux_devices_isolator_tests.cpp
> new file mode 100644
> index 0000000..efaa43b
> --- /dev/null
> +++ b/src/tests/containerizer/linux_devices_isolator_tests.cpp
> @@ -0,0 +1,231 @@
> +// Licensed to the Apache Software Foundation (ASF) under one
> +// or more contributor license agreements.  See the NOTICE file
> +// distributed with this work for additional information
> +// regarding copyright ownership.  The ASF licenses this file
> +// to you under the Apache License, Version 2.0 (the
> +// "License"); you may not use this file except in compliance
> +// with the License.  You may obtain a copy of the License at
> +//
> +//     http://www.apache.org/licenses/LICENSE-2.0
> +//
> +// Unless required by applicable law or agreed to in writing, software
> +// distributed under the License is distributed on an "AS IS" BASIS,
> +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> +// See the License for the specific language governing permissions and
> +// limitations under the License.
> +
> +#include <map>
> +#include <ostream>
> +#include <string>
> +
> +#include <process/future.hpp>
> +#include <process/gtest.hpp>
> +#include <process/owned.hpp>
> +
> +#include <stout/gtest.hpp>
> +#include <stout/option.hpp>
> +#include <stout/path.hpp>
> +#include <stout/strings.hpp>
> +
> +#include "common/parse.hpp"
> +
> +#include "tests/cluster.hpp"
> +#include "tests/mesos.hpp"
> +
> +#include "slave/containerizer/mesos/containerizer.hpp"
> +
> +#include "tests/containerizer/docker_archive.hpp"
> +
> +using process::Future;
> +using process::Owned;
> +
> +using std::map;
> +using std::ostream;
> +using std::string;
> +
> +using mesos::internal::slave::Containerizer;
> +using mesos::internal::slave::Fetcher;
> +using mesos::internal::slave::MesosContainerizer;
> +
> +using mesos::slave::ContainerTermination;
> +
> +namespace mesos {
> +namespace internal {
> +namespace tests {
> +
> +struct DevicesTestParam
> +{
> +  DevicesTestParam(
> +      const string& _containerCheck,
> +      const string& _allowedDevices)
> +    : containerCheck(_containerCheck),
> +      allowedDevices(_allowedDevices) {}
> +
> +  const string containerCheck;
> +  const string allowedDevices;
> +};
> +
> +
> +ostream& operator<<(ostream& stream, const DevicesTestParam& param)
> +{
> +  return stream << param.containerCheck;
> +}
> +
> +
> +class LinuxDevicesIsolatorTest
> +  : public MesosTest,
> +    public ::testing::WithParamInterface<DevicesTestParam>
> +{
> +public:
> +  LinuxDevicesIsolatorTest()
> +    : param(GetParam()) {}
> +
> +  DevicesTestParam param;
> +};
> +
> +
> +TEST_P(LinuxDevicesIsolatorTest, ROOT_PopulateWhitelistedDevices)
> +{
> +  // Verify that all the necessary devices are present on the host.
> +  // All reasonable Linux configuration should have these devices.
> +  ASSERT_TRUE(os::exists("/dev/kmsg"));
> +  ASSERT_TRUE(os::exists("/dev/loop-control"));
> +
> +  slave::Flags flags = CreateSlaveFlags();
> +
> +  flags.isolation = "linux/devices,filesystem/linux,docker/runtime";
> +  flags.docker_registry =  path::join(sandbox.get(), "registry");
> +  flags.docker_store_dir = path::join(sandbox.get(), "store");
> +  flags.image_providers = "docker";
> +
> +  flags.allowed_devices =
> +    flags::parse<DeviceWhitelist>(param.allowedDevices).get();
> +
> +  AWAIT_READY(DockerArchive::create(flags.docker_registry,
> "test_image"));
> +
> +  Fetcher fetcher(flags);
> +
> +  Try<MesosContainerizer*> create =
> +    MesosContainerizer::create(flags, true, &fetcher);
> +
> +  ASSERT_SOME(create);
> +
> +  Owned<Containerizer> containerizer(create.get());
> +
> +  ContainerID containerId;
> +  containerId.set_value(id::UUID::random().toString());
> +
> +  ExecutorInfo executor = createExecutorInfo(
> +      "test_executor",
> +      strings::join(";", "ls -l /dev", param.containerCheck));
> +
> +  executor.mutable_container()->CopyFrom(createContainerInfo("
> test_image"));
> +
> +  string directory = path::join(flags.work_dir, "sandbox");
> +  ASSERT_SOME(os::mkdir(directory));
> +
> +  // Launch the container check command as the non-root user. All the
> +  // check commands are testing for device file access, but root will
> +  // always have access.
> +  Future<Containerizer::LaunchResult> launch = containerizer->launch(
> +      containerId,
> +      createContainerConfig(
> +          None(), executor, directory, os::getenv("SUDO_USER").get()),
> +      map<string, string>(),
> +      None());
> +
> +  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
> +
> +  Future<Option<ContainerTermination>> wait = containerizer->wait(
> containerId);
> +
> +  AWAIT_READY(wait);
> +  ASSERT_SOME(wait.get());
> +  ASSERT_TRUE(wait->get().has_status());
> +  EXPECT_WEXITSTATUS_EQ(0, wait->get().status());
> +}
> +
> +
> +INSTANTIATE_TEST_CASE_P(
> +  DevicesTestParam,
> +  LinuxDevicesIsolatorTest,
> +  ::testing::Values(
> +    // Test that /dev/loop-control is a character device and that
> +    // /dev/kmsg doesn't exist. The latter test ensures that we
> +    // won't succeed by accidentally running on the host.
> +    DevicesTestParam(
> +      "test -c /dev/loop-control && test ! -e /dev/kmsg",
> +      R"~({
> +        "allowed_devices": [
> +          {
> +            "device": {
> +              "path": "/dev/loop-control"
> +            },
> +            "access": {}
> +          }
> +        ]
> +      })~"),
> +    // Test that a device in a subdirectory is populated.
> +    DevicesTestParam(
> +      "test -r /dev/cpu/0/cpuid",
> +      R"~({
> +        "allowed_devices": [
> +          {
> +            "device": {
> +              "path": "/dev/cpu/0/cpuid"
> +            },
> +            "access": {
> +              "read": true
> +            }
> +          }
> +        ]
> +      })~"),
> +    // Test that read-only devices are populated in read-only mode.
> +    DevicesTestParam(
> +      "test -r /dev/loop-control && test ! -w /dev/loop-control",
> +      R"~({
> +        "allowed_devices": [
> +          {
> +            "device": {
> +              "path": "/dev/loop-control"
> +            },
> +            "access": {
> +              "read": true
> +            }
> +          }
> +        ]
> +      })~"),
> +    // Test that write-only devices are populated in write-only mode.
> +    DevicesTestParam(
> +      "test -w /dev/loop-control && test ! -r /dev/loop-control",
> +      R"~({
> +        "allowed_devices": [
> +          {
> +            "device": {
> +              "path": "/dev/loop-control"
> +            },
> +            "access": {
> +              "write": true
> +            }
> +          }
> +        ]
> +      })~"),
> +    // Test that read-write devices are populated in read-write mode.
> +    DevicesTestParam(
> +      "test -w /dev/loop-control && test -r /dev/loop-control",
> +      R"~({
> +        "allowed_devices": [
> +          {
> +            "device": {
> +              "path": "/dev/loop-control"
> +            },
> +            "access": {
> +              "read": true,
> +              "write": true
> +            }
> +          }
> +        ]
> +      })~")));
> +
> +} // namespace tests {
> +} // namespace internal {
> +} // namespace mesos {
>
>

[6/6] mesos git commit: Added `linux/devices` isolator whitelist tests.

Posted by jp...@apache.org.
Added `linux/devices` isolator whitelist tests.

Added test to verify that the `linux/devices` isolator supports
populating devices that are whitelisted by the `allowed_devices`
agent flag.

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


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

Branch: refs/heads/master
Commit: ee6c6cfcbe2b91eaf540afa38bab4521d23b747f
Parents: 68db3f9
Author: James Peach <jp...@apache.org>
Authored: Fri May 25 13:38:14 2018 -0700
Committer: James Peach <jp...@apache.org>
Committed: Fri May 25 13:38:14 2018 -0700

----------------------------------------------------------------------
 src/Makefile.am                                 |   1 +
 src/tests/CMakeLists.txt                        |   1 +
 .../linux_devices_isolator_tests.cpp            | 231 +++++++++++++++++++
 3 files changed, 233 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ee6c6cfc/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index da0d683..b7184ce 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -2666,6 +2666,7 @@ mesos_tests_SOURCES +=						\
   tests/containerizer/cgroups_tests.cpp				\
   tests/containerizer/cni_isolator_tests.cpp			\
   tests/containerizer/docker_volume_isolator_tests.cpp		\
+  tests/containerizer/linux_devices_isolator_tests.cpp		\
   tests/containerizer/linux_filesystem_isolator_tests.cpp	\
   tests/containerizer/fs_tests.cpp				\
   tests/containerizer/memory_pressure_tests.cpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/ee6c6cfc/src/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/tests/CMakeLists.txt b/src/tests/CMakeLists.txt
index 1fef060..b9c906d 100644
--- a/src/tests/CMakeLists.txt
+++ b/src/tests/CMakeLists.txt
@@ -224,6 +224,7 @@ if (LINUX)
     containerizer/docker_volume_isolator_tests.cpp
     containerizer/fs_tests.cpp
     containerizer/linux_capabilities_isolator_tests.cpp
+    containerizer/linux_devices_isolator_tests.cpp
     containerizer/linux_filesystem_isolator_tests.cpp
     containerizer/memory_pressure_tests.cpp
     containerizer/nested_mesos_containerizer_tests.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/ee6c6cfc/src/tests/containerizer/linux_devices_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/linux_devices_isolator_tests.cpp b/src/tests/containerizer/linux_devices_isolator_tests.cpp
new file mode 100644
index 0000000..efaa43b
--- /dev/null
+++ b/src/tests/containerizer/linux_devices_isolator_tests.cpp
@@ -0,0 +1,231 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include <map>
+#include <ostream>
+#include <string>
+
+#include <process/future.hpp>
+#include <process/gtest.hpp>
+#include <process/owned.hpp>
+
+#include <stout/gtest.hpp>
+#include <stout/option.hpp>
+#include <stout/path.hpp>
+#include <stout/strings.hpp>
+
+#include "common/parse.hpp"
+
+#include "tests/cluster.hpp"
+#include "tests/mesos.hpp"
+
+#include "slave/containerizer/mesos/containerizer.hpp"
+
+#include "tests/containerizer/docker_archive.hpp"
+
+using process::Future;
+using process::Owned;
+
+using std::map;
+using std::ostream;
+using std::string;
+
+using mesos::internal::slave::Containerizer;
+using mesos::internal::slave::Fetcher;
+using mesos::internal::slave::MesosContainerizer;
+
+using mesos::slave::ContainerTermination;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+struct DevicesTestParam
+{
+  DevicesTestParam(
+      const string& _containerCheck,
+      const string& _allowedDevices)
+    : containerCheck(_containerCheck),
+      allowedDevices(_allowedDevices) {}
+
+  const string containerCheck;
+  const string allowedDevices;
+};
+
+
+ostream& operator<<(ostream& stream, const DevicesTestParam& param)
+{
+  return stream << param.containerCheck;
+}
+
+
+class LinuxDevicesIsolatorTest
+  : public MesosTest,
+    public ::testing::WithParamInterface<DevicesTestParam>
+{
+public:
+  LinuxDevicesIsolatorTest()
+    : param(GetParam()) {}
+
+  DevicesTestParam param;
+};
+
+
+TEST_P(LinuxDevicesIsolatorTest, ROOT_PopulateWhitelistedDevices)
+{
+  // Verify that all the necessary devices are present on the host.
+  // All reasonable Linux configuration should have these devices.
+  ASSERT_TRUE(os::exists("/dev/kmsg"));
+  ASSERT_TRUE(os::exists("/dev/loop-control"));
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  flags.isolation = "linux/devices,filesystem/linux,docker/runtime";
+  flags.docker_registry =  path::join(sandbox.get(), "registry");
+  flags.docker_store_dir = path::join(sandbox.get(), "store");
+  flags.image_providers = "docker";
+
+  flags.allowed_devices =
+    flags::parse<DeviceWhitelist>(param.allowedDevices).get();
+
+  AWAIT_READY(DockerArchive::create(flags.docker_registry, "test_image"));
+
+  Fetcher fetcher(flags);
+
+  Try<MesosContainerizer*> create =
+    MesosContainerizer::create(flags, true, &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<Containerizer> containerizer(create.get());
+
+  ContainerID containerId;
+  containerId.set_value(id::UUID::random().toString());
+
+  ExecutorInfo executor = createExecutorInfo(
+      "test_executor",
+      strings::join(";", "ls -l /dev", param.containerCheck));
+
+  executor.mutable_container()->CopyFrom(createContainerInfo("test_image"));
+
+  string directory = path::join(flags.work_dir, "sandbox");
+  ASSERT_SOME(os::mkdir(directory));
+
+  // Launch the container check command as the non-root user. All the
+  // check commands are testing for device file access, but root will
+  // always have access.
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
+      containerId,
+      createContainerConfig(
+          None(), executor, directory, os::getenv("SUDO_USER").get()),
+      map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait->get().has_status());
+  EXPECT_WEXITSTATUS_EQ(0, wait->get().status());
+}
+
+
+INSTANTIATE_TEST_CASE_P(
+  DevicesTestParam,
+  LinuxDevicesIsolatorTest,
+  ::testing::Values(
+    // Test that /dev/loop-control is a character device and that
+    // /dev/kmsg doesn't exist. The latter test ensures that we
+    // won't succeed by accidentally running on the host.
+    DevicesTestParam(
+      "test -c /dev/loop-control && test ! -e /dev/kmsg",
+      R"~({
+        "allowed_devices": [
+          {
+            "device": {
+              "path": "/dev/loop-control"
+            },
+            "access": {}
+          }
+        ]
+      })~"),
+    // Test that a device in a subdirectory is populated.
+    DevicesTestParam(
+      "test -r /dev/cpu/0/cpuid",
+      R"~({
+        "allowed_devices": [
+          {
+            "device": {
+              "path": "/dev/cpu/0/cpuid"
+            },
+            "access": {
+              "read": true
+            }
+          }
+        ]
+      })~"),
+    // Test that read-only devices are populated in read-only mode.
+    DevicesTestParam(
+      "test -r /dev/loop-control && test ! -w /dev/loop-control",
+      R"~({
+        "allowed_devices": [
+          {
+            "device": {
+              "path": "/dev/loop-control"
+            },
+            "access": {
+              "read": true
+            }
+          }
+        ]
+      })~"),
+    // Test that write-only devices are populated in write-only mode.
+    DevicesTestParam(
+      "test -w /dev/loop-control && test ! -r /dev/loop-control",
+      R"~({
+        "allowed_devices": [
+          {
+            "device": {
+              "path": "/dev/loop-control"
+            },
+            "access": {
+              "write": true
+            }
+          }
+        ]
+      })~"),
+    // Test that read-write devices are populated in read-write mode.
+    DevicesTestParam(
+      "test -w /dev/loop-control && test -r /dev/loop-control",
+      R"~({
+        "allowed_devices": [
+          {
+            "device": {
+              "path": "/dev/loop-control"
+            },
+            "access": {
+              "read": true,
+              "write": true
+            }
+          }
+        ]
+      })~")));
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {