You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by gi...@apache.org on 2019/07/12 08:39:23 UTC

[mesos] 01/03: Improved `namespaces/ipc` isolator for configurable IPC support.

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

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

commit 6d243ce939706a72e2479d04b43d0fb904984607
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Fri Jul 12 00:40:07 2019 -0700

    Improved `namespaces/ipc` isolator for configurable IPC support.
    
    Review: https://reviews.apache.org/r/70798/
---
 .../mesos/isolators/namespaces/ipc.cpp             | 247 ++++++++++++++++++++-
 .../mesos/isolators/namespaces/ipc.hpp             |   4 +-
 src/slave/containerizer/mesos/paths.cpp            |  72 ++++++
 src/slave/containerizer/mesos/paths.hpp            |  13 ++
 src/tests/containerizer/isolator_tests.cpp         |   2 +-
 5 files changed, 326 insertions(+), 12 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/namespaces/ipc.cpp b/src/slave/containerizer/mesos/isolators/namespaces/ipc.cpp
index 6c8e8ee..327827f 100644
--- a/src/slave/containerizer/mesos/isolators/namespaces/ipc.cpp
+++ b/src/slave/containerizer/mesos/isolators/namespaces/ipc.cpp
@@ -14,15 +14,32 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-#include "slave/containerizer/mesos/isolators/namespaces/ipc.hpp"
+#include <string>
 
 #include <process/future.hpp>
 #include <process/id.hpp>
 
+#include <stout/os.hpp>
+
+#include "common/protobuf_utils.hpp"
+
+#include "linux/fs.hpp"
 #include "linux/ns.hpp"
 
+#include "slave/containerizer/mesos/paths.hpp"
+
+#include "slave/containerizer/mesos/isolators/namespaces/ipc.hpp"
+
+using process::Failure;
 using process::Future;
 
+using std::string;
+
+using mesos::internal::protobuf::slave::createContainerMount;
+
+using mesos::internal::slave::containerizer::paths::AGENT_SHM_DIRECTORY;
+
+using mesos::slave::ContainerClass;
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
 using mesos::slave::Isolator;
@@ -51,13 +68,22 @@ Try<Isolator*> NamespacesIPCIsolatorProcess::create(const Flags& flags)
         "The 'linux' launcher must be used to enable the IPC namespace");
   }
 
+  // Make sure 'filesystem/linux' isolator is used.
+  // NOTE: 'filesystem/linux' isolator will make sure mounts in the
+  // child mount namespace will not be propagated back to the host
+  // mount namespace.
+  if (!strings::contains(flags.isolation, "filesystem/linux")) {
+    return Error("'filesystem/linux' must be used to enable IPC namespace");
+  }
+
   return new MesosIsolator(process::Owned<MesosIsolatorProcess>(
-      new NamespacesIPCIsolatorProcess()));
+      new NamespacesIPCIsolatorProcess(flags)));
 }
 
 
-NamespacesIPCIsolatorProcess::NamespacesIPCIsolatorProcess()
-  : ProcessBase(process::ID::generate("ipc-namespace-isolator")) {}
+NamespacesIPCIsolatorProcess::NamespacesIPCIsolatorProcess(const Flags& _flags)
+  : ProcessBase(process::ID::generate("ipc-namespace-isolator")),
+    flags(_flags) {}
 
 
 bool NamespacesIPCIsolatorProcess::supportsNesting()
@@ -76,20 +102,221 @@ bool NamespacesIPCIsolatorProcess::supportsStandalone()
 // namespace. Neither /proc, nor any of the special SVIPC filesystem need
 // to be remounted for this to work. IPC namespaces are disjoint. That is,
 // once you enter an IPC namespace, IPC objects from the host namespace are
