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/27 00:07:06 UTC

svn commit: r1402689 - /incubator/mesos/trunk/src/tests/cgroups_tests.cpp

Author: benh
Date: Fri Oct 26 22:07:06 2012
New Revision: 1402689

URL: http://svn.apache.org/viewvc?rev=1402689&view=rev
Log:
Refactored the cgroups tests so that they can run in a multitude of
environments. See each bullet below.

* Added new test fixtures:
  (1) CgroupsNoHierarchyTest for running tests where we need to create
      a hierarchy (and thus, most likely, no existing hierarchy can
      exist because it will already have the cpu and or memory
      subsystem attached).
  (2) CgroupsAnyHierarchyTest and subclasses
      CgroupsAnyHierarchyWithCpuMemoryTest and
      CgroupsAnyHierarchyWithCpuMemoryFreezerTest for running tests
      with any hierarchy provided it has necessary subsystems
      attached.

* Renamed cgroups (from "prof", "stu", etc. to "mesos_test") and
  removed nested cgroups by default. The rename was done because we
  might run tests inside of an existing hierarchy and we want to avoid
  name clashes. The nested cgroups were removed in favor of a test
  that explicitly tries to create nested cgroups (since some older
  kernels with particular subsystems attached have a hard time with
  this, and we'd like to detect that case explicitly).

* Created an explicit test for nested cgroups (see above).

* Updated the "write control" test to use a forked process rather than
  the test process (to be more conservative in the presence of errors)

* Updated the "listen event" (i.e., "oom") test to check for the
  proper control first (memory.oom_control).

* Updated the failure mechanism of forked (children) processes to use
  'abort' rather than 'ASSERT_*' and 'FAIL' in order to make test
  output more readable upon failures.

* Updated the notify mechanism from forked (children) processes to
  parent processes to correctly distinguish a closed pipe from a value
  written (to catch more instances of when the test is actually
  failing).

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

Modified:
    incubator/mesos/trunk/src/tests/cgroups_tests.cpp

Modified: incubator/mesos/trunk/src/tests/cgroups_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/cgroups_tests.cpp?rev=1402689&r1=1402688&r2=1402689&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/cgroups_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/cgroups_tests.cpp Fri Oct 26 22:07:06 2012
@@ -19,10 +19,15 @@
 #include <assert.h>
 #include <errno.h>
 #include <signal.h>
+#include <stdio.h>
 #include <stdlib.h>
 #include <string.h>
 #include <unistd.h>
 
+#include <set>
+#include <string>
+#include <vector>
+
 #include <sys/mman.h>
 #include <sys/types.h>
 #include <sys/wait.h>
@@ -31,6 +36,7 @@
 
 #include <stout/option.hpp>
 #include <stout/os.hpp>
+#include <stout/path.hpp>
 #include <stout/stringify.hpp>
 #include <stout/strings.hpp>
 
@@ -40,78 +46,176 @@
 
 using namespace process;
 
+const static std::string HIERARCHY = "/tmp/mesos_cgroups_testing_hierarchy";
+
 
-// Define the test fixture for the cgroups tests.
 class CgroupsTest : public ::testing::Test
 {
 protected:
-  virtual void SetUp()
-  {
-    cleanup();
-    prepare();
-  }
-
-  virtual void TearDown()
-  {
-    cleanup();
-  }
-
-  // Prepare the testing hierarchy and cgroups.
-  void prepare()
+  static void SetUpTestCase()
   {
-    // Create a hierarchy for test.
-    std::string subsystems = "cpu,memory,freezer";
-    ASSERT_SOME(cgroups::createHierarchy(hierarchy, subsystems));
-
-    // Create cgroups for test.
-    ASSERT_SOME(cgroups::createCgroup(hierarchy, "prof"));
-    ASSERT_SOME(cgroups::createCgroup(hierarchy, "stu"));
-    ASSERT_SOME(cgroups::createCgroup(hierarchy, "stu/grad"));
-    ASSERT_SOME(cgroups::createCgroup(hierarchy, "stu/under"));
-    ASSERT_SOME(cgroups::createCgroup(hierarchy, "stu/under/senior"));
+    TearDownTestCase();
   }
 
-  void cleanup()
+  static void TearDownTestCase()
   {
-    if (cgroups::checkHierarchy(hierarchy).isSome()) {
+    // Remove the testing hierarchy.
+    if (cgroups::checkHierarchy(HIERARCHY).isSome()) {
       // Remove all cgroups.
-      Try<std::vector<std::string> > cgroups = cgroups::getCgroups(hierarchy);
+      Try<std::vector<std::string> > cgroups = cgroups::getCgroups(HIERARCHY);
       ASSERT_SOME(cgroups);
       foreach (const std::string& cgroup, cgroups.get()) {
-        ASSERT_SOME(cgroups::removeCgroup(hierarchy, cgroup));
+        ASSERT_SOME(cgroups::removeCgroup(HIERARCHY, cgroup));
       }
 
       // Remove the hierarchy.
-      ASSERT_SOME(cgroups::removeHierarchy(hierarchy));
-    }
+      ASSERT_SOME(cgroups::removeHierarchy(HIERARCHY));
 
-    // Remove the directory if still exists.
-    if (os::exists(hierarchy)) {
-      os::rmdir(hierarchy);
+      // Remove the directory if still exists.
+      if (os::exists(HIERARCHY)) {
+        os::rmdir(HIERARCHY);
+      }
     }
   }
-
-  // Path to the root hierarchy for tests.
-  static const std::string hierarchy;
 };
 
 
-// Define the test fixture for the simple cgroups tests. Simple cgroups tests do
-// not prepare testing hierarchy and cgroups.
-class CgroupsSimpleTest : public CgroupsTest
+// 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 creating\n"
+        << "hierarchies because you have the following hierarchies active:\n"
+        << strings::trim(stringify(hierarchies.get()), " {},") << "\n"
+        << "You can either remove 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:
+  static void SetUpTestCase()
+  {
+    CgroupsTest::SetUpTestCase();
+  }
+
+  static void TearDownTestCase()
+  {
+    CgroupsTest::TearDownTestCase();
+  }
+
   virtual void SetUp()
   {
-    cleanup();
+    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.
+      if (cgroups::checkHierarchy(candidate, subsystems).isSome()) {
+        hierarchy = candidate;
+        break;
+      }
+    }
+
+    if (hierarchy.empty()) {
+      // Create a hierarchy for testing.
+      ASSERT_SOME(cgroups::createHierarchy(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 = HIERARCHY;
+    }
+
+    // Create a cgroup (removing first if necessary) for the tests to use.
+    if (cgroups::checkCgroup(hierarchy, "mesos_test").isSome()) {
+      ASSERT_FUTURE_WILL_SUCCEED(cgroups::destroyCgroup(hierarchy, "mesos_test"))
+        << "-------------------------------------------------------------\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"
+        << "-------------------------------------------------------------";
+    }
+
+    ASSERT_SOME(cgroups::createCgroup(hierarchy, "mesos_test"));
   }
+
+  virtual void TearDown()
+  {
+    // Remove all *our* cgroups.
+    Try<std::vector<std::string> > cgroups = cgroups::getCgroups(hierarchy);
+    ASSERT_SOME(cgroups);
+    foreach (const std::string& cgroup, cgroups.get()) {
+      if (strings::startsWith(cgroup, "mesos_test")) {
+        ASSERT_SOME(cgroups::removeCgroup(hierarchy, cgroup));
+      }
+    }
+
+    // And destroy HIERARCHY in the event it needed to be created.
+    CgroupsTest::TearDownTestCase();
+  }
+
+  const std::string subsystems; // Subsystems required to run tests.
+  std::string hierarchy; // Path to the hierarchy being used.
+};
+
+
+class CgroupsAnyHierarchyWithCpuMemoryTest
+  : public CgroupsAnyHierarchyTest
+{
+public:
+  CgroupsAnyHierarchyWithCpuMemoryTest()
+    : CgroupsAnyHierarchyTest("cpu,memory") {}
 };
 
 
-const std::string CgroupsTest::hierarchy = "/tmp/mesos_cgroups_test_hierarchy";
+class CgroupsAnyHierarchyWithCpuMemoryFreezerTest
+  : public CgroupsAnyHierarchyTest
+{
+public:
+  CgroupsAnyHierarchyWithCpuMemoryFreezerTest()
+    : CgroupsAnyHierarchyTest("cpu,memory,freezer") {}
+};
 
 
-TEST_F(CgroupsSimpleTest, ROOT_CGROUPS_Enabled)
+TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Enabled)
 {
   EXPECT_SOME_TRUE(cgroups::enabled("cpu"));
   EXPECT_SOME_TRUE(cgroups::enabled(",cpu"));
@@ -124,7 +228,7 @@ TEST_F(CgroupsSimpleTest, ROOT_CGROUPS_E
 }
 
 
-TEST_F(CgroupsTest, ROOT_CGROUPS_Busy)
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_Busy)
 {
   EXPECT_ERROR(cgroups::busy("invalid"));
   EXPECT_ERROR(cgroups::busy("cpu,invalid"));
@@ -137,7 +241,7 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_Busy)
 }
 
 
