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/11/26 22:24:05 UTC

svn commit: r1413863 [2/2] - in /incubator/mesos/branches/0.11.0/src: linux/cgroups.cpp linux/cgroups.hpp slave/cgroups_isolation_module.cpp tests/assert.hpp tests/cgroups_tests.cpp

Modified: incubator/mesos/branches/0.11.0/src/tests/cgroups_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/branches/0.11.0/src/tests/cgroups_tests.cpp?rev=1413863&r1=1413862&r2=1413863&view=diff
==============================================================================
--- incubator/mesos/branches/0.11.0/src/tests/cgroups_tests.cpp (original)
+++ incubator/mesos/branches/0.11.0/src/tests/cgroups_tests.cpp Mon Nov 26 21:24:04 2012
@@ -60,16 +60,18 @@ protected:
   static void TearDownTestCase()
   {
     // Remove the testing hierarchy.
-    if (cgroups::checkHierarchy(HIERARCHY).isSome()) {
+    Try<bool> mounted = cgroups::mounted(HIERARCHY);
+    ASSERT_SOME(mounted);
+    if (mounted.get()) {
       // Remove all cgroups.
-      Try<std::vector<std::string> > cgroups = cgroups::getCgroups(HIERARCHY);
+      Try<std::vector<std::string> > cgroups = cgroups::get(HIERARCHY);
       ASSERT_SOME(cgroups);
       foreach (const std::string& cgroup, cgroups.get()) {
-        ASSERT_SOME(cgroups::removeCgroup(HIERARCHY, cgroup));
+        ASSERT_SOME(cgroups::remove(HIERARCHY, cgroup));
       }
 
       // Remove the hierarchy.
-      ASSERT_SOME(cgroups::removeHierarchy(HIERARCHY));
+      ASSERT_SOME(cgroups::unmount(HIERARCHY));
 
       // Remove the directory if still exists.
       if (os::exists(HIERARCHY)) {
@@ -96,10 +98,10 @@ protected:
     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"
+        << "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 remove those hierarchies, or disable\n"
+        << "You can either unmount those hierarchies, or disable\n"
         << "this test case (i.e., --gtest_filter=-CgroupsNoHierarchyTest.*).\n"
         << "-------------------------------------------------------------"
         << std::endl;
@@ -140,15 +142,17 @@ protected:
       }
 
       // Check and see if this candidate meets our subsystem requirements.
-      if (cgroups::checkHierarchy(candidate, subsystems).isSome()) {
+      Try<bool> mounted = cgroups::mounted(candidate, subsystems);
+      ASSERT_SOME(mounted);
+      if (mounted.get()) {
         hierarchy = candidate;
         break;
       }
     }
 
     if (hierarchy.empty()) {
-      // Create a hierarchy for testing.
-      ASSERT_SOME(cgroups::createHierarchy(HIERARCHY, subsystems))
+      // Try to mount a hierarchy for testing.
+      ASSERT_SOME(cgroups::mount(HIERARCHY, subsystems))
         << "-------------------------------------------------------------\n"
         << "We cannot run any cgroups tests that require\n"
         << "a hierarchy with subsystems '" << subsystems << "'\n"
@@ -165,8 +169,10 @@ protected:
     }
 
     // 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"))
+    Try<bool> exists = cgroups::exists(hierarchy, "mesos_test");
+    ASSERT_SOME(exists);
+    if (exists.get()) {
+      ASSERT_FUTURE_WILL_SUCCEED(cgroups::destroy(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"
@@ -174,17 +180,17 @@ protected:
         << "-------------------------------------------------------------";
     }
 
-    ASSERT_SOME(cgroups::createCgroup(hierarchy, "mesos_test"));
+    ASSERT_SOME(cgroups::create(hierarchy, "mesos_test"));
   }
 
   virtual void TearDown()
   {
     // Remove all *our* cgroups.
-    Try<std::vector<std::string> > cgroups = cgroups::getCgroups(hierarchy);
+    Try<std::vector<std::string> > cgroups = cgroups::get(hierarchy);
     ASSERT_SOME(cgroups);
     foreach (const std::string& cgroup, cgroups.get()) {
       if (strings::startsWith(cgroup, "mesos_test")) {
-        ASSERT_SOME(cgroups::removeCgroup(hierarchy, cgroup));
+        ASSERT_SOME(cgroups::remove(hierarchy, cgroup));
       }
     }
 
@@ -217,27 +223,27 @@ public:
 
 TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Enabled)
 {
+  EXPECT_SOME_TRUE(cgroups::enabled(""));
+  EXPECT_SOME_TRUE(cgroups::enabled(","));
   EXPECT_SOME_TRUE(cgroups::enabled("cpu"));
   EXPECT_SOME_TRUE(cgroups::enabled(",cpu"));
   EXPECT_SOME_TRUE(cgroups::enabled("cpu,memory"));
   EXPECT_SOME_TRUE(cgroups::enabled("cpu,memory,"));
   EXPECT_ERROR(cgroups::enabled("invalid"));
   EXPECT_ERROR(cgroups::enabled("cpu,invalid"));
-  EXPECT_ERROR(cgroups::enabled(","));
-  EXPECT_ERROR(cgroups::enabled(""));
 }
 
 
 TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_Busy)
 {
-  EXPECT_ERROR(cgroups::busy("invalid"));
-  EXPECT_ERROR(cgroups::busy("cpu,invalid"));
-  EXPECT_ERROR(cgroups::busy(","));
-  EXPECT_ERROR(cgroups::busy(""));
+  EXPECT_SOME_FALSE(cgroups::busy(""));
+  EXPECT_SOME_FALSE(cgroups::busy(","));
   EXPECT_SOME_TRUE(cgroups::busy("cpu"));
   EXPECT_SOME_TRUE(cgroups::busy(",cpu"));
   EXPECT_SOME_TRUE(cgroups::busy("cpu,memory"));
   EXPECT_SOME_TRUE(cgroups::busy("cpu,memory,"));
+  EXPECT_ERROR(cgroups::busy("invalid"));
+  EXPECT_ERROR(cgroups::busy("cpu,invalid"));
 }
 
 
@@ -281,70 +287,70 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryT
 }
 
 
-TEST_F(CgroupsNoHierarchyTest, ROOT_CGROUPS_CreateRemoveHierarchy)
+TEST_F(CgroupsNoHierarchyTest, ROOT_CGROUPS_MountUnmountHierarchy)
 {
-  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::removeHierarchy("/tmp"));
-  ASSERT_SOME(cgroups::removeHierarchy(HIERARCHY));
+  EXPECT_ERROR(cgroups::mount("/tmp", "cpu"));
+  EXPECT_ERROR(cgroups::mount(HIERARCHY, "invalid"));
+  ASSERT_SOME(cgroups::mount(HIERARCHY, "cpu,memory"));
+  EXPECT_ERROR(cgroups::mount(HIERARCHY, "cpuset"));
+  EXPECT_ERROR(cgroups::unmount("/tmp"));
+  ASSERT_SOME(cgroups::unmount(HIERARCHY));
 }
 
 
-TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_CheckHierarchy)
+TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Mounted)
 {
-  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 + "/not_expected"));
+  EXPECT_SOME_FALSE(cgroups::mounted("/tmp-nonexist"));
+  EXPECT_SOME_FALSE(cgroups::mounted("/tmp"));
+  EXPECT_SOME_FALSE(cgroups::mounted(hierarchy + "/not_expected"));
+  EXPECT_SOME_TRUE(cgroups::mounted(hierarchy));
+  EXPECT_SOME_TRUE(cgroups::mounted(hierarchy + "/"));
 }
 
 
-TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_CheckHierarchySubsystems)
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_MountedSubsystems)
 {
-  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 + "/not_expected", "cpu"));
+  EXPECT_SOME_FALSE(cgroups::mounted("/tmp-nonexist", "cpu"));
+  EXPECT_SOME_FALSE(cgroups::mounted("/tmp", "cpu,memory"));
+  EXPECT_SOME_FALSE(cgroups::mounted("/tmp", "cpu"));
+  EXPECT_SOME_FALSE(cgroups::mounted("/tmp", "invalid"));
+  EXPECT_SOME_TRUE(cgroups::mounted(hierarchy, "cpu,memory"));
+  EXPECT_SOME_TRUE(cgroups::mounted(hierarchy, "memory"));
+  EXPECT_SOME_FALSE(cgroups::mounted(hierarchy, "invalid"));
+  EXPECT_SOME_FALSE(cgroups::mounted(hierarchy + "/not_expected", "cpu"));
 }
 
 
-TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_CreateRemoveCgroup)
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_CreateRemove)
 {
-  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"));
+  EXPECT_ERROR(cgroups::create("/tmp", "test"));
+  EXPECT_ERROR(cgroups::create(hierarchy, "mesos_test_missing/1"));
+  ASSERT_SOME(cgroups::create(hierarchy, "mesos_test_missing"));
+  EXPECT_ERROR(cgroups::remove(hierarchy, "invalid"));
+  ASSERT_SOME(cgroups::remove(hierarchy, "mesos_test_missing"));
 }
 
 
-TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_GetCgroups)
+TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Get)
 {
-  ASSERT_SOME(cgroups::createCgroup(hierarchy, "mesos_test1"));
-  ASSERT_SOME(cgroups::createCgroup(hierarchy, "mesos_test2"));
+  ASSERT_SOME(cgroups::create(hierarchy, "mesos_test1"));
+  ASSERT_SOME(cgroups::create(hierarchy, "mesos_test2"));
 
-  Try<std::vector<std::string> > cgroups = cgroups::getCgroups(hierarchy);
+  Try<std::vector<std::string> > cgroups = cgroups::get(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"));
+  ASSERT_SOME(cgroups::remove(hierarchy, "mesos_test1"));
+  ASSERT_SOME(cgroups::remove(hierarchy, "mesos_test2"));
 }
 
 
 TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_NestedCgroups)
 {
-  ASSERT_SOME(cgroups::createCgroup(hierarchy, "mesos_test/1"))
+  ASSERT_SOME(cgroups::create(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"
@@ -352,53 +358,57 @@ TEST_F(CgroupsAnyHierarchyTest, ROOT_CGR
     << "disable this test.\n"
     << "-------------------------------------------------------------";
 
-  ASSERT_SOME(cgroups::createCgroup(hierarchy, "mesos_test/2"));
+  ASSERT_SOME(cgroups::create(hierarchy, "mesos_test/2"));
 
-  Try<std::vector<std::string> > cgroups =
-    cgroups::getCgroups(hierarchy, "mesos_test");
+  Try<std::vector<std::string> > cgroups = cgroups::get(hierarchy, "mesos_test");
   ASSERT_SOME(cgroups);
   ASSERT_EQ(2u, cgroups.get().size());
 
   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"));
+  ASSERT_SOME(cgroups::remove(hierarchy, "mesos_test/1"));
+  ASSERT_SOME(cgroups::remove(hierarchy, "mesos_test/2"));
 }
 
 
-TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_ReadControl)
+TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Tasks)
 {
-  EXPECT_ERROR(cgroups::readControl(hierarchy, "mesos_test", "invalid"));
+  Try<std::set<pid_t> > pids = cgroups::tasks(hierarchy, "/");
+  ASSERT_SOME(pids);
+  EXPECT_NE(0u, pids.get().count(1));
+  EXPECT_NE(0u, pids.get().count(::getpid()));
+}
+
+
+TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Read)
+{
+  EXPECT_ERROR(cgroups::read(hierarchy, "mesos_test", "invalid"));
 
   std::string pid = stringify(::getpid());
 
-  Try<std::string> result = cgroups::readControl(hierarchy, "/", "tasks");
+  Try<std::string> result = cgroups::read(hierarchy, "/", "tasks");
   ASSERT_SOME(result);
   EXPECT_TRUE(strings::contains(result.get(), pid));
 }
 
 
-TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_WriteControl)
+TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Write)
 {
-  EXPECT_ERROR(cgroups::writeControl(hierarchy,
-                                     "mesos_test",
-                                     "invalid",
-                                     "invalid"));
+  EXPECT_ERROR(cgroups::write(hierarchy, "mesos_test", "invalid", "invalid"));
 
   pid_t pid = ::fork();
   ASSERT_NE(-1, pid);
 
   if (pid) {
     // In parent process.
-    ASSERT_SOME(cgroups::writeControl(
+    ASSERT_SOME(cgroups::write(
                     hierarchy, "mesos_test", "tasks", stringify(pid)));
 
-    Try<std::set<pid_t> > tasks = cgroups::getTasks(hierarchy, "mesos_test");
-    ASSERT_SOME(tasks);
+    Try<std::set<pid_t> > pids = cgroups::tasks(hierarchy, "mesos_test");
+    ASSERT_SOME(pids);
 
-    std::set<pid_t> pids = tasks.get();
-    EXPECT_NE(pids.find(pid), pids.end());
+    EXPECT_NE(0u, pids.get().count(pid));
 
     // Kill the child process.
     ASSERT_NE(-1, ::kill(pid, SIGKILL));
@@ -406,8 +416,10 @@ TEST_F(CgroupsAnyHierarchyTest, ROOT_CGR
     // Wait for the child process.
     int status;
     EXPECT_NE(-1, ::waitpid((pid_t) -1, &status, 0));
+    ASSERT_TRUE(WIFSIGNALED(status));
+    EXPECT_EQ(SIGKILL, WTERMSIG(status));
   } else {
-    // In child process.
+    // In child process, wait for kill signal.
     while (true);
 
     // Should not reach here.
@@ -417,20 +429,9 @@ TEST_F(CgroupsAnyHierarchyTest, ROOT_CGR
 }
 
 
-TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_GetTasks)
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_Listen)
 {
-  Try<std::set<pid_t> > tasks = cgroups::getTasks(hierarchy, "/");
-  ASSERT_SOME(tasks);
-
-  std::set<pid_t> pids = tasks.get();
-  EXPECT_NE(pids.find(1), pids.end());
-  EXPECT_NE(pids.find(::getpid()), pids.end());
-}
-
-
-TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_ListenEvent)
-{
-  ASSERT_SOME(cgroups::checkControl(hierarchy, "mesos_test", "memory.oom_control"))
+  ASSERT_SOME(cgroups::exists(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"
@@ -439,32 +440,30 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryT
     << "-------------------------------------------------------------";
 
   // Disable oom killer.
-  ASSERT_SOME(cgroups::writeControl(hierarchy,
-                                    "mesos_test",
-                                    "memory.oom_control",
-                                    "1"));
+  ASSERT_SOME(cgroups::write(
+                  hierarchy,
+                  "mesos_test",
+                  "memory.oom_control",
+                  "1"));
 
   // Limit the memory usage of "mesos_test" to 64MB.
   size_t limit = 1024 * 1024 * 64;
-  ASSERT_SOME(cgroups::writeControl(hierarchy,
-                                    "mesos_test",
-                                    "memory.limit_in_bytes",
-                                    stringify(limit)));
+  ASSERT_SOME(cgroups::write(
+                  hierarchy,
+                  "mesos_test",
+                  "memory.limit_in_bytes",
+                  stringify(limit)));
 
   // Listen on oom events for "mesos_test" cgroup.
-  Future<uint64_t> future =
-    cgroups::listenEvent(hierarchy,
-                         "mesos_test",
-                         "memory.oom_control");
+  Future<uint64_t> future = cgroups::listen(
+      hierarchy, "mesos_test", "memory.oom_control");
   ASSERT_FALSE(future.isFailed());
 
   // Test the cancellation.
   future.discard();
 
   // Test the normal operation below.
-  future = cgroups::listenEvent(hierarchy,
-                                "mesos_test",
-                                "memory.oom_control");
+  future = cgroups::listen(hierarchy, "mesos_test", "memory.oom_control");
   ASSERT_FALSE(future.isFailed());
 
   pid_t pid = ::fork();
@@ -482,11 +481,12 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryT
     // Wait for the child process.
     int status;
     EXPECT_NE(-1, ::waitpid((pid_t) -1, &status, 0));
+    ASSERT_TRUE(WIFSIGNALED(status));
+    EXPECT_EQ(SIGKILL, WTERMSIG(status));
   } else {
     // In child process. We try to trigger an oom here.
     // Put self into the "mesos_test" cgroup.
-    Try<Nothing> assign =
-      cgroups::assignTask(hierarchy, "mesos_test", ::getpid());
+    Try<Nothing> assign = cgroups::assign(hierarchy, "mesos_test", ::getpid());
     if (assign.isError()) {
       std::cerr << "Failed to assign cgroup: " << assign.error() << std::endl;
       abort();
@@ -520,7 +520,7 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryT
 }
 
 
-TEST_F(CgroupsAnyHierarchyWithCpuMemoryFreezerTest, ROOT_CGROUPS_Freezer)
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryFreezerTest, ROOT_CGROUPS_Freeze)
 {
   int pipes[2];
   int dummy;
@@ -538,13 +538,13 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryF
     ::close(pipes[0]);
 
     // Freeze the "mesos_test" cgroup.
-    Future<bool> freeze = cgroups::freezeCgroup(hierarchy, "mesos_test");
+    Future<bool> freeze = cgroups::freeze(hierarchy, "mesos_test");
     freeze.await(Seconds(5.0));
     ASSERT_TRUE(freeze.isReady());
     EXPECT_EQ(true, freeze.get());
 
     // Thaw the "mesos_test" cgroup.
-    Future<bool> thaw = cgroups::thawCgroup(hierarchy, "mesos_test");
+    Future<bool> thaw = cgroups::thaw(hierarchy, "mesos_test");
     thaw.await(Seconds(5.0));
     ASSERT_TRUE(thaw.isReady());
     EXPECT_EQ(true, thaw.get());
@@ -555,13 +555,14 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryF
     // Wait for the child process.
     int status;
     EXPECT_NE(-1, ::waitpid((pid_t) -1, &status, 0));
+    ASSERT_TRUE(WIFSIGNALED(status));
+    EXPECT_EQ(SIGKILL, WTERMSIG(status));
   } else {
     // In child process.
     ::close(pipes[0]);
 
     // Put self into the "mesos_test" cgroup.
-    Try<Nothing> assign =
-      cgroups::assignTask(hierarchy, "mesos_test", ::getpid());
+    Try<Nothing> assign = cgroups::assign(hierarchy, "mesos_test", ::getpid());
     if (assign.isError()) {
       std::cerr << "Failed to assign cgroup: " << assign.error() << std::endl;
       abort();
@@ -584,7 +585,7 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryF
 }
 
 
-TEST_F(CgroupsAnyHierarchyWithCpuMemoryFreezerTest, ROOT_CGROUPS_KillTasks)
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryFreezerTest, ROOT_CGROUPS_Kill)
 {
   int pipes[2];
   int dummy;
@@ -604,13 +605,13 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryF
     ASSERT_LT(0, ::read(pipes[0], &dummy, sizeof(dummy)));
     ::close(pipes[0]);
 
-    Future<bool> future = cgroups::killTasks(hierarchy, "mesos_test");
-    future.await(Seconds(5.0));
-    ASSERT_TRUE(future.isReady());
-    EXPECT_TRUE(future.get());
+    Try<Nothing> kill = cgroups::kill(hierarchy, "mesos_test", SIGKILL);
+    EXPECT_SOME(kill);
 
     int status;
     EXPECT_NE(-1, ::waitpid((pid_t) -1, &status, 0));
+    ASSERT_TRUE(WIFSIGNALED(status));
+    EXPECT_EQ(SIGKILL, WTERMSIG(status));
   } else {
     // In child process.
 
@@ -620,8 +621,7 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryF
     ::fork();
 
     // Put self into "mesos_test" cgroup.
-    Try<Nothing> assign =
-      cgroups::assignTask(hierarchy, "mesos_test", ::getpid());
+    Try<Nothing> assign = cgroups::assign(hierarchy, "mesos_test", ::getpid());
     if (assign.isError()) {
       std::cerr << "Failed to assign cgroup: " << assign.error() << std::endl;
       abort();
@@ -645,7 +645,8 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryF
 }
 
 
-TEST_F(CgroupsAnyHierarchyWithCpuMemoryFreezerTest, ROOT_CGROUPS_DestroyCgroup)
+// TODO(benh): Write a version of this test with nested cgroups.
+TEST_F(CgroupsAnyHierarchyWithCpuMemoryFreezerTest, ROOT_CGROUPS_Destroy)
 {
   int pipes[2];
   int dummy;
@@ -665,13 +666,15 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryF
     ASSERT_LT(0, ::read(pipes[0], &dummy, sizeof(dummy)));
     ::close(pipes[0]);
 
-    Future<bool> future = cgroups::destroyCgroup(hierarchy, "mesos_test");
+    Future<bool> future = cgroups::destroy(hierarchy, "mesos_test");
     future.await(Seconds(5.0));
     ASSERT_TRUE(future.isReady());
     EXPECT_TRUE(future.get());
 
     int status;
     EXPECT_NE(-1, ::waitpid((pid_t) -1, &status, 0));
+    ASSERT_TRUE(WIFSIGNALED(status));
+    EXPECT_EQ(SIGKILL, WTERMSIG(status));
   } else {
     // In child process.
 
@@ -681,8 +684,7 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryF
     ::fork();
 
     // Put self into "mesos_test" cgroup.
-    Try<Nothing> assign =
-      cgroups::assignTask(hierarchy, "mesos_test", ::getpid());
+    Try<Nothing> assign = cgroups::assign(hierarchy, "mesos_test", ::getpid());
     if (assign.isError()) {
       std::cerr << "Failed to assign cgroup: " << assign.error() << std::endl;
       abort();