You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ab...@apache.org on 2020/03/13 10:00:06 UTC

[mesos] 03/06: Cgroups isolator: added support for nested cgroups during launch.

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

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

commit 220cf1049d33170650059cc3fcd64657b1ea0407
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Tue Mar 3 14:57:24 2020 +0100

    Cgroups isolator: added support for nested cgroups during launch.
    
    This patch adds support for nested cgroups for nested containers.
    Nested cgroups are created only for a nested container with explicitly
    disabled `share_cgroups` flag. The cgroups isolator stores info about
    nested cgroups in the `infos` class variable, which is used to
    determine whether a nested container has its nested cgroup.
    
    Review: https://reviews.apache.org/r/71965/
---
 .../mesos/isolators/cgroups/cgroups.cpp            | 141 +++++++++++++--------
 .../mesos/isolators/cgroups/cgroups.hpp            |   2 +
 2 files changed, 90 insertions(+), 53 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
index b12b73d..09feaf3 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
@@ -44,6 +44,7 @@
 
 using mesos::internal::protobuf::slave::containerSymlinkOperation;
 
+using mesos::slave::ContainerClass;
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
 using mesos::slave::ContainerLimitation;
@@ -408,10 +409,18 @@ Future<Option<ContainerLaunchInfo>> CgroupsIsolatorProcess::prepare(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig)
 {
-  // Only prepare cgroups for top-level containers. Nested container
-  // will inherit cgroups from its root container, so here we just
+  // If the nested container shares cgroups with its parent container,
+  // we don't need to prepare cgroups. In this case, the nested container
+  // will inherit cgroups from its ancestor, so here we just
   // need to do the container-specific cgroups mounts.
-  if (containerId.has_parent()) {
+  const bool shareCgroups =
+    (containerConfig.has_container_info() &&
+     containerConfig.container_info().has_linux_info() &&
+     containerConfig.container_info().linux_info().has_share_cgroups())
+      ? containerConfig.container_info().linux_info().share_cgroups()
+      : true;
+
+  if (containerId.has_parent() && shareCgroups) {
     return __prepare(containerId, containerConfig);
   }
 
@@ -419,11 +428,16 @@ Future<Option<ContainerLaunchInfo>> CgroupsIsolatorProcess::prepare(
     return Failure("Container has already been prepared");
   }
 
+  CHECK(containerConfig.container_class() != ContainerClass::DEBUG);
+
+  CHECK(!containerId.has_parent() || !containerId.parent().has_parent())
+    << "2nd level or greater nested cgroups are not supported";
+
   // We save 'Info' into 'infos' first so that even if 'prepare'
   // fails, we can properly cleanup the *side effects* created below.
   infos[containerId] = Owned<Info>(new Info(
       containerId,
-      path::join(flags.cgroups_root, containerId.value())));
+      containerizer::paths::getCgroupPath(flags.cgroups_root, containerId)));
 
   vector<Future<Nothing>> prepares;
 
@@ -456,7 +470,8 @@ Future<Option<ContainerLaunchInfo>> CgroupsIsolatorProcess::prepare(
           infos[containerId]->cgroup));
     }
 
-    // Chown the cgroup so the executor can create nested cgroups. Do
+    // Chown the cgroup so the executor or a nested container whose
+    // `share_cgroups` is false can create nested cgroups. Do
     // not recurse so the control files are still owned by the slave
     // user and thus cannot be changed by the executor.
     //
@@ -553,10 +568,6 @@ Future<Option<ContainerLaunchInfo>> CgroupsIsolatorProcess::__prepare(
     return None();
   }
 
-  const ContainerID rootContainerId = protobuf::getRootContainerId(containerId);
-
-  CHECK(infos.contains(rootContainerId));
-
   ContainerLaunchInfo launchInfo;
   launchInfo.add_clone_namespaces(CLONE_NEWNS);
 
