You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by id...@apache.org on 2014/06/17 20:04:55 UTC

[2/3] git commit: Refactored cgroups::destroy to use a single pass.

Refactored cgroups::destroy to use a single pass.

internal::Destroyer now uses a single pass of freeze, kill, thaw, reap
to kill all processes in a freezer cgroup. cgroups::destroy will not
complete until all processes have been reaped.

Added tests for destroying a freezer cgroup containing a stopped
process or a traced process.

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


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

Branch: refs/heads/master
Commit: ca2ba6a0b20f93b8f4507b35f8eff427115f2e7e
Parents: 07c3dc5
Author: Ian Downes <id...@twitter.com>
Authored: Fri Apr 25 17:58:56 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue Jun 17 11:00:41 2014 -0700

----------------------------------------------------------------------
 src/linux/cgroups.cpp                           | 258 +++++++------------
 src/linux/cgroups.hpp                           |  18 +-
 .../isolators/cgroups/cpushare.cpp              |   2 +-
 src/slave/containerizer/linux_launcher.cpp      |   2 +-
 src/slave/containerizer/mesos_containerizer.cpp |   2 +-
 src/tests/cgroups_tests.cpp                     | 105 +++++++-
 src/tests/slave_recovery_tests.cpp              |   1 +
 7 files changed, 195 insertions(+), 193 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ca2ba6a0/src/linux/cgroups.cpp
----------------------------------------------------------------------
diff --git a/src/linux/cgroups.cpp b/src/linux/cgroups.cpp
index 723acc1..0b5f95a 100644
--- a/src/linux/cgroups.cpp
+++ b/src/linux/cgroups.cpp
@@ -41,6 +41,7 @@
 #include <process/delay.hpp>
 #include <process/io.hpp>
 #include <process/process.hpp>
+#include <process/reap.hpp>
 
 #include <stout/duration.hpp>
 #include <stout/error.hpp>
@@ -1427,7 +1428,6 @@ protected:
   }
 
 private:
-
   const string hierarchy;
   const string cgroup;
   const Time start;
@@ -1435,142 +1435,43 @@ private:
 };
 
 
-// The process used to wait for a cgroup to become empty (no task in it).
-class EmptyWatcher: public Process<EmptyWatcher>
-{
-public:
-  EmptyWatcher(const string& _hierarchy,
-               const string& _cgroup,
-               const Duration& _interval,
-               unsigned int _retries = EMPTY_WATCHER_RETRIES)
-    : hierarchy(_hierarchy),
-      cgroup(_cgroup),
-      interval(_interval),
-      retries(_retries) {}
-
-  virtual ~EmptyWatcher() {}
-
-  // Return a future indicating the state of the watcher.
-  // There are three outcomes:
-  //   1. true:  the cgroup became empty.
-  //   2. false: the cgroup did not become empty within the retry limit.
-  //   3. error: invalid arguments, or an unexpected error occured.
-  Future<bool> future() { return promise.future(); }
-
-protected:
-  virtual void initialize()
-  {
-    // Stop when no one cares.
-    promise.future().onDiscard(lambda::bind(
-        static_cast<void (*)(const UPID&, bool)>(terminate), self(), true));
-
-    CHECK(interval >= Seconds(0));
-
-    check();
-  }
-
-  virtual void finalize()
-  {
-    promise.discard();
-  }
-
-private:
-  void check(unsigned int attempt = 0)
-  {
-    Try<set<pid_t> > pids = processes(hierarchy, cgroup);
-    if (pids.isError()) {
-      promise.fail("Failed to get processes of cgroup: " + pids.error());
-      terminate(self());
-      return;
-    }
-
-    if (pids.get().empty()) {
-      promise.set(true);
-      terminate(self());
-      return;
-    } else {
-      if (attempt > retries) {
-        promise.set(false);
-        terminate(self());
-        return;
-      }
-
-      // Re-check needed.
-      delay(interval, self(), &EmptyWatcher::check, attempt + 1);
-    }
-  }
-
-  const string hierarchy;
-  const string cgroup;
-  const Duration interval;
-  const unsigned int retries;
-  Promise<bool> promise;
-};
-
-
 // The process used to atomically kill all tasks in a cgroup.
 class TasksKiller : public Process<TasksKiller>
 {
 public:
-  TasksKiller(const string& _hierarchy,
-              const string& _cgroup,
-              const Duration& _interval)
-    : hierarchy(_hierarchy),
-      cgroup(_cgroup),
-      interval(_interval) {}
+  TasksKiller(const string& _hierarchy, const string& _cgroup)
+    : hierarchy(_hierarchy), cgroup(_cgroup) {}
 
   virtual ~TasksKiller() {}
 
   // Return a future indicating the state of the killer.
-  Future<bool> future() { return promise.future(); }
+  // Failure occurs if any process in the cgroup is unable to be
+  // killed.
+  Future<Nothing> future() { return promise.future(); }
 
 protected:
   virtual void initialize()
   {
     // Stop when no one cares.
     promise.future().onDiscard(lambda::bind(
-          static_cast<void(*)(const UPID&, bool)>(terminate), self(), true));
-
-    CHECK(interval >= Seconds(0));
+        static_cast<void (*)(const UPID&, bool)>(terminate), self(), true));
 
     killTasks();
   }
 
   virtual void finalize()
   {
-    // Cancel the chain of operations if the user discards the future.
-    if (promise.future().hasDiscard()) {
-      chain.discard();
-
-      // TODO(benh): Discard our promise only after 'chain' has
-      // completed (ready, failed, or discarded).
-      promise.discard();
-    }
+    chain.discard();
+    promise.discard();
   }
 
 private:
