You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2017/11/29 03:35:14 UTC

[7/9] mesos git commit: Added mount propagation support for HOST_PATH volumes.

Added mount propagation support for HOST_PATH volumes.

This allows a container to declare a host path volume such that the
mounts under that volume can be propagated in both directions.
Previously, all volume mounts are default to use host to container
propagation mode.

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


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

Branch: refs/heads/master
Commit: d6d40ecadf13af74f6fc2bfe702fabbf6f70106b
Parents: 38ccb8c
Author: Jie Yu <yu...@gmail.com>
Authored: Sun Oct 22 21:39:38 2017 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Nov 28 18:45:43 2017 -0800

----------------------------------------------------------------------
 include/mesos/mesos.proto                       |  26 ++++
 include/mesos/v1/mesos.proto                    |  26 ++++
 .../mesos/isolators/volume/host_path.cpp        |  76 ++++++++++--
 src/slave/containerizer/mesos/launch.cpp        | 120 ++++++++++++++++++-
 .../volume_host_path_isolator_tests.cpp         |  63 ++++++++++
 src/tests/mesos.hpp                             |  10 +-
 6 files changed, 308 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d6d40eca/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index e194093..8c28b81 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -2659,6 +2659,31 @@ message Image {
 
 
 /**
+ * Describes how the mount will be propagated for a volume. See the
+ * following doc for more details about mount propagation:
+ * https://www.kernel.org/doc/Documentation/filesystems/sharedsubtree.txt
+ */
+message MountPropagation {
+  enum Mode {
+    UNKNOWN = 0;
+
+    // The volume in a container will receive new mounts from the host
+    // or other containers, but filesystems mounted inside the
+    // container won't be propagated to the host or other containers.
+    // This is currently the default behavior for all volumes.
+    HOST_TO_CONTAINER = 1;
+
+    // The volume in a container will receive new mounts from the host
+    // or other containers, and its own mounts will be propagated from
+    // the container to the host or other containers.
+    BIDIRECTIONAL = 2;
+  }
+
+  optional Mode mode = 1;
+}
+
+
+/**
  * Describes a volume mapping either from host to container or vice
  * versa. Both paths can either refer to a directory or a file.
  */
@@ -2718,6 +2743,7 @@ message Volume {
     // Absolute path pointing to a directory or file on the host.
     message HostPath {
       required string path = 1;
+      optional MountPropagation mount_propagation = 2;
     }
 
     // Describe a path from a container's sandbox. The container can

http://git-wip-us.apache.org/repos/asf/mesos/blob/d6d40eca/include/mesos/v1/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/mesos.proto b/include/mesos/v1/mesos.proto
index 6fb1139..35bc6e1 100644
--- a/include/mesos/v1/mesos.proto
+++ b/include/mesos/v1/mesos.proto
@@ -2640,6 +2640,31 @@ message Image {
 
 
 /**
+ * Describes how the mount will be propagated for a volume. See the
+ * following doc for more details about mount propagation:
+ * https://www.kernel.org/doc/Documentation/filesystems/sharedsubtree.txt
+ */
+message MountPropagation {
+  enum Mode {
+    UNKNOWN = 0;
+
+    // The volume in a container will receive new mounts from the host
+    // or other containers, but filesystems mounted inside the
+    // container won't be propagated to the host or other containers.
+    // This is currently the default behavior for all volumes.
+    HOST_TO_CONTAINER = 1;
+
+    // The volume in a container will receive new mounts from the host
+    // or other containers, and its own mounts will be propagated from
+    // the container to the host or other containers.
+    BIDIRECTIONAL = 2;
+  }
+
+  optional Mode mode = 1;
+}
+
+
+/**
  * Describes a volume mapping either from host to container or vice
  * versa. Both paths can either refer to a directory or a file.
  */
@@ -2699,6 +2724,7 @@ message Volume {
     // Absolute path pointing to a directory or file on the host.
     message HostPath {
       required string path = 1;
+      optional MountPropagation mount_propagation = 2;
     }
 
     // Describe a path from a container's sandbox. The container can

http://git-wip-us.apache.org/repos/asf/mesos/blob/d6d40eca/src/slave/containerizer/mesos/isolators/volume/host_path.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/volume/host_path.cpp b/src/slave/containerizer/mesos/isolators/volume/host_path.cpp
index 241a7f6..409fc14 100644
--- a/src/slave/containerizer/mesos/isolators/volume/host_path.cpp
+++ b/src/slave/containerizer/mesos/isolators/volume/host_path.cpp
@@ -22,6 +22,7 @@
 
 #include <stout/foreach.hpp>
 #include <stout/option.hpp>
+#include <stout/os.hpp>
 #include <stout/path.hpp>
 #include <stout/strings.hpp>
 
@@ -32,6 +33,8 @@
 
 #include "common/validation.hpp"
 
+#include "linux/fs.hpp"
+
 #include "slave/containerizer/mesos/isolators/volume/host_path.hpp"
 
 using std::string;
@@ -40,6 +43,8 @@ using process::Failure;
 using process::Future;
 using process::Owned;
 
+using mesos::MountPropagation;
+
 using mesos::slave::ContainerClass;
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
@@ -109,6 +114,7 @@ Future<Option<ContainerLaunchInfo>> VolumeHostPathIsolatorProcess::prepare(
     }
 
     Option<string> hostPath;
+    bool mountPropagationBidirectional = false;
 
     // NOTE: This is the legacy way of specifying the Volume. The
     // 'host_path' can be relative in legacy mode, representing
@@ -123,13 +129,21 @@ Future<Option<ContainerLaunchInfo>> VolumeHostPathIsolatorProcess::prepare(
         volume.source().type() == Volume::Source::HOST_PATH) {
       CHECK(volume.source().has_host_path());
 
-      if (!path::absolute(volume.source().host_path().path())) {
+      const Volume::Source::HostPath& hostPathInfo =
+        volume.source().host_path();
+
+      if (!path::absolute(hostPathInfo.path())) {
         return Failure(
-            "Path '" + volume.source().host_path().path() + "' "
+            "Path '" + hostPathInfo.path() + "' "
             "in HOST_PATH volume is not absolute");
       }
 
-      hostPath = volume.source().host_path().path();
+      hostPath = hostPathInfo.path();
+
+      mountPropagationBidirectional =
+        hostPathInfo.has_mount_propagation() &&
+        hostPathInfo.mount_propagation().mode() ==
+          MountPropagation::BIDIRECTIONAL;
     }
 
     if (hostPath.isNone()) {
@@ -243,11 +257,57 @@ Future<Option<ContainerLaunchInfo>> VolumeHostPathIsolatorProcess::prepare(
       }
     }
 
-    // TODO(jieyu): Consider the mode in the volume.
-    ContainerMountInfo* mount = launchInfo.add_mounts();
-    mount->set_source(hostPath.get());
-    mount->set_target(mountPoint);
-    mount->set_flags(MS_BIND | MS_REC);
+    if (mountPropagationBidirectional) {
+      // First, find the mount entry that is the parent of the host
+      // volume source. If it is not a shared mount, return a failure.
+
+      // Get realpath here because the mount table uses realpaths.
+      Result<string> realHostPath = os::realpath(hostPath.get());
+      if (!realHostPath.isSome()) {
+        return Failure(
+            "Failed to get the realpath of the host path '" +
+            hostPath.get() + "': " +
+            (realHostPath.isError() ? realHostPath.error() : "Not found"));
+      }
+
+      Try<fs::MountInfoTable::Entry> sourceMountEntry =
+        fs::MountInfoTable::findByTarget(realHostPath.get());
+
+      if (sourceMountEntry.isError()) {
+        return Failure(
+            "Cannot find the mount containing host path '" +
+            hostPath.get() + "': " + sourceMountEntry.error());
+      }
+
+      if (sourceMountEntry->shared().isNone()) {
+        return Failure(
+            "Cannot setup bidirectional mount propagation for host path '" +
+            hostPath.get() + "' because it is not under a shared mount");
+      }
+
+      LOG(INFO) << "Mark '" << sourceMountEntry->target
+                << "' as shared for container " << containerId;
+
+      // This tells the launch helper to NOT mark the mount as slave
+      // (otherwise, the propagation won't work).
+      ContainerMountInfo* mount = launchInfo.add_mounts();
+      mount->set_target(sourceMountEntry->target);
+      mount->set_flags(MS_SHARED);
+    }
+
+    // NOTE: 'hostPath' and 'mountPoint' are equal only when the
+    // container does not define its own image and shares the host
+    // filesystem (otherwise, the mount point should be under
+    // container's rootfs, which won't be equal to 'hostPath'). As a
+    // result, no need for the bind mount because the 'hostPath' is
+    // already accessible in the container.
+    if (hostPath.get() != mountPoint) {
+      // TODO(jieyu): Consider the mode in the volume.
+      ContainerMountInfo* mount = launchInfo.add_mounts();
+      mount->set_source(hostPath.get());
+      mount->set_target(mountPoint);
+      mount->set_flags(MS_BIND | MS_REC);
+    }
   }
 
   return launchInfo;

http://git-wip-us.apache.org/repos/asf/mesos/blob/d6d40eca/src/slave/containerizer/mesos/launch.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index 0c4bc33..4fabf92 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -31,6 +31,7 @@
 
 #include <process/subprocess.hpp>
 
+#include <stout/adaptor.hpp>
 #include <stout/foreach.hpp>
 #include <stout/os.hpp>
 #include <stout/protobuf.hpp>
@@ -249,13 +250,33 @@ static Try<Nothing> prepareMounts(const ContainerLaunchInfo& launchInfo)
   if (!cloneMountNamespace) {
     // Mounts are not supported if the mount namespace is not cloned.
     // Otherwise, we'll pollute the parent mount namespace.
+    if (!launchInfo.mounts().empty()) {
+      return Error(
+          "Mounts are not supported if the mount namespace is not cloned");
+    }
+
     return Nothing();
   }
 
-  // If there is no shared mount (i.e., "bidirectional" propagation),
-  // mark the root as recursively slave propagation (i.e.,
-  // --make-rslave) so that mounts do not leak to parent mount
-  // namespace.
+  // Now, setup the mount propagation for the container.
+  //   1) If there is no shared mount (i.e., "bidirectional"
+  //      propagation), mark the root as recursively slave propagation
+  //      (i.e., --make-rslave) so that mounts do not leak to parent
+  //      mount namespace.
+  //   2) If there exist shared mounts, scan the mount table and mark
+  //      the rest as shared mounts one by one.
+  //
+  // TODO(jieyu): Currently, if the container has its own rootfs, the
+  // 'fs::chroot::enter' function will mark `/` as recursively slave.
+  // This will cause problems for shared mounts. As a result,
+  // bidirectional mount propagation does not work for containers that
+  // have rootfses.
+  //
+  // TODO(jieyu): Another caveat right now is that the CNI isolator
+  // will mark `/` as recursively slave in `isolate()` method if the
+  // container joins a named network. As a result, bidirectional mount
+  // propagation currently does not work for containers that want to
+  // join a CNI network.
   bool hasSharedMount = std::find_if(
       launchInfo.mounts().begin(),
       launchInfo.mounts().end(),
@@ -272,9 +293,100 @@ static Try<Nothing> prepareMounts(const ContainerLaunchInfo& launchInfo)
     }
 
     cout << "Marked '/' as rslave" << endl;
+  } else {
+    hashset<string> sharedMountTargets;
+    foreach (const ContainerMountInfo& mount, launchInfo.mounts()) {
+      // Skip normal mounts.
+      if ((mount.flags() & MS_SHARED) == 0) {
+        continue;
+      }
+
+      sharedMountTargets.insert(mount.target());
+    }
+
+    Try<fs::MountInfoTable> table = fs::MountInfoTable::read();
+    if (table.isError()) {
+      return Error("Failed to get mount table: " + table.error());
+    }
+
+    foreach (const fs::MountInfoTable::Entry& entry,
+             adaptor::reverse(table->entries)) {
+      if (!sharedMountTargets.contains(entry.target)) {
+        Try<Nothing> mnt = fs::mount(
+            None(),
+            entry.target,
+            None(),
+            MS_SLAVE,
+            None());
+
+        if (mnt.isError()) {
+          return Error(
+              "Failed to mark '" + entry.target +
+              "' as slave: " + mnt.error());
+        }
+      }
+    }
   }
 
   foreach (const ContainerMountInfo& mount, launchInfo.mounts()) {
+    // Skip those mounts that are used for setting up propagation.
+    if ((mount.flags() & MS_SHARED) != 0) {
+      continue;
+    }
+
+    // If bidirectional mount exists, we will not mark `/` as
+    // recursively slave (otherwise, the bidirectional mount
+    // propagation won't work).
+    //
+    // At the same time, we want to prevent mounts in the child
+    // process from being propagated to the host mount namespace,
+    // except for the ones that set the propagation mode to be
+    // bidirectional. This ensures a clean host mount table, and
+    // greatly simplifies the container cleanup.
+    //
+    // If the target of a volume mount is under a non shared mount,
+    // the mount won't be propagated to the host mount namespace,
+    // which is what we want. Otherwise, the volume mount will be
+    // propagated to the host mount namespace, which will make proper
+    // cleanup almost impossible. Therefore, we perform a sanity check
+    // here to make sure the propagation to host mount namespace does
+    // not happen.
+    //
+    // One implication of this check is that: if the target of a
+    // volume mount is under a mount that has to be shared (e.g.,
+    // explicitly specified by the user using 'MountPropagation' in
+    // HOST_PATH volume), the volume mount will fail.
+    //
+    // TODO(jieyu): Some isolators are still using `pre_exe_commands`
+    // to do mounts. Those isolators thus will escape this check. We
+    // should consider forcing all isolators to use
+    // `ContainerMountInfo` for volume mounts.
+    if (hasSharedMount) {
+      Result<string> realTargetPath = os::realpath(mount.target());
+      if (!realTargetPath.isSome()) {
+        return Error(
+            "Failed to get the realpath of the mount target '" +
+            mount.target() + "': " +
+            (realTargetPath.isError() ? realTargetPath.error() : "Not found"));
+      }
+
+      Try<fs::MountInfoTable::Entry> entry =
+        fs::MountInfoTable::findByTarget(realTargetPath.get());
+
+      if (entry.isError()) {
+        return Error(
+            "Cannot find the mount containing the mount target '" +
+            mount.target() + "': " + entry.error());
+      }
+
+      if (entry->shared().isSome()) {
+        return Error(
+            "Cannot perform mount '" + stringify(JSON::protobuf(mount)) +
+            "' because the target is under a shared mount "
+            "'" + entry->target + "'");
+      }
+    }
+
     Try<Nothing> mnt = fs::mount(
         (mount.has_source() ? Option<string>(mount.source()) : None()),
         mount.target(),

http://git-wip-us.apache.org/repos/asf/mesos/blob/d6d40eca/src/tests/containerizer/volume_host_path_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/volume_host_path_isolator_tests.cpp b/src/tests/containerizer/volume_host_path_isolator_tests.cpp
index 89710f7..271bbbe 100644
--- a/src/tests/containerizer/volume_host_path_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_host_path_isolator_tests.cpp
@@ -21,6 +21,7 @@
 #include <process/owned.hpp>
 #include <process/gtest.hpp>
 
+#include <stout/format.hpp>
 #include <stout/gtest.hpp>
 #include <stout/os.hpp>
 #include <stout/path.hpp>
@@ -288,6 +289,68 @@ TEST_F(VolumeHostPathIsolatorTest, ROOT_FileVolumeFromHostSandboxMountPoint)
 }
 
 
+TEST_F(VolumeHostPathIsolatorTest, ROOT_MountPropagation)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "filesystem/linux";
+
+  Fetcher fetcher(flags);
+
+  Try<MesosContainerizer*> create =
+    MesosContainerizer::create(flags, true, &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<Containerizer> containerizer(create.get());
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  string mountDirectory = path::join(flags.work_dir, "mount_directory");
+  string mountPoint = path::join(mountDirectory, "mount_point");
+  string filePath = path::join(mountPoint, "foo");
+
+  ASSERT_SOME(os::mkdir(mountPoint));
+
+  ExecutorInfo executor = createExecutorInfo(
+      "test_executor",
+      strings::format(
+          "mount -t tmpfs tmpfs %s; touch %s",
+          mountPoint,
+          filePath).get());
+
+  executor.mutable_container()->CopyFrom(createContainerInfo(
+      None(),
+      {createVolumeHostPath(
+          mountDirectory,
+          mountDirectory,
+          Volume::RW,
+          MountPropagation::BIDIRECTIONAL)}));
+
+  string directory = path::join(flags.work_dir, "sandbox");
+  ASSERT_SOME(os::mkdir(directory));
+
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
+      containerId,
+      createContainerConfig(None(), executor, directory),
+      map<string, string>(),
+      None());
+
+  AWAIT_READY(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());
+
+  // If the mount propagation has been setup properly, we should see
+  // the file we touch'ed in 'mountPoint'.
+  EXPECT_TRUE(os::exists(filePath));
+}
+
+
 class VolumeHostPathIsolatorMesosTest
   : public MesosTest,
     public WithParamInterface<ParamExecutorType> {};

http://git-wip-us.apache.org/repos/asf/mesos/blob/d6d40eca/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 18f3623..68f78d0 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -673,7 +673,8 @@ template <typename TVolume>
 inline TVolume createVolumeHostPath(
     const std::string& containerPath,
     const std::string& hostPath,
-    const typename TVolume::Mode& mode)
+    const typename TVolume::Mode& mode,
+    const Option<MountPropagation::Mode>& mountPropagationMode = None())
 {
   TVolume volume;
   volume.set_container_path(containerPath);
@@ -683,6 +684,13 @@ inline TVolume createVolumeHostPath(
   source->set_type(TVolume::Source::HOST_PATH);
   source->mutable_host_path()->set_path(hostPath);
 
+  if (mountPropagationMode.isSome()) {
+    source
+      ->mutable_host_path()
+      ->mutable_mount_propagation()
+      ->set_mode(mountPropagationMode.get());
+  }
+
   return volume;
 }