You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ya...@apache.org on 2015/09/18 00:09:59 UTC

mesos git commit: Implemented a TODO to clean up host mounts irrelevant to the container in the container's mount namespace.

Repository: mesos
Updated Branches:
  refs/heads/master 9b3c8610b -> cf7e9261c


Implemented a TODO to clean up host mounts irrelevant to the container in the container's mount namespace.

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


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

Branch: refs/heads/master
Commit: cf7e9261c39fa2d956bb5560cb655cf62d0bddae
Parents: 9b3c861
Author: Jiang Yan Xu <ya...@jxu.me>
Authored: Wed Sep 16 23:13:58 2015 -0700
Committer: Jiang Yan Xu <ya...@jxu.me>
Committed: Thu Sep 17 15:09:22 2015 -0700

----------------------------------------------------------------------
 .../isolators/filesystem/linux.cpp              |  46 +++-
 .../isolators/filesystem/linux.hpp              |   1 +
 .../containerizer/filesystem_isolator_tests.cpp | 274 ++++++++++++++++---
 3 files changed, 262 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/cf7e9261/src/slave/containerizer/isolators/filesystem/linux.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/linux.cpp b/src/slave/containerizer/isolators/filesystem/linux.cpp
index b674634..152acc3 100644
--- a/src/slave/containerizer/isolators/filesystem/linux.cpp
+++ b/src/slave/containerizer/isolators/filesystem/linux.cpp
@@ -360,17 +360,14 @@ Future<Option<ContainerPrepareInfo>> LinuxFilesystemIsolatorProcess::__prepare(
   // namespace right after forking the executor process. We use these
   // commands to mount those volumes specified in the container info
   // so that they don't pollute the host mount namespace.
-  if (executorInfo.has_container() &&
-      executorInfo.container().volumes_size() > 0) {
-    Try<string> _script = script(executorInfo, directory, rootfs);
-    if (_script.isError()) {
-      return Failure("Failed to generate isolation script: " + _script.error());
-    }
-
-    CommandInfo* command = prepareInfo.add_commands();
-    command->set_value(_script.get());
+  Try<string> _script = script(containerId, executorInfo, directory, rootfs);
+  if (_script.isError()) {
+    return Failure("Failed to generate isolation script: " + _script.error());
   }
 
+  CommandInfo* command = prepareInfo.add_commands();
+  command->set_value(_script.get());
+
   return update(containerId, executorInfo.resources())
     .then([prepareInfo]() -> Future<Option<ContainerPrepareInfo>> {
       return prepareInfo;
@@ -379,12 +376,11 @@ Future<Option<ContainerPrepareInfo>> LinuxFilesystemIsolatorProcess::__prepare(
 
 
 Try<string> LinuxFilesystemIsolatorProcess::script(
+    const ContainerID& containerId,
     const ExecutorInfo& executorInfo,
     const string& directory,
     const Option<string>& rootfs)
 {
-  CHECK(executorInfo.has_container());
-
   ostringstream out;
   out << "#!/bin/sh\n";
   out << "set -x -e\n";
@@ -393,9 +389,31 @@ Try<string> LinuxFilesystemIsolatorProcess::script(
   // propagate back to the host mount namespace.
   out << "mount --make-rslave /\n";
 
-  // TODO(jieyu): Try to unmount work directory mounts and persistent
-  // volume mounts for other containers to release the extra
-  // references to those mounts.
+  // Try to unmount work directory mounts and persistent volume mounts
+  // for other containers to release the extra references to them.
+  // NOTE:
+  // 1) This doesn't completely eliminate the race condition between
+  //    this container copying mount table and other containers being
+  //    cleaned up. This is instead a best-effort attempt.
+  // 2) This script assumes that all the mounts the container needs
+  //    under the slave work directory have its container ID in the
+  //    path either for the mount source (e.g. sandbox self-bind mount)
+  //    or the mount target (e.g. mounting sandbox into new rootfs).
+  //
+  // TODO(xujyan): This command may fail if --work_dir is not specified
+  // with a real path as real paths are used in the mount table. It
+  // doesn't work when the paths contain reserved characters such as
+  // spaces either because such characters in mount info are encoded
+  // in the escaped form (i.e. '\0xx').
+  out << "grep '" << flags.work_dir << "' /proc/self/mountinfo | "
+      << "grep -v '" << containerId.value() << "' | "
+      << "cut -d' ' -f5 | " // '-f5' is the mount target. See MountInfoTable.
+      << "xargs --no-run-if-empty umount -l || "
+      << "true \n"; // We mask errors in this command.
+
+  if (!executorInfo.has_container()) {
+    return out.str();
+  }
 
   foreach (const Volume& volume, executorInfo.container().volumes()) {
     if (!volume.has_host_path()) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/cf7e9261/src/slave/containerizer/isolators/filesystem/linux.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/linux.hpp b/src/slave/containerizer/isolators/filesystem/linux.hpp
index 99f939f..041af7b 100644
--- a/src/slave/containerizer/isolators/filesystem/linux.hpp
+++ b/src/slave/containerizer/isolators/filesystem/linux.hpp
@@ -100,6 +100,7 @@ private:
       const Option<std::string>& rootfs);
 
   Try<std::string> script(
+      const ContainerID& containerId,
       const ExecutorInfo& executorInfo,
       const std::string& directory,
       const Option<std::string>& rootfs);

http://git-wip-us.apache.org/repos/asf/mesos/blob/cf7e9261/src/tests/containerizer/filesystem_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/filesystem_isolator_tests.cpp b/src/tests/containerizer/filesystem_isolator_tests.cpp
index ca9f423..e2eb049 100644
--- a/src/tests/containerizer/filesystem_isolator_tests.cpp
+++ b/src/tests/containerizer/filesystem_isolator_tests.cpp
@@ -26,6 +26,7 @@
 #include <stout/error.hpp>
 #include <stout/foreach.hpp>
 #include <stout/gtest.hpp>
+#include <stout/hashmap.hpp>
 #include <stout/os.hpp>
 #include <stout/path.hpp>
 #include <stout/uuid.hpp>
@@ -40,6 +41,8 @@
 
 #include "slave/containerizer/mesos/containerizer.hpp"
 
+#include "slave/containerizer/provisioner/paths.hpp"
+
 #include "tests/flags.hpp"
 #include "tests/mesos.hpp"
 
@@ -75,15 +78,17 @@ public:
   // LinuxFilesystemIsolator. The filesystem isolator takes a
   // TestAppcProvisioner which provisions APPC images by copying files
   // from the host filesystem.
+  // 'images' is a map of imageName -> rootfsPath.
+  // TODO(xujyan): The current assumption of one rootfs per image name
+  // is inconsistent with the real provisioner and we should fix it.
   Try<Owned<MesosContainerizer>> createContainerizer(
       const slave::Flags& flags,
-      const vector<string>& imageNames)
+      const hashmap<string, string>& images)
   {
     // Create the root filesystems.
     hashmap<string, Shared<Rootfs>> rootfses;
-    foreach (const string& imageName, imageNames) {
-      Try<Owned<Rootfs>> rootfs =
-        LinuxRootfs::create(path::join(os::getcwd(), "rootfses", imageName));
+    foreachpair (const string& imageName, const string& rootfsPath, images) {
+      Try<Owned<Rootfs>> rootfs = LinuxRootfs::create(rootfsPath);
 
       if (rootfs.isError()) {
         return Error("Failed to create LinuxRootfs: " + rootfs.error());
@@ -181,14 +186,19 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ChangeRootFilesystem)
 {
   slave::Flags flags = CreateSlaveFlags();
 
-  Try<Owned<MesosContainerizer>> containerizer =
-    createContainerizer(flags, {"test_image"});
-
-  ASSERT_SOME(containerizer);
-
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
+  string rootfsesDir = slave::provisioner::paths::getContainerDir(
+      slave::paths::getProvisionerDir(flags.work_dir),
+      containerId);
+
+  Try<Owned<MesosContainerizer>> containerizer = createContainerizer(
+      flags,
+      {{"test_image", path::join(rootfsesDir, "test_image")}});
+
+  ASSERT_SOME(containerizer);
+
   ExecutorInfo executor = CREATE_EXECUTOR_INFO(
       "test_executor",
       "[ ! -d '" + os::getcwd() + "' ]");
@@ -229,14 +239,19 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromSandbox)
 {
   slave::Flags flags = CreateSlaveFlags();
 
-  Try<Owned<MesosContainerizer>> containerizer =
-    createContainerizer(flags, {"test_image"});
-
-  ASSERT_SOME(containerizer);
-
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
+  string rootfsesDir = slave::provisioner::paths::getContainerDir(
+      slave::paths::getProvisionerDir(flags.work_dir),
+      containerId);
+
+  Try<Owned<MesosContainerizer>> containerizer = createContainerizer(
+      flags,
+      {{"test_image", path::join(rootfsesDir, "test_image")}});
+
+  ASSERT_SOME(containerizer);
+
   ExecutorInfo executor = CREATE_EXECUTOR_INFO(
       "test_executor",
       "echo abc > /tmp/file");
@@ -281,14 +296,19 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromHost)
 {
   slave::Flags flags = CreateSlaveFlags();
 
-  Try<Owned<MesosContainerizer>> containerizer =
-    createContainerizer(flags, {"test_image"});
-
-  ASSERT_SOME(containerizer);
-
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
+  string rootfsesDir = slave::provisioner::paths::getContainerDir(
+      slave::paths::getProvisionerDir(flags.work_dir),
+      containerId);
+
+  Try<Owned<MesosContainerizer>> containerizer = createContainerizer(
+      flags,
+      {{"test_image", path::join(rootfsesDir, "test_image")}});
+
+  ASSERT_SOME(containerizer);
+
   ExecutorInfo executor = CREATE_EXECUTOR_INFO(
       "test_executor",
       "test -d /tmp/sandbox");
@@ -331,14 +351,19 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromHostSandboxMountPoint)
 {
   slave::Flags flags = CreateSlaveFlags();
 
-  Try<Owned<MesosContainerizer>> containerizer =
-    createContainerizer(flags, {"test_image"});
-
-  ASSERT_SOME(containerizer);
-
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
+  string rootfsesDir = slave::provisioner::paths::getContainerDir(
+      slave::paths::getProvisionerDir(flags.work_dir),
+      containerId);
+
+  Try<Owned<MesosContainerizer>> containerizer = createContainerizer(
+      flags,
+      {{"test_image", path::join(rootfsesDir, "test_image")}});
+
+  ASSERT_SOME(containerizer);
+
   ExecutorInfo executor = CREATE_EXECUTOR_INFO(
       "test_executor",
       "test -d mountpoint/sandbox");
@@ -379,16 +404,24 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromHostSandboxMountPoint)
 TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithRootFilesystem)
 {
   slave::Flags flags = CreateSlaveFlags();
-  flags.work_dir = os::getcwd();
 
-  Try<Owned<MesosContainerizer>> containerizer =
-    createContainerizer(flags, {"test_image"});
-
-  ASSERT_SOME(containerizer);
+  // Need this otherwise the persistent volumes are not created
+  // within the slave work_dir and thus not retrievable.
+  flags.work_dir = os::getcwd();
 
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
+  string rootfsesDir = slave::provisioner::paths::getContainerDir(
+      slave::paths::getProvisionerDir(flags.work_dir),
+      containerId);
+
+  Try<Owned<MesosContainerizer>> containerizer = createContainerizer(
+      flags,
+      {{"test_image", path::join(rootfsesDir, "test_image")}});
+
+  ASSERT_SOME(containerizer);
+
   ExecutorInfo executor = CREATE_EXECUTOR_INFO(
       "test_executor",
       "echo abc > volume/file");
@@ -441,16 +474,24 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithRootFilesystem)
 TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithoutRootFilesystem)
 {
   slave::Flags flags = CreateSlaveFlags();
-  flags.work_dir = os::getcwd();
-
-  Try<Owned<MesosContainerizer>> containerizer =
-    createContainerizer(flags, {"test_image"});
 
-  ASSERT_SOME(containerizer);
+  // Need this otherwise the persistent volumes are not created
+  // within the slave work_dir and thus not retrievable.
+  flags.work_dir = os::getcwd();
 
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
+  string rootfsesDir = slave::provisioner::paths::getContainerDir(
+      slave::paths::getProvisionerDir(flags.work_dir),
+      containerId);
+
+  Try<Owned<MesosContainerizer>> containerizer = createContainerizer(
+      flags,
+      {{"test_image", path::join(rootfsesDir, "test_image")}});
+
+  ASSERT_SOME(containerizer);
+
   ExecutorInfo executor = CREATE_EXECUTOR_INFO(
       "test_executor",
       "echo abc > volume/file");
@@ -471,8 +512,14 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithoutRootFilesystem)
 
   ASSERT_SOME(os::mkdir(volume));
 
-  string directory = path::join(os::getcwd(), "sandbox");
-  ASSERT_SOME(os::mkdir(directory));
+  // To make sure the sandbox directory has the container ID in its
+  // path so it doesn't get unmounted by the launcher.
+  string directory = slave::paths::createExecutorDirectory(
+      flags.work_dir,
+      SlaveID(),
+      FrameworkID(),
+      executor.executor_id(),
+      containerId);
 
   Future<bool> launch = containerizer.get()->launch(
       containerId,
@@ -507,14 +554,19 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ImageInVolumeWithoutRootFilesystem)
 {
   slave::Flags flags = CreateSlaveFlags();
 
-  Try<Owned<MesosContainerizer>> containerizer =
-    createContainerizer(flags, {"test_image"});
-
-  ASSERT_SOME(containerizer);
-
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
+  string rootfsesDir = slave::provisioner::paths::getContainerDir(
+      slave::paths::getProvisionerDir(flags.work_dir),
+      containerId);
+
+  Try<Owned<MesosContainerizer>> containerizer = createContainerizer(
+      flags,
+      {{"test_image", path::join(rootfsesDir, "test_image")}});
+
+  ASSERT_SOME(containerizer);
+
   ExecutorInfo executor = CREATE_EXECUTOR_INFO(
       "test_executor",
       "test -d rootfs/bin");
@@ -557,14 +609,21 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ImageInVolumeWithRootFilesystem)
 {
   slave::Flags flags = CreateSlaveFlags();
 
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  string rootfsesDir = slave::provisioner::paths::getContainerDir(
+      slave::paths::getProvisionerDir(flags.work_dir),
+      containerId);
+
   Try<Owned<MesosContainerizer>> containerizer =
-    createContainerizer(flags, {"test_image_rootfs", "test_image_volume"});
+    createContainerizer(
+        flags,
+        {{"test_image_rootfs", path::join(rootfsesDir, "test_image_rootfs")},
+         {"test_image_volume", path::join(rootfsesDir, "test_image_volume")}});
 
   ASSERT_SOME(containerizer);
 
-  ContainerID containerId;
-  containerId.set_value(UUID::random().toString());
-
   ExecutorInfo executor = CREATE_EXECUTOR_INFO(
       "test_executor",
       "[ ! -d '" + os::getcwd() + "' ] && [ -d rootfs/bin ]");
@@ -598,6 +657,131 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ImageInVolumeWithRootFilesystem)
   EXPECT_TRUE(wait.get().has_status());
   EXPECT_EQ(0, wait.get().status());
 }
+
+
+// This test verifies that multiple containers with images can be
+// launched simultaneously with no interference.
+TEST_F(LinuxFilesystemIsolatorTest, ROOT_MultipleContainers)
+{
+  slave::Flags flags = CreateSlaveFlags();
+
+  // Need this otherwise the persistent volumes are not created
+  // within the slave work_dir and thus not retrievable.
+  flags.work_dir = os::getcwd();
+
+  ContainerID containerId1;
+  containerId1.set_value(UUID::random().toString());
+
+  ContainerID containerId2;
+  containerId2.set_value(UUID::random().toString());
+
+  string rootfsesDir1 = slave::provisioner::paths::getContainerDir(
+      slave::paths::getProvisionerDir(flags.work_dir),
+      containerId1);
+
+  string rootfsesDir2 = slave::provisioner::paths::getContainerDir(
+      slave::paths::getProvisionerDir(flags.work_dir),
+      containerId2);
+
+  Try<Owned<MesosContainerizer>> containerizer =
+    createContainerizer(
+        flags,
+        {{"test_image1", path::join(rootfsesDir1, "test_image1")},
+         {"test_image2", path::join(rootfsesDir2, "test_image2")}});
+
+  ASSERT_SOME(containerizer);
+
+  SlaveID slaveId;
+  slaveId.set_value("test_slave");
+
+  // First launch container 1 which has a long running task which
+  // guarantees that its work directory mount is in the host mount
+  // table when container 2 is launched.
+  ExecutorInfo executor1 = CREATE_EXECUTOR_INFO(
+      "test_executor1",
+      "sleep 1000"); // Long running task.
+
+  executor1.mutable_container()->CopyFrom(createContainerInfo("test_image1"));
+
+  // Create a persistent volume for container 1. We do this because
+  // we want to test container 2 cleaning up multiple mounts.
+  executor1.add_resources()->CopyFrom(createPersistentVolume(
+      Megabytes(32),
+      "test_role",
+      "persistent_volume_id",
+      "volume"));
+
+  string volume = slave::paths::getPersistentVolumePath(
+      os::getcwd(),
+      "test_role",
+      "persistent_volume_id");
+
+  ASSERT_SOME(os::mkdir(volume));
+
+  string directory1 = slave::paths::createExecutorDirectory(
+      flags.work_dir,
+      slaveId,
+      DEFAULT_FRAMEWORK_INFO.id(),
+      executor1.executor_id(),
+      containerId1);
+
+  Future<bool> launch1 = containerizer.get()->launch(
+      containerId1,
+      executor1,
+      directory1,
+      None(),
+      slaveId,
+      PID<Slave>(),
+      false);
+
+  // Wait for the launch to complete.
+  AWAIT_READY(launch1);
+
+  // Now launch container 2 which will copy the host mount table with
+  // container 1's work directory mount in it.
+  ExecutorInfo executor2 = CREATE_EXECUTOR_INFO(
+      "test_executor2",
+      "[ ! -d '" + os::getcwd() + "' ]");
+
+  executor2.mutable_container()->CopyFrom(createContainerInfo("test_image2"));
+
+  string directory2 = slave::paths::createExecutorDirectory(
+      flags.work_dir,
+      slaveId,
+      DEFAULT_FRAMEWORK_INFO.id(),
+      executor2.executor_id(),
+      containerId2);
+
+  Future<bool> launch2 = containerizer.get()->launch(
+      containerId2,
+      executor2,
+      directory2,
+      None(),
+      slaveId,
+      PID<Slave>(),
+      false);
+
+  AWAIT_READY(launch2);
+
+  containerizer.get()->destroy(containerId1);
+
+  // Wait on the containers.
+  Future<containerizer::Termination> wait1 =
+    containerizer.get()->wait(containerId1);
+  Future<containerizer::Termination> wait2 =
+    containerizer.get()->wait(containerId2);
+
+  AWAIT_READY(wait1);
+  AWAIT_READY(wait2);
+
+  // Executor 1 was forcefully killed.
+  EXPECT_TRUE(wait1.get().has_status());
+  EXPECT_EQ(9, wait1.get().status());
+
+  // Executor 2 exited normally.
+  EXPECT_TRUE(wait2.get().has_status());
+  EXPECT_EQ(0, wait2.get().status());
+}
 #endif // __linux__
 
 } // namespace tests {