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:03 UTC

[mesos] branch master updated (9ec5e2b -> e452533)

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

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


    from 9ec5e2b  Fixed the broken PathTest.PathIteration on windows.
     new e855f1e  Moved cgroup path helpers to `paths.hpp`.
     new 81555e8  Fixed `cgroups::create` for nested cgroups.
     new 220cf10  Cgroups isolator: added support for nested cgroups during launch.
     new 969836b  Cgroups isolator: added support for nested cgroups during recovery.
     new 96291c0  Updated nested mesos containerizer tests to support `share_cgroups`.
     new e452533  Added a test `LaunchNestedShareCgroups`.

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/linux/cgroups.cpp                              |  26 ++-
 .../mesos/isolators/cgroups/cgroups.cpp            | 182 +++++++++++++--------
 .../mesos/isolators/cgroups/cgroups.hpp            |   2 +
 .../mesos/isolators/network/ports.cpp              |   5 +-
 src/slave/containerizer/mesos/linux_launcher.cpp   |  77 ++-------
 src/slave/containerizer/mesos/linux_launcher.hpp   |   6 -
 src/slave/containerizer/mesos/paths.cpp            |  60 +++++++
 src/slave/containerizer/mesos/paths.hpp            |  14 ++
 .../nested_mesos_containerizer_tests.cpp           | 138 ++++++++++++++--
 9 files changed, 359 insertions(+), 151 deletions(-)


[mesos] 04/06: Cgroups isolator: added support for nested cgroups during recovery.

Posted by ab...@apache.org.
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 969836bd653437fd131530520e8875ac55125bc2
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Tue Mar 3 14:57:34 2020 +0100

    Cgroups isolator: added support for nested cgroups during recovery.
    
    This patch enables recovery for nested cgroups and implements
    the detection of orphaned nested cgroups.
    
    Review: https://reviews.apache.org/r/71966/
---
 .../mesos/isolators/cgroups/cgroups.cpp            | 41 +++++++++++++++-------
 1 file changed, 29 insertions(+), 12 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