-  // The sequence of operations to kill a cgroup is as follows:
-  // SIGSTOP -> SIGKILL -> empty -> freeze -> SIGKILL -> thaw -> empty
-  // This process is repeated until the cgroup becomes empty.
   void killTasks() {
-    // Chain together the steps needed to kill the tasks. Note that we
-    // ignore the return values of freeze, kill, and thaw because,
-    // provided there are no errors, we'll just retry the chain as
-    // long as tasks still exist.
-    // Send stop signal to all tasks.
-    chain = kill(SIGSTOP)
-      // Now send kill signal.
-      .then(defer(self(), &Self::kill, SIGKILL))
-      // Wait until cgroup is empty.
-      .then(defer(self(), &Self::empty))
-      // Freeze cgroup.
-      .then(defer(self(), &Self::freeze))
-      // Send kill signal to any remaining tasks.
-      .then(defer(self(), &Self::kill, SIGKILL))
-      // Thaw cgroup to deliver signals.
-      .then(defer(self(), &Self::thaw))
-      // Wait until cgroup is empty.
-      .then(defer(self(), &Self::empty));
+    // Chain together the steps needed to kill all tasks in the cgroup.
+    chain = freeze()                     // Freeze the cgroup.
+      .then(defer(self(), &Self::kill))  // Send kill signal.
+      .then(defer(self(), &Self::thaw))  // Thaw cgroup to deliver signal.
+      .then(defer(self(), &Self::reap)); // Wait until all pids are reaped.
 
     chain.onAny(defer(self(), &Self::finished, lambda::_1));
   }
@@ -1580,12 +1481,24 @@ private:
     return cgroups::freezer::freeze(hierarchy, cgroup);
   }
 
-  Future<Nothing> kill(const int signal)
+  Future<Nothing> kill()
   {
-    Try<Nothing> kill = cgroups::kill(hierarchy, cgroup, signal);
+    Try<set<pid_t> > processes = cgroups::processes(hierarchy, cgroup);
+    if (processes.isError()) {
+      return Failure(processes.error());
+    }
+
+    // Reaping the frozen pids before we kill (and thaw) ensures we reap the
+    // correct pids.
+    foreach (const pid_t pid, processes.get()) {
+      statuses.push_back(process::reap(pid));
+    }
+
+    Try<Nothing> kill = cgroups::kill(hierarchy, cgroup, SIGKILL);
     if (kill.isError()) {
       return Failure(kill.error());
     }
+
     return Nothing();
   }
 
