You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2014/09/22 22:39:20 UTC

git commit: Allowed co-mounted cgroup subsystems to enable Mesos on machines with systemd.

Repository: mesos
Updated Branches:
  refs/heads/master 39c1da541 -> 31337348c


Allowed co-mounted cgroup subsystems to enable Mesos on machines with
systemd.

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


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

Branch: refs/heads/master
Commit: 31337348cef29719890bffb59fbf8df8b18b39d0
Parents: 39c1da5
Author: Jie Yu <yu...@gmail.com>
Authored: Fri Sep 19 17:18:38 2014 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Sep 22 13:39:10 2014 -0700

----------------------------------------------------------------------
 src/linux/cgroups.cpp                           | 171 ++++++-----
 .../isolators/cgroups/cpushare.cpp              | 287 +++++++++++--------
 .../isolators/cgroups/cpushare.hpp              |   9 +-
 .../containerizer/isolators/cgroups/mem.cpp     |  17 +-
 .../isolators/cgroups/perf_event.cpp            |   4 +-
 src/slave/containerizer/linux_launcher.cpp      |  13 +
 src/slave/slave.cpp                             |  32 +--
 7 files changed, 301 insertions(+), 232 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/31337348/src/linux/cgroups.cpp
----------------------------------------------------------------------
diff --git a/src/linux/cgroups.cpp b/src/linux/cgroups.cpp
index 5093b4c..62df4b7 100644
--- a/src/linux/cgroups.cpp
+++ b/src/linux/cgroups.cpp
@@ -64,8 +64,14 @@ using namespace process;
 // TODO(benh): Move linux/fs.hpp out of 'mesos- namespace.
 using namespace mesos::internal;
 
+using std::dec;
+using std::getline;
+using std::ifstream;
+using std::istringstream;
 using std::list;
 using std::map;
+using std::ofstream;
+using std::ostringstream;
 using std::set;
 using std::string;
 using std::vector;
