You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2012/10/31 17:39:25 UTC

svn commit: r1404238 - in /incubator/mesos/trunk/src/slave: cgroups_isolation_module.cpp cgroups_isolation_module.hpp

Author: benh
Date: Wed Oct 31 16:39:24 2012
New Revision: 1404238

URL: http://svn.apache.org/viewvc?rev=1404238&view=rev
Log:
Organized cgroups used by the slave in a hierarchical structure.

From: Jie Yu <yu...@gmail.com>
Review: https://reviews.apache.org/r/6501

Modified:
    incubator/mesos/trunk/src/slave/cgroups_isolation_module.cpp
    incubator/mesos/trunk/src/slave/cgroups_isolation_module.hpp

Modified: incubator/mesos/trunk/src/slave/cgroups_isolation_module.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/cgroups_isolation_module.cpp?rev=1404238&r1=1404237&r2=1404238&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/cgroups_isolation_module.cpp (original)
+++ incubator/mesos/trunk/src/slave/cgroups_isolation_module.cpp Wed Oct 31 16:39:24 2012
@@ -21,11 +21,15 @@
 
 #include <sys/types.h>
 
+#include <set>
 #include <sstream>
+#include <string>
+#include <vector>
 
 #include <process/defer.hpp>
 #include <process/dispatch.hpp>
 
+#include <stout/fatal.hpp>
 #include <stout/foreach.hpp>
 #include <stout/lambda.hpp>
 #include <stout/option.hpp>
@@ -40,8 +44,12 @@
 
 #include "slave/cgroups_isolation_module.hpp"
 
-using namespace process;
+using process::defer;
+using process::Future;
 