-TEST_F(CgroupsSimpleTest, ROOT_CGROUPS_Subsystems)
+TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Subsystems)
 {
   Try<std::set<std::string> > names = cgroups::subsystems();
   ASSERT_SOME(names);
@@ -157,7 +261,7 @@ TEST_F(CgroupsSimpleTest, ROOT_CGROUPS_S
 }
 
 
-TEST_F(CgroupsTest, ROOT_CGROUPS_SubsystemsHierarchy)
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_SubsystemsHierarchy)
 {
   Try<std::set<std::string> > names = cgroups::subsystems(hierarchy);
   ASSERT_SOME(names);
@@ -177,77 +281,95 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_Subsyst
 }
 
 
-TEST_F(CgroupsSimpleTest, ROOT_CGROUPS_CreateRemoveHierarchy)
+TEST_F(CgroupsNoHierarchyTest, ROOT_CGROUPS_CreateRemoveHierarchy)
 {
   EXPECT_ERROR(cgroups::createHierarchy("/tmp", "cpu"));
-
-  EXPECT_ERROR(cgroups::createHierarchy(hierarchy, "invalid"));
-
-  ASSERT_SOME(cgroups::createHierarchy(hierarchy, "cpu,memory"));
-
-  EXPECT_ERROR(cgroups::createHierarchy(hierarchy, "cpuset"));
-
+  EXPECT_ERROR(cgroups::createHierarchy(HIERARCHY, "invalid"));
+  ASSERT_SOME(cgroups::createHierarchy(HIERARCHY, "cpu,memory"));
+  EXPECT_ERROR(cgroups::createHierarchy(HIERARCHY, "cpuset"));
   EXPECT_ERROR(cgroups::removeHierarchy("/tmp"));
-
-  ASSERT_SOME(cgroups::removeHierarchy(hierarchy));
+  ASSERT_SOME(cgroups::removeHierarchy(HIERARCHY));
 }
 
 
