You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by vi...@apache.org on 2013/03/13 07:22:51 UTC

svn commit: r1455809 - in /incubator/mesos/trunk/src: linux/cgroups.cpp linux/cgroups.hpp slave/cgroups_isolation_module.cpp slave/cgroups_isolation_module.hpp tests/cgroups_tests.cpp tests/utils.hpp

Author: vinodkone
Date: Wed Mar 13 06:22:51 2013
New Revision: 1455809

URL: http://svn.apache.org/r1455809
Log:
Added fixtures to enable templatized isolation module tests
and fixed a bug in cgroups isolation module.

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

Modified:
    incubator/mesos/trunk/src/linux/cgroups.cpp
    incubator/mesos/trunk/src/linux/cgroups.hpp
    incubator/mesos/trunk/src/slave/cgroups_isolation_module.cpp
    incubator/mesos/trunk/src/slave/cgroups_isolation_module.hpp
    incubator/mesos/trunk/src/tests/cgroups_tests.cpp
    incubator/mesos/trunk/src/tests/utils.hpp

Modified: incubator/mesos/trunk/src/linux/cgroups.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/linux/cgroups.cpp?rev=1455809&r1=1455808&r2=1455809&view=diff
==============================================================================
--- incubator/mesos/trunk/src/linux/cgroups.cpp (original)
+++ incubator/mesos/trunk/src/linux/cgroups.cpp Wed Mar 13 06:22:51 2013
@@ -43,6 +43,7 @@
 #include <stout/duration.hpp>
 #include <stout/error.hpp>
 #include <stout/foreach.hpp>
+#include <stout/hashset.hpp>
 #include <stout/lambda.hpp>
 #include <stout/none.hpp>
 #include <stout/option.hpp>
@@ -433,6 +434,34 @@ Try<set<string> > hierarchies()
 }
 
 
+Result<std::string> hierarchy(const std::string& subsystems)
+{
+  Result<std::string> hierarchy = None();
+  Try<std::set<std::string> > hierarchies = cgroups::hierarchies();
+  if (hierarchies.isError()) {
+    return Error(hierarchies.error());
+  }
+
+  foreach (const std::string& candidate, hierarchies.get()) {
+    if (subsystems.empty()) {
+      hierarchy = candidate;
+      break;
+    }
+
+    // Check and see if this candidate meets our subsystem requirements.
+    Try<bool> mounted = cgroups::mounted(candidate, subsystems);
+    if (mounted.isError()) {
+      return Error(mounted.error());
+    } else if (mounted.get()) {
+      hierarchy = candidate;
+      break;
+    }
+  }
+
+  return hierarchy;
+}
+
+
 Try<bool> enabled(const string& subsystems)
 {
   Try<map<string, internal::SubsystemInfo> > infosResult =
@@ -1616,11 +1645,6 @@ Future<bool> destroy(
     const string& cgroup,
     const Duration& interval)
 {
-  Option<string> error = verify(hierarchy, cgroup, "freezer.state");
-  if (error.isSome()) {
-    return Future<bool>::failed(error.get());
-  }
-
   if (interval < Seconds(0)) {
     return Future<bool>::failed("Interval should be non-negative");
   }
@@ -1632,16 +1656,83 @@ Future<bool> destroy(
         "Failed to get nested cgroups: " + cgroups.error());
   }
 
-  vector<string> toDestroy = cgroups.get();
+  vector<string> candidates = cgroups.get();
   if (cgroup != "/") {
-    toDestroy.push_back(cgroup);
+    candidates.push_back(cgroup);
   }
 
-  internal::Destroyer* destroyer =
-    new internal::Destroyer(hierarchy, toDestroy, interval);
-  Future<bool> future = destroyer->future();
-  spawn(destroyer, true);
-  return future;
+  if (candidates.empty()) {
+    return true;
+  }
+
+  // If the freezer subsystem is available, destroy the cgroups.
+  Option<string> error = verify(hierarchy, cgroup, "freezer.state");
+  if (error.isNone()) {
+    internal::Destroyer* destroyer =
+      new internal::Destroyer(hierarchy, candidates, interval);
+    Future<bool> future = destroyer->future();
+    spawn(destroyer, true);
+    return future;
+  } else {
+    // Otherwise, attempt to remove the cgroups in bottom-up fashion.
+    foreach (const std::string& cgroup, candidates) {
+      Try<Nothing> remove = cgroups::remove(hierarchy, cgroup);
+      if (remove.isError()) {
+        return Future<bool>::failed(remove.error());
+      }
+    }
+  }
+
+  return true;
+}
+
+
+// Forward declaration.
+Future<bool> _cleanup(const string& hierarchy);
+
+
+Future<bool> cleanup(const string& hierarchy)
+{
+  Try<bool> mounted = cgroups::mounted(hierarchy);
+  if (mounted.isError()) {
+    return Future<bool>::failed(mounted.error());
+  }
+
+  if (mounted.get()) {
+    // Destroy all cgroups and then cleanup.
+    return destroy(hierarchy)
+      .then(lambda::bind(_cleanup, hierarchy));
+  } else {
+    // Remove the directory if it still exists.
+    if (os::exists(hierarchy)) {
+      Try<Nothing> rmdir = os::rmdir(hierarchy);
+      if (rmdir.isError()) {
+        return Future<bool>::failed(rmdir.error());
+      }
+    }
+  }
+
+  return true;
+}
+
+
+Future<bool> _cleanup(const string& hierarchy)
+{
+  // Remove the hierarchy.
+  Try<Nothing> unmount = cgroups::unmount(hierarchy);
+  if (unmount.isError()) {
+    return Future<bool>::failed(unmount.error());
+  }
+
+  // Remove the directory if it still exists.
+  if (os::exists(hierarchy)) {
+    Try<Nothing> rmdir = os::rmdir(hierarchy);
+    if (rmdir.isError()) {
+      return Future<bool>::failed(rmdir.error());
+    }
+  }
+
+  return true;
 }
 
 