-// no longer visible (and vice versa). Since IPC namespaces do not nest,
-// we always place nested containers into the IPC namespace of the parent
-// container. That is, containers in the same group share an IPC namespace,
-// but groups are isolated from each other.
+// no longer visible (and vice versa).
 Future<Option<ContainerLaunchInfo>> NamespacesIPCIsolatorProcess::prepare(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig)
 {
   ContainerLaunchInfo launchInfo;
+  Option<LinuxInfo::IpcMode> ipcMode;
+  Option<Bytes> shmSize;
+
+  // Get the container's IPC mode and size of /dev/shm.
+  if (containerConfig.has_container_info() &&
+      containerConfig.container_info().has_linux_info()) {
+    if (containerConfig.container_info().linux_info().has_ipc_mode()) {
+      ipcMode = containerConfig.container_info().linux_info().ipc_mode();
+    }
+
+    if (containerConfig.container_info().linux_info().has_shm_size()) {
+      shmSize =
+        Megabytes(containerConfig.container_info().linux_info().shm_size());
+    } else if (flags.default_shm_size.isSome()) {
+      shmSize = flags.default_shm_size.get();
+    }
+  }
 
   if (containerId.has_parent()) {
-    launchInfo.add_enter_namespaces(CLONE_NEWIPC);
+    // Debug container always shares its parent container's IPC namespace
+    // and /dev/shm. Please note that `filesystem/linux` isolator will
+    // ensure debug container enters its parent container's mount namespace
+    // so it will share its parent container's /dev/shm.
+    if (containerConfig.has_container_class() &&
+        containerConfig.container_class() == ContainerClass::DEBUG) {
+      launchInfo.add_enter_namespaces(CLONE_NEWIPC);
+      return launchInfo;
+    }
+
+    if (ipcMode.isNone()) {
+      // If IPC mode is not set, for backward compatibility we will keep the
+      // previous behavior: Nested container will share the IPC namespace from
+      // its parent container, and if it does not have its own rootfs, it will
+      // share agent's /dev/shm, otherwise it will have its own /dev/shm.
+      launchInfo.add_enter_namespaces(CLONE_NEWIPC);
+
+      if (containerConfig.has_rootfs()) {
+        *launchInfo.add_mounts() = createContainerMount(
+            "tmpfs",
+            path::join(containerConfig.rootfs(), "/dev/shm"),
+            "tmpfs",
+            "mode=1777",
+            MS_NOSUID | MS_NODEV | MS_STRICTATIME);
+      }
+    } else {
+      switch (ipcMode.get()) {
+        case LinuxInfo::PRIVATE: {
+          // If IPC mode is `PRIVATE`, nested container will have its own
+          // IPC namespace and /dev/shm.
+          launchInfo.add_clone_namespaces(CLONE_NEWIPC);
+
+          // Create a tmpfs mount in agent host for nested container's /dev/shm.
+          const string shmPath = containerizer::paths::getContainerShmPath(
+              flags.runtime_dir, containerId);
+
+          Try<Nothing> mkdir = os::mkdir(shmPath);
+          if (mkdir.isError()) {
+            return Failure(
+                "Failed to create container shared memory directory: " +
+                mkdir.error());
+          }
+
+          Try<Nothing> mnt = fs::mount(
+              "tmpfs",
+              shmPath,
+              "tmpfs",
+              MS_NOSUID | MS_NODEV | MS_STRICTATIME,
+              shmSize.isSome() ?
+                strings::format("mode=1777,size=%d", shmSize->bytes()).get() :
+                "mode=1777");
+
+          if (mnt.isError()) {
+            return Failure("Failed to mount '" + shmPath + "': " + mnt.error());
+          }
+
+          // Bind mount the tmpfs mount at /dev/shm in nested container's mount
+          // namespace.
+          *launchInfo.add_mounts() = createContainerMount(
+              shmPath,
+              containerConfig.has_rootfs()
+                ? path::join(containerConfig.rootfs(), "/dev/shm")
+                : "/dev/shm",
+              MS_BIND);
+
+          break;
+        }
+        case LinuxInfo::SHARE_PARENT: {
+          // If IPC mode is `SHARE_PARENT`, nested container will its parent
+          // container's IPC namespace and /dev/shm.
+          launchInfo.add_enter_namespaces(CLONE_NEWIPC);
+
+          Try<string> parentShmPath = containerizer::paths::getParentShmPath(
+              flags.runtime_dir,
+              containerId);
+
+          if (parentShmPath.isError()) {
+            return Failure(
+                "Failed to get parent shared memory path: " +
+                parentShmPath.error());
+          } else if (parentShmPath.get() != AGENT_SHM_DIRECTORY ||
+                     containerConfig.has_rootfs()) {
+            // To share parent container's /dev/shm, we need to bind mount
+            // parent container's /dev/shm at /dev/shm in nested container's
+            // mount namespace. Please note that we do not need to do this if
+            // the parent container uses agent's /dev/shm and the nested
+            // container does not has its own rootfs in which case the nested
+            // container can directly access the agent's /dev/shm.
+            *launchInfo.add_mounts() = createContainerMount(
+                parentShmPath.get(),
+                containerConfig.has_rootfs()
+                  ? path::join(containerConfig.rootfs(), "/dev/shm")
+                  : "/dev/shm",
+                MS_BIND);
+          }
+
+          break;
+        }
+        case LinuxInfo::UNKNOWN: {
+          return Failure("Unknown IPC mode");
+        }
+      }
+    }
   } else {
-    launchInfo.add_clone_namespaces(CLONE_NEWIPC);
+    // This is the case of top-level container.
+    if (ipcMode.isNone()) {
+      // If IPC mode is not set, for backward compatibility we will keep the
+      // previous behavior: Top-level container will have its own IPC namespace,
+      // and if it does not have its own rootfs, it will share agent's /dev/shm,
+      // otherwise it will have its own /dev/shm.
+      launchInfo.add_clone_namespaces(CLONE_NEWIPC);
+
+      if (containerConfig.has_rootfs()) {
+        *launchInfo.add_mounts() = createContainerMount(
+            "tmpfs",
+            path::join(containerConfig.rootfs(), "/dev/shm"),
+            "tmpfs",
+            "mode=1777",
+            MS_NOSUID | MS_NODEV | MS_STRICTATIME);
+      }
+    } else {
+      switch (ipcMode.get()) {
+        case LinuxInfo::PRIVATE: {
+          // If IPC mode is `PRIVATE`, top-level container will have its own
+          // IPC namespace and /dev/shm.
+          launchInfo.add_clone_namespaces(CLONE_NEWIPC);
+
+          // Create a tmpfs mount in agent host for top-level container's
+          // /dev/shm.
+          const string shmPath = containerizer::paths::getContainerShmPath(
+              flags.runtime_dir, containerId);
+
+          Try<Nothing> mkdir = os::mkdir(shmPath);
+          if (mkdir.isError()) {
+            return Failure(
+                "Failed to create container shared memory directory: " +
+                mkdir.error());
+          }
+
+          Try<Nothing> mnt = fs::mount(
+              "tmpfs",
+              shmPath,
+              "tmpfs",
+              MS_NOSUID | MS_NODEV | MS_STRICTATIME,
+              shmSize.isSome() ?
+                strings::format("mode=1777,size=%d", shmSize->bytes()).get() :
+                "mode=1777");
+
+          if (mnt.isError()) {
+            return Failure("Failed to mount '" + shmPath + "': " + mnt.error());
+          }
+
+          // Bind mount the tmpfs mount at /dev/shm in top-level container's
+          // mount namespace.
+          *launchInfo.add_mounts() = createContainerMount(
+              shmPath,
+              containerConfig.has_rootfs() ?
+                path::join(containerConfig.rootfs(), "/dev/shm") :
+                "/dev/shm",
+              MS_BIND);
+
+          break;
+        }
+        case LinuxInfo::SHARE_PARENT: {
+          if (flags.disallow_sharing_agent_ipc_namespace) {
+            return Failure(
+                "Sharing agent IPC namespace with "
+                "top-level container is not allowed");
+          }
+
+          // If top-level container has its own rootfs, we will bind mount
+          // agent's /dev/shm at /dev/shm in its mount namespace, otherwise
+          // we do not need to anything since it can directly access agent's
+          // /dev/shm.
+          if (containerConfig.has_rootfs()) {
+            *launchInfo.add_mounts() = createContainerMount(
+                AGENT_SHM_DIRECTORY,
+                containerConfig.has_rootfs() ?
+                  path::join(containerConfig.rootfs(), "/dev/shm") :
+                  "/dev/shm",
+                MS_BIND);
+          }
+
+          break;
+        }
+        case LinuxInfo::UNKNOWN: {
+          return Failure("Unknown IPC mode");
+        }
+      }
+    }
   }
 
   return launchInfo;
diff --git a/src/slave/containerizer/mesos/isolators/namespaces/ipc.hpp b/src/slave/containerizer/mesos/isolators/namespaces/ipc.hpp
index 32c8883..00c9d7e 100644
--- a/src/slave/containerizer/mesos/isolators/namespaces/ipc.hpp
+++ b/src/slave/containerizer/mesos/isolators/namespaces/ipc.hpp
@@ -40,7 +40,9 @@ public:
       const mesos::slave::ContainerConfig& containerConfig) override;
 
 private:
-  NamespacesIPCIsolatorProcess();
+  NamespacesIPCIsolatorProcess(const Flags& flags);
+
+  const Flags flags;
 };
 
 } // namespace slave {
diff --git a/src/slave/containerizer/mesos/paths.cpp b/src/slave/containerizer/mesos/paths.cpp
index 4281abc..b028795 100644
--- a/src/slave/containerizer/mesos/paths.cpp
+++ b/src/slave/containerizer/mesos/paths.cpp
@@ -522,6 +522,78 @@ Try<ContainerID> parseSandboxPath(
   return currentContainerId;
 }
 
+
+string getContainerShmPath(
+    const string& runtimeDir,
+    const ContainerID& containerId)
+{
+  return path::join(
+      getRuntimePath(runtimeDir, containerId),
+      CONTAINER_SHM_DIRECTORY);
+}
+
+
+Try<string> getParentShmPath(
+    const string runtimeDir,
+    const ContainerID& containerId)
+{
+  CHECK(containerId.has_parent());
+
+  ContainerID parentId = containerId.parent();
+
+  Result<ContainerConfig> parentConfig =
+    getContainerConfig(runtimeDir, parentId);
+
+  if (parentConfig.isNone()) {
+    return Error(
+        "Failed to find config for container " + stringify(parentId));
+  } else if (parentConfig.isError()) {
+    return Error(parentConfig.error());
+  }
+
+  string parentShmPath;
+
+  if (parentConfig->has_container_info() &&
+      parentConfig->container_info().has_linux_info() &&
+      parentConfig->container_info().linux_info().has_ipc_mode()) {
+    switch (parentConfig->container_info().linux_info().ipc_mode()) {
+      case LinuxInfo::PRIVATE: {
+        const string shmPath = getContainerShmPath(runtimeDir, parentId);
+        if (!os::exists(shmPath)) {
+          return Error(
+              "The shared memory path '" + shmPath + "' of container "
+              + stringify(parentId) + " does not exist");
+        }
+
+        parentShmPath = shmPath;
+        break;
+      }
+      case LinuxInfo::SHARE_PARENT: {
+        if (parentId.has_parent()) {
+          return getParentShmPath(runtimeDir, parentId);
+        }
+
+        parentShmPath = AGENT_SHM_DIRECTORY;
+        break;
+      }
+      case LinuxInfo::UNKNOWN: {
+        LOG(FATAL) << "The IPC mode of container " << parentId << " is UNKNOWN";
+      }
+    }
+  } else {
+    if (parentConfig->has_rootfs()) {
+      return Error(
+          "The shared memory of container " + stringify(parentId) +
+          " cannot be shared with any other containers because it"
+          " is only in the container's own mount namespace");
+    }
+
+    parentShmPath = AGENT_SHM_DIRECTORY;
+  }
+
+  return parentShmPath;
+}
+
 } // namespace paths {
 } // namespace containerizer {
 } // namespace slave {
