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 2018/06/25 21:43:33 UTC

[2/5] mesos git commit: Added container-specific cgroup FS mounts.

Added container-specific cgroup FS mounts.

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


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

Branch: refs/heads/master
Commit: ae90054f6d8fcd7387bac26a4caa458e4ddb597c
Parents: eb61bbc
Author: Qian Zhang <zh...@gmail.com>
Authored: Mon Jun 25 14:39:36 2018 -0700
Committer: Gilbert Song <so...@gmail.com>
Committed: Mon Jun 25 14:39:36 2018 -0700

----------------------------------------------------------------------
 src/launcher/executor.cpp                       | 41 +++++++-
 src/linux/fs.cpp                                |  7 ++
 .../mesos/isolators/cgroups/cgroups.cpp         | 99 +++++++++++++++++++-
 .../mesos/isolators/cgroups/cgroups.hpp         | 12 ++-
 4 files changed, 147 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ae90054f/src/launcher/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.cpp b/src/launcher/executor.cpp
index 541ca5b..9fb43c0 100644
--- a/src/launcher/executor.cpp
+++ b/src/launcher/executor.cpp
@@ -111,6 +111,8 @@ using mesos::Environment;
 using mesos::executor::Call;
 using mesos::executor::Event;
 
+using mesos::slave::ContainerLaunchInfo;
+
 using mesos::v1::executor::Mesos;
 using mesos::v1::executor::MesosBase;
 using mesos::v1::executor::V0ToV1Adapter;
@@ -131,6 +133,7 @@ public:
       const Option<Environment>& _taskEnvironment,
       const Option<CapabilityInfo>& _effectiveCapabilities,
       const Option<CapabilityInfo>& _boundingCapabilities,
+      const Option<ContainerLaunchInfo>& _taskLaunchInfo,
       const FrameworkID& _frameworkId,
       const ExecutorID& _executorId,
       const Duration& _shutdownGracePeriod)
@@ -155,6 +158,7 @@ public:
       taskEnvironment(_taskEnvironment),
       effectiveCapabilities(_effectiveCapabilities),
       boundingCapabilities(_boundingCapabilities),
+      taskLaunchInfo(_taskLaunchInfo),
       frameworkId(_frameworkId),
       executorId(_executorId),
       lastTaskStatus(None()) {}
@@ -406,12 +410,13 @@ protected:
       const Option<string>& sandboxDirectory,
       const Option<string>& workingDirectory,
       const Option<CapabilityInfo>& effectiveCapabilities,
-      const Option<CapabilityInfo>& boundingCapabilities)
+      const Option<CapabilityInfo>& boundingCapabilities,
+      const Option<ContainerLaunchInfo>& taskLaunchInfo)
   {
     // Prepare the flags to pass to the launch process.
     slave::MesosContainerizerLaunch::Flags launchFlags;
 
-    ::mesos::slave::ContainerLaunchInfo launchInfo;
+    ContainerLaunchInfo launchInfo;
     launchInfo.mutable_command()->CopyFrom(command);
 
 #ifndef __WINDOWS__
@@ -473,6 +478,15 @@ protected:
           boundingCapabilities.get());
     }
 
+    if (taskLaunchInfo.isSome()) {
+      launchInfo.mutable_mounts()->CopyFrom(taskLaunchInfo->mounts());
+      launchInfo.mutable_pre_exec_commands()->CopyFrom(
+          taskLaunchInfo->pre_exec_commands());
+
+      launchInfo.mutable_clone_namespaces()->CopyFrom(
+          taskLaunchInfo->clone_namespaces());
+    }
+
     launchFlags.launch_info = JSON::protobuf(launchInfo);
 
     // TODO(tillt): Consider using a flag allowing / disallowing the
@@ -673,7 +687,8 @@ protected:
         sandboxDirectory,
         workingDirectory,
         effectiveCapabilities,
-        boundingCapabilities);
+        boundingCapabilities,
+        taskLaunchInfo);
 
     LOG(INFO) << "Forked command at " << pid.get();
 
@@ -1203,6 +1218,7 @@ private:
   Option<Environment> taskEnvironment;
   Option<CapabilityInfo> effectiveCapabilities;
   Option<CapabilityInfo> boundingCapabilities;
+  Option<ContainerLaunchInfo> taskLaunchInfo;
   const FrameworkID frameworkId;
   const ExecutorID executorId;
   Owned<MesosBase> mesos;