Modified: incubator/mesos/trunk/src/linux/cgroups.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/linux/cgroups.hpp?rev=1455809&r1=1455808&r2=1455809&view=diff
==============================================================================
--- incubator/mesos/trunk/src/linux/cgroups.hpp (original)
+++ incubator/mesos/trunk/src/linux/cgroups.hpp Wed Mar 13 06:22:51 2013
@@ -68,6 +68,18 @@ bool enabled();
 Try<std::set<std::string> > hierarchies();
 
 
+// Get an already mounted hierarchy that has 'subsystems' attached.
+// This function will return an error if we are unable to find the
+// hierarchies or if we are unable to find if the subsystems are
+// mounted at a given hierarchy.
+// @param subsystems Comma-separated subsystem names.
+// @return Path to the hierarchy root, if a hierarchy with all the
+//         given subsystems mounted exists.
+//         None, if no such hierarchy exists.
+//         Error, if the operation fails.
+Result<std::string> hierarchy(const std::string& subsystems);
+
+
 // Check whether all the given subsystems are enabled on the current machine.
 // @param   subsystems  Comma-separated subsystem names.
 // @return  True if all the given subsystems are enabled.
@@ -81,7 +93,7 @@ Try<bool> enabled(const std::string& sub
 // @param   subsystems  Comma-separated subsystem names.
 // @return  True if any of the given subsystems is being attached.
 //          False if non of the given subsystems is being attached.
-//          Error if some unexpected happens.
+//          Error if something unexpected happens.
 Try<bool> busy(const std::string& subsystems);
 
 
@@ -183,7 +195,7 @@ Try<std::vector<std::string> > get(
 // @param   cgroup      Path to the cgroup relative to the hierarchy root.
 // @param   signal      The signal to send to all tasks within the cgroup.
 // @return  Some on success.
-//          Error if some unexpected happens.
+//          Error if something unexpected happens.
 Try<Nothing> kill(
     const std::string& hierarchy,
     const std::string& cgroup,
@@ -267,7 +279,7 @@ Try<Nothing> assign(
 // @param   control     Name of the control file.
 // @param   args        Control specific arguments.
 // @return  A future which contains the value read from the file when ready.
-//          Error if some unexpected happens.
+//          Error if something unexpected happens.
 process::Future<uint64_t> listen(
     const std::string& hierarchy,
     const std::string& cgroup,
@@ -293,7 +305,7 @@ process::Future<uint64_t> listen(
 //                      indicates infinite retries. (default: 50 attempts).
 // @return  A future which will become true when all processes are frozen, or
 //          false when all retries have occurred unsuccessfully.
-//          Error if some unexpected happens.
+//          Error if something unexpected happens.
 process::Future<bool> freeze(
     const std::string& hierarchy,
     const std::string& cgroup,
@@ -310,32 +322,44 @@ process::Future<bool> freeze(
 // @param   interval    The time interval between two state check
 //                      requests (default: 0.1 seconds).
 // @return  A future which will become ready when all processes are thawed.
-//          Error if some unexpected happens.
+//          Error if something unexpected happens.
 process::Future<bool> thaw(
     const std::string& hierarchy,
     const std::string& cgroup,
     const Duration& interval = Seconds(0.1));
 
 
-// Destroy a cgroup under a given hierarchy. This function is different from
-// removeCgroup in that it tries to kill all tasks in the given cgroup so that
-// this cgroup can be removed. It will also recursively remove sub-cgroups if
-// exist. The given cgroup itself will also be destroyed. However, if the given
-// cgroup is the root cgroup, it will not be destroyed (cannot destroy a root
-// cgroup). The function returns a future indicating the state of the destroy
-// process. The future will become ready when the destroy operation finishes.
-// @param   hierarchy   Path to the hierarchy root.
+// Destroy a cgroup under a given hierarchy. It will also recursively
+// destroy any sub-cgroups. If the freezer subsystem is attached to
+// the hierarchy, we attempt to kill all tasks in a given cgroup,
+// before removing it. Otherwise, we just attempt to remove the
+// cgroup. This function will return an error if the given hierarchy
+// or the given cgroup does not exist or if we failed to destroy any
+// of the cgroups.
+// NOTE: If cgroup is "/" (default), all cgroups under the
+// hierarchy are destroyed.
+// TODO(vinod): Add support for killing tasks when freezer subsystem
+// is not present.
+// @param   hierarchy Path to the hierarchy root.
 // @param   cgroup      Path to the cgroup relative to the hierarchy root.
 // @param   interval    The time interval between two state check
 //                      requests (default: 0.1 seconds).
 // @return  A future which will become ready when the operation is done.
-//          Error if some unexpected happens.
+//          Error if something unexpected happens.
 process::Future<bool> destroy(
     const std::string& hierarchy,
     const std::string& cgroup = "/",
     const Duration& interval = Seconds(0.1));
 
 
+// Cleanup the hierarchy, by first destroying all the underlying
+// cgroups, unmounting the hierarchy and deleting the mount point.
+// @param   hierarchy Path to the hierarchy root.
+// @return  A future which will become ready when the operation is done.
+//          Error if something unexpected happens.
+process::Future<bool> cleanup(const std::string& hierarchy);
+
+
 // Returns the stat information from the given file.
 // @param   hierarchy   Path to the hierarchy root.
 // @param   cgroup      Path to the cgroup relative to the hierarchy root.

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=1455809&r1=1455808&r2=1455809&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/cgroups_isolation_module.cpp (original)
+++ incubator/mesos/trunk/src/slave/cgroups_isolation_module.cpp Wed Mar 13 06:22:51 2013
@@ -484,7 +484,7 @@ void CgroupsIsolationModule::launchExecu
   const ExecutorID& executorId = executorInfo.executor_id();
 
   // Register the cgroup information.
-  CgroupInfo* info = registerCgroupInfo(frameworkId, executorId);
+  CgroupInfo* info = registerCgroupInfo(frameworkId, executorId, flags);
 
   LOG(INFO) << "Launching " << executorId
             << " (" << executorInfo.command().value() << ")"
@@ -1003,7 +1003,8 @@ void CgroupsIsolationModule::_killExecut
 
 CgroupsIsolationModule::CgroupInfo* CgroupsIsolationModule::registerCgroupInfo(
     const FrameworkID& frameworkId,
-    const ExecutorID& executorId)
+    const ExecutorID& executorId,
+    const Flags& flags)
 {
   CgroupInfo* info = new CgroupInfo;
   info->frameworkId = frameworkId;
@@ -1014,6 +1015,7 @@ CgroupsIsolationModule::CgroupInfo* Cgro
   info->destroyed = false;
   info->status = -1;
   info->reason = "";
+  info->flags = flags;
   if (subsystems.contains("cpuset")) {
     info->cpuset = new Cpuset();
   } else {

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=1455809&r1=1455808&r2=1455809&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/cgroups_isolation_module.hpp (original)
+++ incubator/mesos/trunk/src/slave/cgroups_isolation_module.hpp Wed Mar 13 06:22:51 2013
@@ -134,7 +134,7 @@ private:
       out << "framework_" << frameworkId
           << "_executor_" << executorId
           << "_tag_" << tag;
-      return path::join("mesos", out.str());
+      return path::join(flags.cgroups_root, out.str());
     }
 
     FrameworkID frameworkId;
@@ -159,6 +159,8 @@ private:
 
     int status; // Exit status of the executor.
 
+    Flags flags; // Slave flags.
+
     // Used to cancel the OOM listening.
     process::Future<uint64_t> oomNotifier;
 
@@ -236,10 +238,12 @@ private:
   // Register a cgroup in the isolation module.
   // @param   frameworkId   The id of the given framework.
   // @param   executorId    The id of the given executor.
+  // @param   flags         The slave flags.
   // @return  A pointer to the cgroup info registered.
   CgroupInfo* registerCgroupInfo(
       const FrameworkID& frameworkId,
-      const ExecutorID& executorId);
+      const ExecutorID& executorId,
+      const Flags& flags);
 
   // Unregister a cgroup in the isolation module.
   // @param   frameworkId   The id of the given framework.

Modified: incubator/mesos/trunk/src/tests/cgroups_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/cgroups_tests.cpp?rev=1455809&r1=1455808&r2=1455809&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/cgroups_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/cgroups_tests.cpp Wed Mar 13 06:22:51 2013
@@ -51,6 +51,114 @@ using namespace mesos::internal::tests;
 using namespace process;
 
 
+class CgroupsTest : public ::testing::Test
+{
+public:
+  static void SetUpTestCase()
+  {
+    // Clean up the testing hierarchy, in case it wasn't cleaned up
+    // properly from previous tests.
+    ASSERT_FUTURE_WILL_SUCCEED(cgroups::cleanup(TEST_CGROUPS_HIERARCHY));
+  }
+
+  static void TearDownTestCase()
+  {
+    ASSERT_FUTURE_WILL_SUCCEED(cgroups::cleanup(TEST_CGROUPS_HIERARCHY));
+  }
+};
+
+
+// A fixture which is used to name tests that expect NO hierarchy to
+// exist in order to test the ability to create a hierarchy (since
+// most likely existing hierarchies will have all or most subsystems
+// attached rendering our ability to create a hierarchy fruitless).
+class CgroupsNoHierarchyTest : public CgroupsTest
+{
+public:
+  static void SetUpTestCase()
+  {
+    CgroupsTest::SetUpTestCase();
+
+    Try<std::set<std::string> > hierarchies = cgroups::hierarchies();
+    ASSERT_SOME(hierarchies);
+    if (!hierarchies.get().empty()) {
+      std::cerr
+        << "-------------------------------------------------------------\n"
+        << "We cannot run any cgroups tests that require mounting\n"
+        << "hierarchies because you have the following hierarchies mounted:\n"
+        << strings::trim(stringify(hierarchies.get()), " {},") << "\n"
+        << "You can either unmount those hierarchies, or disable\n"
+        << "this test case (i.e., --gtest_filter=-CgroupsNoHierarchyTest.*).\n"
+        << "-------------------------------------------------------------"
+        << std::endl;
+    }
+  }
+};
+
+
+// A fixture that assumes ANY hierarchy is acceptable for use provided
+// it has the subsystems attached that were specified in the
+// constructor. If no hierarchy could be found that has all the
+// required subsystems then we attempt to create a new hierarchy.
+class CgroupsAnyHierarchyTest : public CgroupsTest
+{
+public:
+  CgroupsAnyHierarchyTest(const std::string& _subsystems = "cpu")
+    : subsystems(_subsystems) {}
+
+protected:
+  virtual void SetUp()
+  {
+    Result<std::string> hierarchy_ = cgroups::hierarchy(subsystems);
+    ASSERT_FALSE(hierarchy_.isError());
+    if (hierarchy_.isNone()) {
+      // Try to mount a hierarchy for testing.
+      ASSERT_SOME(cgroups::mount(TEST_CGROUPS_HIERARCHY, subsystems))
+        << "-------------------------------------------------------------\n"
+        << "We cannot run any cgroups tests that require\n"
+        << "a hierarchy with subsystems '" << subsystems << "'\n"
+        << "because we failed to find an existing hierarchy\n"
+        << "or create a new one. You can either remove all existing\n"
+        << "hierarchies, or disable this test case\n"
+        << "(i.e., --gtest_filter=-"
+        << ::testing::UnitTest::GetInstance()
+             ->current_test_info()
+             ->test_case_name() << ".*).\n"
+        << "-------------------------------------------------------------";
+
+      hierarchy = TEST_CGROUPS_HIERARCHY;
+    } else {
+      hierarchy = hierarchy_.get();
+    }
+
+    // Create a cgroup (removing first if necessary) for the tests to use.
+    Try<bool> exists = cgroups::exists(hierarchy, TEST_CGROUPS_ROOT);
+    ASSERT_SOME(exists);
+    if (exists.get()) {
+     ASSERT_FUTURE_WILL_SUCCEED(cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT));
+    }
+    ASSERT_SOME(cgroups::create(hierarchy, TEST_CGROUPS_ROOT));
+  }
+
+  virtual void TearDown()
+  {
+    // Remove all *our* cgroups.
+    Try<bool> exists = cgroups::exists(hierarchy, TEST_CGROUPS_ROOT);
+    ASSERT_SOME(exists);
+    if (exists.get()) {
+     ASSERT_FUTURE_WILL_SUCCEED(cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT));
+    }
+
+    // And cleanup TEST_CGROUPS_HIERARCHY in the event it is needed
+    // to be created.
+    ASSERT_FUTURE_WILL_SUCCEED(cgroups::cleanup(TEST_CGROUPS_HIERARCHY));
+  }
+
+  const std::string subsystems; // Subsystems required to run tests.
+  std::string hierarchy; // Path to the hierarchy being used.
+};
+
+
 class CgroupsAnyHierarchyWithCpuMemoryTest
   : public CgroupsAnyHierarchyTest
 {

Modified: incubator/mesos/trunk/src/tests/utils.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/utils.hpp?rev=1455809&r1=1455808&r2=1455809&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/utils.hpp (original)
+++ incubator/mesos/trunk/src/tests/utils.hpp Wed Mar 13 06:22:51 2013
@@ -59,6 +59,9 @@
 
 #include "messages/messages.hpp"
 
+#ifdef __linux__
+#include "slave/cgroups_isolation_module.hpp"
+#endif
 #include "slave/isolation_module.hpp"
 #include "slave/slave.hpp"
 
@@ -78,6 +81,8 @@ extern flags::Flags<logging::Flags, Flag
 
 
 #ifdef __linux__
+using slave::CgroupsIsolationModule;
+
 // Cgroups hierarchy used by the cgroups related tests.
 const static std::string TEST_CGROUPS_HIERARCHY = "/tmp/mesos_test_cgroup";
 
@@ -122,10 +127,6 @@ protected:
 
     slaveFlags.work_dir = directory.get();
     slaveFlags.launcher_dir = path::join(tests::flags.build_dir, "src");
-#ifdef __linux__
-    slaveFlags.cgroups_hierarchy = TEST_CGROUPS_HIERARCHY;
-    slaveFlags.cgroups_root = TEST_CGROUPS_ROOT;
-#endif
 
     // For locating killtree.sh.
     os::setenv("MESOS_SOURCE_DIR",tests::flags.source_dir);
@@ -146,107 +147,39 @@ protected:
   }
 
   flags::Flags<logging::Flags, slave::Flags> slaveFlags;
+  const std::string hierarchy;
 };
 
 
+template <typename T>
+class IsolationTest : public MesosTest
+{};
+
+
 #ifdef __linux__
-class CgroupsTest : public MesosTest
+template <>
+class IsolationTest<CgroupsIsolationModule> : public MesosTest
 {
-protected:
+public:
   static void SetUpTestCase()
   {
     // Clean up the testing hierarchy, in case it wasn't cleaned up
     // properly from previous tests.
-    TearDownTestCase();
+    ASSERT_FUTURE_WILL_SUCCEED(cgroups::cleanup(TEST_CGROUPS_HIERARCHY));
   }
 
   static void TearDownTestCase()
   {
-    // Remove the testing hierarchy.
-    Try<bool> mounted = cgroups::mounted(TEST_CGROUPS_HIERARCHY);
-    ASSERT_SOME(mounted);
-    if (mounted.get()) {
-      // Remove all cgroups.
-      Try<std::vector<std::string> > cgroups =
-        cgroups::get(TEST_CGROUPS_HIERARCHY);
-
-      ASSERT_SOME(cgroups);
-      foreach (const std::string& cgroup, cgroups.get()) {
-        ASSERT_FUTURE_WILL_SUCCEED(
-            cgroups::destroy(TEST_CGROUPS_HIERARCHY, cgroup));
-      }
-
-      // Remove the hierarchy.
-      ASSERT_SOME(cgroups::unmount(TEST_CGROUPS_HIERARCHY));
-
-      // Remove the directory if still exists.
-      if (os::exists(TEST_CGROUPS_HIERARCHY)) {
-        os::rmdir(TEST_CGROUPS_HIERARCHY);
-      }
-    }
+    ASSERT_FUTURE_WILL_SUCCEED(cgroups::cleanup(TEST_CGROUPS_HIERARCHY));
   }
-};
-
-
-// A fixture which is used to name tests that expect NO hierarchy to
-// exist in order to test the ability to create a hierarchy (since
-// most likely existing hierarchies will have all or most subsystems
-// attached rendering our ability to create a hierarchy fruitless).
-class CgroupsNoHierarchyTest : public CgroupsTest
-{
-protected:
-  static void SetUpTestCase()
-  {
-    CgroupsTest::SetUpTestCase();
-
-    Try<std::set<std::string> > hierarchies = cgroups::hierarchies();
-    ASSERT_SOME(hierarchies);
-    if (!hierarchies.get().empty()) {
-      std::cerr
-        << "-------------------------------------------------------------\n"
-        << "We cannot run any cgroups tests that require mounting\n"
-        << "hierarchies because you have the following hierarchies mounted:\n"
-        << strings::trim(stringify(hierarchies.get()), " {},") << "\n"
-        << "You can either unmount those hierarchies, or disable\n"
-        << "this test case (i.e., --gtest_filter=-CgroupsNoHierarchyTest.*).\n"
-        << "-------------------------------------------------------------"
-        << std::endl;
-    }
-  }
-};
-
-
-// A fixture that assumes ANY hierarchy is acceptable for use provided
-// it has the subsystems attached that were specified in the
-// constructor. If no hierarchy could be found that has all the
-// required subsystems then we attempt to create a new hierarchy.
-class CgroupsAnyHierarchyTest : public CgroupsTest
-{
-public:
-  CgroupsAnyHierarchyTest(const std::string& _subsystems = "cpu")
-    : subsystems(_subsystems) {}
 
 protected:
   virtual void SetUp()
   {
-    Try<std::set<std::string> > hierarchies = cgroups::hierarchies();
-    ASSERT_SOME(hierarchies);
-    foreach (const std::string& candidate, hierarchies.get()) {
-      if (subsystems.empty()) {
-        hierarchy = candidate;
-        break;
-      }
-
-      // Check and see if this candidate meets our subsystem requirements.
-      Try<bool> mounted = cgroups::mounted(candidate, subsystems);
-      ASSERT_SOME(mounted);
-      if (mounted.get()) {
-        hierarchy = candidate;
-        break;
-      }
-    }
-
-    if (hierarchy.empty()) {
+    const std::string subsystems = "cpu,cpuacct,memory,freezer";
+    Result<std::string> hierarchy_ = cgroups::hierarchy(subsystems);
+    ASSERT_FALSE(hierarchy_.isError());
+    if (hierarchy_.isNone()) {
       // Try to mount a hierarchy for testing.
       ASSERT_SOME(cgroups::mount(TEST_CGROUPS_HIERARCHY, subsystems))
         << "-------------------------------------------------------------\n"
@@ -262,41 +195,30 @@ protected:
         << "-------------------------------------------------------------";
 
       hierarchy = TEST_CGROUPS_HIERARCHY;
+    } else {
+      hierarchy = hierarchy_.get();
     }
 
-    // Create a cgroup (removing first if necessary) for the tests to use.
-    Try<bool> exists = cgroups::exists(hierarchy, TEST_CGROUPS_ROOT);
-    ASSERT_SOME(exists);
-    if (exists.get()) {
-      ASSERT_FUTURE_WILL_SUCCEED(cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT))
-        << "-------------------------------------------------------------\n"
-        << "We failed to destroy our \"testing\" cgroup (most likely left\n"
-        << "around from a previously failing test). This is a pretty\n"
-        << "serious error, please report a bug!\n"
-        << "-------------------------------------------------------------";
-    }
+    MesosTest::SetUp();
 
-    ASSERT_SOME(cgroups::create(hierarchy, TEST_CGROUPS_ROOT));
+    // Set slave's cgroup flags.
+    slaveFlags.cgroups_hierarchy = hierarchy;
+    slaveFlags.cgroups_root = TEST_CGROUPS_ROOT;
   }
 
   virtual void TearDown()
   {
-    // Remove all *our* cgroups.
-    Try<std::vector<std::string> > cgroups = cgroups::get(hierarchy);
-    ASSERT_SOME(cgroups);
-    foreach (const std::string& cgroup, cgroups.get()) {
-      if (strings::startsWith(cgroup, TEST_CGROUPS_ROOT)) {
-        ASSERT_FUTURE_WILL_SUCCEED(cgroups::destroy(hierarchy, cgroup));
-      }
-    }
+    MesosTest::TearDown();
 
-    // And destroy TEST_CGROUPS_HIERARCHY in the event it is needed
-    // to be created.
-    CgroupsTest::TearDownTestCase();
+    Try<bool> exists = cgroups::exists(hierarchy, TEST_CGROUPS_ROOT);
+    ASSERT_SOME(exists);
+    if (exists.get()) {
+     ASSERT_FUTURE_WILL_SUCCEED(cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT));
+    }
   }
 
-  const std::string subsystems; // Subsystems required to run tests.
-  std::string hierarchy; // Path to the hierarchy being used.
+private:
+  std::string hierarchy;
 };
 #endif