-TEST_F(CgroupsTest, ROOT_CGROUPS_CheckHierarchy)
+TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_CheckHierarchy)
 {
   EXPECT_ERROR(cgroups::checkHierarchy("/tmp-nonexist"));
-
   EXPECT_ERROR(cgroups::checkHierarchy("/tmp"));
-
   EXPECT_SOME(cgroups::checkHierarchy(hierarchy));
-
   EXPECT_SOME(cgroups::checkHierarchy(hierarchy + "/"));
-
-  EXPECT_ERROR(cgroups::checkHierarchy(hierarchy + "/stu"));
+  EXPECT_ERROR(cgroups::checkHierarchy(hierarchy + "/not_expected"));
 }
 
 
-TEST_F(CgroupsTest, ROOT_CGROUPS_CheckHierarchySubsystems)
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_CheckHierarchySubsystems)
 {
   EXPECT_ERROR(cgroups::checkHierarchy("/tmp-nonexist", "cpu"));
-
   EXPECT_ERROR(cgroups::checkHierarchy("/tmp", "cpu,memory"));
-
   EXPECT_ERROR(cgroups::checkHierarchy("/tmp", "cpu"));
-
   EXPECT_ERROR(cgroups::checkHierarchy("/tmp", "invalid"));
-
   EXPECT_SOME(cgroups::checkHierarchy(hierarchy, "cpu,memory"));
-
   EXPECT_SOME(cgroups::checkHierarchy(hierarchy, "memory"));
-
   EXPECT_ERROR(cgroups::checkHierarchy(hierarchy, "invalid"));
-
-  EXPECT_ERROR(cgroups::checkHierarchy(hierarchy + "/stu", "cpu"));
+  EXPECT_ERROR(cgroups::checkHierarchy(hierarchy + "/not_expected", "cpu"));
 }
 
 