@@ -1594,37 +1507,43 @@ private:
     return cgroups::freezer::thaw(hierarchy, cgroup);
   }
 
-  Future<bool> empty()
+  Future<list<Option<int> > > reap()
   {
-    EmptyWatcher* watcher = new EmptyWatcher(hierarchy, cgroup, interval);
-    Future<bool> future = watcher->future();
-    spawn(watcher, true);
-    return future;
+    // Wait until we've reaped all processes.
+    return collect(statuses);
   }
 
-  void finished(const Future<bool>& empty)
+  void finished(const Future<list<Option<int> > >& future)
   {
-    if (empty.isDiscarded()) {
-      promise.discard();
+    if (future.isDiscarded()) {
+      promise.fail("Unexpected discard of future");
       terminate(self());
-    } else if (empty.isFailed()) {
-      promise.fail(empty.failure());
+      return;
+    } else if (future.isFailed()) {
+      promise.fail(future.failure());
       terminate(self());
-    } else if (empty.get()) {
-      promise.set(true);
+      return;
+    }
+
+    // Verify the cgroup is now empty.
+    Try<set<pid_t> > processes = cgroups::processes(hierarchy, cgroup);
+    if (processes.isError() || !processes.get().empty()) {
+      promise.fail("Failed to kill all processes in cgroup: " +
+                   (processes.isError() ? processes.error()
+                                        : "processes remain"));
       terminate(self());
-    } else {
-      // The cgroup was not empty after the retry limit.
-      // We need to re-attempt the freeze/kill/thaw/watch chain.
-      killTasks();
+      return;
     }
+
+    promise.set(Nothing());
+    terminate(self());
   }
 
   const string hierarchy;
   const string cgroup;
-  const Duration interval;
-  Promise<bool> promise;
-  Future<bool> chain; // Used to discard the "chain" of operations.
+  Promise<Nothing> promise;
+  list<Future<Option<int> > > statuses; // List of statuses for processes.
+  Future<list<Option<int> > > chain; // Used to discard all operations.
 };
 
 
@@ -1632,32 +1551,27 @@ private:
 class Destroyer : public Process<Destroyer>
 {
 public:
-  Destroyer(const string& _hierarchy,
-            const vector<string>& _cgroups,
-            const Duration& _interval)
-    : hierarchy(_hierarchy),
-      cgroups(_cgroups),
-      interval(_interval) {}
+  Destroyer(const string& _hierarchy, const vector<string>& _cgroups)
+    : hierarchy(_hierarchy), cgroups(_cgroups) {}
 
   virtual ~Destroyer() {}
 
   // Return a future indicating the state of the destroyer.
-  Future<bool> future() { return promise.future(); }
+  // Failure occurs if any cgroup fails to be destroyed.
+  Future<Nothing> future() { return promise.future(); }
 
 protected:
   virtual void initialize()
   {
     // Stop when no one cares.
     promise.future().onDiscard(lambda::bind(
-          static_cast<void(*)(const UPID&, bool)>(terminate), self(), true));
-
-    CHECK(interval >= Seconds(0));
+        static_cast<void (*)(const UPID&, bool)>(terminate), self(), true));
 
     // Kill tasks in the given cgroups in parallel. Use collect mechanism to
     // wait until all kill processes finish.
     foreach (const string& cgroup, cgroups) {
       internal::TasksKiller* killer =
-        new internal::TasksKiller(hierarchy, cgroup, interval);
+        new internal::TasksKiller(hierarchy, cgroup);
       killers.push_back(killer->future());
       spawn(killer, true);
     }
@@ -1668,18 +1582,12 @@ protected:
 
   virtual void finalize()
   {
-    // Cancel the operation if the user discards the future.
-    if (promise.future().hasDiscard()) {
-      discard<bool>(killers);
-
-      // TODO(benh): Discard our promise only after all 'killers' have
-      // completed (ready, failed, or discarded).
-      promise.discard();
-    }
+    discard(killers);
+    promise.discard();
   }
 
 private:
-  void killed(const Future<list<bool> >& kill)
+  void killed(const Future<list<Nothing> >& kill)
   {
     if (kill.isReady()) {
       remove();
@@ -1687,7 +1595,8 @@ private:
       promise.discard();
       terminate(self());
     } else if (kill.isFailed()) {
-      promise.fail("Failed to kill tasks in nested cgroups: " + kill.failure());
+      promise.fail("Failed to kill tasks in nested cgroups: " +
+                   kill.failure());
       terminate(self());
     }
   }
@@ -1704,31 +1613,23 @@ private:
       }
     }
 
-    promise.set(true);
+    promise.set(Nothing());
     terminate(self());
   }
 
   const string hierarchy;
   const vector<string> cgroups;
-  const Duration interval;
-  Promise<bool> promise;
+  Promise<Nothing> promise;
 
   // The killer processes used to atomically kill tasks in each cgroup.
-  list<Future<bool> > killers;
+  list<Future<Nothing> > killers;
 };
 
 } // namespace internal {
 
 
-Future<bool> destroy(
-    const string& hierarchy,
-    const string& cgroup,
-    const Duration& interval)
+Future<Nothing> destroy(const string& hierarchy, const string& cgroup)
 {
-  if (interval < Seconds(0)) {
-    return Failure("Interval should be non-negative");
-  }
-
   // Construct the vector of cgroups to destroy.
   Try<vector<string> > cgroups = cgroups::get(hierarchy, cgroup);
   if (cgroups.isError()) {
@@ -1742,15 +1643,15 @@ Future<bool> destroy(
   }
 
   if (candidates.empty()) {
-    return true;
+    return Nothing();
   }
 
   // If the freezer subsystem is available, destroy the cgroups.
   Option<Error> error = verify(hierarchy, cgroup, "freezer.state");
   if (error.isNone()) {
     internal::Destroyer* destroyer =
-      new internal::Destroyer(hierarchy, candidates, interval);
-    Future<bool> future = destroyer->future();
+      new internal::Destroyer(hierarchy, candidates);
+    Future<Nothing> future = destroyer->future();
     spawn(destroyer, true);
     return future;
   } else {
@@ -1763,7 +1664,22 @@ Future<bool> destroy(
     }
   }
 
-  return true;
+  return Nothing();
+}
+
+
+Future<Nothing> destroyTimedOut(
+    Future<Nothing> future,
+    const string& hierarchy,
+    const string& cgroup,
+    const Duration& timeout)
+{
+  LOG(WARNING) << "Failed to destroy cgroup '" << path::join(hierarchy, cgroup)
+               << "' after " << timeout;
+
+  future.discard();
+
+  return future;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/ca2ba6a0/src/linux/cgroups.hpp
----------------------------------------------------------------------
diff --git a/src/linux/cgroups.hpp b/src/linux/cgroups.hpp
index 599badb..38e345f 100644
--- a/src/linux/cgroups.hpp
+++ b/src/linux/cgroups.hpp
@@ -41,7 +41,6 @@
 namespace cgroups {
 
 const Duration DESTROY_TIMEOUT = Seconds(60);
-const unsigned int EMPTY_WATCHER_RETRIES = 50;
 
 // Default number of assign attempts when moving threads to a cgroup.
 const unsigned int THREAD_ASSIGN_RETRIES = 100;
@@ -334,14 +333,21 @@ process::Future<uint64_t> listen(
 // 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 something unexpected happens.
-process::Future<bool> destroy(
+process::Future<Nothing> destroy(
     const std::string& hierarchy,
-    const std::string& cgroup = "/",
-    const Duration& interval = Milliseconds(100));
+    const std::string& cgroup = "/");
+
+
+// Destroy a cgroup under a given hierarchy. This is a convenience
+// function which wraps the cgroups::destroy() to add a timeout: if
+// the cgroup(s) cannot be destroyed after timeout the operation will
+// be discarded.
+process::Future<Nothing> destroy(
+    const std::string& hierarchy,
+    const std::string& cgroup,
+    const Duration& timeout);
 
 
 // Cleanup the hierarchy, by first destroying all the underlying

http://git-wip-us.apache.org/repos/asf/mesos/blob/ca2ba6a0/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 eb8933f..1e2db0f 100644
--- a/src/slave/containerizer/isolators/cgroups/cpushare.cpp
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
@@ -441,7 +441,7 @@ Future<Nothing> CgroupsCpushareIsolatorProcess::cleanup(
 
   Info* info = CHECK_NOTNULL(infos[containerId]);
 
-  list<Future<bool> > futures;
+  list<Future<Nothing> > futures;
   futures.push_back(cgroups::destroy(hierarchies["cpu"], info->cgroup));
   futures.push_back(cgroups::destroy(hierarchies["cpuacct"], info->cgroup));
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/ca2ba6a0/src/slave/containerizer/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/linux_launcher.cpp b/src/slave/containerizer/linux_launcher.cpp
index c17724b..9d0205e 100644
--- a/src/slave/containerizer/linux_launcher.cpp
+++ b/src/slave/containerizer/linux_launcher.cpp
@@ -271,7 +271,7 @@ Try<pid_t> LinuxLauncher::fork(
 
 Future<Nothing> _destroy(
     const ContainerID& containerId,
-    process::Future<bool> destroyed)
+    process::Future<Nothing> destroyed)
 {
   if (destroyed.isFailed()) {
     LOG(ERROR) << "Failed to destroy freezer cgroup for '"

http://git-wip-us.apache.org/repos/asf/mesos/blob/ca2ba6a0/src/slave/containerizer/mesos_containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos_containerizer.cpp b/src/slave/containerizer/mesos_containerizer.cpp
index d6df9a1..4d97d49 100644
--- a/src/slave/containerizer/mesos_containerizer.cpp
+++ b/src/slave/containerizer/mesos_containerizer.cpp
@@ -926,7 +926,7 @@ Future<Nothing> MesosContainerizerProcess::update(
 
   // Wait for all isolators to complete.
   return collect(futures)
-    .then(lambda::bind(_nothing));
+    .then(lambda::bind(&_nothing));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/ca2ba6a0/src/tests/cgroups_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cgroups_tests.cpp b/src/tests/cgroups_tests.cpp
index c00c30a..bfb5858 100644
--- a/src/tests/cgroups_tests.cpp
+++ b/src/tests/cgroups_tests.cpp
@@ -30,6 +30,7 @@
 #include <vector>
 
 #include <sys/mman.h>
+#include <sys/ptrace.h>
 #include <sys/types.h>
 #include <sys/wait.h>
 
@@ -756,15 +757,13 @@ TEST_F(CgroupsAnyHierarchyWithFreezerTest, ROOT_CGROUPS_Destroy)
     ASSERT_LT(0, ::read(pipes[0], &dummy, sizeof(dummy)));
     ::close(pipes[0]);
 
-    Future<bool> future = cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT);
-    future.await(Seconds(5));
-    ASSERT_TRUE(future.isReady());
-    EXPECT_TRUE(future.get());
+    AWAIT_READY(cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT));
 
+    // cgroups::destroy will reap all processes in the cgroup so we should
+    // *not* be able to reap it now.
     int status;
-    EXPECT_NE(-1, ::waitpid((pid_t) -1, &status, 0));
-    ASSERT_TRUE(WIFSIGNALED(status));
-    EXPECT_EQ(SIGKILL, WTERMSIG(status));
+    EXPECT_EQ(-1, ::waitpid(pid, &status, 0));
+    EXPECT_EQ(ECHILD, errno);
   } else {
     // In child process.
 
@@ -855,10 +854,91 @@ TEST_F(CgroupsAnyHierarchyWithFreezerTest, ROOT_CGROUPS_AssignThreads)
   CHECK_SOME(cgroups::assign(hierarchy, "", ::getpid()));
 
   // Destroy the cgroup.
-  Future<bool> future = cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT);
-  future.await(Seconds(5));
-  ASSERT_TRUE(future.isReady());
-  EXPECT_TRUE(future.get());
+  AWAIT_READY(cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT));
+}
+
+
+TEST_F(CgroupsAnyHierarchyWithFreezerTest, ROOT_CGROUPS_DestroyStoppedProcess)
+{
+  std::string hierarchy = path::join(baseHierarchy, "freezer");
+  ASSERT_SOME(cgroups::create(hierarchy, TEST_CGROUPS_ROOT));
+
+  pid_t pid = ::fork();
+  ASSERT_NE(-1, pid);
+
+  if (pid == 0) {
+    // In child process.
+    while (true) { sleep(1); }
+
+    ABORT("Child should not reach this statement");
+  }
+
+  // In parent process.
+
+  // Put child into the freezer cgroup.
+  Try<Nothing> assign = cgroups::assign(hierarchy, TEST_CGROUPS_ROOT, pid);
+
+  // Stop the child process.
+  EXPECT_EQ(0, kill(pid, SIGSTOP));
+
+  AWAIT_READY(cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT));
+
+  // cgroups::destroy will reap all processes in the cgroup so we should
+  // *not* be able to reap it now.
+  int status;
+  EXPECT_EQ(-1, ::waitpid(pid, &status, 0));
+  EXPECT_EQ(ECHILD, errno);
+}
+
+
+TEST_F(CgroupsAnyHierarchyWithFreezerTest, ROOT_CGROUPS_DestroyTracedProcess)
+{
+  std::string hierarchy = path::join(baseHierarchy, "freezer");
+  ASSERT_SOME(cgroups::create(hierarchy, TEST_CGROUPS_ROOT));
+
+  pid_t pid = ::fork();
+  ASSERT_NE(-1, pid);
+
+  if (pid == 0) {
+    // In child process.
+    while (true) { sleep(1); }
+
+    ABORT("Child should not reach this statement");
+  }
+
+  // In parent process.
+  Try<Nothing> assign = cgroups::assign(hierarchy, TEST_CGROUPS_ROOT, pid);
+  ASSERT_SOME(assign);
+
+  // Attach to the child process.
+  ASSERT_EQ(0, ptrace(PT_ATTACH, pid, NULL, NULL));
+
+  // Wait until the process is in traced state ('t' or 'T').
+  Duration elapsed = Duration::zero();
+  while (true) {
+    Result<proc::ProcessStatus> process = proc::status(pid);
+    ASSERT_SOME(process);
+
+    if (process.get().state == 'T' || process.get().state == 't') {
+      break;
+    }
+
+    if (elapsed > Seconds(1)) {
+      FAIL() << "Failed to wait for process to be traced";
+    }
+
+    os::sleep(Milliseconds(5));
+    elapsed += Milliseconds(5);
+  }
+
+  // Now destroy the cgroup.
+  AWAIT_READY(cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT));
+
+  // cgroups::destroy will reap all processes in the cgroup so we should
+  // *not* be able to reap it now.
+  int status;
+  EXPECT_EQ(-1, ::waitpid(pid, &status, 0));
+  EXPECT_EQ(ECHILD, errno);
 }
 
 
@@ -937,7 +1017,6 @@ TEST_F(CgroupsAnyHierarchyWithPerfEventTest, ROOT_CGROUPS_Perf)
   EXPECT_EQ(SIGKILL, WTERMSIG(status));
 
   // Destroy the cgroup.
-  Future<bool> destroy = cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT);
+  Future<Nothing> destroy = cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT);
   AWAIT_READY(destroy);
-  EXPECT_TRUE(destroy.get());
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/ca2ba6a0/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index 7044327..8c6421b 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -3232,6 +3232,7 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, CGROUPS_ROOT_PerfRollForward)
   // isolator.
   slave::Flags flags = this->CreateSlaveFlags();
   flags.isolation = "cgroups/cpu,cgroups/mem";
+  flags.slave_subsystems = "";
 
   Try<MesosContainerizer*> containerizer1 =
     MesosContainerizer::create(flags, true);