@@ -105,7 +111,7 @@ struct SubsystemInfo
 static Try<map<string, SubsystemInfo> > subsystems()
 {
   // TODO(benh): Use os::read to get better error information.
-  std::ifstream file("/proc/cgroups");
+  ifstream file("/proc/cgroups");
 
   if (!file.is_open()) {
     return Error("Failed to open /proc/cgroups");
@@ -115,7 +121,7 @@ static Try<map<string, SubsystemInfo> > subsystems()
 
   while (!file.eof()) {
     string line;
-    std::getline(file, line);
+    getline(file, line);
 
     if (file.fail()) {
       if (!file.eof()) {
@@ -136,8 +142,8 @@ static Try<map<string, SubsystemInfo> > subsystems()
         int cgroups;
         bool enabled;
 
-        std::istringstream ss(line);
-        ss >> std::dec >> name >> hierarchy >> cgroups >> enabled;
+        istringstream ss(line);
+        ss >> dec >> name >> hierarchy >> cgroups >> enabled;
 
         // Check for any read/parse errors.
         if (ss.fail() && !ss.eof()) {
@@ -355,17 +361,17 @@ static Try<string> read(
   // TODO(benh): Use os::read. Note that we do not use os::read
   // currently because it cannot correctly read /proc or cgroups
   // control files since lseek (in os::read) will return error.
-  std::ifstream file(path.c_str());
+  ifstream file(path.c_str());
 
   if (!file.is_open()) {
     return Error("Failed to open file " + path);
   }
 
-  std::ostringstream ss;
+  ostringstream ss;
   ss << file.rdbuf();
 
   if (file.fail()) {
-    ErrnoError error; // TODO(jieyu): Does std::ifstream actually set errno?
+    ErrnoError error; // TODO(jieyu): Does ifstream actually set errno?
     file.close();
     return error;
   }
@@ -389,7 +395,7 @@ static Try<Nothing> write(
     const string& value)
 {
   string path = path::join(hierarchy, cgroup, control);
-  std::ofstream file(path.c_str());
+  ofstream file(path.c_str());
 
   if (!file.is_open()) {
     return Error("Failed to open file " + path);
@@ -401,7 +407,7 @@ static Try<Nothing> write(
   file << value;
 
   if (file.fail()) {
-    ErrnoError error; // TODO(jieyu): Does std::ifstream actually set errno?
+    ErrnoError error; // TODO(jieyu): Does ifstream actually set errno?
     file.close();
     return error;
   }
@@ -414,13 +420,10 @@ static Try<Nothing> write(
 
 
 Try<string> prepare(
-    const std::string& baseHierarchy,
-    const std::string& subsystem,
-    const std::string& cgroup)
+    const string& baseHierarchy,
+    const string& subsystem,
+    const string& cgroup)
 {
-  // Construct the hierarchy for this subsystem.
-  std::string hierarchy = path::join(baseHierarchy, subsystem);
-
   // Ensure cgroups are enabled in the kernel.
   if (!cgroups::enabled()) {
     return Error("No cgroups support detected in this kernel");
@@ -431,103 +434,98 @@ Try<string> prepare(
     return Error("Using cgroups requires root permissions");
   }
 
-  // Check if the hierarchy is already mounted, and if not, mount it.
-  Try<bool> mounted = cgroups::mounted(hierarchy);
-
-  if (mounted.isError()) {
-    return Error("Failed to determine if " + hierarchy +
-                 " is already mounted: " + mounted.error());
+  // Check if the specified subsystem has already been attached to
+  // some hierarchy. If not, create and mount the hierarchy according
+  // to the given baseHierarchy and subsystem.
+  Result<string> hierarchy = cgroups::hierarchy(subsystem);
+  if (hierarchy.isError()) {
+    return Error(
+        "Failed to determine the hierarchy where the subsystem " +
+        subsystem + " is attached");
   }
 
-  if (mounted.get()) {
-    // Make sure that desired subsystem is attached to the already
-    // mounted hierarchy.
-    Try<std::set<std::string> > attached = cgroups::subsystems(hierarchy);
-    if (attached.isError()) {
-      return Error(string("Failed to determine the attached subsystems") +
-                   "for the cgroup hierarchy at " + hierarchy + ": " +
-                   attached.error());
-    }
-
-    if (attached.get().count(subsystem) == 0) {
-      return Error("The cgroups hierarchy at " + hierarchy +
-                   " can not be used because it does not have the '" +
-                   subsystem + "' subsystem attached");
-    }
-
-    if (attached.get().size() > 1) {
-      return Error("The " + subsystem + " subsystem is co-mounted at " +
-                   hierarchy + " with other subsytems");
-    }
-  } else {
+  if (hierarchy.isNone()) {
     // Attempt to mount the hierarchy ourselves.
-    if (os::exists(hierarchy)) {
+    hierarchy = path::join(baseHierarchy, subsystem);
+
+    if (os::exists(hierarchy.get())) {
       // The path specified by the given hierarchy already exists in
       // the file system. We try to remove it if it is an empty
       // directory. This will helps us better deal with slave restarts
       // since we won't need to manually remove the directory.
-      Try<Nothing> rmdir = os::rmdir(hierarchy, false);
+      Try<Nothing> rmdir = os::rmdir(hierarchy.get(), false);
       if (rmdir.isError()) {
-        return Error("Failed to mount cgroups hierarchy at '" + hierarchy +
-                     "' because we could not remove the existing directory: " +
-                     rmdir.error());
+        return Error(
+            "Failed to mount cgroups hierarchy at '" + hierarchy.get() +
+            "' because we could not remove the existing directory: " +
+            rmdir.error());
       }
     }
 
     // Mount the subsystem.
-    Try<Nothing> mount = cgroups::mount(hierarchy, subsystem);
+    Try<Nothing> mount = cgroups::mount(hierarchy.get(), subsystem);
     if (mount.isError()) {
-      return Error("Failed to mount cgroups hierarchy at '" + hierarchy +
-                   "': " + mount.error());
+      return Error(
+          "Failed to mount cgroups hierarchy at '" + hierarchy.get() +
+          "': " + mount.error());
     }
   }
 
+  CHECK_SOME(hierarchy);
+
   // Create the cgroup if it doesn't exist.
-  Try<bool> exists = cgroups::exists(hierarchy, cgroup);
+  Try<bool> exists = cgroups::exists(hierarchy.get(), cgroup);
   if (exists.isError()) {
-    return Error("Failed to check existence of root cgroup " +
-                 path::join(hierarchy, cgroup) +
-                 ": " + exists.error());
+    return Error(
+        "Failed to check existence of root cgroup " +
+        path::join(hierarchy.get(), cgroup) +
+        ": " + exists.error());
   }
 
   if (!exists.get()) {
     // No cgroup exists, create it.
-    Try<Nothing> create = cgroups::create(hierarchy, cgroup, true);
+    Try<Nothing> create = cgroups::create(hierarchy.get(), cgroup, true);
     if (create.isError()) {
-      return Error("Failed to create root cgroup " +
-                   path::join(hierarchy, cgroup) +
-                   ": " + create.error());
+      return Error(
+          "Failed to create root cgroup " +
+          path::join(hierarchy.get(), cgroup) +
+          ": " + create.error());
     }
   }
 
+  // Test for nested cgroup support.
+  // TODO(jieyu): Consider doing this test only once.
   const string& testCgroup = path::join(cgroup, "test");
+
   // Create a nested test cgroup if it doesn't exist.
-  exists = cgroups::exists(hierarchy, testCgroup);
+  exists = cgroups::exists(hierarchy.get(), testCgroup);
   if (exists.isError()) {
-    return Error("Failed to check existence nested of test cgroup " +
-                 path::join(hierarchy, testCgroup) +
-                 ": " + exists.error());
+    return Error(
+        "Failed to check existence of the nested test cgroup " +
+        path::join(hierarchy.get(), testCgroup) +
+        ": " + exists.error());
   }
 
   if (!exists.get()) {
     // Make sure this kernel supports creating nested cgroups.
-    Try<Nothing> create = cgroups::create(hierarchy, testCgroup);
+    Try<Nothing> create = cgroups::create(hierarchy.get(), testCgroup);
     if (create.isError()) {
-      return Error(string("Failed to create a nested 'test' cgroup.") +
-                   " Your kernel might be too old to use the" +
-                   " cgroups isolator: " + create.error());
+      return Error(
+          "Your kernel might be too old to support nested cgroup: " +
+          create.error());
     }
   }
 
   // Remove the nested 'test' cgroup.
-  Try<Nothing> remove = cgroups::remove(hierarchy, testCgroup);
+  Try<Nothing> remove = cgroups::remove(hierarchy.get(), testCgroup);
   if (remove.isError()) {
     return Error("Failed to remove the nested test cgroup: " + remove.error());
   }
 
-  return hierarchy;
+  return hierarchy.get();
 }
 
+
 // Returns some error string if either (a) hierarchy is not mounted,
 // (b) cgroup does not exist, or (c) control file does not exist.
 static Option<Error> verify(
@@ -594,15 +592,15 @@ Try<set<string> > hierarchies()
 }
 
 
-Result<std::string> hierarchy(const std::string& subsystems)
+Result<string> hierarchy(const string& subsystems)
 {
-  Result<std::string> hierarchy = None();
-  Try<std::set<std::string> > hierarchies = cgroups::hierarchies();
+  Result<string> hierarchy = None();
+  Try<set<string> > hierarchies = cgroups::hierarchies();
   if (hierarchies.isError()) {
     return Error(hierarchies.error());
   }
 
-  foreach (const std::string& candidate, hierarchies.get()) {
+  foreach (const string& candidate, hierarchies.get()) {
     if (subsystems.empty()) {
       hierarchy = candidate;
       break;
@@ -723,9 +721,10 @@ Try<set<string> > subsystems(const string& hierarchy)
              : "No such file or directory"));
       }
 
-      // Seems that a directory can be mounted more than once. Previous mounts
-      // are obscured by the later mounts. Therefore, we must see all entries to
-      // make sure we find the last one that matches.
+      // Seems that a directory can be mounted more than once.
+      // Previous mounts are obscured by the later mounts. Therefore,
+      // we must see all entries to make sure we find the last one
+      // that matches.
       if (dirAbsPath.get() == hierarchyAbsPath.get()) {
         hierarchyEntry = entry;
       }
@@ -736,9 +735,9 @@ Try<set<string> > subsystems(const string& hierarchy)
     return Error("'" + hierarchy + "' is not a valid hierarchy");
   }
 
-  // Get the intersection of the currently enabled subsystems and mount
-  // options. Notice that mount options may contain somethings (e.g. rw) that
-  // are not in the set of enabled subsystems.
+  // Get the intersection of the currently enabled subsystems and
+  // mount options. Notice that mount options may contain somethings
+  // (e.g. rw) that are not in the set of enabled subsystems.
   Try<set<string> > names = subsystems();
   if (names.isError()) {
     return Error(names.error());
@@ -1042,8 +1041,8 @@ Try<set<pid_t> > tasks(
   // Parse the values read from the control file and insert into a set. This
   // ensures they are unique (and also sorted).
   set<pid_t> pids;
-  std::istringstream ss(value.get());
-  ss >> std::dec;
+  istringstream ss(value.get());
+  ss >> dec;
   while (!ss.eof()) {
     pid_t pid;
     ss >> pid;
@@ -1162,8 +1161,8 @@ static Try<int> registerNotifier(
   }
 
   // Write the event control file (cgroup.event_control).
-  std::ostringstream out;
-  out << std::dec << efd << " " << cfd.get();
+  ostringstream out;
+  out << dec << efd << " " << cfd.get();
   if (args.isSome()) {
     out << " " << args.get();
   }
@@ -1691,7 +1690,7 @@ Future<Nothing> destroy(const string& hierarchy, const string& cgroup)
     return future;
   } else {
     // Otherwise, attempt to remove the cgroups in bottom-up fashion.
-    foreach (const std::string& cgroup, candidates) {
+    foreach (const string& cgroup, candidates) {
       Try<Nothing> remove = cgroups::remove(hierarchy, cgroup);
       if (remove.isError()) {
         return Failure(remove.error());
@@ -1796,7 +1795,7 @@ Try<hashmap<string, uint64_t> > stat(
     const string& cgroup,
     const string& file)
 {
-  Try<std::string> contents = cgroups::read(hierarchy, cgroup, file);
+  Try<string> contents = cgroups::read(hierarchy, cgroup, file);
 
   if (contents.isError()) {
     return Error(contents.error());
@@ -1814,7 +1813,7 @@ Try<hashmap<string, uint64_t> > stat(
     uint64_t value;
 
     // Expected line format: "%s %llu".
-    std::istringstream stream(line);
+    istringstream stream(line);
     stream >> name >> value;
 
     if (stream.fail()) {
@@ -1902,7 +1901,7 @@ Try<uint64_t> shares(
   }
 
   uint64_t shares;
-  std::istringstream ss(read.get());
+  istringstream ss(read.get());
 
   ss >> shares;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/31337348/src/slave/containerizer/isolators/cgroups/cpushare.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/cpushare.cpp b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
index b1cad47..7164ecc 100644
--- a/src/slave/containerizer/isolators/cgroups/cpushare.cpp
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
@@ -48,6 +48,7 @@
 using namespace process;
 
 using std::list;
+using std::set;
 using std::string;
 using std::vector;
 
@@ -61,45 +62,107 @@ static Future<Option<T> > none() { return None(); }
 
 CgroupsCpushareIsolatorProcess::CgroupsCpushareIsolatorProcess(
     const Flags& _flags,
-    const hashmap<string, string>& _hierarchies)
-  : flags(_flags), hierarchies(_hierarchies) {}
+    const hashmap<string, string>& _hierarchies,
+    const vector<string>& _subsystems)
+  : flags(_flags),
+    hierarchies(_hierarchies),
+    subsystems(_subsystems) {}
 
 
 CgroupsCpushareIsolatorProcess::~CgroupsCpushareIsolatorProcess() {}
 
 
-Try<Isolator*> CgroupsCpushareIsolatorProcess::create(
-    const Flags& flags)
+Try<Isolator*> CgroupsCpushareIsolatorProcess::create(const Flags& flags)
 {
-  hashmap<string, string> hierarchies;
+  Try<string> hierarchyCpu = cgroups::prepare(
+        flags.cgroups_hierarchy,
+        "cpu",
+        flags.cgroups_root);
+
+  if (hierarchyCpu.isError()) {
+    return Error(
+        "Failed to prepare hierarchy for cpu subsystem: " +
+        hierarchyCpu.error());
+  }
 
-  vector<string> subsystems;
-  subsystems.push_back("cpu");
-  subsystems.push_back("cpuacct");
+  Try<string> hierarchyCpuacct = cgroups::prepare(
+        flags.cgroups_hierarchy,
+        "cpuacct",
+        flags.cgroups_root);
 
-  foreach (const string& subsystem, subsystems) {
-    Try<string> hierarchy = cgroups::prepare(
-        flags.cgroups_hierarchy, subsystem, flags.cgroups_root);
+  if (hierarchyCpuacct.isError()) {
+    return Error(
+        "Failed to prepare hierarchy for cpuacct subsystem: " +
+        hierarchyCpuacct.error());
+  }
+
+  hashmap<string, string> hierarchies;
+  vector<string> subsystems;
 
-    if (hierarchy.isError()) {
-      return Error("Failed to create isolator: " + hierarchy.error());
+  hierarchies["cpu"] = hierarchyCpu.get();
+  hierarchies["cpuacct"] = hierarchyCpuacct.get();
+
+  if (hierarchyCpu.get() == hierarchyCpuacct.get()) {
+    // Subsystem cpu and cpuacct are co-mounted (e.g., systemd).
+    hierarchies["cpu,cpuacct"] = hierarchyCpu.get();
+    subsystems.push_back("cpu,cpuacct");
+
+    // Ensure that no other subsystem is attached to the hierarchy.
+    Try<set<string> > _subsystems = cgroups::subsystems(hierarchyCpu.get());
+    if (_subsystems.isError()) {
+      return Error(
+          "Failed to get the list of attached subsystems for hierarchy " +
+          hierarchyCpu.get());
+    } else if (_subsystems.get().size() != 2) {
+      return Error(
+          "Unexpected subsystems found attached to the hierarchy " +
+          hierarchyCpu.get());
+    }
+  } else {
+    // Subsystem cpu and cpuacct are mounted separately.
+    subsystems.push_back("cpu");
+    subsystems.push_back("cpuacct");
+
+    // Ensure that no other subsystem is attached to each of the
+    // hierarchy.
+    Try<set<string> > _subsystems = cgroups::subsystems(hierarchyCpu.get());
+    if (_subsystems.isError()) {
+      return Error(
+          "Failed to get the list of attached subsystems for hierarchy " +
+          hierarchyCpu.get());
+    } else if (_subsystems.get().size() != 1) {
+      return Error(
+          "Unexpected subsystems found attached to the hierarchy " +
+          hierarchyCpu.get());
     }
 
-    hierarchies[subsystem] = hierarchy.get();
+    _subsystems = cgroups::subsystems(hierarchyCpuacct.get());
+    if (_subsystems.isError()) {
+      return Error(
+          "Failed to get the list of attached subsystems for hierarchy " +
+          hierarchyCpuacct.get());
+    } else if (_subsystems.get().size() != 1) {
+      return Error(
+          "Unexpected subsystems found attached to the hierarchy " +
+          hierarchyCpuacct.get());
+    }
   }
 
   if (flags.cgroups_enable_cfs) {
     Try<bool> exists = cgroups::exists(
-        hierarchies["cpu"], flags.cgroups_root, "cpu.cfs_quota_us");
+        hierarchies["cpu"],
+        flags.cgroups_root,
+        "cpu.cfs_quota_us");
 
     if (exists.isError() || !exists.get()) {
-      return Error("Failed to find 'cpu.cfs_quota_us'. Your kernel "
-                   "might be too old to use the CFS cgroups feature.");
+      return Error(
+          "Failed to find 'cpu.cfs_quota_us'. Your kernel "
+          "might be too old to use the CFS cgroups feature.");
     }
   }
 
   process::Owned<IsolatorProcess> process(
-      new CgroupsCpushareIsolatorProcess(flags, hierarchies));
+      new CgroupsCpushareIsolatorProcess(flags, hierarchies, subsystems));
 
   return new Isolator(process);
 }
@@ -128,15 +191,15 @@ Future<Nothing> CgroupsCpushareIsolatorProcess::recover(
         delete info;
       }
       infos.clear();
-      return Failure("Failed to check cgroup for container '" +
-                     stringify(containerId) + "'");
+      return Failure(
+          "Failed to check cgroup for container " + stringify(containerId));
     }
 
     if (!exists.get()) {
-      // This may occur if the executor has exited and the isolator has
-      // destroyed the cgroup but the slave dies before noticing this. This
-      // will be detected when the containerizer tries to monitor the
-      // executor's pid.
+      // This may occur if the executor has exited and the isolator
+      // has destroyed the cgroup but the slave dies before noticing
+      // this. This will be detected when the containerizer tries to
+      // monitor the executor's pid.
       LOG(WARNING) << "Couldn't find cgroup for container " << containerId;
       continue;
     }
@@ -145,58 +208,38 @@ Future<Nothing> CgroupsCpushareIsolatorProcess::recover(
     cgroups.insert(cgroup);
   }
 
-  // Remove orphans in the cpu hierarchy.
-  Try<vector<string> > orphans = cgroups::get(
-      hierarchies["cpu"], flags.cgroups_root);
-  if (orphans.isError()) {
-    foreachvalue (Info* info, infos) {
-      delete info;
-    }
-    infos.clear();
-    return Failure(orphans.error());
-  }
+  // Remove orphans.
+  foreach (const string& subsystem, subsystems) {
+    Try<vector<string> > orphans = cgroups::get(
+        hierarchies[subsystem],
+        flags.cgroups_root);
 
-  foreach (const string& orphan, orphans.get()) {
-    // Ignore the slave cgroup (see the --slave_subsystems flag).
-    // TODO(idownes): Remove this when the cgroups layout is updated,
-    // see MESOS-1185.
-    if (orphan == path::join(flags.cgroups_root, "slave")) {
-      continue;
+    if (orphans.isError()) {
+      foreachvalue (Info* info, infos) {
+        delete info;
+      }
+      infos.clear();
+      return Failure(orphans.error());
     }
 
-    if (!cgroups.contains(orphan)) {
-      LOG(INFO) << "Removing orphaned cgroup"
-                << " '" << path::join("cpu", orphan) << "'";
-      // We don't wait on the destroy as we don't want to block recovery.
-      cgroups::destroy(
-          hierarchies["cpu"], orphan, cgroups::DESTROY_TIMEOUT);
-    }
-  }
+    foreach (const string& orphan, orphans.get()) {
+      // Ignore the slave cgroup (see the --slave_subsystems flag).
+      // TODO(idownes): Remove this when the cgroups layout is
+      // updated, see MESOS-1185.
+      if (orphan == path::join(flags.cgroups_root, "slave")) {
+        continue;
+      }
 
-  // Remove orphans in the cpuacct hierarchy.
-  orphans = cgroups::get(hierarchies["cpuacct"], flags.cgroups_root);
-  if (orphans.isError()) {
-    foreachvalue (Info* info, infos) {
-      delete info;
-    }
-    infos.clear();
-    return Failure(orphans.error());
-  }
+      if (!cgroups.contains(orphan)) {
+        LOG(INFO) << "Removing orphaned cgroup" << " '"
+                  << path::join(subsystem, orphan) << "'";
 
-  foreach (const string& orphan, orphans.get()) {
-    // Ignore the slave cgroup (see the --slave_subsystems flag).
-    // TODO(idownes): Remove this when the cgroups layout is updated,
-    // see MESOS-1185.
-    if (orphan == path::join(flags.cgroups_root, "slave")) {
-      continue;
-    }
-
-    if (!cgroups.contains(orphan)) {
-      LOG(INFO) << "Removing orphaned cgroup"
-                << " '" << path::join("cpuacct", orphan) << "'";
-      // We don't wait on the destroy as we don't want to block recovery.
-      cgroups::destroy(
-          hierarchies["cpuacct"], orphan, cgroups::DESTROY_TIMEOUT);
+        // We don't wait on the destroy as we don't want to block recovery.
+        cgroups::destroy(
+            hierarchies[subsystem],
+            orphan,
+            cgroups::DESTROY_TIMEOUT);
+      }
     }
   }
 
@@ -214,39 +257,25 @@ Future<Option<CommandInfo> > CgroupsCpushareIsolatorProcess::prepare(
 
   // TODO(bmahler): Don't insert into 'infos' unless we create the
   // cgroup successfully. It's safe for now because 'cleanup' gets
-  // called if we return a Failure, but cleanup will fail because
-  // the cgroup does not exist when cgroups::destroy is called.
+  // called if we return a Failure, but cleanup will fail because the
+  // cgroup does not exist when cgroups::destroy is called.
   Info* info = new Info(
       containerId, path::join(flags.cgroups_root, containerId.value()));
 
   infos[containerId] = info;
 
-  // Create a 'cpu' cgroup for this container.
-  Try<bool> exists = cgroups::exists(hierarchies["cpu"], info->cgroup);
-
-  if (exists.isError()) {
-    return Failure("Failed to prepare isolator: " + exists.error());
-  } else if (exists.get()) {
-    return Failure("Failed to prepare isolator: cgroup already exists");
-  }
-
-  Try<Nothing> create = cgroups::create(hierarchies["cpu"], info->cgroup);
-  if (create.isError()) {
-    return Failure("Failed to prepare isolator: " + create.error());
-  }
-
-  // Create a 'cpuacct' cgroup for this container.
-  exists = cgroups::exists(hierarchies["cpuacct"], info->cgroup);
-
-  if (exists.isError()) {
-    return Failure("Failed to prepare isolator: " + exists.error());
-  } else if (exists.get()) {
-    return Failure("Failed to prepare isolator: cgroup already exists");
-  }
+  foreach (const string& subsystem, subsystems) {
+    Try<bool> exists = cgroups::exists(hierarchies[subsystem], info->cgroup);
+    if (exists.isError()) {
+      return Failure("Failed to prepare isolator: " + exists.error());
+    } else if (exists.get()) {
+      return Failure("Failed to prepare isolator: cgroup already exists");
+    }
 
-  create = cgroups::create(hierarchies["cpuacct"], info->cgroup);
-  if (create.isError()) {
-    return Failure("Failed to prepare isolator: " + create.error());
+    Try<Nothing> create = cgroups::create(hierarchies[subsystem], info->cgroup);
+    if (create.isError()) {
+      return Failure("Failed to prepare isolator: " + create.error());
+    }
   }
 
   return update(containerId, executorInfo.resources())
@@ -267,22 +296,20 @@ Future<Nothing> CgroupsCpushareIsolatorProcess::isolate(
   CHECK(info->pid.isNone());
   info->pid = pid;
 
-  Try<Nothing> assign = cgroups::assign(hierarchies["cpu"], info->cgroup, pid);
-  if (assign.isError()) {
-    LOG(ERROR) << "Failed to assign container '" << info->containerId
-               << " to its own cgroup '"
-               << path::join(hierarchies["cpu"], info->cgroup)
-               << "' : " << assign.error();
-    return Failure("Failed to isolate container: " + assign.error());
-  }
-
-  assign = cgroups::assign(hierarchies["cpuacct"], info->cgroup, pid);
-  if (assign.isError()) {
-    LOG(ERROR) << "Failed to assign container '" << info->containerId
-               << " to its own cgroup '"
-               << path::join(hierarchies["cpuacct"], info->cgroup)
-               << "' : " << assign.error();
-    return Failure("Failed to isolate container: " + assign.error());
+  foreach (const string& subsystem, subsystems) {
+    Try<Nothing> assign = cgroups::assign(
+        hierarchies[subsystem],
+        info->cgroup,
+        pid);
+
+    if (assign.isError()) {
+      LOG(ERROR) << "Failed to assign container '" << info->containerId
+                 << " to its own cgroup '"
+                 << path::join(hierarchies[subsystem], info->cgroup)
+                 << "' : " << assign.error();
+
+      return Failure("Failed to isolate container: " + assign.error());
+    }
   }
 
   return Nothing();
@@ -315,7 +342,6 @@ Future<Nothing> CgroupsCpushareIsolatorProcess::update(
   }
 
   const Option<string>& hierarchy = hierarchies.get("cpu");
-
   if (hierarchy.isNone()) {
     return Failure("No 'cpu' hierarchy");
   }
@@ -329,7 +355,9 @@ Future<Nothing> CgroupsCpushareIsolatorProcess::update(
     std::max((uint64_t) (CPU_SHARES_PER_CPU * cpus), MIN_CPU_SHARES);
 
   Try<Nothing> write = cgroups::cpu::shares(
-      hierarchy.get(), info->cgroup, shares);
+      hierarchy.get(),
+      info->cgroup,
+      shares);
 
   if (write.isError()) {
     return Failure("Failed to update 'cpu.shares': " + write.error());
@@ -342,7 +370,9 @@ Future<Nothing> CgroupsCpushareIsolatorProcess::update(
   // Set cfs quota if enabled.
   if (flags.cgroups_enable_cfs) {
     write = cgroups::cpu::cfs_period_us(
-        hierarchy.get(), info->cgroup, CPU_CFS_PERIOD);
+        hierarchy.get(),
+        info->cgroup,
+        CPU_CFS_PERIOD);
 
     if (write.isError()) {
       return Failure("Failed to update 'cpu.cfs_period_us': " + write.error());
@@ -351,7 +381,6 @@ Future<Nothing> CgroupsCpushareIsolatorProcess::update(
     Duration quota = std::max(CPU_CFS_PERIOD * cpus, MIN_CPU_CFS_QUOTA);
 
     write = cgroups::cpu::cfs_quota_us(hierarchy.get(), info->cgroup, quota);
-
     if (write.isError()) {
       return Failure("Failed to update 'cpu.cfs_quota_us': " + write.error());
     }
@@ -383,8 +412,10 @@ Future<ResourceStatistics> CgroupsCpushareIsolatorProcess::usage(
   PCHECK(ticks > 0) << "Failed to get sysconf(_SC_CLK_TCK)";
 
   // Add the cpuacct.stat information.
-  Try<hashmap<string, uint64_t> > stat =
-    cgroups::stat(hierarchies["cpuacct"], info->cgroup, "cpuacct.stat");
+  Try<hashmap<string, uint64_t> > stat = cgroups::stat(
+      hierarchies["cpuacct"],
+      info->cgroup,
+      "cpuacct.stat");
 
   if (stat.isError()) {
     return Failure("Failed to read cpuacct.stat: " + stat.error());
@@ -403,7 +434,6 @@ Future<ResourceStatistics> CgroupsCpushareIsolatorProcess::usage(
   // Add the cpu.stat information only if CFS is enabled.
   if (flags.cgroups_enable_cfs) {
     stat = cgroups::stat(hierarchies["cpu"], info->cgroup, "cpu.stat");
-
     if (stat.isError()) {
       return Failure("Failed to read cpu.stat: " + stat.error());
     }
@@ -443,15 +473,18 @@ Future<Nothing> CgroupsCpushareIsolatorProcess::cleanup(
   if (!infos.contains(containerId)) {
     VLOG(1) << "Ignoring cleanup request for unknown container: "
             << containerId;
+
     return Nothing();
   }
 
   Info* info = CHECK_NOTNULL(infos[containerId]);
 
   list<Future<Nothing> > futures;
-  foreachvalue (const string& hierarchy, hierarchies) {
-    futures.push_back(
-        cgroups::destroy(hierarchy, info->cgroup, cgroups::DESTROY_TIMEOUT));
+  foreach (const string& subsystem, subsystems) {
+    futures.push_back(cgroups::destroy(
+        hierarchies[subsystem],
+        info->cgroup,
+        cgroups::DESTROY_TIMEOUT));
   }
 
   return collect(futures)
@@ -474,9 +507,9 @@ Future<list<Nothing> > CgroupsCpushareIsolatorProcess::_cleanup(
   CHECK_NOTNULL(infos[containerId]);
 
   if (!future.isReady()) {
-    return Failure("Failed to clean up container " + stringify(containerId) +
-                   " : " + (future.isFailed() ? future.failure()
-                                              : "discarded"));
+    return Failure(
+        "Failed to clean up container " + stringify(containerId) +
+        " : " + (future.isFailed() ? future.failure() : "discarded"));
   }
 
   delete infos[containerId];

http://git-wip-us.apache.org/repos/asf/mesos/blob/31337348/src/slave/containerizer/isolators/cgroups/cpushare.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/cpushare.hpp b/src/slave/containerizer/isolators/cgroups/cpushare.hpp
index d4df5f3..2187c29 100644
--- a/src/slave/containerizer/isolators/cgroups/cpushare.hpp
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.hpp
@@ -69,7 +69,8 @@ public:
 private:
   CgroupsCpushareIsolatorProcess(
       const Flags& flags,
-      const hashmap<std::string, std::string>& hierarchies);
+      const hashmap<std::string, std::string>& hierarchies,
+      const std::vector<std::string>& subsystems);
 
   virtual process::Future<std::list<Nothing> > _cleanup(
       const ContainerID& containerId,
@@ -92,6 +93,12 @@ private:
   // Map from subsystem to hierarchy.
   hashmap<std::string, std::string> hierarchies;
 
+  // Subsystems used for this isolator. Typically, there are two
+  // elements in the vector: 'cpu' and 'cpuacct'. If cpu and cpuacct
+  // systems are co-mounted (e.g., systems using systemd), then there
+  // will be only one element in the vector which is 'cpu,cpuacct'.
+  std::vector<std::string> subsystems;
+
   // TODO(bmahler): Use Owned<Info>.
   hashmap<ContainerID, Info*> infos;
 };

http://git-wip-us.apache.org/repos/asf/mesos/blob/31337348/src/slave/containerizer/isolators/cgroups/mem.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/mem.cpp b/src/slave/containerizer/isolators/cgroups/mem.cpp
index fb3db88..b3d4a5d 100644
--- a/src/slave/containerizer/isolators/cgroups/mem.cpp
+++ b/src/slave/containerizer/isolators/cgroups/mem.cpp
@@ -48,6 +48,7 @@ using namespace process;
 
 using std::list;
 using std::ostringstream;
+using std::set;
 using std::string;
 using std::vector;
 
@@ -74,12 +75,26 @@ CgroupsMemIsolatorProcess::~CgroupsMemIsolatorProcess() {}
 Try<Isolator*> CgroupsMemIsolatorProcess::create(const Flags& flags)
 {
   Try<string> hierarchy = cgroups::prepare(
-      flags.cgroups_hierarchy, "memory", flags.cgroups_root);
+      flags.cgroups_hierarchy,
+      "memory",
+      flags.cgroups_root);
 
   if (hierarchy.isError()) {
     return Error("Failed to create memory cgroup: " + hierarchy.error());
   }
 
+  // Ensure that no other subsystem is attached to the hierarchy.
+  Try<set<string> > subsystems = cgroups::subsystems(hierarchy.get());
+  if (subsystems.isError()) {
+    return Error(
+        "Failed to get the list of attached subsystems for hierarchy " +
+        hierarchy.get());
+  } else if (subsystems.get().size() != 1) {
+    return Error(
+        "Unexpected subsystems found attached to the hierarchy " +
+        hierarchy.get());
+  }
+
   // Make sure the kernel OOM-killer is enabled.
   // The Mesos OOM handler, as implemented, is not capable of handling
   // the oom condition by itself safely given the limitations Linux

http://git-wip-us.apache.org/repos/asf/mesos/blob/31337348/src/slave/containerizer/isolators/cgroups/perf_event.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/perf_event.cpp b/src/slave/containerizer/isolators/cgroups/perf_event.cpp
index ff047d3..4ced508 100644
--- a/src/slave/containerizer/isolators/cgroups/perf_event.cpp
+++ b/src/slave/containerizer/isolators/cgroups/perf_event.cpp
@@ -93,7 +93,9 @@ Try<Isolator*> CgroupsPerfEventIsolatorProcess::create(const Flags& flags)
   }
 
   Try<string> hierarchy = cgroups::prepare(
-      flags.cgroups_hierarchy, "perf_event", flags.cgroups_root);
+      flags.cgroups_hierarchy,
+      "perf_event",
+      flags.cgroups_root);
 
   if (hierarchy.isError()) {
     return Error("Failed to create perf_event cgroup: " + hierarchy.error());

http://git-wip-us.apache.org/repos/asf/mesos/blob/31337348/src/slave/containerizer/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/linux_launcher.cpp b/src/slave/containerizer/linux_launcher.cpp
index d5ef1d6..f7bc894 100644
--- a/src/slave/containerizer/linux_launcher.cpp
+++ b/src/slave/containerizer/linux_launcher.cpp
@@ -40,6 +40,7 @@ using namespace process;
 
 using std::list;
 using std::map;
+using std::set;
 using std::string;
 using std::vector;
 
@@ -75,6 +76,18 @@ Try<Launcher*> LinuxLauncher::create(const Flags& flags)
     return Error("Failed to create Linux launcher: " + hierarchy.error());
   }
 
+  // Ensure that no other subsystem is attached to the hierarchy.
+  Try<set<string> > subsystems = cgroups::subsystems(hierarchy.get());
+  if (subsystems.isError()) {
+    return Error(
+        "Failed to get the list of attached subsystems for hierarchy " +
+        hierarchy.get());
+  } else if (subsystems.get().size() != 1) {
+    return Error(
+        "Unexpected subsystems found attached to the hierarchy " +
+        hierarchy.get());
+  }
+
   LOG(INFO) << "Using " << hierarchy.get()
             << " as the freezer hierarchy for the Linux launcher";
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/31337348/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 28eb028..9a6646f 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -178,16 +178,19 @@ void Slave::initialize()
   }
 
 #ifdef __linux__
-  // Move the slave into its own cgroup for each of the specified subsystems.
-  // NOTE: Any subsystem configuration is inherited from the mesos root cgroup
-  // for that subsystem, e.g., by default the memory cgroup will be unlimited.
+  // Move the slave into its own cgroup for each of the specified
+  // subsystems.
+  // NOTE: Any subsystem configuration is inherited from the mesos
+  // root cgroup for that subsystem, e.g., by default the memory
+  // cgroup will be unlimited.
   if (flags.slave_subsystems.isSome()) {
     foreach (const string& subsystem,
             strings::tokenize(flags.slave_subsystems.get(), ",")) {
       LOG(INFO) << "Moving slave process into its own cgroup for"
                 << " subsystem: " << subsystem;
 
-      // Ensure the subsystem is mounted and the Mesos root cgroup is present.
+      // Ensure the subsystem is mounted and the Mesos root cgroup is
+      // present.
       Try<string> hierarchy = cgroups::prepare(
           flags.cgroups_hierarchy,
           subsystem,
@@ -196,20 +199,18 @@ void Slave::initialize()
       if (hierarchy.isError()) {
         EXIT(1) << "Failed to prepare cgroup " << flags.cgroups_root
                 << " for subsystem " << subsystem
-                << " under hierarchy " << hierarchy.get()
-                << " for slave: " + hierarchy.error();
+                << ": " << hierarchy.error();
       }
 
       // Create a cgroup for the slave.
       string cgroup = path::join(flags.cgroups_root, "slave");
 
       Try<bool> exists = cgroups::exists(hierarchy.get(), cgroup);
-
       if (exists.isError()) {
         EXIT(1) << "Failed to find cgroup " << cgroup
                 << " for subsystem " << subsystem
                 << " under hierarchy " << hierarchy.get()
-                << " for slave: " + exists.error();
+                << " for slave: " << exists.error();
       }
 
       if (!exists.get()) {
@@ -218,7 +219,7 @@ void Slave::initialize()
           EXIT(1) << "Failed to create cgroup " << cgroup
                   << " for subsystem " << subsystem
                   << " under hierarchy " << hierarchy.get()
-                  << " for slave: " + create.error();
+                  << " for slave: " << create.error();
         }
       }
 
@@ -229,17 +230,16 @@ void Slave::initialize()
         EXIT(1) << "Failed to check for existing threads in cgroup " << cgroup
                 << " for subsystem " << subsystem
                 << " under hierarchy " << hierarchy.get()
-                << " for slave: " + processes.error();
+                << " for slave: " << processes.error();
       }
 
-      // TODO(idownes): Re-evaluate this behavior if it's observed, possibly
-      // automatically killing any running processes and moving this code to
-      // during recovery.
+      // TODO(idownes): Re-evaluate this behavior if it's observed,
+      // possibly automatically killing any running processes and
+      // moving this code to during recovery.
       if (!processes.get().empty()) {
         EXIT(1) << "A slave (or child process) is still running, "
                 << "please check the process(es) '"
-                << stringify(processes.get())
-                << "' listed in "
+                << stringify(processes.get()) << "' listed in "
                 << path::join(hierarchy.get(), cgroup, "cgroups.proc");
       }
 
@@ -249,7 +249,7 @@ void Slave::initialize()
         EXIT(1) << "Failed to move slave into cgroup " << cgroup
                 << " for subsystem " << subsystem
                 << " under hierarchy " << hierarchy.get()
-                << " for slave: " + assign.error();
+                << " for slave: " << assign.error();
       }
     }
   }