-TEST_F(CgroupsSimpleTest, ROOT_CGROUPS_CreateRemoveCgroup)
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_CreateRemoveCgroup)
 {
   EXPECT_ERROR(cgroups::createCgroup("/tmp", "test"));
+  EXPECT_ERROR(cgroups::createCgroup(hierarchy, "mesos_test_missing/1"));
+  ASSERT_SOME(cgroups::createCgroup(hierarchy, "mesos_test_missing"));
+  EXPECT_ERROR(cgroups::removeCgroup(hierarchy, "invalid"));
+  ASSERT_SOME(cgroups::removeCgroup(hierarchy, "mesos_test_missing"));
+}
 
-  ASSERT_SOME(cgroups::createHierarchy(hierarchy, "cpu,memory"));
 
-  EXPECT_ERROR(cgroups::createCgroup(hierarchy, "test/1"));
+TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_GetCgroups)
+{
+  ASSERT_SOME(cgroups::createCgroup(hierarchy, "mesos_test1"));
+  ASSERT_SOME(cgroups::createCgroup(hierarchy, "mesos_test2"));
 
-  ASSERT_SOME(cgroups::createCgroup(hierarchy, "test"));
+  Try<std::vector<std::string> > cgroups = cgroups::getCgroups(hierarchy);
+  ASSERT_SOME(cgroups);
+
+  EXPECT_EQ(cgroups.get()[0], "mesos_test2");
+  EXPECT_EQ(cgroups.get()[1], "mesos_test1");
+  EXPECT_EQ(cgroups.get()[2], "mesos_test");
+
+  ASSERT_SOME(cgroups::removeCgroup(hierarchy, "mesos_test1"));
+  ASSERT_SOME(cgroups::removeCgroup(hierarchy, "mesos_test2"));
+}
 
-  EXPECT_ERROR(cgroups::removeCgroup(hierarchy, "invalid"));
 
-  ASSERT_SOME(cgroups::removeCgroup(hierarchy, "test"));
+TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_NestedCgroups)
+{
+  ASSERT_SOME(cgroups::createCgroup(hierarchy, "mesos_test/1"))
+    << "-------------------------------------------------------------\n"
+    << "We cannot run this test because it appears you do not have\n"
+    << "a modern enough version of the Linux kernel. You won't be\n"
+    << "able to use the cgroups isolation module, but feel free to\n"
+    << "disable this test.\n"
+    << "-------------------------------------------------------------";
+
+  ASSERT_SOME(cgroups::createCgroup(hierarchy, "mesos_test/2"));
+
+  Try<std::vector<std::string> > cgroups =
+    cgroups::getCgroups(hierarchy, "mesos_test");
+  ASSERT_SOME(cgroups);
+  ASSERT_EQ(2u, cgroups.get().size());
 
-  ASSERT_SOME(cgroups::removeHierarchy(hierarchy));
+  EXPECT_EQ(cgroups.get()[0], "mesos_test/2");
+  EXPECT_EQ(cgroups.get()[1], "mesos_test/1");
+
+  ASSERT_SOME(cgroups::removeCgroup(hierarchy, "mesos_test/1"));
+  ASSERT_SOME(cgroups::removeCgroup(hierarchy, "mesos_test/2"));
 }
 
 