@@ -1262,6 +1278,10 @@ public:
         "bounding_capabilities",
         "The bounding set of capabilities the command can use.");
 
+    add(&Flags::task_launch_info,
+        "task_launch_info",
+        "The launch info to run the task.");
+
     add(&Flags::launcher_dir,
         "launcher_dir",
         "Directory path of Mesos binaries.",
@@ -1279,6 +1299,7 @@ public:
   Option<Environment> task_environment;
   Option<mesos::CapabilityInfo> effective_capabilities;
   Option<mesos::CapabilityInfo> bounding_capabilities;
+  Option<JSON::Object> task_launch_info;
   string launcher_dir;
 };
 
@@ -1343,6 +1364,19 @@ int main(int argc, char** argv)
     shutdownGracePeriod = parse.get();
   }
 
+  Option<ContainerLaunchInfo> task_launch_info;
+  if (flags.task_launch_info.isSome()) {
+    Try<ContainerLaunchInfo> parse =
+      protobuf::parse<ContainerLaunchInfo>(flags.task_launch_info.get());
+
+    if (parse.isError()) {
+      EXIT(EXIT_FAILURE)
+        << "Failed to parse task launch info: " << parse.error();
+    }
+
+    task_launch_info = parse.get();
+  }
+
   process::initialize();
 
   Owned<mesos::internal::CommandExecutor> executor(
@@ -1356,6 +1390,7 @@ int main(int argc, char** argv)
           flags.task_environment,
           flags.effective_capabilities,
           flags.bounding_capabilities,
+          task_launch_info,
           frameworkId,
           executorId,
           shutdownGracePeriod));