index 09feaf3..bf2a4d8 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
@@ -206,10 +206,17 @@ Future<Nothing> CgroupsIsolatorProcess::recover(
   // Recover active containers first.
   vector<Future<Nothing>> recovers;
   foreach (const ContainerState& state, states) {
-    // If we are a nested container, we do not need to recover
-    // anything since only top-level containers will have cgroups
+    // If we are a nested container with shared cgroups, we do not
+    // need to recover anything since its ancestor will have cgroups
     // created for them.
-    if (state.container_id().has_parent()) {
+    const bool shareCgroups =
+      (state.has_container_info() &&
+       state.container_info().has_linux_info() &&
+       state.container_info().linux_info().has_share_cgroups())
+        ? state.container_info().linux_info().share_cgroups()
+        : true;
+
+    if (state.container_id().has_parent() && shareCgroups) {
       continue;
     }
 
@@ -248,7 +255,6 @@ Future<Nothing> CgroupsIsolatorProcess::_recover(
   hashset<ContainerID> unknownOrphans;
 
   foreach (const string& hierarchy, subsystems.keys()) {
-    // TODO(jieyu): Use non-recursive version of `cgroups::get`.
     Try<vector<string>> cgroups = cgroups::get(
         hierarchy,
         flags.cgroups_root);
@@ -267,18 +273,28 @@ Future<Nothing> CgroupsIsolatorProcess::_recover(
         continue;
       }
 
-      ContainerID containerId;
-      containerId.set_value(Path(cgroup).basename());
+      // Need to parse the cgroup to see if it's one we created (i.e.,
+      // matches our separator structure) or one that someone else
+      // created (e.g., in the future we might have nested containers
+      // that are managed by something else rooted within the cgroup
+      // hierarchy).
+      Option<ContainerID> containerId =
+        containerizer::paths::parseCgroupPath(flags.cgroups_root, cgroup);
+
+      if (containerId.isNone()) {
+        LOG(INFO) << "Not recovering cgroup " << cgroup;
+        continue;
+      }
 
       // Skip containerId which already have been recovered.
-      if (infos.contains(containerId)) {
+      if (infos.contains(containerId.get())) {
         continue;
       }
 
-      if (orphans.contains(containerId)) {
-        knownOrphans.insert(containerId);
+      if (orphans.contains(containerId.get())) {
+        knownOrphans.insert(containerId.get());
       } else {
-        unknownOrphans.insert(containerId);
+        unknownOrphans.insert(containerId.get());
       }
     }
   }
@@ -335,7 +351,8 @@ Future<Nothing> CgroupsIsolatorProcess::__recover(
 Future<Nothing> CgroupsIsolatorProcess::___recover(
     const ContainerID& containerId)
 {
-  const string cgroup = path::join(flags.cgroups_root, containerId.value());
+  const string cgroup =
+    containerizer::paths::getCgroupPath(flags.cgroups_root, containerId);
 
   vector<Future<Nothing>> recovers;
   hashset<string> recoveredSubsystems;
@@ -397,7 +414,7 @@ Future<Nothing> CgroupsIsolatorProcess::____recover(
 
   infos[containerId] = Owned<Info>(new Info(
       containerId,
-      path::join(flags.cgroups_root, containerId.value())));
+      containerizer::paths::getCgroupPath(flags.cgroups_root, containerId)));
 
   infos[containerId]->subsystems = recoveredSubsystems;
 


[mesos] 06/06: Added a test `LaunchNestedShareCgroups`.

Posted by ab...@apache.org.
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 e452533a29baecd8ae748852dda2d4385d2270a6
Author: Andrei Budnik <ab...@apache.org>
AuthorDate: Fri Feb 28 16:15:47 2020 +0100

    Added a test `LaunchNestedShareCgroups`.
    
    Review: https://reviews.apache.org/r/72190
---
 .../nested_mesos_containerizer_tests.cpp           | 89 +++++++++++++++++++++-
 1 file changed, 88 insertions(+), 1 deletion(-)

diff --git a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
index 13c6d28..8aaf80a 100644
--- a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
@@ -53,7 +53,7 @@
 using mesos::internal::slave::Containerizer;
 using mesos::internal::slave::Fetcher;
 using mesos::internal::slave::MesosContainerizer;
-
+using mesos::internal::slave::containerizer::paths::getCgroupPath;
 using mesos::internal::slave::containerizer::paths::getContainerConfig;
 using mesos::internal::slave::containerizer::paths::getRuntimePath;
 using mesos::internal::slave::containerizer::paths::getSandboxPath;
@@ -274,6 +274,93 @@ TEST_P(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNested)
 }
 
 
+// This test verifies that a separate cgroup is created for a nested
+// container only if it does not share cgroups with its parent container.
+TEST_P(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNestedShareCgroups)
+{
+  const bool shareCgroups = GetParam();
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "linux";
+  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
+
+  Fetcher fetcher(flags);
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      false,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(id::UUID::random().toString());
+
+  Try<string> directory = environment->mkdtemp();
+  ASSERT_SOME(directory);
+
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
+      containerId,
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+          directory.get()),
+      map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Now launch nested container.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(id::UUID::random().toString());
+
+  launch = containerizer->launch(
+      nestedContainerId,
+      createNestedContainerConfig("cpus:0.1", createCommandInfo("sleep 1000")),
+      map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Check that a separate cgroup is created for a nested container only
+  // if `share_cgroups` field is set to false.
+  Result<string> cpuHierarchy = cgroups::hierarchy("cpu");
+  ASSERT_SOME(cpuHierarchy);
+
+  const string cgroup = getCgroupPath(flags.cgroups_root, nestedContainerId);
+
+  ASSERT_NE(shareCgroups, cgroups::exists(cpuHierarchy.get(), cgroup));
+
+  Future<Option<ContainerTermination>> nestedTermination =
+    containerizer->destroy(nestedContainerId);
+
+  AWAIT_READY(nestedTermination);
+  ASSERT_SOME(nestedTermination.get());
+  ASSERT_TRUE(nestedTermination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, nestedTermination.get()->status());
+
+  Future<Option<ContainerTermination>> termination =
+    containerizer->destroy(containerId);
+
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
+
+  // Check that the cgroups isolator cleaned up a nested cgroup
+  // for the nested container.
+  ASSERT_FALSE(cgroups::exists(cpuHierarchy.get(), cgroup));
+}
+
+
 // This test verifies that a debug container inherits the
 // environment of its parent even after agent failover.
 TEST_F(NestedMesosContainerizerTest,


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

Posted by ab...@apache.org.
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


[mesos] 05/06: Updated nested mesos containerizer tests to support `share_cgroups`.

Posted by ab...@apache.org.
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 96291c0e157e0d3c125b75868186dc4661986c96
Author: Andrei Budnik <ab...@apache.org>
AuthorDate: Tue Mar 3 14:53:25 2020 +0100

    Updated nested mesos containerizer tests to support `share_cgroups`.
    
    Parametrized some `NestedMesosContainerizerTest` tests on whether we
    are launching nested containers with `share_cgroups=false` or not.
    Previously, all nested containers shared cgroups with their parent by
    default. Now, since we've added support for nested containers with
    their own cgroups, we need to verify that a nested container with
    `share_cgroups=false` can be successfully launched and it does not
    lead to problems during the recovery of Mesos containerizer.
    
    Review: https://reviews.apache.org/r/72189
---
 .../nested_mesos_containerizer_tests.cpp           | 49 ++++++++++++++++++----
 1 file changed, 40 insertions(+), 9 deletions(-)

diff --git a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
index c6f96e6..13c6d28 100644
--- a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
@@ -87,7 +87,8 @@ namespace internal {
 namespace tests {
 
 class NestedMesosContainerizerTest
-  : public ContainerizerTest<slave::MesosContainerizer>
+  : public ContainerizerTest<slave::MesosContainerizer>,
+    public ::testing::WithParamInterface<bool>
 {
 protected:
   Try<SlaveState> createSlaveState(
@@ -135,9 +136,39 @@ protected:
 
     return slaveState;
   }
+
+  template <typename... Args>
+  mesos::slave::ContainerConfig createNestedContainerConfig(
+      const string& resources, Args... args) const
+  {
+    mesos::slave::ContainerConfig containerConfig =
+      createContainerConfig(std::forward<Args>(args)...);
+
+    const bool shareCgroups = GetParam();
+
+    ContainerInfo* container = containerConfig.mutable_container_info();
+    container->set_type(ContainerInfo::MESOS);
+    container->mutable_linux_info()->set_share_cgroups(shareCgroups);
+
+    if (!shareCgroups) {
+      containerConfig.mutable_resources()->CopyFrom(
+        Resources::parse(resources).get());
+    }
+
+    return containerConfig;
+  }
 };
 
 
+// Some nested containerizer tests are parameterized by the boolean
+// `shared_cgroups` flag that specifies whether cgroups are shared
+// between nested containers and their parent container.
+INSTANTIATE_TEST_CASE_P(
+    NestedContainerShareCgroups,
+    NestedMesosContainerizerTest,
+    ::testing::Values(true, false));
+
+
 TEST_F(NestedMesosContainerizerTest, NestedContainerID)
 {
   ContainerID id1;
@@ -173,7 +204,7 @@ TEST_F(NestedMesosContainerizerTest, NestedContainerID)
 }
 
 
-TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNested)
+TEST_P(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNested)
 {
   slave::Flags flags = CreateSlaveFlags();
   flags.launcher = "linux";
@@ -219,7 +250,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNested)
 
   launch = containerizer->launch(
       nestedContainerId,
-      createContainerConfig(createCommandInfo("exit 42")),
+      createNestedContainerConfig("cpus:0.1", createCommandInfo("exit 42")),
       map<string, string>(),
       None());
 
@@ -1600,7 +1631,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentSigterm)
 }
 
 
-TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNested)
+TEST_P(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNested)
 {
   slave::Flags flags = CreateSlaveFlags();
   flags.launcher = "linux";
@@ -1659,7 +1690,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNested)
 
   launch = containerizer->launch(
       nestedContainerId,
-      createContainerConfig(createCommandInfo("sleep 1000")),
+      createNestedContainerConfig("cpus:0.1", createCommandInfo("sleep 1000")),
       map<string, string>(),
       None());
 
@@ -1740,7 +1771,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNested)
 // This test verifies that the agent could recover if the agent
 // metadata is empty but container runtime dir is not cleaned
 // up. This is a regression test for MESOS-8416.
-TEST_F(NestedMesosContainerizerTest,
+TEST_P(NestedMesosContainerizerTest,
        ROOT_CGROUPS_RecoverNestedWithoutSlaveState)
 {
   slave::Flags flags = CreateSlaveFlags();
@@ -1798,7 +1829,7 @@ TEST_F(NestedMesosContainerizerTest,
 
   launch = containerizer->launch(
       nestedContainerId,
-      createContainerConfig(createCommandInfo("sleep 1000")),
+      createNestedContainerConfig("cpus:0.1", createCommandInfo("sleep 1000")),
       map<string, string>(),
       None());
 
@@ -1844,7 +1875,7 @@ TEST_F(NestedMesosContainerizerTest,
 }
 
 
-TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNestedWithoutConfig)
+TEST_P(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNestedWithoutConfig)
 {
   slave::Flags flags = CreateSlaveFlags();
   flags.launcher = "linux";
@@ -1903,7 +1934,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNestedWithoutConfig)
 
   launch = containerizer->launch(
       nestedContainerId,
-      createContainerConfig(createCommandInfo("sleep 1000")),
+      createNestedContainerConfig("cpus:0.1", createCommandInfo("sleep 1000")),
       map<string, string>(),
       None());
 


[mesos] 01/06: Moved cgroup path helpers to `paths.hpp`.

Posted by ab...@apache.org.
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 e855f1ee7fdcc812cf2bc48d198054a49c5320f2
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Tue Mar 3 14:57:09 2020 +0100

    Moved cgroup path helpers to `paths.hpp`.
    
    Review: https://reviews.apache.org/r/72121/
---
 .../mesos/isolators/network/ports.cpp              |  5 +-
 src/slave/containerizer/mesos/linux_launcher.cpp   | 77 ++++------------------
 src/slave/containerizer/mesos/linux_launcher.hpp   |  6 --
 src/slave/containerizer/mesos/paths.cpp            | 60 +++++++++++++++++
 src/slave/containerizer/mesos/paths.hpp            | 14 ++++
 5 files changed, 88 insertions(+), 74 deletions(-)

diff --git a/src/slave/containerizer/mesos/isolators/network/ports.cpp b/src/slave/containerizer/mesos/isolators/network/ports.cpp
index fec74d3..86d3053 100644
--- a/src/slave/containerizer/mesos/isolators/network/ports.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/ports.cpp
@@ -38,7 +38,7 @@
 
 #include "slave/constants.hpp"
 
-#include "slave/containerizer/mesos/linux_launcher.hpp"
+#include "slave/containerizer/mesos/paths.hpp"
 
 using std::list;
 using std::set;
@@ -94,7 +94,8 @@ collectContainerListeners(
 
   foreach (const ContainerID& containerId, containerIds) {
     // Reconstruct the cgroup path from the container ID.
-    string cgroup = LinuxLauncher::cgroup(cgroupsRoot, containerId);
+    string cgroup =
+      containerizer::paths::getCgroupPath(cgroupsRoot, containerId);
 
     VLOG(1) << "Checking processes for container " << containerId
             << " in cgroup " << cgroup;
diff --git a/src/slave/containerizer/mesos/linux_launcher.cpp b/src/slave/containerizer/mesos/linux_launcher.cpp
index c10092b..0c8c890 100644
--- a/src/slave/containerizer/mesos/linux_launcher.cpp
+++ b/src/slave/containerizer/mesos/linux_launcher.cpp
@@ -37,7 +37,6 @@
 
 #include "mesos/resources.hpp"
 
-#include "slave/containerizer/mesos/constants.hpp"
 #include "slave/containerizer/mesos/linux_launcher.hpp"
 #include "slave/containerizer/mesos/paths.hpp"
 
@@ -105,10 +104,6 @@ private:
 
   Future<Nothing> _destroy(const ContainerID& containerId);
 
-  // Helper for parsing the cgroup path to determine the container ID
-  // it belongs to.
-  Option<ContainerID> parse(const string& cgroup);
-
   static const string subsystem;
   const Flags flags;
   const string freezerHierarchy;
@@ -194,19 +189,6 @@ bool LinuxLauncher::available()
 }
 
 
-string LinuxLauncher::cgroup(
-    const string& cgroupsRoot,
-    const ContainerID& containerId)
-{
-  return path::join(
-      cgroupsRoot,
-      containerizer::paths::buildPath(
-          containerId,
-          CGROUP_SEPARATOR,
-          containerizer::paths::JOIN));
-}
-
-
 LinuxLauncher::LinuxLauncher(
     const Flags& flags,
     const string& freezerHierarchy,
@@ -326,7 +308,9 @@ Future<hashset<ContainerID>> LinuxLauncherProcess::recover(
     // created (e.g., in the future we might have nested containers
     // that are managed by something else rooted within the freezer
     // hierarchy).
-    Option<ContainerID> containerId = parse(cgroup);
+    Option<ContainerID> containerId =
+      containerizer::paths::parseCgroupPath(flags.cgroups_root, cgroup);
+
     if (containerId.isNone()) {
       LOG(INFO) << "Not recovering cgroup " << cgroup;
       continue;
@@ -510,7 +494,9 @@ Try<pid_t> LinuxLauncherProcess::fork(
   parentHooks.emplace_back(Subprocess::ParentHook([=](pid_t child) {
     return cgroups::isolate(
         freezerHierarchy,
-        LinuxLauncher::cgroup(this->flags.cgroups_root, containerId),
+        containerizer::paths::getCgroupPath(
+            this->flags.cgroups_root,
+            containerId),
         child);
   }));
 
@@ -519,7 +505,9 @@ Try<pid_t> LinuxLauncherProcess::fork(
     parentHooks.emplace_back(Subprocess::ParentHook([=](pid_t child) {
       return cgroups::isolate(
           systemdHierarchy.get(),
-          LinuxLauncher::cgroup(this->flags.cgroups_root, containerId),
+          containerizer::paths::getCgroupPath(
+              this->flags.cgroups_root,
+              containerId),
           child);
     }));
   }
@@ -591,7 +579,7 @@ Future<Nothing> LinuxLauncherProcess::destroy(const ContainerID& containerId)
   }
 
   const string cgroup =
-    LinuxLauncher::cgroup(flags.cgroups_root, container->id);
+    containerizer::paths::getCgroupPath(flags.cgroups_root, container->id);
 
   // We remove the container so that we don't attempt multiple
   // destroys simultaneously and no other functions will return
@@ -641,7 +629,7 @@ Future<Nothing> LinuxLauncherProcess::_destroy(const ContainerID& containerId)
   }
 
   const string cgroup =
-    LinuxLauncher::cgroup(flags.cgroups_root, containerId);
+    containerizer::paths::getCgroupPath(flags.cgroups_root, containerId);
 
   if (!cgroups::exists(systemdHierarchy.get(), cgroup)) {
     return Nothing();
@@ -675,49 +663,6 @@ Future<ContainerStatus> LinuxLauncherProcess::status(
   return status;
 }
 
-
-Option<ContainerID> LinuxLauncherProcess::parse(const string& cgroup)
-{
-  Option<ContainerID> current;
-
-  // Start not expecting to see a separator and adjust after each
-  // non-separator we see.
-  bool separator = false;
-
-  vector<string> tokens = strings::tokenize(
-      strings::remove(cgroup, flags.cgroups_root, strings::PREFIX),
-      stringify(os::PATH_SEPARATOR));
-
-  for (size_t i = 0; i < tokens.size(); i++) {
-    if (separator && tokens[i] == CGROUP_SEPARATOR) {
-      separator = false;
-
-      // If the cgroup has CGROUP_SEPARATOR as the last segment,
-      // should just ignore it because this cgroup belongs to us.
-      if (i == tokens.size() - 1) {
-        return None();
-      } else {
-        continue;
-      }
-    } else if (separator) {
-      return None();
-    } else {
-      separator = true;
-    }
-
-    ContainerID id;
-    id.set_value(tokens[i]);
-
-    if (current.isSome()) {
-      id.mutable_parent()->CopyFrom(current.get());
-    }
-
-    current = id;
-  }
-
-  return current;
-}
-
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {
diff --git a/src/slave/containerizer/mesos/linux_launcher.hpp b/src/slave/containerizer/mesos/linux_launcher.hpp
index 993a264..8d63411 100644
--- a/src/slave/containerizer/mesos/linux_launcher.hpp
+++ b/src/slave/containerizer/mesos/linux_launcher.hpp
@@ -37,12 +37,6 @@ public:
   // Returns 'true' if prerequisites for using LinuxLauncher are available.
   static bool available();
 
-  // Helper for determining the cgroup for a container (i.e., the path
-  // in a cgroup subsystem).
-  static std::string cgroup(
-      const std::string& cgroupsRoot,
-      const ContainerID& containerId);
-
   ~LinuxLauncher() override;
 
   process::Future<hashset<ContainerID>> recover(
diff --git a/src/slave/containerizer/mesos/paths.cpp b/src/slave/containerizer/mesos/paths.cpp
index b028795..8230028 100644
--- a/src/slave/containerizer/mesos/paths.cpp
+++ b/src/slave/containerizer/mesos/paths.cpp
@@ -22,7 +22,9 @@
 #include "common/protobuf_utils.hpp"
 #include "common/resources_utils.hpp"
 
+#include "slave/containerizer/mesos/constants.hpp"
 #include "slave/containerizer/mesos/paths.hpp"
+
 #include "slave/state.hpp"
 
 #ifndef __WINDOWS__
@@ -594,6 +596,64 @@ Try<string> getParentShmPath(
   return parentShmPath;
 }
 
+
+string getCgroupPath(
+    const string& cgroupsRoot,
+    const ContainerID& containerId)
+{
+  return path::join(
+      cgroupsRoot,
+      containerizer::paths::buildPath(
+          containerId,
+          CGROUP_SEPARATOR,
+          containerizer::paths::JOIN));
+}
+
+
+Option<ContainerID> parseCgroupPath(
+    const string& cgroupsRoot,
+    const string& cgroup)
+{
+  Option<ContainerID> current;
+
+  // Start not expecting to see a separator and adjust after each
+  // non-separator we see.
+  bool separator = false;
+
+  vector<string> tokens = strings::tokenize(
+      strings::remove(cgroup, cgroupsRoot, strings::PREFIX),
+      stringify(os::PATH_SEPARATOR));
+
+  for (size_t i = 0; i < tokens.size(); i++) {
+    if (separator && tokens[i] == CGROUP_SEPARATOR) {
+      separator = false;
+
+      // If the cgroup has CGROUP_SEPARATOR as the last segment,
+      // should just ignore it because this cgroup belongs to us.
+      if (i == tokens.size() - 1) {
+        return None();
+      } else {
+        continue;
+      }
+    } else if (separator) {
+      return None();
+    } else {
+      separator = true;
+    }
+
+    ContainerID id;
+    id.set_value(tokens[i]);
+
+    if (current.isSome()) {
+      id.mutable_parent()->CopyFrom(current.get());
+    }
+
+    current = id;
+  }
+
+  return current;
+}
+
 } // namespace paths {
 } // namespace containerizer {
 } // namespace slave {
diff --git a/src/slave/containerizer/mesos/paths.hpp b/src/slave/containerizer/mesos/paths.hpp
index e35d380..5f188e5 100644
--- a/src/slave/containerizer/mesos/paths.hpp
+++ b/src/slave/containerizer/mesos/paths.hpp
@@ -272,6 +272,20 @@ Try<std::string> getParentShmPath(
     const std::string runtimeDir,
     const ContainerID& containerId);
 
+
+// Helper for determining the cgroup for a container (i.e., the path
+// in a cgroup subsystem).
+std::string getCgroupPath(
+    const std::string& cgroupsRoot,
+    const ContainerID& containerId);
+
+
+// Helper for parsing the cgroup path to determine the container ID
+// it belongs to.
+Option<ContainerID> parseCgroupPath(
+    const std::string& cgroupsRoot,
+    const std::string& cgroup);
+
 } // namespace paths {
 } // namespace containerizer {
 } // namespace slave {


[mesos] 02/06: Fixed `cgroups::create` for nested cgroups.

Posted by ab...@apache.org.
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 81555e8d73507afcc28bc6ee92c2ef456adbaf87
Author: Andrei Budnik <ab...@mesosphere.com>
AuthorDate: Tue Mar 3 14:57:49 2020 +0100

    Fixed `cgroups::create` for nested cgroups.
    
    This patch modifies `cgroups::create` function to call
    `cloneCpusetCpusMems` for all absent nested cgroups along
    the path to a cgroup that is accepted as an argument to this function.
    For instance, if `cgroups::create` is called to create three
    non-existent cgroups recursively for the path `/a/b/c`, then
    `cloneCpusetCpusMems` is called to clone both `cpuset.cpus` and
    `cpuset.mems` for `/a` from its parent, then `/a/b` from `/a`,
    and so on down the path.
    
    Review: https://reviews.apache.org/r/72122/
---
 src/linux/cgroups.cpp | 26 ++++++++++++++++++++++++--
 1 file changed, 24 insertions(+), 2 deletions(-)

diff --git a/src/linux/cgroups.cpp b/src/linux/cgroups.cpp
index 73646c9..2234f0d 100644
--- a/src/linux/cgroups.cpp
+++ b/src/linux/cgroups.cpp
@@ -646,7 +646,19 @@ Try<Nothing> create(
     const string& cgroup,
     bool recursive)
 {
+  vector<string> missingCgroups;
+  string currentCgroup;
+  Path cgroupPath(cgroup);
+  for (auto it = cgroupPath.begin(); it != cgroupPath.end(); ++it) {
+    currentCgroup = path::join(currentCgroup, *it);
+    if (!missingCgroups.empty() ||
+        !os::exists(path::join(hierarchy, currentCgroup))) {
+      missingCgroups.push_back(currentCgroup);
+    }
+  }
+
   string path = path::join(hierarchy, cgroup);
+
   Try<Nothing> mkdir = os::mkdir(path, recursive);
   if (mkdir.isError()) {
     return Error(
@@ -661,8 +673,18 @@ Try<Nothing> create(
         "Failed to determine if hierarchy '" + hierarchy +
         "' has the 'cpuset' subsystem attached: " + attached.error());
   } else if (attached->count("cpuset") > 0) {
-    string parent = Path(path::join("/", cgroup)).dirname();
-    return internal::cloneCpusetCpusMems(hierarchy, parent, cgroup);
+    foreach (const string& cgroup, missingCgroups) {
+      string parent = Path(cgroup).dirname();
+
+      Try<Nothing> clone =
+        internal::cloneCpusetCpusMems(hierarchy, parent, cgroup);
+
+      if (clone.isError()) {
+        return Error(
+            "Failed to clone `cpuset.cpus` and `cpuset.mems` from '" +
+            parent + "' to '" + cgroup + "': " + clone.error());
+      }
+    }
   }
 
   return Nothing();