-TEST_F(CgroupsTest, ROOT_CGROUPS_ReadControl)
+TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_ReadControl)
 {
-  EXPECT_ERROR(cgroups::readControl(hierarchy, "/stu", "invalid"));
+  EXPECT_ERROR(cgroups::readControl(hierarchy, "mesos_test", "invalid"));
 
   std::string pid = stringify(::getpid());
 
@@ -257,53 +379,45 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_ReadCon
 }
 
 
-TEST_F(CgroupsTest, ROOT_CGROUPS_WriteControl)
+TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_WriteControl)
 {
   EXPECT_ERROR(cgroups::writeControl(hierarchy,
-                                    "/prof",
-                                    "invalid",
-                                    "invalid"));
-
-  std::string pid = stringify(::getpid());
-
-  ASSERT_SOME(cgroups::writeControl(hierarchy, "/prof", "tasks", pid));
-
-  Try<std::set<pid_t> > tasks = cgroups::getTasks(hierarchy, "/prof");
-  ASSERT_SOME(tasks);
-
-  std::set<pid_t> pids = tasks.get();
-  EXPECT_NE(pids.find(::getpid()), pids.end());
-
-  ASSERT_SOME(cgroups::writeControl(hierarchy, "/", "tasks", pid));
-}
+                                     "mesos_test",
+                                     "invalid",
+                                     "invalid"));
 
+  pid_t pid = ::fork();
+  ASSERT_NE(-1, pid);
 
-TEST_F(CgroupsTest, ROOT_CGROUPS_GetCgroups)
-{
-  Try<std::vector<std::string> > cgroups = cgroups::getCgroups(hierarchy);
-  ASSERT_SOME(cgroups);
+  if (pid) {
+    // In parent process.
+    ASSERT_SOME(cgroups::writeControl(
+                    hierarchy, "mesos_test", "tasks", stringify(pid)));
 
-  EXPECT_EQ(cgroups.get()[0], "/stu/under/senior");
-  EXPECT_EQ(cgroups.get()[1], "/stu/under");
-  EXPECT_EQ(cgroups.get()[2], "/stu/grad");
-  EXPECT_EQ(cgroups.get()[3], "/stu");
-  EXPECT_EQ(cgroups.get()[4], "/prof");
+    Try<std::set<pid_t> > tasks = cgroups::getTasks(hierarchy, "mesos_test");
+    ASSERT_SOME(tasks);
 
-  cgroups = cgroups::getCgroups(hierarchy, "/stu");
-  ASSERT_SOME(cgroups);
+    std::set<pid_t> pids = tasks.get();
+    EXPECT_NE(pids.find(pid), pids.end());
 
-  EXPECT_EQ(cgroups.get()[0], "/stu/under/senior");
-  EXPECT_EQ(cgroups.get()[1], "/stu/under");
-  EXPECT_EQ(cgroups.get()[2], "/stu/grad");
+    // Kill the child process.
+    ASSERT_NE(-1, ::kill(pid, SIGKILL));
 
-  cgroups = cgroups::getCgroups(hierarchy, "/prof");
-  ASSERT_SOME(cgroups);
+    // Wait for the child process.
+    int status;
+    EXPECT_NE(-1, ::waitpid((pid_t) -1, &status, 0));
+  } else {
+    // In child process.
+    while (true);
 
-  EXPECT_TRUE(cgroups.get().empty());
+    // Should not reach here.
+    std::cerr << "Reach an unreachable statement!" << std::endl;
+    abort();
+  }
 }
 
 
-TEST_F(CgroupsTest, ROOT_CGROUPS_GetTasks)
+TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_GetTasks)
 {
   Try<std::set<pid_t> > tasks = cgroups::getTasks(hierarchy, "/");
   ASSERT_SOME(tasks);
@@ -314,25 +428,33 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_GetTask
 }
 
 