+using std::set;
+using std::string;
+using std::vector;
 
 namespace mesos {
 namespace internal {
@@ -83,14 +91,14 @@ void CgroupsIsolationModule::initialize(
   local = _local;
   slave = _slave;
 
-  // Make sure that we have root permission.
-  if (os::user() != "root") {
-    LOG(FATAL) << "Cgroups isolation module needs root permission";
-  }
-
   // Make sure that cgroups is enabled by the kernel.
   if (!cgroups::enabled()) {
-    LOG(FATAL) << "Cgroups is not supported by the kernel";
+    fatal("No cgroups support detected on this kernel");
+  }
+
+  // Make sure that we have root permissions.
+  if (geteuid() != 0) {
+    fatal("The cgroups isolation module requires root permissions");
   }
 
   // Configure cgroups hierarchy root path.
@@ -99,25 +107,25 @@ void CgroupsIsolationModule::initialize(
   LOG(INFO) << "Using " << hierarchy << " as cgroups hierarchy root";
 
   // Configure required/optional subsystems.
-  hashset<std::string> requiredSubsystems;
-  hashset<std::string> optionalSubsystems;
+  hashset<string> requiredSubsystems;
+  hashset<string> optionalSubsystems;
 
   requiredSubsystems.insert("cpu");
-  requiredSubsystems.insert("cpuset");
   requiredSubsystems.insert("memory");
   requiredSubsystems.insert("freezer");
 
+  optionalSubsystems.insert("cpuset");
   optionalSubsystems.insert("blkio");
 
   // Probe cgroups subsystems.
-  hashset<std::string> enabledSubsystems;
-  hashset<std::string> busySubsystems;
+  hashset<string> enabledSubsystems;
+  hashset<string> busySubsystems;
 
-  Try<std::set<std::string> > enabled = cgroups::subsystems();
+  Try<set<string> > enabled = cgroups::subsystems();
   if (enabled.isError()) {
     LOG(FATAL) << "Failed to probe cgroups subsystems: " << enabled.error();
   } else {
-    foreach (const std::string& name, enabled.get()) {
+    foreach (const string& name, enabled.get()) {
       enabledSubsystems.insert(name);
 
       Try<bool> busy = cgroups::busy(name);
@@ -132,7 +140,7 @@ void CgroupsIsolationModule::initialize(
   }
 
   // Make sure that all the required subsystems are enabled by the kernel.
-  foreach (const std::string& name, requiredSubsystems) {
+  foreach (const string& name, requiredSubsystems) {
     if (!enabledSubsystems.contains(name)) {
       LOG(FATAL) << "Required subsystem " << name
                  << " is not enabled by the kernel";
@@ -150,17 +158,17 @@ void CgroupsIsolationModule::initialize(
       // remove the residue directory after a slave reboot.
       if (::rmdir(hierarchy.c_str()) < 0) {
         LOG(FATAL) << "Cannot create cgroups hierarchy root at " << hierarchy
-                   << ". Consider removing it.";
+                   << ". Consider removing it";
       }
     }
 
     // The comma-separated subsystem names which will be passed to
     // cgroups::createHierarchy to create the hierarchy root.
-    std::string subsystems;
+    string subsystems;
 
     // Make sure that all the required subsystems are not busy so that we can
     // activate them in the given cgroups hierarchy root.
-    foreach (const std::string& name, requiredSubsystems) {
+    foreach (const string& name, requiredSubsystems) {
       if (busySubsystems.contains(name)) {
         LOG(FATAL) << "Required subsystem " << name << " is busy";
       }
@@ -169,7 +177,7 @@ void CgroupsIsolationModule::initialize(
     }
 
     // Also activate those optional subsystems that are not busy.
-    foreach (const std::string& name, optionalSubsystems) {
+    foreach (const string& name, optionalSubsystems) {
       if (enabledSubsystems.contains(name) && !busySubsystems.contains(name)) {
         subsystems.append(name + ",");
       }
@@ -185,31 +193,35 @@ void CgroupsIsolationModule::initialize(
   }
 
   // Probe activated subsystems in the cgroups hierarchy root.
-  Try<std::set<std::string> > activated = cgroups::subsystems(hierarchy);
-  foreach (const std::string& name, activated.get()) {
+  Try<set<string> > activated = cgroups::subsystems(hierarchy);
+  foreach (const string& name, activated.get()) {
     activatedSubsystems.insert(name);
   }
 
   // Make sure that all the required subsystems are activated.
-  foreach (const std::string& name, requiredSubsystems) {
+  foreach (const string& name, requiredSubsystems) {
     if (!activatedSubsystems.contains(name)) {
       LOG(FATAL) << "Required subsystem " << name
                  << " is not activated in hierarchy " << hierarchy;
     }
   }
 
-  // Try to cleanup the cgroups in the cgroups hierarchy root that belong to
-  // this module (which are created in the previous executions).
-  Try<std::vector<std::string> > cgroups = cgroups::getCgroups(hierarchy);
-  if (cgroups.isError()) {
-    LOG(FATAL) << "Failed to peek cgroups in hierarchy " << hierarchy
-               << ": " << cgroups.error();
-  }
-
-  foreach (const std::string cgroup, cgroups.get()) {
-    if (isValidCgroupName(cgroup)) {
-      LOG(INFO) << "Removing stale cgroup " << cgroup
-                << " in hierarchy " << hierarchy;
+  // Create the root "mesos" cgroup and cleanup any orphaned cgroups
+  // that were created in previous executions.
+  if (cgroups::checkCgroup(hierarchy, "mesos").isError()) {
+    // No root cgroup exists, create it.
+    Try<Nothing> create = cgroups::createCgroup(hierarchy, "mesos");
+    CHECK(create.isSome())
+      << "Failed to create the \"mesos\" cgroup: "
+      << create.error();
+  } else {
+    // The root cgroup already exists, so cleanup any orphaned cgroups.
+    Try<vector<string> > cgroups = cgroups::getCgroups(hierarchy, "mesos");
+    CHECK(cgroups.isSome())
+      << "Failed to get nested cgroups of \"mesos\": "
+      << cgroups.error();
+    foreach (const string& cgroup, cgroups.get()) {
+      LOG(INFO) << "Removing orphaned cgroup '" << cgroup << "'";
       cgroups::destroyCgroup(hierarchy, cgroup)
         .onAny(defer(PID<CgroupsIsolationModule>(this),
                      &CgroupsIsolationModule::destroyWaited,
@@ -218,6 +230,33 @@ void CgroupsIsolationModule::initialize(
     }
   }
 
+  // Make sure this kernel supports creating nested cgroups.
+  Try<Nothing> create = cgroups::createCgroup(hierarchy, "mesos/test");
+  if (create.isError()) {
+    fatal("Failed to create a nested \"test\" cgroup, your kernel "
+          "might be too old to use the cgroups isolation module"
+          ": %s", create.error().c_str()); // TODO(benh): Update fatal.
+  }
+
+  Try<Nothing> remove = cgroups::removeCgroup(hierarchy, "mesos/test");
+  CHECK(remove.isSome())
+    << "Failed to remove the nested \"test\" cgroup:" << remove.error();
+
+
+  Try<Nothing> check =
+    cgroups::checkControl(hierarchy, "mesos", "memory.oom_control");
+  if (check.isError()) {
+    fatal("Failed to find 'memory.oom_control', your kernel "
+          "might be too old to use the cgroups isolation module"
+          ": %s", check.error().c_str()); // TODO(benh): Update fatal.
+  }
+
+  // Disable the OOM killer so that we can capture 'memory.stat'.
+  Try<Nothing> disable =
+    cgroups::writeControl(hierarchy, "mesos", "memory.oom_control", "1");
+  CHECK(disable.isSome())
+    << "Failed to disable OOM killer: " << disable.error();
+
   // Configure resource subsystem mapping.
   resourceSubsystemMap["cpus"] = "cpu";
   resourceSubsystemMap["mem"] = "memory";
@@ -234,7 +273,7 @@ void CgroupsIsolationModule::launchExecu
     const FrameworkID& frameworkId,
     const FrameworkInfo& frameworkInfo,
     const ExecutorInfo& executorInfo,
-    const std::string& directory,
+    const string& directory,
     const Resources& resources)
 {
   CHECK(initialized) << "Cannot launch executors before initialization";
@@ -242,14 +281,14 @@ void CgroupsIsolationModule::launchExecu
   const ExecutorID& executorId = executorInfo.executor_id();
 
   // Register the cgroup information.
-  registerCgroupInfo(frameworkId, executorId);
+  CgroupInfo* info = registerCgroupInfo(frameworkId, executorId);
 
   LOG(INFO) << "Launching " << executorId
             << " (" << executorInfo.command().value() << ")"
             << " in " << directory
             << " with resources " << resources
             << " for framework " << frameworkId
-            << " in cgroup " << getCgroupName(frameworkId, executorId);
+            << " in cgroup " << info->name();
 
   // First fetch the executor.
   launcher::ExecutorLauncher launcher(
@@ -284,8 +323,7 @@ void CgroupsIsolationModule::launchExecu
   }
 
   // Create a new cgroup for the executor.
-  Try<Nothing> create =
-    cgroups::createCgroup(hierarchy, getCgroupName(frameworkId, executorId));
+  Try<Nothing> create = cgroups::createCgroup(hierarchy, info->name());
   if (create.isError()) {
     LOG(FATAL) << "Failed to create cgroup for executor " << executorId
                << " of framework " << frameworkId
@@ -323,9 +361,7 @@ void CgroupsIsolationModule::launchExecu
     // In child process.
     // Put self into the newly created cgroup.
     Try<Nothing> assign =
-      cgroups::assignTask(hierarchy,
-                          getCgroupName(frameworkId, executorId),
-                          ::getpid());
+      cgroups::assignTask(hierarchy, info->name(), ::getpid());
     if (assign.isError()) {
       LOG(FATAL) << "Failed to assign for executor " << executorId
                  << " of framework " << frameworkId
@@ -362,10 +398,10 @@ void CgroupsIsolationModule::killExecuto
   // wait for it to succeed as we don't want to block the isolation module.
   // Instead, we register a callback which will be invoked when its result is
   // ready.
-  cgroups::destroyCgroup(hierarchy, getCgroupName(frameworkId, executorId))
+  cgroups::destroyCgroup(hierarchy, info->name())
     .onAny(defer(PID<CgroupsIsolationModule>(this),
                  &CgroupsIsolationModule::destroyWaited,
-                 getCgroupName(frameworkId, executorId),
+                 info->name(),
                  lambda::_1));
 
   // We do not unregister the cgroup info here, instead, we ask the process
@@ -395,7 +431,7 @@ void CgroupsIsolationModule::resourcesCh
   for (Resources::const_iterator it = resources.begin();
        it != resources.end(); ++it) {
     const Resource& resource = *it;
-    const std::string& name = resource.name();
+    const string& name = resource.name();
 
     if (resourceChangedHandlers.contains(name)) {
       // We only call the resource changed handler either if the resource does
@@ -403,9 +439,7 @@ void CgroupsIsolationModule::resourcesCh
       if (!resourceSubsystemMap.contains(name) ||
           activatedSubsystems.contains(resourceSubsystemMap[name])) {
         Try<Nothing> result =
-          (this->*resourceChangedHandlers[name])(frameworkId,
-                                                 executorId,
-                                                 resources);
+          (this->*resourceChangedHandlers[name])(info, resources);
         if (result.isError()) {
           LOG(ERROR) << result.error();
         }
@@ -441,8 +475,7 @@ void CgroupsIsolationModule::processExit
 
 
 Try<Nothing> CgroupsIsolationModule::cpusChanged(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId,
+    const CgroupInfo* info,
     const Resources& resources)
 {
   Resource r;
@@ -452,7 +485,7 @@ Try<Nothing> CgroupsIsolationModule::cpu
   Option<Resource> cpusResource = resources.get(r);
   if (cpusResource.isNone()) {
     LOG(WARNING) << "Resource cpus cannot be retrieved for executor "
-                 << executorId << " of framework " << frameworkId;
+                 << info->executorId << " of framework " << info->frameworkId;
   } else {
     double cpus = cpusResource.get().scalar().value();
     size_t cpuShares =
@@ -460,7 +493,7 @@ Try<Nothing> CgroupsIsolationModule::cpu
 
     Try<Nothing> set =
       cgroups::writeControl(hierarchy,
-                            getCgroupName(frameworkId, executorId),
+                            info->name(),
                             "cpu.shares",
                             stringify(cpuShares));
     if (set.isError()) {
@@ -468,8 +501,8 @@ Try<Nothing> CgroupsIsolationModule::cpu
     }
 
     LOG(INFO) << "Write cpu.shares = " << cpuShares
-              << " for executor " << executorId
-              << " of framework " << frameworkId;
+              << " for executor " << info->executorId
+              << " of framework " << info->frameworkId;
   }
 
   return Nothing();
@@ -477,8 +510,7 @@ Try<Nothing> CgroupsIsolationModule::cpu
 
 
 Try<Nothing> CgroupsIsolationModule::memChanged(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId,
+    const CgroupInfo* info,
     const Resources& resources)
 {
   Resource r;
@@ -488,7 +520,7 @@ Try<Nothing> CgroupsIsolationModule::mem
   Option<Resource> memResource = resources.get(r);
   if (memResource.isNone()) {
     LOG(WARNING) << "Resource mem cannot be retrieved for executor "
-                 << executorId << " of framework " << frameworkId;
+                 << info->executorId << " of framework " << info->frameworkId;
   } else {
     double mem = memResource.get().scalar().value();
     size_t limitInBytes =
@@ -496,7 +528,7 @@ Try<Nothing> CgroupsIsolationModule::mem
 
     Try<Nothing> set =
       cgroups::writeControl(hierarchy,
-                            getCgroupName(frameworkId, executorId),
+                            info->name(),
                             "memory.limit_in_bytes",
                             stringify(limitInBytes));
     if (set.isError()) {
@@ -504,8 +536,8 @@ Try<Nothing> CgroupsIsolationModule::mem
     }
 
     LOG(INFO) << "Write memory.limit_in_bytes = " << limitInBytes
-              << " for executor " << executorId
-              << " of framework " << frameworkId;
+              << " for executor " << info->executorId
+              << " of framework " << info->frameworkId;
   }
 
   return Nothing();
@@ -520,9 +552,7 @@ void CgroupsIsolationModule::oomListen(
   CHECK(info != NULL) << "Cgroup info is not registered";
 
   info->oomNotifier =
-    cgroups::listenEvent(hierarchy,
-                         getCgroupName(frameworkId, executorId),
-                         "memory.oom_control");
+    cgroups::listenEvent(hierarchy, info->name(), "memory.oom_control");
 
   // If the listening fails immediately, something very wrong happened.
   // Therefore, we report a fatal error here.
@@ -548,7 +578,7 @@ void CgroupsIsolationModule::oomListen(
 void CgroupsIsolationModule::oomWaited(
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const std::string& tag,
+    const string& tag,
     const Future<uint64_t>& future)
 {
   LOG(INFO) << "OOM notifier is triggered for executor "
@@ -573,12 +603,8 @@ void CgroupsIsolationModule::oomWaited(
 void CgroupsIsolationModule::oom(
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const std::string& tag)
+    const string& tag)
 {
-  LOG(INFO) << "OOM detected in executor " << executorId
-            << " of framework " << frameworkId
-            << " with tag " << tag;
-
   CgroupInfo* info = findCgroupInfo(frameworkId, executorId);
   if (info == NULL) {
     // It is likely that processExited is executed before this function (e.g.
@@ -599,6 +625,23 @@ void CgroupsIsolationModule::oom(
   // Therefore, we should not be able to reach this point.
   CHECK(!info->killed) << "OOM detected for a killed executor";
 
+  LOG(INFO) << "OOM detected for executor " << executorId
+            << " of framework " << frameworkId
+            << " with tag " << tag;
+
+  // Output 'memory.limit_in_bytes' of the cgroup to help with debugging.
+  Try<string> read =
+    cgroups::readControl(hierarchy, info->name(), "memory.limit_in_bytes");
+  if (read.isSome()) {
+    LOG(INFO) << "MEMORY LIMIT: " << strings::trim(read.get()) << " bytes";
+  }
+
+  // Output 'memory.stat' of the cgroup to help with debugging.
+  read = cgroups::readControl(hierarchy, info->name(), "memory.stat");
+  if (read.isSome()) {
+    LOG(INFO) << "MEMORY STATISTICS: \n" << read.get();
+  }
+
   // TODO(jieyu): Have a mechanism to use a different policy (e.g. freeze the
   // executor) when OOM happens.
   killExecutor(frameworkId, executorId);
@@ -606,7 +649,7 @@ void CgroupsIsolationModule::oom(
 
 
 void CgroupsIsolationModule::destroyWaited(
-    const std::string& cgroup,
+    const string& cgroup,
     const Future<bool>& future)
 {
   if (future.isReady()) {
@@ -675,30 +718,6 @@ CgroupsIsolationModule::CgroupInfo* Cgro
   return NULL;
 }
 
-
-std::string CgroupsIsolationModule::getCgroupName(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId)
-{
-  CgroupInfo* info = findCgroupInfo(frameworkId, executorId);
-  CHECK(info != NULL) << "Cgroup info is not registered";
-
-  std::ostringstream out;
-  out << "mesos_cgroup_framework_" << frameworkId
-      << "_executor_" << executorId
-      << "_tag_" << info->tag;
-  return out.str();
-}
-
-
-bool CgroupsIsolationModule::isValidCgroupName(const std::string& name)
-{
-  return
-    strings::startsWith(name, "mesos_cgroup_framework_") &&
-    strings::contains(name, "_executor_") &&
-    strings::contains(name, "_tag_");
-}
-
 } // namespace mesos {
 } // namespace internal {
 } // namespace slave {

Modified: incubator/mesos/trunk/src/slave/cgroups_isolation_module.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/cgroups_isolation_module.hpp?rev=1404238&r1=1404237&r2=1404238&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/cgroups_isolation_module.hpp (original)
+++ incubator/mesos/trunk/src/slave/cgroups_isolation_module.hpp Wed Oct 31 16:39:24 2012
@@ -19,6 +19,7 @@
 #ifndef __CGROUPS_ISOLATION_MODULE_HPP__
 #define __CGROUPS_ISOLATION_MODULE_HPP__
 
+#include <sstream>
 #include <string>
 
 #include <process/future.hpp>
@@ -27,6 +28,7 @@
 #include <stout/hashmap.hpp>
 #include <stout/hashset.hpp>
 #include <stout/lambda.hpp>
+#include <stout/path.hpp>
 
 #include "launcher/launcher.hpp"
 
@@ -48,22 +50,26 @@ public:
 
   virtual ~CgroupsIsolationModule();
 
-  virtual void initialize(const Flags& flags,
-                          bool local,
-                          const process::PID<Slave>& slave);
-
-  virtual void launchExecutor(const FrameworkID& frameworkId,
-                              const FrameworkInfo& frameworkInfo,
-                              const ExecutorInfo& executorInfo,
-                              const std::string& directory,
-                              const Resources& resources);
-
-  virtual void killExecutor(const FrameworkID& frameworkId,
-                            const ExecutorID& executorId);
-
-  virtual void resourcesChanged(const FrameworkID& frameworkId,
-                                const ExecutorID& executorId,
-                                const Resources& resources);
+  virtual void initialize(
+      const Flags& flags,
+      bool local,
+      const process::PID<Slave>& slave);
+
+  virtual void launchExecutor(
+      const FrameworkID& frameworkId,
+      const FrameworkInfo& frameworkInfo,
+      const ExecutorInfo& executorInfo,
+      const std::string& directory,
+      const Resources& resources);
+
+  virtual void killExecutor(
+      const FrameworkID& frameworkId,
+      const ExecutorID& executorId);
+
+  virtual void resourcesChanged(
+      const FrameworkID& frameworkId,
+      const ExecutorID& executorId,
+      const Resources& resources);
 
   virtual void processExited(pid_t pid, int status);
 
@@ -75,6 +81,16 @@ private:
   // The cgroup information for each live executor.
   struct CgroupInfo
   {
+    // Returns the canonicalized name of the cgroup in the filesystem.
+    std::string name() const
+    {
+      std::ostringstream out;
+      out << "framework_" << frameworkId
+          << "_executor_" << executorId
+          << "_tag_" << tag;
+      return path::join("mesos", out.str());
+    }
+
     FrameworkID frameworkId;
     ExecutorID executorId;
 
@@ -97,61 +113,67 @@ private:
   // @param   executorId    The id of the given executor.
   // @param   resources     The handle for the resources.
   // @return  Whether the operation successes.
-  Try<Nothing> cpusChanged(const FrameworkID& frameworkId,
-                           const ExecutorID& executorId,
-                           const Resources& resources);
+  Try<Nothing> cpusChanged(
+      const CgroupInfo* info,
+      const Resources& resources);
 
   // The callback which will be invoked when "mem" resource has changed.
   // @param   frameworkId   The id of the given framework.
   // @param   executorId    The id of the given executor.
   // @param   resources     The handle for the resources.
   // @return  Whether the operation successes.
-  Try<Nothing> memChanged(const FrameworkID& frameworkId,
-                          const ExecutorID& executorId,
-                          const Resources& resources);
+  Try<Nothing> memChanged(
+      const CgroupInfo* info,
+      const Resources& resources);
 
   // Start listening on OOM events. This function will create an eventfd and
   // start polling on it.
   // @param   frameworkId   The id of the given framework.
   // @param   executorId    The id of the given executor.
-  void oomListen(const FrameworkID& frameworkId,
-                 const ExecutorID& executorId);
+  void oomListen(
+      const FrameworkID& frameworkId,
+      const ExecutorID& executorId);
 
   // This function is invoked when the polling on eventfd has a result.
   // @param   frameworkId   The id of the given framework.
   // @param   executorId    The id of the given executor.
   // @param   tag           The uuid tag.
-  void oomWaited(const FrameworkID& frameworkId,
-                 const ExecutorID& executorId,
-                 const std::string& tag,
-                 const process::Future<uint64_t>& future);
+  void oomWaited(
+      const FrameworkID& frameworkId,
+      const ExecutorID& executorId,
+      const std::string& tag,
+      const process::Future<uint64_t>& future);
 
   // This function is invoked when the OOM event happens.
   // @param   frameworkId   The id of the given framework.
   // @param   executorId    The id of the given executor.
   // @param   tag           The uuid tag.
-  void oom(const FrameworkID& frameworkId,
-           const ExecutorID& executorId,
-           const std::string& tag);
+  void oom(
+      const FrameworkID& frameworkId,
+      const ExecutorID& executorId,
+      const std::string& tag);
 
   // This callback is invoked when destroy cgroup has a result.
   // @param   cgroup        The cgroup that is being destroyed.
   // @param   future        The future describing the destroy process.
-  void destroyWaited(const std::string& cgroup,
-                     const process::Future<bool>& future);
+  void destroyWaited(
+      const std::string& cgroup,
+      const process::Future<bool>& future);
 
   // Register a cgroup in the isolation module.
   // @param   frameworkId   The id of the given framework.
   // @param   executorId    The id of the given executor.
   // @return  A pointer to the cgroup info registered.
-  CgroupInfo* registerCgroupInfo(const FrameworkID& frameworkId,
-                                 const ExecutorID& executorId);
+  CgroupInfo* registerCgroupInfo(
+      const FrameworkID& frameworkId,
+      const ExecutorID& executorId);
 
   // Unregister a cgroup in the isolation module.
   // @param   frameworkId   The id of the given framework.
   // @param   executorId    The id of the given executor.
-  void unregisterCgroupInfo(const FrameworkID& frameworkId,
-                            const ExecutorID& executorId);
+  void unregisterCgroupInfo(
+      const FrameworkID& frameworkId,
+      const ExecutorID& executorId);
 
   // Find a registered cgroup by the PID of the leading process.
   // @param   pid           The PID of the leading process in the cgroup.
@@ -162,22 +184,9 @@ private:
   // @param   frameworkId   The id of the given framework.
   // @param   executorId    The id of the given executor.
   // @return  A pointer to the cgroup info if found, NULL otherwise.
-  CgroupInfo* findCgroupInfo(const FrameworkID& frameworkId,
-                             const ExecutorID& executorId);
-
-  // Return the canonicalized name of the cgroup used by a given executor in a
-  // given framework.
-  // @param   frameworkId   The id of the given framework.
-  // @param   executorId    The id of the given executor.
-  // @return  The canonicalized name of the cgroup.
-  std::string getCgroupName(const FrameworkID& frameworkId,
-                            const ExecutorID& executorId);
-
-  // Return true if the given name is a valid cgroup name used by this isolation
-  // module.
-  // @param   name          The name to check.
-  // @return  True if the given name is valid cgroup name, False otherwise.
-  bool isValidCgroupName(const std::string& name);
+  CgroupInfo* findCgroupInfo(
+      const FrameworkID& frameworkId,
+      const ExecutorID& executorId);
 
   Flags flags;
   bool local;
@@ -201,8 +210,7 @@ private:
   // handler function.
   hashmap<std::string,
           Try<Nothing>(CgroupsIsolationModule::*)(
-              const FrameworkID&,
-              const ExecutorID&,
+              const CgroupInfo*,
               const Resources&)> resourceChangedHandlers;
 };