http://git-wip-us.apache.org/repos/asf/mesos/blob/ae90054f/src/linux/fs.cpp
----------------------------------------------------------------------
diff --git a/src/linux/fs.cpp b/src/linux/fs.cpp
index 6b38b4a..9055ef4 100644
--- a/src/linux/fs.cpp
+++ b/src/linux/fs.cpp
@@ -781,6 +781,13 @@ Try<Nothing> mountSpecialFilesystems(const string& root)
     },
     {
       "tmpfs",
+      "/sys/fs/cgroup",
+      "tmpfs",
+      "mode=755",
+      MS_NOSUID | MS_NOEXEC | MS_NODEV
+    },
+    {
+      "tmpfs",
       "/dev",
       "tmpfs",
       "mode=755",

http://git-wip-us.apache.org/repos/asf/mesos/blob/ae90054f/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
index 6d14672..316d2c0 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
@@ -32,6 +32,8 @@
 #include "common/protobuf_utils.hpp"
 
 #include "linux/cgroups.hpp"
+#include "linux/fs.hpp"
+#include "linux/ns.hpp"
 
 #include "slave/containerizer/mesos/isolators/cgroups/cgroups.hpp"
 #include "slave/containerizer/mesos/isolators/cgroups/constants.hpp"
@@ -39,6 +41,7 @@
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
 using mesos::slave::ContainerLimitation;
+using mesos::slave::ContainerMountInfo;
 using mesos::slave::ContainerState;
 using mesos::slave::Isolator;
 
@@ -416,11 +419,11 @@ Future<Option<ContainerLaunchInfo>> CgroupsIsolatorProcess::prepare(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig)
 {
-  // If we are a nested container, we do not need to prepare
-  // anything since only top-level containers should have cgroups
-  // created for them.
+  // Only prepare cgroups for top-level containers. Nested container
+  // will inherit cgroups from its root container, so here we just
+  // need to do the container-specific cgroups mounts.
   if (containerId.has_parent()) {
-    return None();
+    return __prepare(containerId, containerConfig);
   }
 
   if (infos.contains(containerId)) {
@@ -550,7 +553,93 @@ Future<Option<ContainerLaunchInfo>> CgroupsIsolatorProcess::_prepare(
   }
 
   return update(containerId, containerConfig.resources())
-    .then([]() { return Option<ContainerLaunchInfo>::none(); });
+    .then(defer(
+        PID<CgroupsIsolatorProcess>(this),
+        &CgroupsIsolatorProcess::__prepare,
+        containerId,
+        containerConfig));
+}
+
+
+Future<Option<ContainerLaunchInfo>> CgroupsIsolatorProcess::__prepare(
+    const ContainerID& containerId,
+    const ContainerConfig& containerConfig)
+{
+  // We will do container-specific cgroups mounts
+  // only for the container with rootfs.
+  if (!containerConfig.has_rootfs()) {
+    return None();
+  }
+
+  const ContainerID rootContainerId = protobuf::getRootContainerId(containerId);
+
+  CHECK(infos.contains(rootContainerId));
+
+  ContainerLaunchInfo launchInfo;
+  launchInfo.add_clone_namespaces(CLONE_NEWNS);
+
+  // For the comounted subsystems (e.g., cpu & cpuacct, net_cls & net_prio),
+  // we need to create a symbolic link for each of them to the mount point.
+  // E.g.: ln -s /sys/fs/cgroup/cpu,cpuacct /sys/fs/cgroup/cpu
+  //       ln -s /sys/fs/cgroup/cpu,cpuacct /sys/fs/cgroup/cpuacct
+  foreach (const string& hierarchy, subsystems.keys()) {
+    if (subsystems.get(hierarchy).size() > 1) {
+      foreach (const Owned<Subsystem>& subsystem, subsystems.get(hierarchy)) {
+        CommandInfo* command = launchInfo.add_pre_exec_commands();
+        command->set_shell(false);
+        command->set_value("ln");
+        command->add_arguments("ln");
+        command->add_arguments("-s");
+        command->add_arguments(
+            path::join("/sys/fs/cgroup", Path(hierarchy).basename()));
+
+        command->add_arguments(path::join(
+            containerConfig.rootfs(),
+            "/sys/fs/cgroup",
+            subsystem->name()));
+      }
+    }
+  }
+
+  // For the subsystem loaded by this isolator, do the container-specific
+  // cgroups mount, e.g.:
+  //   mount --bind /sys/fs/cgroup/memory/mesos/<containerId> /sys/fs/cgroup/memory // NOLINT(whitespace/line_length)
+  foreach (const string& hierarchy, subsystems.keys()) {
+    ContainerMountInfo* mount = launchInfo.add_mounts();
+    mount->set_source(path::join(hierarchy, infos[rootContainerId]->cgroup));
+
+    mount->set_target(path::join(
+        containerConfig.rootfs(),
+        "/sys/fs/cgroup",
+        Path(hierarchy).basename()));
+
+    mount->set_flags(MS_BIND | MS_REC);
+  }
+
+  // TODO(qianzhang): This is a hack to pass the container-specific cgroups
+  // mounts and the symbolic links to the command executor to do for the
+  // command task. The reasons that we do it in this way are:
+  //   1. We need to ensure the container-specific cgroups mounts are done
+  //      only in the command task's mount namespace but not in the command
+  //      executor's mount namespace.
+  //   2. Even it's acceptable to do the container-specific cgroups mounts
+  //      in the command executor's mount namespace and the command task
+  //      inherit them from there (i.e., here we just return `launchInfo`
+  //      rather than passing it via `--task_launch_info`), the container
+  //      specific cgroups mounts will be hidden by the `sysfs` mounts done in
+  //      `mountSpecialFilesystems()` when the command executor launches the
+  //      command task.
+  if (containerConfig.has_task_info()) {
+    ContainerLaunchInfo _launchInfo;
+
+    _launchInfo.mutable_command()->add_arguments(
+        "--task_launch_info=" +
+        stringify(JSON::protobuf(launchInfo)));
+
+    return _launchInfo;
+  }
+
+  return launchInfo;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/ae90054f/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
index 81c9343..480aedf 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
@@ -114,18 +114,22 @@ private:
       const std::vector<process::Future<Nothing>>& futures);
 
   process::Future<Nothing> ___recover(
-    const ContainerID& containerId);
+      const ContainerID& containerId);
 
   process::Future<Nothing> ____recover(
-    const ContainerID& containerId,
-    const hashset<std::string>& recoveredSubsystems,
-    const std::vector<process::Future<Nothing>>& futures);
+      const ContainerID& containerId,
+      const hashset<std::string>& recoveredSubsystems,
+      const std::vector<process::Future<Nothing>>& futures);
 
   process::Future<Option<mesos::slave::ContainerLaunchInfo>> _prepare(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::vector<process::Future<Nothing>>& futures);
 
+  process::Future<Option<mesos::slave::ContainerLaunchInfo>> __prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig);
+
   process::Future<Nothing> _isolate(
       const std::vector<process::Future<Nothing>>& futures);