-TEST_F(CgroupsTest, ROOT_CGROUPS_ListenEvent)
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_ListenEvent)
 {
+  ASSERT_SOME(cgroups::checkControl(hierarchy, "mesos_test", "memory.oom_control"))
+    << "-------------------------------------------------------------\n"
+    << "We cannot run this test because it appears you do not have\n"
+    << "a modern enough version of the Linux kernel. You won't be\n"
+    << "able to use the cgroups isolation module, but feel free to\n"
+    << "disable this test.\n"
+    << "-------------------------------------------------------------";
+
   // Disable oom killer.
   ASSERT_SOME(cgroups::writeControl(hierarchy,
-                                    "/prof",
+                                    "mesos_test",
                                     "memory.oom_control",
                                     "1"));
 
-  // Limit the memory usage of "/prof" to 64MB.
+  // Limit the memory usage of "mesos_test" to 64MB.
   size_t limit = 1024 * 1024 * 64;
   ASSERT_SOME(cgroups::writeControl(hierarchy,
-                                    "/prof",
+                                    "mesos_test",
                                     "memory.limit_in_bytes",
                                     stringify(limit)));
 
-  // Listen on oom events for "/prof" cgroup.
+  // Listen on oom events for "mesos_test" cgroup.
   Future<uint64_t> future =
     cgroups::listenEvent(hierarchy,
-                         "/prof",
+                         "mesos_test",
                          "memory.oom_control");
   ASSERT_FALSE(future.isFailed());
 
@@ -341,7 +463,7 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_ListenE
 
   // Test the normal operation below.
   future = cgroups::listenEvent(hierarchy,
-                                "/prof",
+                                "mesos_test",
                                 "memory.oom_control");
   ASSERT_FALSE(future.isFailed());
 
@@ -362,12 +484,12 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_ListenE
     EXPECT_NE(-1, ::waitpid((pid_t) -1, &status, 0));
   } else {
     // In child process. We try to trigger an oom here.
-    // Put self into the "/prof" cgroup.
-    Try<Nothing> assignResult = cgroups::assignTask(hierarchy,
-                                                    "/prof",
-                                                    ::getpid());
-    if (assignResult.isError()) {
-      FAIL() << "Failed to assign cgroup: " << assignResult.error();
+    // Put self into the "mesos_test" cgroup.
+    Try<Nothing> assign =
+      cgroups::assignTask(hierarchy, "mesos_test", ::getpid());
+    if (assign.isError()) {
+      std::cerr << "Failed to assign cgroup: " << assign.error() << std::endl;
+      abort();
     }
 
     // Blow up the memory.
@@ -375,27 +497,30 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_ListenE
     void* buffer = NULL;
 
     if (posix_memalign(&buffer, getpagesize(), limit) != 0) {
-      FAIL() << "Failed to allocate page-aligned memory, posix_memalign: "
-             << strerror(errno);
+      perror("Failed to allocate page-aligned memory, posix_memalign");
+      abort();
     }
 
     // We use mlock and memset here to make sure that the memory
     // actually gets paged in and thus accounted for.
     if (mlock(buffer, limit) != 0) {
-      FAIL() << "Failed to lock memory, mlock: " << strerror(errno);
+      perror("Failed to lock memory, mlock");
+      abort();
     }
 
     if (memset(buffer, 1, limit) != 0) {
-      FAIL() << "Failed to fill memory, memset: " << strerror(errno);
+      perror("Failed to fill memory, memset");
+      abort();
     }
 
     // Should not reach here.
-    FAIL() << "OOM does not happen!";
+    std::cerr << "OOM does not happen!" << std::endl;
+    abort();
   }
 }
 
 