diff --git a/src/slave/containerizer/mesos/paths.hpp b/src/slave/containerizer/mesos/paths.hpp
index a5e0920..c003335 100644
--- a/src/slave/containerizer/mesos/paths.hpp
+++ b/src/slave/containerizer/mesos/paths.hpp
@@ -63,6 +63,7 @@ namespace paths {
 //           |-- mnt
 //           |   |-- host_proc
 //           |-- pid
+//           |-- shm
 //           |-- standalone.marker
 //           |-- status
 //           |-- termination
@@ -81,6 +82,8 @@ 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";
+constexpr char CONTAINER_SHM_DIRECTORY[] = "shm";
+constexpr char AGENT_SHM_DIRECTORY[] = "/dev/shm";
 
 
 enum Mode
@@ -258,6 +261,16 @@ Try<ContainerID> parseSandboxPath(
     const std::string& rootSandboxPath,
     const std::string& path);
 
+
+std::string getContainerShmPath(
+    const std::string& runtimeDir,
+    const ContainerID& containerId);
+
+
+Try<std::string> getParentShmPath(
+    const std::string runtimeDir,
+    const ContainerID& containerId);
+
 } // namespace paths {
 } // namespace containerizer {
 } // namespace slave {
diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index 9c14f3a..bb2cda4 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -260,7 +260,7 @@ TEST_F(NamespacesIsolatorTest, ROOT_SharePidNamespaceWhenDisallow)
 TEST_F(NamespacesIsolatorTest, ROOT_IPCNamespace)
 {
   Try<Owned<MesosContainerizer>> containerizer =
-    createContainerizer("namespaces/ipc");
+    createContainerizer("filesystem/linux,namespaces/ipc");
 
   ASSERT_SOME(containerizer);