@@ -580,9 +591,16 @@ Future<Option<ContainerLaunchInfo>> CgroupsIsolatorProcess::__prepare(
   // 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)
+  Owned<Info> info = findCgroupInfo(containerId);
+  if (!info.get()) {
+    return Failure(
+        "Failed to find cgroup for container " +
+        stringify(containerId));
+  }
+
   foreach (const string& hierarchy, subsystems.keys()) {
     *launchInfo.add_mounts() = protobuf::slave::createContainerMount(
-        path::join(hierarchy, infos[rootContainerId]->cgroup),
+        path::join(hierarchy, info->cgroup),
         path::join(
             containerConfig.rootfs(),
             "/sys/fs/cgroup",
@@ -664,11 +682,11 @@ Future<Nothing> CgroupsIsolatorProcess::isolate(
     pid_t pid)
 {
   // We currently can't call `subsystem->isolate()` on nested
-  // containers, because we don't call `prepare()`, `recover()`, or
-  // `cleanup()` on them either. If we were to call `isolate()` on
-  // them, the call would likely fail because the subsystem doesn't
-  // know about the container. This is currently OK because the only
-  // cgroup isolator that even implements `isolate()` is the
+  // containers with shared cgroups, because we don't call `prepare()`,
+  // `recover()`, or `cleanup()` on them either. If we were to call
+  // `isolate()` on them, the call would likely fail because the subsystem
+  // doesn't know about the container. This is currently OK because
+  // the only cgroup isolator that even implements `isolate()` is the
   // `NetClsSubsystem` and it doesn't do anything with the `pid`
   // passed in.
   //
@@ -678,7 +696,7 @@ Future<Nothing> CgroupsIsolatorProcess::isolate(
 
   vector<Future<Nothing>> isolates;
 
-  if (!containerId.has_parent()) {
+  if (infos.contains(containerId)) {
     foreachvalue (const Owned<Subsystem>& subsystem, subsystems) {
       isolates.push_back(subsystem->isolate(
           containerId,
@@ -722,14 +740,16 @@ Future<Nothing> CgroupsIsolatorProcess::_isolate(
         strings::join(";", errors));
   }
 
-  // If we are a nested container, we inherit the cgroup from our parent.
-  const ContainerID rootContainerId = protobuf::getRootContainerId(containerId);
-
-  if (!infos.contains(rootContainerId)) {
-    return Failure("Failed to isolate the container: Unknown root container");
+  // If we are a nested container with shared cgroups,
+  // we inherit the cgroup from our parent.
+  Owned<Info> info = findCgroupInfo(containerId);
+  if (!info.get()) {
+    return Failure(
+        "Failed to find cgroup for container " +
+        stringify(containerId));
   }
 
-  const string& cgroup = infos[rootContainerId]->cgroup;
+  const string& cgroup = info->cgroup;
 
   // TODO(haosdent): Use foreachkey once MESOS-5037 is resolved.
   foreach (const string& hierarchy, subsystems.keys()) {
@@ -737,7 +757,8 @@ Future<Nothing> CgroupsIsolatorProcess::_isolate(
     // upgrade, the newly added cgroup subsystems do not
     // exist on old container's cgroup hierarchy. So skip
     // assigning the pid to this cgroup subsystem.
-    if (containerId.has_parent() && !cgroups::exists(hierarchy, cgroup)) {
+    if (containerId.has_parent() && containerId != info->containerId &&
+        !cgroups::exists(hierarchy, cgroup)) {
       LOG(INFO) << "Skipping assigning pid " << stringify(pid)
                 << " to cgroup at '" << path::join(hierarchy, cgroup)
                 << "' for container " << containerId
@@ -770,15 +791,21 @@ Future<Nothing> CgroupsIsolatorProcess::_isolate(
 Future<ContainerLimitation> CgroupsIsolatorProcess::watch(
     const ContainerID& containerId)
 {
-  // Since we do not maintain cgroups for nested containers
-  // directly, we simply return a pending future here, indicating
+  // Since we do not maintain cgroups for nested containers with shared
+  // cgroups directly, we simply return a pending future here, indicating
   // that the limit for the nested container will never be reached.
-  if (containerId.has_parent()) {
-    return Future<ContainerLimitation>();
-  }
-
   if (!infos.contains(containerId)) {
-    return Failure("Unknown container");
+    // TODO(abudnik): We should return a failure for the nested container
+    // whose `share_cgroups` is false but `infos` does not contain it.
+    // This may happen due to a bug in our code.
+    //
+    // NOTE: We return a pending future for a non-existent nested container
+    // whose ancestor is known to the cgroups isolator.
+    if (findCgroupInfo(containerId).get()) {
+      return Future<ContainerLimitation>();
+    } else {
+      return Failure("Unknown container");
+    }
   }
 
   foreachvalue (const Owned<Subsystem>& subsystem, subsystems) {
@@ -814,10 +841,6 @@ Future<Nothing> CgroupsIsolatorProcess::update(
     const ContainerID& containerId,
     const Resources& resources)
 {
-  if (containerId.has_parent()) {
-    return Failure("Not supported for nested containers");
-  }
-
   if (!infos.contains(containerId)) {
     return Failure("Unknown container");
   }
@@ -865,10 +888,6 @@ Future<Nothing> CgroupsIsolatorProcess::_update(
 Future<ResourceStatistics> CgroupsIsolatorProcess::usage(
     const ContainerID& containerId)
 {
-  if (containerId.has_parent()) {
-    return Failure("Not supported for nested containers");
-  }
-
   if (!infos.contains(containerId)) {
     return Failure("Unknown container");
   }
@@ -905,15 +924,17 @@ Future<ResourceStatistics> CgroupsIsolatorProcess::usage(
 Future<ContainerStatus> CgroupsIsolatorProcess::status(
     const ContainerID& containerId)
 {
-  // TODO(jieyu): Currently, all nested containers share the same
-  // cgroup as their parent container. Revisit this once this is no
-  // long true.
-  if (containerId.has_parent()) {
-    return status(containerId.parent());
-  }
-
+  // If we are a nested container unknown to the isolator,
+  // we try to find the status of its ancestor.
   if (!infos.contains(containerId)) {
-    return Failure("Unknown container");
+    // TODO(abudnik): We should return a failure for the nested container
+    // whose `share_cgroups` is false but `infos` does not contain it.
+    // This may happen due to a bug in our code.
+    if (containerId.has_parent()) {
+      return status(containerId.parent());
+    } else {
+      return Failure("Unknown container");
+    }
   }
 
   vector<Future<ContainerStatus>> statuses;
@@ -947,15 +968,8 @@ Future<ContainerStatus> CgroupsIsolatorProcess::status(
 Future<Nothing> CgroupsIsolatorProcess::cleanup(
     const ContainerID& containerId)
 {
-  // If we are a nested container, we do not need to clean anything up
-  // since only top-level containers should have cgroups created for them.
-  if (containerId.has_parent()) {
-    return Nothing();
-  }
-
   if (!infos.contains(containerId)) {
     VLOG(1) << "Ignoring cleanup request for unknown container " << containerId;
-
     return Nothing();
   }
 
@@ -1049,6 +1063,27 @@ Future<Nothing> CgroupsIsolatorProcess::__cleanup(
   return Nothing();
 }
 
+
+Owned<CgroupsIsolatorProcess::Info> CgroupsIsolatorProcess::findCgroupInfo(
+    const ContainerID& containerId) const
+{
+  Option<ContainerID> current = containerId;
+  while (current.isSome()) {
+    Option<Owned<Info>> info = infos.get(current.get());
+    if (info.isSome()) {
+      return info.get();
+    }
+
+    if (!current->has_parent()) {
+      break;
+    }
+
+    current = current->parent();
+  }
+
+  return nullptr;
+}
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
index 4bd3d6d..8718b7a 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
@@ -149,6 +149,8 @@ private:
       const ContainerID& containerId,
       const std::vector<process::Future<Nothing>>& futures);
 
+  process::Owned<Info> findCgroupInfo(const ContainerID& containerId) const;
+
   const Flags flags;
 
   // We map hierarchy path and `Subsystem` in subsystems. Same hierarchy may