-TEST_F(CgroupsTest, ROOT_CGROUPS_Freezer)
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryFreezerTest, ROOT_CGROUPS_Freezer)
 {
   int pipes[2];
   int dummy;
@@ -409,17 +534,17 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_Freezer
     ::close(pipes[1]);
 
     // Wait until child has assigned the cgroup.
-    ASSERT_NE(-1, ::read(pipes[0], &dummy, sizeof(dummy)));
+    ASSERT_LT(0, ::read(pipes[0], &dummy, sizeof(dummy)));
     ::close(pipes[0]);
 
-    // Freeze the "/prof" cgroup.
-    Future<bool> freeze = cgroups::freezeCgroup(hierarchy, "/prof");
+    // Freeze the "mesos_test" cgroup.
+    Future<bool> freeze = cgroups::freezeCgroup(hierarchy, "mesos_test");
     freeze.await(Seconds(5.0));
     ASSERT_TRUE(freeze.isReady());
     EXPECT_EQ(true, freeze.get());
 
-    // Thaw the "/prof" cgroup.
-    Future<bool> thaw = cgroups::thawCgroup(hierarchy, "/prof");
+    // Thaw the "mesos_test" cgroup.
+    Future<bool> thaw = cgroups::thawCgroup(hierarchy, "mesos_test");
     thaw.await(Seconds(5.0));
     ASSERT_TRUE(thaw.isReady());
     EXPECT_EQ(true, thaw.get());
@@ -432,32 +557,34 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_Freezer
     EXPECT_NE(-1, ::waitpid((pid_t) -1, &status, 0));
   } else {
     // In child process.
-    close(pipes[0]);
+    ::close(pipes[0]);
 
-    // Put self into the "/prof" cgroup.
-    Try<Nothing> assign = cgroups::assignTask(hierarchy,
-                                              "/prof",
-                                              ::getpid());
+    // Put self into the "mesos_test" cgroup.
+    Try<Nothing> assign =
+      cgroups::assignTask(hierarchy, "mesos_test", ::getpid());
     if (assign.isError()) {
-      FAIL() << "Failed to assign cgroup: " << assign.error();
+      std::cerr << "Failed to assign cgroup: " << assign.error() << std::endl;
+      abort();
     }
 
     // Notify the parent.
     if (::write(pipes[1], &dummy, sizeof(dummy)) != sizeof(dummy)) {
-      FAIL() << "Failed to notify the parent";
+      perror("Failed to notify the parent");
+      abort();
     }
     ::close(pipes[1]);
 
     // Infinite loop here.
-    while (true) ;
+    while (true);
 
     // Should not reach here.
-    FAIL() << "Reach an unreachable statement!";
+    std::cerr << "Reach an unreachable statement!" << std::endl;
+    abort();
   }
 }
 
 
-TEST_F(CgroupsTest, ROOT_CGROUPS_KillTasks)
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryFreezerTest, ROOT_CGROUPS_KillTasks)
 {
   int pipes[2];
   int dummy;
@@ -471,13 +598,13 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_KillTas
     ::close(pipes[1]);
 
     // Wait until all children have assigned the cgroup.
-    ASSERT_NE(-1, ::read(pipes[0], &dummy, sizeof(dummy)));
-    ASSERT_NE(-1, ::read(pipes[0], &dummy, sizeof(dummy)));
-    ASSERT_NE(-1, ::read(pipes[0], &dummy, sizeof(dummy)));
-    ASSERT_NE(-1, ::read(pipes[0], &dummy, sizeof(dummy)));
+    ASSERT_LT(0, ::read(pipes[0], &dummy, sizeof(dummy)));
+    ASSERT_LT(0, ::read(pipes[0], &dummy, sizeof(dummy)));
+    ASSERT_LT(0, ::read(pipes[0], &dummy, sizeof(dummy)));
+    ASSERT_LT(0, ::read(pipes[0], &dummy, sizeof(dummy)));
     ::close(pipes[0]);
 
-    Future<bool> future = cgroups::killTasks(hierarchy, "/prof");
+    Future<bool> future = cgroups::killTasks(hierarchy, "mesos_test");
     future.await(Seconds(5.0));
     ASSERT_TRUE(future.isReady());
     EXPECT_TRUE(future.get());
@@ -486,38 +613,40 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_KillTas
     EXPECT_NE(-1, ::waitpid((pid_t) -1, &status, 0));
   } else {
     // In child process.
+
+    // We create 4 child processes here using two forks to test the case in
+    // which there are multiple active processes in the given cgroup.
     ::fork();
     ::fork();
 
-    // Put self into "/prof" cgroup.
-    Try<Nothing> assign = cgroups::assignTask(hierarchy, "/prof", ::getpid());
+    // Put self into "mesos_test" cgroup.
+    Try<Nothing> assign =
+      cgroups::assignTask(hierarchy, "mesos_test", ::getpid());
     if (assign.isError()) {
-      FAIL() << "Failed to assign cgroup: " << assign.error();
+      std::cerr << "Failed to assign cgroup: " << assign.error() << std::endl;
+      abort();
     }
 
     // Notify the parent.
-    ::close(pipes[0]);
+    ::close(pipes[0]); // TODO(benh): Close after first fork?
     if (::write(pipes[1], &dummy, sizeof(dummy)) != sizeof(dummy)) {
-      FAIL() << "Failed to notify the parent";
+      perror("Failed to notify the parent");
+      abort();
     }
     ::close(pipes[1]);
 
     // Wait kill signal from parent.
-    while (true) ;
+    while (true);
 
     // Should not reach here.
-    FAIL() << "Reach an unreachable statement!";
+    std::cerr << "Reach an unreachable statement!" << std::endl;
+    abort();
   }
 }
 
 
-TEST_F(CgroupsTest, ROOT_CGROUPS_DestroyCgroup)
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryFreezerTest, ROOT_CGROUPS_DestroyCgroup)
 {
-  Future<bool> future = cgroups::destroyCgroup(hierarchy, "/stu/under");
-  future.await(Seconds(5.0));
-  ASSERT_TRUE(future.isReady());
-  EXPECT_TRUE(future.get());
-
   int pipes[2];
   int dummy;
   ASSERT_NE(-1, ::pipe(pipes));
@@ -530,13 +659,13 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_Destroy
     ::close(pipes[1]);
 
     // Wait until all children have assigned the cgroup.
-    ASSERT_NE(-1, ::read(pipes[0], &dummy, sizeof(dummy)));
-    ASSERT_NE(-1, ::read(pipes[0], &dummy, sizeof(dummy)));
-    ASSERT_NE(-1, ::read(pipes[0], &dummy, sizeof(dummy)));
-    ASSERT_NE(-1, ::read(pipes[0], &dummy, sizeof(dummy)));
+    ASSERT_LT(0, ::read(pipes[0], &dummy, sizeof(dummy)));
+    ASSERT_LT(0, ::read(pipes[0], &dummy, sizeof(dummy)));
+    ASSERT_LT(0, ::read(pipes[0], &dummy, sizeof(dummy)));
+    ASSERT_LT(0, ::read(pipes[0], &dummy, sizeof(dummy)));
     ::close(pipes[0]);
 
-    Future<bool> future = cgroups::destroyCgroup(hierarchy, "/");
+    Future<bool> future = cgroups::destroyCgroup(hierarchy, "mesos_test");
     future.await(Seconds(5.0));
     ASSERT_TRUE(future.isReady());
     EXPECT_TRUE(future.get());
@@ -545,21 +674,25 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_Destroy
     EXPECT_NE(-1, ::waitpid((pid_t) -1, &status, 0));
   } else {
     // In child process.
+
     // We create 4 child processes here using two forks to test the case in
     // which there are multiple active processes in the given cgroup.
     ::fork();
     ::fork();
 
-    // Put self into "/prof" cgroup.
-    Try<Nothing> assign = cgroups::assignTask(hierarchy, "/prof", ::getpid());
+    // Put self into "mesos_test" cgroup.
+    Try<Nothing> assign =
+      cgroups::assignTask(hierarchy, "mesos_test", ::getpid());
     if (assign.isError()) {
-      FAIL() << "Failed to assign cgroup: " << assign.error();
+      std::cerr << "Failed to assign cgroup: " << assign.error() << std::endl;
+      abort();
     }
 
     // Notify the parent.
-    ::close(pipes[0]);
+    ::close(pipes[0]); // TODO(benh): Close after first fork?
     if (::write(pipes[1], &dummy, sizeof(dummy)) != sizeof(dummy)) {
-      FAIL() << "Failed to notify the parent";
+      perror("Failed to notify the parent");
+      abort();
     }
     ::close(pipes[1]);
 
@@ -567,6 +700,7 @@ TEST_F(CgroupsTest, ROOT_CGROUPS_Destroy
     while (true) ;
 
     // Should not reach here.
-    FAIL() << "Reach an unreachable statement!";
+    std::cerr << "Reach an unreachable statement!" << std::endl;
+    abort();
   }
 }