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 2014/02/12 02:31:17 UTC

[11/11] git commit: Containerizer - updated tests (part 5).

Containerizer - updated tests (part 5).

A few isolator specific tests haven't been updated.

The tests require a different cgroup mount configuration; please see
https://issues.apache.org/jira/browse/MESOS-926 for preliminary
documentation.

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


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

Branch: refs/heads/master
Commit: 0f3f8f35a73dd5f9b35d657b3912449f570243d3
Parents: d0db1ba
Author: Ian Downes <ia...@gmail.com>
Authored: Tue Feb 11 16:31:22 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Tue Feb 11 16:59:51 2014 -0800

----------------------------------------------------------------------
 src/Makefile.am                                 |   1 +
 src/examples/balloon_framework.cpp              |   7 +
 src/slave/containerizer/mesos_containerizer.cpp |   3 +-
 src/tests/allocator_tests.cpp                   |   8 +-
 src/tests/balloon_framework_test.sh             |  11 +-
 src/tests/cgroups_isolator_tests.cpp            | 271 +----------
 src/tests/cgroups_tests.cpp                     | 161 ++++---
 src/tests/cluster.hpp                           |  60 ++-
 src/tests/containerizer.cpp                     | 195 ++++++++
 src/tests/containerizer.hpp                     | 112 +++++
 src/tests/environment.cpp                       |  13 +-
 src/tests/fault_tolerance_tests.cpp             |  30 +-
 src/tests/gc_tests.cpp                          |  34 +-
 src/tests/isolator.hpp                          | 211 ---------
 src/tests/isolator_tests.cpp                    | 417 ++++++++++++----
 src/tests/master_contender_detector_tests.cpp   |   1 -
 src/tests/master_tests.cpp                      | 124 ++---
 src/tests/mesos.cpp                             | 187 +++++---
 src/tests/mesos.hpp                             |  41 +-
 src/tests/monitor_tests.cpp                     |  61 +--
 src/tests/paths_tests.cpp                       |  14 +-
 src/tests/slave_recovery_tests.cpp              | 471 +++++++++++--------
 src/tests/test_framework_test.sh                |   2 +
 23 files changed, 1334 insertions(+), 1101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 8a6a976..aa8bb2b 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -886,6 +886,7 @@ mesos_tests_LDADD = ../$(LIBPROCESS)/3rdparty/libgmock.la libmesos.la
 mesos_tests_DEPENDENCIES = # Initialized to allow += below.
 
 if OS_LINUX
+  mesos_tests_SOURCES += tests/cgroups_isolator_tests.cpp
   mesos_tests_SOURCES += tests/cgroups_tests.cpp
   mesos_tests_SOURCES += tests/fs_tests.cpp
 endif

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/examples/balloon_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/balloon_framework.cpp b/src/examples/balloon_framework.cpp
index d7abf1f..400764d 100644
--- a/src/examples/balloon_framework.cpp
+++ b/src/examples/balloon_framework.cpp
@@ -105,6 +105,13 @@ public:
         resource->set_type(Value::SCALAR);
         resource->mutable_scalar()->set_value(mem - EXECUTOR_MEMORY_MB);
 
+        // And all the CPU.
+        double cpus = getScalarResource(offer, "cpus");
+        resource = task.add_resources();
+        resource->set_name("cpus");
+        resource->set_type(Value::SCALAR);
+        resource->mutable_scalar()->set_value(cpus);
+
         tasks.push_back(task);
         driver->launchTasks(offer.id(), tasks);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/slave/containerizer/mesos_containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos_containerizer.cpp b/src/slave/containerizer/mesos_containerizer.cpp
index c61739b..04b9fa1 100644
--- a/src/slave/containerizer/mesos_containerizer.cpp
+++ b/src/slave/containerizer/mesos_containerizer.cpp
@@ -235,7 +235,8 @@ Future<Nothing> MesosContainerizerProcess::_recover(
 void asyncSafeFatal(const char* message)
 {
   // Ignore the return value from write() to silence compiler warning.
-  (void) write(STDERR_FILENO, message, strlen(message));
+  while (write(STDERR_FILENO, message, strlen(message)) == -1 &&
+      errno == EINTR);
   _exit(1);
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/allocator_tests.cpp b/src/tests/allocator_tests.cpp
index 751a6c4..31cc836 100644
--- a/src/tests/allocator_tests.cpp
+++ b/src/tests/allocator_tests.cpp
@@ -35,7 +35,7 @@
 #include "master/hierarchical_allocator_process.hpp"
 #include "master/master.hpp"
 
-#include "tests/isolator.hpp"
+#include "tests/containerizer.hpp"
 #include "tests/mesos.hpp"
 
 using namespace mesos;
@@ -1089,11 +1089,11 @@ TYPED_TEST(AllocatorTest, FrameworkExited)
   MockExecutor exec1(executor1.executor_id());
   MockExecutor exec2(executor2.executor_id());
 
-  map<ExecutorID, Executor*> execs;
+  hashmap<ExecutorID, Executor*> execs;
   execs[executor1.executor_id()] = &exec1;
   execs[executor2.executor_id()] = &exec2;
 
-  TestingIsolator isolator(execs);
+  TestContainerizer containerizer(execs);
 
   slave::Flags flags = this->CreateSlaveFlags();
 
@@ -1101,7 +1101,7 @@ TYPED_TEST(AllocatorTest, FrameworkExited)
 
   EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator, flags);
+  Try<PID<Slave> > slave = this->StartSlave(&containerizer, flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched1;

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/balloon_framework_test.sh
----------------------------------------------------------------------
diff --git a/src/tests/balloon_framework_test.sh b/src/tests/balloon_framework_test.sh
index e7bf4e6..11e9167 100755
--- a/src/tests/balloon_framework_test.sh
+++ b/src/tests/balloon_framework_test.sh
@@ -6,8 +6,13 @@
 source ${MESOS_SOURCE_DIR}/support/colors.sh
 source ${MESOS_SOURCE_DIR}/support/atexit.sh
 
-# TODO(benh): Look for an existing hierarchy first.
-TEST_CGROUP_HIERARCHY=/tmp/mesos_test_cgroup
+EXISTING_FREEZER_HIERARCHY=$(cat /proc/mounts | grep ^freezer | cut -f 2 -d ' ')
+if [[ -n ${EXISTING_FREEZER_HIERARCHY} ]]; then
+  # Strip off the subsystem component.
+  TEST_CGROUP_HIERARCHY=${EXISTING_FREEZER_HIERARCHY%/*}
+else
+  TEST_CGROUP_HIERARCHY=/tmp/mesos_test_cgroup
+fi
 TEST_CGROUP_ROOT=mesos_test
 
 # Check if the hierarchy exists. If it doesn't, we want to make sure we
@@ -63,7 +68,7 @@ BALLOON_FRAMEWORK=${MESOS_BUILD_DIR}/src/balloon-framework
 # to correspond to flags, so we unset these here.
 unset MESOS_BUILD_DIR
 unset MESOS_SOURCE_DIR
-unset MESOS_LAUNCHER_DIR
+#unset MESOS_LAUNCHER_DIR # leave this so we can find mesos-fetcher.
 unset MESOS_VERBOSE
 
 # Launch master.

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/cgroups_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cgroups_isolator_tests.cpp b/src/tests/cgroups_isolator_tests.cpp
index 1f5ce76..5a9704d 100644
--- a/src/tests/cgroups_isolator_tests.cpp
+++ b/src/tests/cgroups_isolator_tests.cpp
@@ -25,7 +25,7 @@
 #include <stout/proc.hpp>
 #include <stout/stringify.hpp>
 
-#include "slave/cgroups_isolator.hpp"
+#include "slave/containerizer/mesos_containerizer.hpp"
 
 #include "tests/script.hpp"
 
@@ -35,272 +35,7 @@ using namespace mesos::internal::slave;
 
 using std::map;
 
-// Run the balloon framework under the cgroups isolator.
-TEST_SCRIPT(CgroupsIsolatorTest,
+// Run the balloon framework under a mesos containerizer.
+TEST_SCRIPT(ContainerizerTest,
             ROOT_CGROUPS_BalloonFramework,
             "balloon_framework_test.sh")
-
-
-#define GROW_USAGE(delta, cpuset, usage)                                   \
-  ({                                                                       \
-    const map<proc::CPU, double>& allocation = cpuset.grow(delta, usage);  \
-    foreachpair (const proc::CPU& cpu, double allocated, allocation) {     \
-      usage[cpu] += allocated;                                             \
-      ASSERT_LT(usage[cpu], 1.001);                                        \
-    }                                                                      \
-  })
-
-
-#define SHRINK_USAGE(delta, cpuset, usage)                                 \
-  ({                                                                       \
-    const map<proc::CPU, double>& deallocation = cpuset.shrink(delta);     \
-    foreachpair (const proc::CPU& cpu, double deallocated, deallocation) { \
-      usage[cpu] -= deallocated;                                           \
-      ASSERT_GT(usage[cpu], -0.001);                                       \
-    }                                                                      \
-  })
-
-
-TEST(CgroupsCpusetTest, OneCPUOneCpuset)
-{
-  Cpuset cpuset;
-
-  map<proc::CPU, double> usage;
-  // NOTE: Using the [] operator here led to a warning with gcc 4.4.3.
-  usage.insert(std::make_pair(proc::CPU(0, 0, 0), 0.0));
-
-  // Saturate the CPU.
-  GROW_USAGE(0.2, cpuset, usage);
-  GROW_USAGE(0.1, cpuset, usage);
-  GROW_USAGE(0.2, cpuset, usage);
-  GROW_USAGE(0.5, cpuset, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(0,0,0)], 1.0, 0.001);
-  ASSERT_NEAR(cpuset.usage(), 1.0, 0.001);
-
-  ASSERT_EQ(stringify(cpuset), "0");
-
-  // Empty the CPU.
-  SHRINK_USAGE(0.5, cpuset, usage);
-  SHRINK_USAGE(0.2, cpuset, usage);
-  SHRINK_USAGE(0.1, cpuset, usage);
-  SHRINK_USAGE(0.2, cpuset, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(0,0,0)], 0.0, 0.001);
-  ASSERT_NEAR(cpuset.usage(), 0.0, 0.001);
-
-  ASSERT_EQ(stringify(cpuset), "");
-}
-
-
-TEST(CgroupsCpusetTest, OneCPUManyCpusets)
-{
-  Cpuset cpuset1, cpuset2, cpuset3;
-
-  map<proc::CPU, double> usage;
-  // NOTE: Using the [] operator here led to a warning with gcc 4.4.3.
-  usage.insert(std::make_pair(proc::CPU(0, 0, 0), 0.0));
-
-  // Saturate the CPU.
-  GROW_USAGE(0.2, cpuset1, usage);
-  GROW_USAGE(0.1, cpuset2, usage);
-  GROW_USAGE(0.2, cpuset3, usage);
-  GROW_USAGE(0.5, cpuset1, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(0,0,0)], 1.0, 0.001);
-  ASSERT_NEAR(cpuset1.usage(), 0.7, 0.001);
-  ASSERT_NEAR(cpuset2.usage(), 0.1, 0.001);
-  ASSERT_NEAR(cpuset3.usage(), 0.2, 0.001);
-
-  ASSERT_EQ(stringify(cpuset1), "0");
-  ASSERT_EQ(stringify(cpuset2), "0");
-  ASSERT_EQ(stringify(cpuset3), "0");
-
-  // Empty the CPU.
-  SHRINK_USAGE(0.5, cpuset1, usage);
-  SHRINK_USAGE(0.2, cpuset3, usage);
-  SHRINK_USAGE(0.1, cpuset2, usage);
-  SHRINK_USAGE(0.2, cpuset1, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(0,0,0)], 0.0, 0.001);
-  ASSERT_NEAR(cpuset1.usage(), 0.0, 0.001);
-  ASSERT_NEAR(cpuset2.usage(), 0.0, 0.001);
-  ASSERT_NEAR(cpuset3.usage(), 0.0, 0.001);
-
-  ASSERT_EQ(stringify(cpuset1), "");
-  ASSERT_EQ(stringify(cpuset2), "");
-  ASSERT_EQ(stringify(cpuset3), "");
-}
-
-
-TEST(CgroupsCpusetTest, ManyCPUOneCpuset)
-{
-  Cpuset cpuset;
-
-  map<proc::CPU, double> usage;
-  // NOTE: Using the [] operator here led to a warning with gcc 4.4.3.
-  usage.insert(std::make_pair(proc::CPU(0, 0, 0), 0.0));
-  usage.insert(std::make_pair(proc::CPU(1, 0, 0), 0.0));
-  usage.insert(std::make_pair(proc::CPU(2, 0, 0), 0.0));
-
-  // Saturate the first CPU.
-  GROW_USAGE(0.2, cpuset, usage);
-  GROW_USAGE(0.1, cpuset, usage);
-  GROW_USAGE(0.2, cpuset, usage);
-  GROW_USAGE(0.5, cpuset, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(0,0,0)], 1.0, 0.001);
-  ASSERT_NEAR(cpuset.usage(), 1.0, 0.001);
-
-  ASSERT_EQ(stringify(cpuset), "0");
-
-  // Saturate the second CPU.
-  GROW_USAGE(0.6, cpuset, usage);
-  GROW_USAGE(0.4, cpuset, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(0,0,0)], 1.0, 0.001);
-  ASSERT_NEAR(usage[proc::CPU(1,0,0)], 1.0, 0.001);
-  ASSERT_NEAR(cpuset.usage(), 2.0, 0.001);
-
-  ASSERT_EQ(stringify(cpuset), "0,1");
-
-  // Partial third CPU.
-  GROW_USAGE(0.1, cpuset, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(2,0,0)], 0.1, 0.001);
-  ASSERT_NEAR(cpuset.usage(), 2.1, 0.001);
-
-  ASSERT_EQ(stringify(cpuset), "0,1,2");
-
-  // Empty the CPU.
-  SHRINK_USAGE(0.5, cpuset, usage);
-  SHRINK_USAGE(0.2, cpuset, usage);
-  SHRINK_USAGE(0.1, cpuset, usage);
-  SHRINK_USAGE(0.1, cpuset, usage);
-  SHRINK_USAGE(0.2, cpuset, usage);
-  SHRINK_USAGE(0.4, cpuset, usage);
-  SHRINK_USAGE(0.6, cpuset, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(0,0,0)], 0.0, 0.001);
-  ASSERT_NEAR(cpuset.usage(), 0.0, 0.001);
-
-  ASSERT_EQ(stringify(cpuset), "");
-}
-
-
-TEST(CgroupsCpusetTest, ManyCPUManyCpusets)
-{
-  Cpuset cpuset1, cpuset2, cpuset3;
-
-  map<proc::CPU, double> usage;
-  // NOTE: Using the [] operator here led to a warning with gcc 4.4.3.
-  usage.insert(std::make_pair(proc::CPU(0, 0, 0), 0.0));
-  usage.insert(std::make_pair(proc::CPU(1, 0, 0), 0.0));
-  usage.insert(std::make_pair(proc::CPU(2, 0, 0), 0.0));
-
-  // Saturate the first CPU.
-  GROW_USAGE(0.2, cpuset1, usage);
-  GROW_USAGE(0.1, cpuset2, usage);
-  GROW_USAGE(0.2, cpuset3, usage);
-  GROW_USAGE(0.5, cpuset1, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(0,0,0)], 1.0, 0.001);
-  ASSERT_NEAR(cpuset1.usage(), 0.7, 0.001);
-  ASSERT_NEAR(cpuset2.usage(), 0.1, 0.001);
-  ASSERT_NEAR(cpuset3.usage(), 0.2, 0.001);
-
-  ASSERT_EQ(stringify(cpuset1), "0");
-  ASSERT_EQ(stringify(cpuset2), "0");
-  ASSERT_EQ(stringify(cpuset3), "0");
-
-  // Saturate the second CPU.
-  GROW_USAGE(0.6, cpuset3, usage);
-  GROW_USAGE(0.4, cpuset2, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(0,0,0)], 1.0, 0.001);
-  ASSERT_NEAR(usage[proc::CPU(1,0,0)], 1.0, 0.001);
-  ASSERT_NEAR(cpuset2.usage(), 0.5, 0.001);
-  ASSERT_NEAR(cpuset3.usage(), 0.8, 0.001);
-
-  ASSERT_EQ(stringify(cpuset2), "0,1");
-  ASSERT_EQ(stringify(cpuset3), "0,1");
-
-  // Partial third CPU.
-  GROW_USAGE(0.1, cpuset2, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(2,0,0)], 0.1, 0.001);
-  ASSERT_NEAR(cpuset2.usage(), 0.6, 0.001);
-
-  ASSERT_EQ(stringify(cpuset2), "0,1,2");
-
-  // Empty the CPU.
-  SHRINK_USAGE(0.5, cpuset1, usage);
-  SHRINK_USAGE(0.2, cpuset1, usage);
-  SHRINK_USAGE(0.1, cpuset2, usage);
-  SHRINK_USAGE(0.1, cpuset2, usage);
-  SHRINK_USAGE(0.2, cpuset3, usage);
-  SHRINK_USAGE(0.4, cpuset2, usage);
-  SHRINK_USAGE(0.6, cpuset3, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(0,0,0)], 0.0, 0.001);
-  ASSERT_NEAR(usage[proc::CPU(1,0,0)], 0.0, 0.001);
-  ASSERT_NEAR(usage[proc::CPU(2,0,0)], 0.0, 0.001);
-
-  ASSERT_NEAR(cpuset1.usage(), 0.0, 0.001);
-  ASSERT_NEAR(cpuset2.usage(), 0.0, 0.001);
-  ASSERT_NEAR(cpuset3.usage(), 0.0, 0.001);
-
-  ASSERT_EQ(stringify(cpuset1), "");
-  ASSERT_EQ(stringify(cpuset2), "");
-  ASSERT_EQ(stringify(cpuset3), "");
-}
-
-
-TEST(CgroupsCpusetTest, IntegerAllocations)
-{
-  // Ensure no fragmentation occurs.
-  Cpuset cpuset1, cpuset2, cpuset3;
-
-  map<proc::CPU, double> usage;
-  // NOTE: Using the [] operator here led to a warning with gcc 4.4.3.
-  usage.insert(std::make_pair(proc::CPU(0, 0, 0), 0.0));
-  usage.insert(std::make_pair(proc::CPU(1, 0, 0), 0.0));
-  usage.insert(std::make_pair(proc::CPU(2, 0, 0), 0.0));
-  usage.insert(std::make_pair(proc::CPU(3, 0, 0), 0.0));
-
-  // Saturate the CPUs.
-  GROW_USAGE(1.0, cpuset1, usage);
-  GROW_USAGE(2.0, cpuset2, usage);
-  GROW_USAGE(1.0, cpuset3, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(0,0,0)], 1.0, 0.001);
-  ASSERT_NEAR(usage[proc::CPU(1,0,0)], 1.0, 0.001);
-  ASSERT_NEAR(usage[proc::CPU(2,0,0)], 1.0, 0.001);
-  ASSERT_NEAR(usage[proc::CPU(3,0,0)], 1.0, 0.001);
-
-  ASSERT_NEAR(cpuset1.usage(), 1.0, 0.001);
-  ASSERT_NEAR(cpuset2.usage(), 2.0, 0.001);
-  ASSERT_NEAR(cpuset3.usage(), 1.0, 0.001);
-
-  ASSERT_EQ(stringify(cpuset1), "0");
-  ASSERT_EQ(stringify(cpuset2), "1,2");
-  ASSERT_EQ(stringify(cpuset3), "3");
-
-  // Empty the CPU.
-  SHRINK_USAGE(1.0, cpuset1, usage);
-  SHRINK_USAGE(2.0, cpuset2, usage);
-  SHRINK_USAGE(1.0, cpuset3, usage);
-
-  ASSERT_NEAR(usage[proc::CPU(0,0,0)], 0.0, 0.001);
-  ASSERT_NEAR(usage[proc::CPU(1,0,0)], 0.0, 0.001);
-  ASSERT_NEAR(usage[proc::CPU(2,0,0)], 0.0, 0.001);
-  ASSERT_NEAR(usage[proc::CPU(3,0,0)], 0.0, 0.001);
-
-  ASSERT_NEAR(cpuset1.usage(), 0.0, 0.001);
-  ASSERT_NEAR(cpuset2.usage(), 0.0, 0.001);
-  ASSERT_NEAR(cpuset3.usage(), 0.0, 0.001);
-
-  ASSERT_EQ(stringify(cpuset1), "");
-  ASSERT_EQ(stringify(cpuset2), "");
-  ASSERT_EQ(stringify(cpuset3), "");
-}

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/cgroups_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cgroups_tests.cpp b/src/tests/cgroups_tests.cpp
index 0e9316d..f0dead7 100644
--- a/src/tests/cgroups_tests.cpp
+++ b/src/tests/cgroups_tests.cpp
@@ -109,55 +109,65 @@ public:
 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, retrying as necessary since the
-      // previous unmount might not have taken effect yet due to a bug in
-      // Ubuntu 12.04.
-      ASSERT_SOME(cgroups::mount(TEST_CGROUPS_HIERARCHY, subsystems, 10))
-        << "-------------------------------------------------------------\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();
+    foreach (const std::string& subsystem, strings::tokenize(subsystems, ",")) {
+      // Establish the base hierarchy if this is the first subsystem checked.
+      if (baseHierarchy.empty()) {
+        Result<std::string> hierarchy = cgroups::hierarchy(subsystem);
+        ASSERT_FALSE(hierarchy.isError());
+
+        if (hierarchy.isNone()) {
+          baseHierarchy = TEST_CGROUPS_HIERARCHY;
+        } else {
+          // Strip the subsystem to get the base hierarchy.
+          baseHierarchy = strings::remove(
+              hierarchy.get(),
+              subsystem,
+              strings::SUFFIX);
+        }
+      }
+
+      // Mount the subsystem if necessary.
+      std::string hierarchy = path::join(baseHierarchy, subsystem);
+      Try<bool> mounted = cgroups::mounted(hierarchy, subsystem);
+      ASSERT_SOME(mounted);
+      if (!mounted.get()) {
+        ASSERT_SOME(cgroups::mount(hierarchy, subsystem))
+          << "-------------------------------------------------------------\n"
+          << "We cannot run any cgroups tests that require\n"
+          << "a hierarchy with subsystem '" << subsystem << "'\n"
+          << "because we failed to find an existing hierarchy\n"
+          << "or create a new one (tried '" << hierarchy << "').\n"
+          << "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"
+          << "-------------------------------------------------------------";
+      }
     }
-
-    // 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()) {
-     AWAIT_READY(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()) {
-     AWAIT_READY(cgroups::destroy(hierarchy, TEST_CGROUPS_ROOT));
+    foreach (const std::string& subsystem, strings::tokenize(subsystems, ",")) {
+      std::string hierarchy = path::join(baseHierarchy, subsystem);
+
+      Try<std::vector<std::string> > cgroups = cgroups::get(hierarchy);
+      CHECK_SOME(cgroups);
+
+      foreach (const std::string& cgroup, cgroups.get()) {
+        // Remove any cgroups that start with TEST_CGROUPS_ROOT.
+        if (cgroup == TEST_CGROUPS_ROOT) {
+          AWAIT_READY(cgroups::destroy(hierarchy, cgroup));
+        }
+      }
     }
-
-    // And cleanup TEST_CGROUPS_HIERARCHY in the event it is needed
-    // to be created.
-    AWAIT_READY(cgroups::cleanup(TEST_CGROUPS_HIERARCHY));
   }
 
   const std::string subsystems; // Subsystems required to run tests.
-  std::string hierarchy; // Path to the hierarchy being used.
+  std::string baseHierarchy; // Path to the hierarchy being used.
 };
 
 
@@ -227,7 +237,9 @@ TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Subsystems)
 
 TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_SubsystemsHierarchy)
 {
-  Try<std::set<std::string> > names = cgroups::subsystems(hierarchy);
+  std::string cpuHierarchy = path::join(baseHierarchy, "cpu");
+
+  Try<std::set<std::string> > names = cgroups::subsystems(cpuHierarchy);
   ASSERT_SOME(names);
 
   Option<std::string> cpu;
@@ -241,6 +253,22 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_SubsystemsHierarchy)
   }
 
   EXPECT_SOME(cpu);
+  EXPECT_NONE(memory);
+
+  std::string memoryHierarchy = path::join(baseHierarchy, "memory");
+  names = cgroups::subsystems(memoryHierarchy);
+  ASSERT_SOME(names);
+
+  cpu = None();
+  memory = None();
+  foreach (const std::string& name, names.get()) {
+    if (name == "cpu") {
+      cpu = name;
+    } else if (name == "memory") {
+      memory = name;
+    }
+  }
+  EXPECT_NONE(cpu);
   EXPECT_SOME(memory);
 }
 
@@ -264,9 +292,8 @@ TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Mounted)
 {
   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 + "/"));
+  EXPECT_SOME_FALSE(cgroups::mounted(baseHierarchy + "/not_expected"));
+  EXPECT_SOME_TRUE(cgroups::mounted(baseHierarchy + "/cpu"));
 }
 
 
@@ -276,25 +303,30 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_MountedSubsystems)
   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"));
+  EXPECT_SOME_TRUE(cgroups::mounted(path::join(baseHierarchy, "cpu"), "cpu"));
+  EXPECT_SOME_TRUE(cgroups::mounted(
+        path::join(baseHierarchy, "memory"), "memory"));
+  EXPECT_SOME_FALSE(cgroups::mounted(baseHierarchy, "invalid"));
+  EXPECT_SOME_FALSE(cgroups::mounted(baseHierarchy + "/not_expected", "cpu"));
 }
 
 
 TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_CreateRemove)
 {
   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"));
+  EXPECT_ERROR(cgroups::create(baseHierarchy, "mesos_test_missing/1"));
+  ASSERT_SOME(cgroups::create(
+        path::join(baseHierarchy, "cpu"), "mesos_test_missing"));
+  EXPECT_ERROR(cgroups::remove(baseHierarchy, "invalid"));
+  ASSERT_SOME(cgroups::remove(
+        path::join(baseHierarchy, "cpu"), "mesos_test_missing"));
 }
 
 
 TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Get)
 {
+  std::string hierarchy = path::join(baseHierarchy, "cpu");
+
   ASSERT_SOME(cgroups::create(hierarchy, "mesos_test1"));
   ASSERT_SOME(cgroups::create(hierarchy, "mesos_test2"));
 
@@ -303,7 +335,6 @@ TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Get)
 
   EXPECT_EQ(cgroups.get()[0], "mesos_test2");
   EXPECT_EQ(cgroups.get()[1], "mesos_test1");
-  EXPECT_EQ(cgroups.get()[2], TEST_CGROUPS_ROOT);
 
   ASSERT_SOME(cgroups::remove(hierarchy, "mesos_test1"));
   ASSERT_SOME(cgroups::remove(hierarchy, "mesos_test2"));
@@ -312,6 +343,8 @@ TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Get)
 
 TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_NestedCgroups)
 {
+  std::string hierarchy = path::join(baseHierarchy, "cpu");
+  ASSERT_SOME(cgroups::create(hierarchy, TEST_CGROUPS_ROOT));
   ASSERT_SOME(cgroups::create(hierarchy, path::join(TEST_CGROUPS_ROOT, "1")))
     << "-------------------------------------------------------------\n"
     << "We cannot run this test because it appears you do not have\n"
@@ -338,6 +371,7 @@ TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_NestedCgroups)
 
 TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Tasks)
 {
+  std::string hierarchy = path::join(baseHierarchy, "cpu");
   Try<std::set<pid_t> > pids = cgroups::processes(hierarchy, "/");
   ASSERT_SOME(pids);
   EXPECT_NE(0u, pids.get().count(1));
@@ -347,6 +381,7 @@ TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Tasks)
 
 TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Read)
 {
+  std::string hierarchy = path::join(baseHierarchy, "cpu");
   EXPECT_ERROR(cgroups::read(hierarchy, TEST_CGROUPS_ROOT, "invalid"));
 
   std::string pid = stringify(::getpid());
@@ -359,12 +394,15 @@ TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Read)
 
 TEST_F(CgroupsAnyHierarchyTest, ROOT_CGROUPS_Write)
 {
+  std::string hierarchy = path::join(baseHierarchy, "cpu");
   EXPECT_ERROR(
       cgroups::write(hierarchy, TEST_CGROUPS_ROOT, "invalid", "invalid"));
 
   pid_t pid = ::fork();
   ASSERT_NE(-1, pid);
 
+  ASSERT_SOME(cgroups::create(hierarchy, TEST_CGROUPS_ROOT));
+
   if (pid > 0) {
     // In parent process.
     ASSERT_SOME(
@@ -405,17 +443,19 @@ public:
 
 TEST_F(CgroupsAnyHierarchyWithCpuAcctMemoryTest, ROOT_CGROUPS_Stat)
 {
-  EXPECT_ERROR(cgroups::stat(hierarchy, TEST_CGROUPS_ROOT, "invalid"));
+  EXPECT_ERROR(cgroups::stat(baseHierarchy, TEST_CGROUPS_ROOT, "invalid"));
 
   Try<hashmap<std::string, uint64_t> > result =
-    cgroups::stat(hierarchy, "/", "cpuacct.stat");
+    cgroups::stat(
+        path::join(baseHierarchy, "cpuacct"), "/", "cpuacct.stat");
   ASSERT_SOME(result);
   EXPECT_TRUE(result.get().contains("user"));
   EXPECT_TRUE(result.get().contains("system"));
   EXPECT_GT(result.get()["user"], 0llu);
   EXPECT_GT(result.get()["system"], 0llu);
 
-  result = cgroups::stat(hierarchy, "/", "memory.stat");
+  result = cgroups::stat(
+      path::join(baseHierarchy, "memory"), "/", "memory.stat");
   ASSERT_SOME(result);
   EXPECT_TRUE(result.get().contains("rss"));
   EXPECT_GT(result.get()["rss"], 0llu);
@@ -424,6 +464,8 @@ TEST_F(CgroupsAnyHierarchyWithCpuAcctMemoryTest, ROOT_CGROUPS_Stat)
 
 TEST_F(CgroupsAnyHierarchyWithCpuMemoryTest, ROOT_CGROUPS_Listen)
 {
+  std::string hierarchy = path::join(baseHierarchy, "memory");
+  ASSERT_SOME(cgroups::create(hierarchy, TEST_CGROUPS_ROOT));
   ASSERT_SOME(
       cgroups::exists(hierarchy, TEST_CGROUPS_ROOT, "memory.oom_control"))
     << "-------------------------------------------------------------\n"
@@ -519,6 +561,9 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryFreezerTest, ROOT_CGROUPS_Freeze)
   int dummy;
   ASSERT_NE(-1, ::pipe(pipes));
 
+  std::string hierarchy = path::join(baseHierarchy, "freezer");
+  ASSERT_SOME(cgroups::create(hierarchy, TEST_CGROUPS_ROOT));
+
   pid_t pid = ::fork();
   ASSERT_NE(-1, pid);
 
@@ -586,6 +631,9 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryFreezerTest, ROOT_CGROUPS_Kill)
   int dummy;
   ASSERT_NE(-1, ::pipe(pipes));
 
+  std::string hierarchy = path::join(baseHierarchy, "freezer");
+  ASSERT_SOME(cgroups::create(hierarchy, TEST_CGROUPS_ROOT));
+
   pid_t pid = ::fork();
   ASSERT_NE(-1, pid);
 
@@ -649,6 +697,9 @@ TEST_F(CgroupsAnyHierarchyWithCpuMemoryFreezerTest, ROOT_CGROUPS_Destroy)
   int dummy;
   ASSERT_NE(-1, ::pipe(pipes));
 
+  std::string hierarchy = path::join(baseHierarchy, "freezer");
+  ASSERT_SOME(cgroups::create(hierarchy, TEST_CGROUPS_ROOT));
+
   pid_t pid = ::fork();
   ASSERT_NE(-1, pid);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/cluster.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cluster.hpp b/src/tests/cluster.hpp
index 1334800..5100959 100644
--- a/src/tests/cluster.hpp
+++ b/src/tests/cluster.hpp
@@ -44,8 +44,7 @@
 #include "master/registrar.hpp"
 
 #include "slave/flags.hpp"
-#include "slave/isolator.hpp"
-#include "slave/process_isolator.hpp"
+#include "slave/containerizer/containerizer.hpp"
 #include "slave/slave.hpp"
 
 #include "state/leveldb.hpp"
@@ -142,7 +141,7 @@ public:
     // The isolator is expected to outlive the launched slave (i.e.,
     // until it is stopped via Slaves::stop).
     Try<process::PID<slave::Slave> > start(
-        slave::Isolator* isolator,
+        slave::Containerizer* containerizer,
         const slave::Flags& flags = slave::Flags());
 
     // Start and manage a new slave injecting the specified Master
@@ -153,7 +152,7 @@ public:
         const slave::Flags& flags = slave::Flags());
 
     Try<process::PID<slave::Slave> > start(
-        slave::Isolator* isolator,
+        slave::Containerizer* containerizer,
         process::Owned<MasterDetector> detector,
         const slave::Flags& flags = slave::Flags());
 
@@ -176,13 +175,13 @@ public:
     struct Slave
     {
       Slave()
-        : isolator(NULL),
+        : containerizer(NULL),
           slave(NULL),
           detector(NULL) {}
 
-      // Only register the isolator here if it is created within the
+      // Only register the containerizer here if it is created within the
       // Cluster.
-      slave::Isolator* isolator;
+      slave::Containerizer* containerizer;
       slave::Slave* slave;
       process::Owned<MasterDetector> detector;
     };
@@ -424,15 +423,18 @@ inline Try<process::PID<slave::Slave> > Cluster::Slaves::start(
 
   Slave slave;
 
-  // Create a new process isolator for this slave.
-  slave.isolator = new slave::ProcessIsolator();
-  process::spawn(slave.isolator);
+  // Create a new containerizer for this slave.
+  Try<slave::Containerizer*> containerizer =
+    slave::Containerizer::create(flags, true);
+  CHECK_SOME(containerizer);
+
+  slave.containerizer = containerizer.get();
 
   // Get a detector for the master(s).
   slave.detector = masters->detector();
 
-  slave.slave = new slave::Slave(flags, true, slave.detector.get(), slave.isolator,
-      &cluster->files);
+  slave.slave = new slave::Slave(
+      flags, slave.detector.get(), slave.containerizer, &cluster->files);
   process::PID<slave::Slave> pid = process::spawn(slave.slave);
 
   slaves[pid] = slave;
@@ -442,10 +444,10 @@ inline Try<process::PID<slave::Slave> > Cluster::Slaves::start(
 
 
 inline Try<process::PID<slave::Slave> > Cluster::Slaves::start(
-    slave::Isolator* isolator,
+    slave::Containerizer* containerizer,
     const slave::Flags& flags)
 {
-  return start(isolator, masters->detector(), flags);
+  return start(containerizer, masters->detector(), flags);
 }
 
 
@@ -453,12 +455,32 @@ inline Try<process::PID<slave::Slave> > Cluster::Slaves::start(
     process::Owned<MasterDetector> detector,
     const slave::Flags& flags)
 {
-  return start(new slave::ProcessIsolator(), detector, flags);
+  // TODO(benh): Create a work directory if using the default.
+
+  Slave slave;
+
+  // Create a new containerizer for this slave.
+  Try<slave::Containerizer*> containerizer =
+    slave::Containerizer::create(flags, true);
+  CHECK_SOME(containerizer);
+
+  slave.containerizer = containerizer.get();
+
+  // Get a detector for the master(s).
+  slave.detector = detector;
+
+  slave.slave = new slave::Slave(
+      flags, slave.detector.get(), slave.containerizer, &cluster->files);
+  process::PID<slave::Slave> pid = process::spawn(slave.slave);
+
+  slaves[pid] = slave;
+
+  return pid;
 }
 
 
 inline Try<process::PID<slave::Slave> > Cluster::Slaves::start(
-    slave::Isolator* isolator,
+    slave::Containerizer* containerizer,
     process::Owned<MasterDetector> detector,
     const slave::Flags& flags)
 {
@@ -469,8 +491,8 @@ inline Try<process::PID<slave::Slave> > Cluster::Slaves::start(
   // Get a detector for the master(s).
   slave.detector = detector;
 
-  slave.slave = new slave::Slave(flags, true, slave.detector.get(),
-      isolator, &cluster->files);
+  slave.slave = new slave::Slave(
+      flags, slave.detector.get(), containerizer, &cluster->files);
   process::PID<slave::Slave> pid = process::spawn(slave.slave);
 
   slaves[pid] = slave;
@@ -497,7 +519,7 @@ inline Try<Nothing> Cluster::Slaves::stop(
   process::wait(slave.slave);
   delete slave.slave;
 
-  delete slave.isolator; // May be NULL.
+  delete slave.containerizer; // May be NULL.
 
   slaves.erase(pid);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.cpp b/src/tests/containerizer.cpp
new file mode 100644
index 0000000..bfb9341
--- /dev/null
+++ b/src/tests/containerizer.cpp
@@ -0,0 +1,195 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "tests/containerizer.hpp"
+#include "tests/mesos.hpp"
+
+using std::map;
+using std::string;
+
+using namespace process;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+
+TestContainerizer::TestContainerizer(
+    const hashmap<ExecutorID, Executor*>& _executors)
+  : executors(_executors)
+{
+  setup();
+}
+
+
+TestContainerizer::TestContainerizer(
+    const ExecutorID& executorId,
+    Executor* executor)
+{
+  executors[executorId] = executor;
+  setup();
+}
+
+
+TestContainerizer::TestContainerizer(MockExecutor* executor)
+{
+  executors[executor->id] = executor;
+  setup();
+}
+
+
+TestContainerizer::TestContainerizer()
+{
+  setup();
+}
+
+
+TestContainerizer::~TestContainerizer()
+{
+  foreachvalue (const Owned<MesosExecutorDriver>& driver, drivers) {
+    driver->stop();
+    driver->join();
+  }
+  drivers.clear();
+}
+
+
+Future<Nothing> TestContainerizer::launch(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo,
+    const string& directory,
+    const Option<string>& user,
+    const SlaveID& slaveId,
+    const PID<slave::Slave>& slavePid,
+    bool checkpoint)
+{
+  CHECK(!drivers.contains(containerId))
+    << "Failed to launch executor " << executorInfo.executor_id()
+    << " of framework " << executorInfo.framework_id()
+    << " because it is already launched";
+
+  CHECK(executors.contains(executorInfo.executor_id()))
+    << "Failed to launch executor " << executorInfo.executor_id()
+    << " of framework " << executorInfo.framework_id()
+    << " because it is unknown to the containerizer";
+
+  // Store mapping from (frameworkId, executorId) -> containerId to facilitate
+  // easy destroy from tests.
+  std::pair<FrameworkID, ExecutorID> key(executorInfo.framework_id(),
+                                         executorInfo.executor_id());
+  containers[key] = containerId;
+
+  Executor* executor = executors[executorInfo.executor_id()];
+  Owned<MesosExecutorDriver> driver(new MesosExecutorDriver(executor));
+  drivers[containerId] = driver;
+
+  // Prepare additional environment variables for the executor.
+  const map<string, string>& env = executorEnvironment(
+      executorInfo,
+      directory,
+      slaveId,
+      slavePid,
+      checkpoint,
+      Duration::zero());
+
+  foreachpair (const string& name, const string variable, env) {
+    os::setenv(name, variable);
+  }
+
+  foreach (const Environment_Variable& variable,
+      executorInfo.command().environment().variables()) {
+    os::setenv(variable.name(), variable.value());
+  }
+  os::setenv("MESOS_LOCAL", "1");
+
+  driver->start();
+
+  foreachkey (const string& name, env) {
+    os::unsetenv(name);
+  }
+
+  foreach(const Environment_Variable& variable,
+      executorInfo.command().environment().variables()) {
+    os::unsetenv(variable.name());
+  }
+  os::unsetenv("MESOS_LOCAL");
+
+  Owned<Promise<slave::Containerizer::Termination> > promise(
+      new Promise<slave::Containerizer::Termination>());
+  promises[containerId] = promise;
+
+  return Nothing();
+}
+
+
+Future<slave::Containerizer::Termination> TestContainerizer::wait(
+    const ContainerID& containerId)
+{
+  CHECK(promises.contains(containerId))
+    << "Container " << containerId << "not started";
+
+  return promises[containerId]->future();
+}
+
+
+void TestContainerizer::destroy(
+    const FrameworkID& frameworkId,
+    const ExecutorID& executorId)
+{
+  std::pair<FrameworkID, ExecutorID> key(frameworkId, executorId);
+  if (!containers.contains(key)) {
+    LOG(WARNING) << "Ignoring destroy of unknown container for executor '"
+                  << executorId << "' of framework '" << frameworkId << "'";
+    return;
+  }
+  destroy(containers[key]);
+}
+
+
+void TestContainerizer::destroy(const ContainerID& containerId)
+{
+  CHECK(drivers.contains(containerId))
+    << "Failed to terminate container " << containerId
+    << " because it is has not been started";
+
+  Owned<MesosExecutorDriver> driver = drivers[containerId];
+  driver->stop();
+  driver->join();
+  drivers.erase(containerId);
+
+  promises[containerId]->set(
+      slave::Containerizer::Termination(0, false, "Killed executor"));
+  promises.erase(containerId);
+}
+
+
+void TestContainerizer::setup()
+{
+  EXPECT_CALL(*this, recover(testing::_))
+    .WillRepeatedly(testing::Return(Nothing()));
+
+  EXPECT_CALL(*this, usage(testing::_))
+    .WillRepeatedly(testing::Return(ResourceStatistics()));
+
+  EXPECT_CALL(*this, update(testing::_, testing::_))
+    .WillRepeatedly(testing::Return(Nothing()));
+}
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.hpp b/src/tests/containerizer.hpp
new file mode 100644
index 0000000..5686398
--- /dev/null
+++ b/src/tests/containerizer.hpp
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __TEST_CONTAINERIZER_HPP__
+#define __TEST_CONTAINERIZER_HPP__
+
+#include <unistd.h>
+
+#include <gmock/gmock.h>
+
+#include <map>
+#include <string>
+
+#include <process/dispatch.hpp>
+#include <process/future.hpp>
+#include <process/pid.hpp>
+
+#include <stout/hashmap.hpp>
+#include <stout/os.hpp>
+#include <stout/try.hpp>
+#include <stout/uuid.hpp>
+
+#include "mesos/executor.hpp"
+#include "mesos/mesos.hpp"
+#include "mesos/resources.hpp"
+
+#include "slave/containerizer/containerizer.hpp"
+
+#include "slave/slave.hpp"
+#include "slave/state.hpp"
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+// Forward declaration.
+class MockExecutor;
+
+class TestContainerizer : public slave::Containerizer
+{
+public:
+  TestContainerizer(const hashmap<ExecutorID, Executor*>& executors);
+
+  TestContainerizer(const ExecutorID& executorId, Executor* executor);
+
+  TestContainerizer(MockExecutor* executor);
+
+  TestContainerizer();
+
+  virtual ~TestContainerizer();
+
+  virtual process::Future<Nothing> launch(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo,
+      const std::string& directory,
+      const Option<std::string>& user,
+      const SlaveID& slaveId,
+      const process::PID<slave::Slave>& slavePid,
+      bool checkpoint);
+
+  virtual process::Future<slave::Containerizer::Termination> wait(
+      const ContainerID& containerId);
+
+  // Additional destroy method for testing because we won't know the
+  // ContainerID created for each container.
+  void destroy(const FrameworkID& frameworkId, const ExecutorID& executorId);
+
+  virtual void destroy(const ContainerID& containerId);
+
+  MOCK_METHOD1(
+      recover,
+      process::Future<Nothing>(const Option<slave::state::SlaveState>&));
+
+  MOCK_METHOD2(
+      update,
+      process::Future<Nothing>(const ContainerID&, const Resources&));
+
+  MOCK_METHOD1(
+      usage,
+      process::Future<ResourceStatistics>(const ContainerID&));
+
+private:
+  void setup();
+
+  hashmap<ExecutorID, Executor*> executors;
+
+  hashmap<std::pair<FrameworkID, ExecutorID>, ContainerID> containers;
+  hashmap<ContainerID, process::Owned<MesosExecutorDriver> > drivers;
+  hashmap<ContainerID,
+          process::Owned<process::Promise<slave::Containerizer::Termination> > > promises;
+};
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __TEST_CONTAINERIZER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/environment.cpp
----------------------------------------------------------------------
diff --git a/src/tests/environment.cpp b/src/tests/environment.cpp
index 41b8a71..585d73e 100644
--- a/src/tests/environment.cpp
+++ b/src/tests/environment.cpp
@@ -108,21 +108,10 @@ static bool enable(const ::testing::TestInfo& test)
   // Now check the type parameter.
   if (test.type_param() != NULL) {
     const string& type = test.type_param();
-    if (strings::contains(type, "CgroupsIsolator") &&
+    if (strings::contains(type, "Cgroups") &&
         (os::user() != "root" || !os::exists("/proc/cgroups"))) {
       return false;
     }
-#ifdef __APPLE__
-    if (strings::contains(test.test_case_name(), "IsolatorTest") &&
-        strings::contains(test.name(), "Usage") &&
-        strings::contains(type, "ProcessIsolator") &&
-        os::user() != "root") {
-      // We can't run the Isolator resource usage test when we're not
-      // the root user on OSX because proc_pidinfo() only returns
-      // memory and CPU usage reliably when running as root.
-      return false;
-    }
-#endif
   }
 
   return true;

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/fault_tolerance_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fault_tolerance_tests.cpp b/src/tests/fault_tolerance_tests.cpp
index 130272c..b0711cf 100644
--- a/src/tests/fault_tolerance_tests.cpp
+++ b/src/tests/fault_tolerance_tests.cpp
@@ -41,10 +41,9 @@
 
 #include "master/master.hpp"
 
-#include "slave/isolator.hpp"
 #include "slave/slave.hpp"
 
-#include "tests/isolator.hpp"
+#include "tests/containerizer.hpp"
 #include "tests/mesos.hpp"
 
 using namespace mesos;
@@ -54,7 +53,6 @@ using namespace mesos::internal::tests;
 
 using mesos::internal::master::Master;
 
-using mesos::internal::slave::Isolator;
 using mesos::internal::slave::Slave;
 using mesos::internal::slave::STATUS_UPDATE_RETRY_INTERVAL_MIN;
 
@@ -483,9 +481,9 @@ TEST_F(FaultToleranceTest, PartitionedSlaveExitedExecutor)
   DROP_MESSAGES(Eq("PONG"), _, _);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestingIsolator isolator(&exec);
+  TestContainerizer containerizer(&exec);
 
-  Try<PID<Slave> > slave = StartSlave(&isolator);
+  Try<PID<Slave> > slave = StartSlave(&containerizer);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -582,10 +580,8 @@ TEST_F(FaultToleranceTest, PartitionedSlaveExitedExecutor)
   shutdownMessage = FUTURE_PROTOBUF(ShutdownMessage(), _, slave.get());
 
   // Induce an ExitedExecutorMessage from the slave.
-  dispatch(isolator,
-           &Isolator::killExecutor,
-           frameworkId.get(),
-           DEFAULT_EXECUTOR_INFO.executor_id());
+  containerizer.destroy(
+      frameworkId.get(), DEFAULT_EXECUTOR_INFO.executor_id());
 
   // Upon receiving the message, the master will shutdown the slave.
   AWAIT_READY(shutdownMessage);
@@ -1113,10 +1109,11 @@ TEST_F(FaultToleranceTest, ReregisterFrameworkExitedExecutor)
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestingIsolator isolator(&exec);
+  TestContainerizer containerizer(&exec);
+
   Owned<MasterDetector> slaveDetector(
       new StandaloneMasterDetector(master.get()));
-  Try<PID<Slave> > slave = StartSlave(&isolator, slaveDetector);
+  Try<PID<Slave> > slave = StartSlave(&containerizer, slaveDetector);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -1189,7 +1186,7 @@ TEST_F(FaultToleranceTest, ReregisterFrameworkExitedExecutor)
   DROP_PROTOBUFS(StatusUpdateMessage(), _, _);
 
   // Now kill the executor.
-  dispatch(isolator, &Isolator::killExecutor, frameworkId, DEFAULT_EXECUTOR_ID);
+  containerizer.destroy(frameworkId, DEFAULT_EXECUTOR_ID);
 
   AWAIT_READY(executorExitedMessage);
 
@@ -1683,12 +1680,12 @@ TEST_F(FaultToleranceTest, SlaveReregisterTerminatedExecutor)
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestingIsolator isolator(&exec);
+  TestContainerizer containerizer(&exec);
 
   StandaloneMasterDetector* detector =
     new StandaloneMasterDetector(master.get());
   Try<PID<Slave> > slave =
-    StartSlave(&isolator, Owned<MasterDetector>(detector));
+    StartSlave(&containerizer, Owned<MasterDetector>(detector));
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -1726,10 +1723,7 @@ TEST_F(FaultToleranceTest, SlaveReregisterTerminatedExecutor)
     FUTURE_PROTOBUF(ExitedExecutorMessage(), _, _);
 
   // Now kill the executor.
-  dispatch(isolator,
-           &Isolator::killExecutor,
-           frameworkId.get(),
-           DEFAULT_EXECUTOR_ID);
+  containerizer.destroy(frameworkId.get(), DEFAULT_EXECUTOR_ID);
 
   AWAIT_READY(executorExitedMessage);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/gc_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/gc_tests.cpp b/src/tests/gc_tests.cpp
index 6638a4a..ac547cf 100644
--- a/src/tests/gc_tests.cpp
+++ b/src/tests/gc_tests.cpp
@@ -49,11 +49,10 @@
 #include "slave/constants.hpp"
 #include "slave/flags.hpp"
 #include "slave/gc.hpp"
-#include "slave/isolator.hpp"
 #include "slave/paths.hpp"
 #include "slave/slave.hpp"
 
-#include "tests/isolator.hpp"
+#include "tests/containerizer.hpp"
 #include "tests/mesos.hpp"
 #include "tests/utils.hpp"
 
@@ -65,7 +64,6 @@ using mesos::internal::master::Master;
 
 using mesos::internal::slave::GarbageCollector;
 using mesos::internal::slave::GarbageCollectorProcess;
-using mesos::internal::slave::Isolator;
 using mesos::internal::slave::Slave;
 
 using process::Clock;
@@ -472,14 +470,14 @@ TEST_F(GarbageCollectorIntegrationTest, ExitedExecutor)
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
 
-  TestingIsolator isolator(&exec);
+  TestContainerizer containerizer(&exec);
 
   Future<SlaveRegisteredMessage> slaveRegisteredMessage =
     FUTURE_PROTOBUF(SlaveRegisteredMessage(), _, _);
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = StartSlave(&isolator, flags);
+  Try<PID<Slave> > slave = StartSlave(&containerizer, flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(slaveRegisteredMessage);
@@ -537,10 +535,7 @@ TEST_F(GarbageCollectorIntegrationTest, ExitedExecutor)
     .Times(AtMost(1)); // Ignore TASK_LOST from killed executor.
 
   // Kill the executor and inform the slave.
-  dispatch(isolator,
-           &Isolator::killExecutor,
-           frameworkId.get(),
-           DEFAULT_EXECUTOR_ID);
+  containerizer.destroy(frameworkId.get(), DEFAULT_EXECUTOR_ID);
 
   AWAIT_READY(schedule);
 
@@ -574,14 +569,14 @@ TEST_F(GarbageCollectorIntegrationTest, DiskUsage)
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
 
-  TestingIsolator isolator(&exec);
+  TestContainerizer containerizer(&exec);
 
   Future<SlaveRegisteredMessage> slaveRegisteredMessage =
     FUTURE_PROTOBUF(SlaveRegisteredMessage(), _, _);
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = StartSlave(&isolator, flags);
+  Try<PID<Slave> > slave = StartSlave(&containerizer, flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(slaveRegisteredMessage);
@@ -639,10 +634,7 @@ TEST_F(GarbageCollectorIntegrationTest, DiskUsage)
     .Times(AtMost(1)); // Ignore TASK_LOST from killed executor.
 
   // Kill the executor and inform the slave.
-  dispatch(isolator,
-           &Isolator::killExecutor,
-           frameworkId.get(),
-           DEFAULT_EXECUTOR_ID);
+  containerizer.destroy(frameworkId.get(), DEFAULT_EXECUTOR_ID);
 
   AWAIT_READY(schedule);
 
@@ -698,15 +690,15 @@ TEST_F(GarbageCollectorIntegrationTest, Unschedule)
   MockExecutor exec1(executor1.executor_id());
   MockExecutor exec2(executor2.executor_id());
 
-  map<ExecutorID, Executor*> execs;
+  hashmap<ExecutorID, Executor*> execs;
   execs[executor1.executor_id()] = &exec1;
   execs[executor2.executor_id()] = &exec2;
 
-  TestingIsolator isolator(execs);
+  TestContainerizer containerizer(execs);
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = StartSlave(&isolator, flags);
+  Try<PID<Slave> > slave = StartSlave(&containerizer, flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(slaveRegistered);
@@ -784,11 +776,7 @@ TEST_F(GarbageCollectorIntegrationTest, Unschedule)
   Clock::pause();
 
   // Kill the first executor.
-  process::dispatch(
-      isolator,
-      &Isolator::killExecutor,
-      frameworkId.get(),
-      exec1.id);
+  containerizer.destroy(frameworkId.get(), exec1.id);
 
   AWAIT_READY(scheduleExecutorRunWork);
   AWAIT_READY(scheduleExecutorWork);

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/isolator.hpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator.hpp b/src/tests/isolator.hpp
deleted file mode 100644
index 6431dd2..0000000
--- a/src/tests/isolator.hpp
+++ /dev/null
@@ -1,211 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#ifndef __TESTS_ISOLATOR_HPP__
-#define __TESTS_ISOLATOR_HPP__
-
-#include "unistd.h"
-
-#include <map>
-#include <string>
-
-#include <process/dispatch.hpp>
-#include <process/future.hpp>
-#include <process/pid.hpp>
-
-#include <stout/os.hpp>
-#include <stout/try.hpp>
-#include <stout/uuid.hpp>
-
-#include "mesos/executor.hpp"
-#include "mesos/mesos.hpp"
-
-#include "slave/isolator.hpp"
-
-#include "tests/mesos.hpp" // For MockExecutor.
-
-namespace mesos {
-namespace internal {
-namespace tests {
-
-class TestingIsolator : public slave::Isolator
-{
-public:
-  TestingIsolator()
-  {
-    setup();
-  }
-
-  TestingIsolator(const std::map<ExecutorID, Executor*>& _executors)
-    : executors(_executors)
-  {
-    setup();
-  }
-
-  TestingIsolator(const ExecutorID& executorId, Executor* executor)
-  {
-    executors[executorId] = executor;
-    setup();
-  }
-
-  TestingIsolator(MockExecutor* executor)
-  {
-    executors[executor->id] = executor;
-    setup();
-  }
-
-  virtual ~TestingIsolator()
-  {
-    foreachvalue (MesosExecutorDriver* driver, drivers) {
-      driver->stop();
-      driver->join();
-      delete driver;
-    }
-    drivers.clear();
-  }
-
-  virtual void initialize(
-      const slave::Flags& flags,
-      const Resources& resources,
-      bool local,
-      const process::PID<slave::Slave>& _slave)
-  {
-    slave = _slave;
-  }
-
-  virtual void launchExecutor(
-      const SlaveID& slaveId,
-      const FrameworkID& frameworkId,
-      const FrameworkInfo& frameworkInfo,
-      const ExecutorInfo& executorInfo,
-      const UUID& uuid,
-      const std::string& directory,
-      const Resources& resources)
-  {
-    // TODO(vinod): Currently TestingIsolator doesn't support 2
-    // different frameworks launching an executor with the same
-    // executorID! This is tricky to support because most of the
-    // tests do not known the framework id when they setup the
-    // TestingIsolator.
-    if (drivers.count(executorInfo.executor_id()) > 0) {
-      FAIL() << "Failed to launch executor " << executorInfo.executor_id()
-             << " of framework " << frameworkId
-             << " because it is already launched";
-    }
-
-    if (executors.count(executorInfo.executor_id()) == 0) {
-      FAIL() << "Failed to launch executor " << executorInfo.executor_id()
-             << " of framework " << frameworkId
-             << " because it is unknown to the isolator";
-    }
-
-    Executor* executor = executors[executorInfo.executor_id()];
-    MesosExecutorDriver* driver = new MesosExecutorDriver(executor);
-    drivers[executorInfo.executor_id()] = driver;
-
-    os::setenv("MESOS_LOCAL", "1");
-    os::setenv("MESOS_DIRECTORY", directory);
-    os::setenv("MESOS_SLAVE_PID", slave);
-    os::setenv("MESOS_SLAVE_ID", slaveId.value());
-    os::setenv("MESOS_FRAMEWORK_ID", frameworkId.value());
-    os::setenv("MESOS_EXECUTOR_ID", executorInfo.executor_id().value());
-    os::setenv("MESOS_CHECKPOINT", frameworkInfo.checkpoint() ? "1" : "0");
-
-    driver->start();
-
-    os::unsetenv("MESOS_LOCAL");
-    os::unsetenv("MESOS_DIRECTORY");
-    os::unsetenv("MESOS_SLAVE_PID");
-    os::unsetenv("MESOS_SLAVE_ID");
-    os::unsetenv("MESOS_FRAMEWORK_ID");
-    os::unsetenv("MESOS_EXECUTOR_ID");
-    os::unsetenv("MESOS_CHECKPOINT");
-
-    process::dispatch(
-        slave,
-        &slave::Slave::executorStarted,
-        frameworkId,
-        executorInfo.executor_id(),
-        getpid());
-  }
-
-  virtual void killExecutor(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId)
-  {
-    if (drivers.count(executorId) > 0) {
-      MesosExecutorDriver* driver = drivers[executorId];
-      driver->stop();
-      driver->join();
-      delete driver;
-      drivers.erase(executorId);
-
-      process::dispatch(
-          slave,
-          &slave::Slave::executorTerminated,
-          frameworkId,
-          executorId,
-          0,
-          false,
-          "Killed executor");
-    } else {
-      FAIL() << "Failed to kill executor " << executorId
-             << " of framework " << frameworkId
-             << " because it is not launched";
-    }
-  }
-
-  // Mocked so tests can check that the resources reflect all started tasks.
-  MOCK_METHOD3(resourcesChanged, void(const FrameworkID&,
-                                      const ExecutorID&,
-                                      const Resources&));
-
-  MOCK_METHOD2(
-      usage,
-      process::Future<ResourceStatistics>(
-          const FrameworkID&,
-          const ExecutorID&));
-
-  MOCK_METHOD1(
-      recover,
-      process::Future<Nothing>(const Option<slave::state::SlaveState>&));
-
-private:
-  // Helper to setup default expectations.
-  void setup()
-  {
-    EXPECT_CALL(*this, resourcesChanged(testing::_, testing::_, testing::_))
-      .Times(testing::AnyNumber());
-
-    EXPECT_CALL(*this, usage(testing::_, testing::_))
-      .WillRepeatedly(testing::Return(ResourceStatistics()));
-
-    EXPECT_CALL(*this, recover(testing::_))
-      .WillRepeatedly(testing::Return(Nothing()));
-  }
-
-  std::map<ExecutorID, Executor*> executors;
-  std::map<ExecutorID, MesosExecutorDriver*> drivers;
-  process::PID<slave::Slave> slave;
-};
-
-} // namespace tests {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __TESTS_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator_tests.cpp b/src/tests/isolator_tests.cpp
index 45a41ca..5c36659 100644
--- a/src/tests/isolator_tests.cpp
+++ b/src/tests/isolator_tests.cpp
@@ -26,6 +26,8 @@
 #include <mesos/resources.hpp>
 
 #include <process/future.hpp>
+#include <process/owned.hpp>
+#include <process/reap.hpp>
 
 #include <stout/os.hpp>
 #include <stout/path.hpp>
@@ -34,13 +36,19 @@
 #include "master/detector.hpp"
 
 #include "slave/flags.hpp"
-#ifdef __linux__
-#include "slave/cgroups_isolator.hpp"
-#endif
-#include "slave/process_isolator.hpp"
 #include "slave/slave.hpp"
 
+#include "slave/containerizer/isolator.hpp"
+#include "slave/containerizer/launcher.hpp"
+
+#include "slave/containerizer/isolators/posix.hpp"
+#ifdef __linux__
+#include "slave/containerizer/isolators/cgroups/cpushare.hpp"
+#include "slave/containerizer/isolators/cgroups/mem.hpp"
+#endif // __linux__
+
 #include "tests/mesos.hpp"
+#include "tests/utils.hpp"
 
 using namespace mesos;
 using namespace mesos::internal;
@@ -49,13 +57,17 @@ using namespace mesos::internal::tests;
 using namespace process;
 
 using mesos::internal::master::Master;
-
 #ifdef __linux__
-using mesos::internal::slave::CgroupsIsolator;
-#endif
+using mesos::internal::slave::CgroupsCpushareIsolatorProcess;
+using mesos::internal::slave::CgroupsMemIsolatorProcess;
+#endif // __linux__
 using mesos::internal::slave::Isolator;
-using mesos::internal::slave::ProcessIsolator;
-using mesos::internal::slave::Slave;
+using mesos::internal::slave::IsolatorProcess;
+using mesos::internal::slave::Launcher;
+using mesos::internal::slave::PosixLauncher;
+using mesos::internal::slave::PosixCpuIsolatorProcess;
+using mesos::internal::slave::PosixMemIsolatorProcess;
+using mesos::internal::slave::Flags;
 
 using std::string;
 using std::vector;
@@ -66,139 +78,348 @@ using testing::Return;
 using testing::SaveArg;
 
 
+int execute(const std::string& command, int pipes[2])
+{
+  // In child process
+  ::close(pipes[1]);
+
+  // Wait until the parent signals us to continue.
+  int buf;
+  ::read(pipes[0], &buf, sizeof(buf));
+  ::close(pipes[0]);
+
+  execl("/bin/sh", "sh", "-c", command.c_str(), (char*) NULL);
+
+  std::cerr << "Should not reach here!" << std::endl;
+  abort();
+}
+
+
+template <typename T>
+class CpuIsolatorTest : public MesosTest {};
+
 #ifdef __linux__
-typedef ::testing::Types<ProcessIsolator, CgroupsIsolator> IsolatorTypes;
+typedef ::testing::Types<PosixCpuIsolatorProcess,
+                         CgroupsCpushareIsolatorProcess> CpuIsolatorTypes;
 #else
-typedef ::testing::Types<ProcessIsolator> IsolatorTypes;
-#endif
+typedef ::testing::Types<PosixCpuIsolatorProcess> CpuIsolatorTypes;
+#endif // __linux__
 
-TYPED_TEST_CASE(IsolatorTest, IsolatorTypes);
+TYPED_TEST_CASE(CpuIsolatorTest, CpuIsolatorTypes);
 
-TYPED_TEST(IsolatorTest, Usage)
+TYPED_TEST(CpuIsolatorTest, UserCpuUsage)
 {
-  Try<PID<Master> > master = this->StartMaster();
-  ASSERT_SOME(master);
+  Flags flags;
 
-  TypeParam isolator;
+  Try<Isolator*> isolator = TypeParam::create(flags);
+  CHECK_SOME(isolator);
 
-  slave::Flags flags = this->CreateSlaveFlags();
+  // A PosixLauncher is sufficient even when testing a cgroups isolator.
+  Try<Launcher*> launcher = PosixLauncher::create(flags);
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator, flags);
-  ASSERT_SOME(slave);
+  ExecutorInfo executorInfo;
+  executorInfo.mutable_resources()->CopyFrom(
+      Resources::parse("cpus:1.0").get());
 
-  MockScheduler sched;
-  MesosSchedulerDriver driver(
-      &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
+  ContainerID containerId;
+  containerId.set_value("user_cpu_usage");
 
-  Future<FrameworkID> frameworkId;
-  EXPECT_CALL(sched, registered(&driver, _, _))
-    .WillOnce(FutureArg<1>(&frameworkId));
+  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo));
 
-  Future<vector<Offer> > offers;
-  EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(FutureArg<1>(&offers))
-    .WillRepeatedly(Return()); // Ignore subsequent offers.
+  Try<string> dir = os::mkdtemp();
+  ASSERT_SOME(dir);
+  const string& file = path::join(dir.get(), "mesos_isolator_test_ready");
 
-  driver.start();
+  // Max out a single core in userspace. This will run for at most one second.
+  string command = "while true ; do true ; done &"
+    "touch " + file + "; " // Signals the command is running.
+    "sleep 60";
 
-  AWAIT_READY(frameworkId);
-  AWAIT_READY(offers);
+  int pipes[2];
+  ASSERT_NE(-1, ::pipe(pipes));
 
-  EXPECT_NE(0u, offers.get().size());
+  lambda::function<int()> inChild = lambda::bind(&execute, command, pipes);
 
-  TaskInfo task;
-  task.set_name("isolator_test");
-  task.mutable_task_id()->set_value("1");
-  task.mutable_slave_id()->MergeFrom(offers.get()[0].slave_id());
-  task.mutable_resources()->MergeFrom(offers.get()[0].resources());
+  Try<pid_t> pid = launcher.get()->fork(containerId, inChild);
+  ASSERT_SOME(pid);
 
-  Resources resources(offers.get()[0].resources());
-  Option<Bytes> mem = resources.mem();
-  ASSERT_SOME(mem);
-  Option<double> cpus = resources.cpus();
-  ASSERT_SOME(cpus);
+  // Reap the forked child.
+  Future<Option<int> > status = process::reap(pid.get());
 
-  const std::string& file = path::join(flags.work_dir, "ready");
+  // Continue in the parent.
+  ::close(pipes[0]);
 
-  // This task induces user/system load in a child process by
-  // running top in a child process for ten seconds.
-  task.mutable_command()->set_value(
-#ifdef __APPLE__
-      // Use logging mode with 30,000 samples with no interval.
-      "top -l 30000 -s 0 2>&1 > /dev/null & "
-#else
-      // Batch mode, with 30,000 samples with no interval.
-      "top -b -d 0 -n 30000 2>&1 > /dev/null & "
-#endif
-      "touch " + file +  "; " // Signals that the top command is running.
-      "sleep 60");
+  // Isolate the forked child.
+  AWAIT_READY(isolator.get()->isolate(containerId, pid.get()));
+
+  // Now signal the child to continue.
+  int buf;
+  ASSERT_LT(0, ::write(pipes[1],  &buf, sizeof(buf)));
+  ::close(pipes[1]);
+
+  // Wait for the command to start.
+  while (!os::exists(file));
+
+  // Wait up to 1 second for the child process to induce 1/8 of a second of
+  // user cpu time.
+  ResourceStatistics statistics;
+  Duration waited = Duration::zero();
+  do {
+    Future<ResourceStatistics> usage = isolator.get()->usage(containerId);
+    AWAIT_READY(usage);
+
+    statistics = usage.get();
+
+    // If we meet our usage expectations, we're done!
+    if (statistics.cpus_user_time_secs() >= 0.125) {
+      break;
+    }
 
-  vector<TaskInfo> tasks;
-  tasks.push_back(task);
+    os::sleep(Milliseconds(200));
+    waited += Milliseconds(200);
+  } while (waited < Seconds(1));
 
-  Future<TaskStatus> status;
-  EXPECT_CALL(sched, statusUpdate(&driver, _))
-    .WillOnce(FutureArg<1>(&status));
+  EXPECT_LE(0.125, statistics.cpus_user_time_secs());
 
-  driver.launchTasks(offers.get()[0].id(), tasks);
+  // Shouldn't be any appreciable system time.
+  EXPECT_GT(0.025, statistics.cpus_system_time_secs());
 
+  // Ensure all processes are killed.
+  AWAIT_READY(launcher.get()->destroy(containerId));
+
+  // Make sure the child was reaped.
   AWAIT_READY(status);
 
-  EXPECT_EQ(TASK_RUNNING, status.get().state());
+  // Let the isolator clean up.
+  AWAIT_READY(isolator.get()->cleanup(containerId));
 
-  // Wait for the task to begin inducing cpu time.
-  while (!os::exists(file));
+  delete isolator.get();
+  delete launcher.get();
+
+  CHECK_SOME(os::rmdir(dir.get()));
+}
+
+
+TYPED_TEST(CpuIsolatorTest, SystemCpuUsage)
+{
+  Flags flags;
+
+  Try<Isolator*> isolator = TypeParam::create(flags);
+  CHECK_SOME(isolator);
+
+  // A PosixLauncher is sufficient even when testing a cgroups isolator.
+  Try<Launcher*> launcher = PosixLauncher::create(flags);
+
+  ExecutorInfo executorInfo;
+  executorInfo.mutable_resources()->CopyFrom(
+      Resources::parse("cpus:1.0").get());
+
+  ContainerID containerId;
+  containerId.set_value("system_cpu_usage");
 
-  ExecutorID executorId;
-  executorId.set_value(task.task_id().value());
+  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo));
 
-  // We'll wait up to 10 seconds for the child process to induce
-  // 1/8 of a second of user and system cpu time in total.
-  // TODO(bmahler): Also induce rss memory consumption, by re-using
-  // the balloon framework.
+  Try<string> dir = os::mkdtemp();
+  ASSERT_SOME(dir);
+  const string& file = path::join(dir.get(), "mesos_isolator_test_ready");
+
+  // Generating random numbers is done by the kernel and will max out a single
+  // core and run almost exclusively in the kernel, i.e., system time.
+  string command = "cat /dev/urandom > /dev/null & "
+    "touch " + file + "; " // Signals the command is running.
+    "sleep 60";
+
+  int pipes[2];
+  ASSERT_NE(-1, ::pipe(pipes));
+
+  lambda::function<int()> inChild = lambda::bind(&execute, command, pipes);
+
+  Try<pid_t> pid = launcher.get()->fork(containerId, inChild);
+  ASSERT_SOME(pid);
+
+  // Reap the forked child.
+  Future<Option<int> > status = process::reap(pid.get());
+
+  // Continue in the parent.
+  ::close(pipes[0]);
+
+  // Isolate the forked child.
+  AWAIT_READY(isolator.get()->isolate(containerId, pid.get()));
+
+  // Now signal the child to continue.
+  int buf;
+  ASSERT_LT(0, ::write(pipes[1],  &buf, sizeof(buf)));
+  ::close(pipes[1]);
+
+  // Wait for the command to start.
+  while (!os::exists(file));
+
+  // Wait up to 1 second for the child process to induce 1/8 of a second of
+  // system cpu time.
   ResourceStatistics statistics;
   Duration waited = Duration::zero();
   do {
-    Future<ResourceStatistics> usage =
-      process::dispatch(
-          (Isolator*) &isolator, // TODO(benh): Fix after reaper changes.
-          &Isolator::usage,
-          frameworkId.get(),
-          executorId);
-
+    Future<ResourceStatistics> usage = isolator.get()->usage(containerId);
     AWAIT_READY(usage);
 
     statistics = usage.get();
 
     // If we meet our usage expectations, we're done!
-    if (statistics.cpus_user_time_secs() >= 0.125 &&
-        statistics.cpus_system_time_secs() >= 0.125 &&
-        statistics.mem_rss_bytes() >= 1024u) {
+    if (statistics.cpus_system_time_secs() >= 0.125) {
       break;
     }
 
-    os::sleep(Milliseconds(100));
-    waited += Milliseconds(100);
-  } while (waited < Seconds(10));
-
+    os::sleep(Milliseconds(200));
+    waited += Milliseconds(200);
+  } while (waited < Seconds(1));
 
-  EXPECT_GE(statistics.cpus_user_time_secs(), 0.125);
-  EXPECT_GE(statistics.cpus_system_time_secs(), 0.125);
-  EXPECT_EQ(statistics.cpus_limit(), cpus.get());
-  EXPECT_GE(statistics.mem_rss_bytes(), 1024u);
-  EXPECT_EQ(statistics.mem_limit_bytes(), mem.get().bytes());
+  EXPECT_LE(0.125, statistics.cpus_system_time_secs());
 
-  EXPECT_CALL(sched, statusUpdate(&driver, _))
-    .WillOnce(FutureArg<1>(&status));
+  // Shouldn't be any appreciable user time.
+  EXPECT_GT(0.025, statistics.cpus_user_time_secs());
 
-  driver.killTask(task.task_id());
+  // Ensure all processes are killed.
+  AWAIT_READY(launcher.get()->destroy(containerId));
 
+  // Make sure the child was reaped.
   AWAIT_READY(status);
 
-  EXPECT_EQ(TASK_KILLED, status.get().state());
+  // Let the isolator clean up.
+  AWAIT_READY(isolator.get()->cleanup(containerId));
+
+  delete isolator.get();
+  delete launcher.get();
+
+  CHECK_SOME(os::rmdir(dir.get()));
+}
+
+
+template <typename T>
+class MemIsolatorTest : public MesosTest {};
+
+#ifdef __linux__
+typedef ::testing::Types<PosixMemIsolatorProcess,
+                         CgroupsMemIsolatorProcess> MemIsolatorTypes;
+#else
+typedef ::testing::Types<PosixMemIsolatorProcess> MemIsolatorTypes;
+#endif // __linux__
+
+TYPED_TEST_CASE(MemIsolatorTest, MemIsolatorTypes);
+
+
+// This function should be async-signal-safe but it isn't: at least
+// posix_memalign, mlock, memset and perror are not safe.
+int consumeMemory(const Bytes& _size, const Duration& duration, int pipes[2])
+{
+  // In child process
+  ::close(pipes[1]);
+
+  int buf;
+  // Wait until the parent signals us to continue.
+  ::read(pipes[0], &buf, sizeof(buf));
+  ::close(pipes[0]);
+
+  size_t size = static_cast<size_t>(_size.bytes());
+  void* buffer = NULL;
+
+  if (posix_memalign(&buffer, getpagesize(), size) != 0) {
+    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, size) != 0) {
+    perror("Failed to lock memory, mlock");
+    abort();
+  }
+
+  if (memset(buffer, 1, size) != buffer) {
+    perror("Failed to fill memory, memset");
+    abort();
+  }
+
+  os::sleep(duration);
+
+  return 0;
+}
+
+
+TYPED_TEST(MemIsolatorTest, MemUsage)
+{
+  Flags flags;
+
+  Try<Isolator*> isolator = TypeParam::create(flags);
+  CHECK_SOME(isolator);
+
+  // A PosixLauncher is sufficient even when testing a cgroups isolator.
+  Try<Launcher*> launcher = PosixLauncher::create(flags);
+
+  ExecutorInfo executorInfo;
+  executorInfo.mutable_resources()->CopyFrom(
+      Resources::parse("mem:1024").get());
+
+  ContainerID containerId;
+  containerId.set_value("memory_usage");
+
+  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo));
+
+  int pipes[2];
+  ASSERT_NE(-1, ::pipe(pipes));
+
+  lambda::function<int()> inChild = lambda::bind(
+      &consumeMemory,
+      Megabytes(256),
+      Seconds(10),
+      pipes);
+
+  Try<pid_t> pid = launcher.get()->fork(containerId, inChild);
+  ASSERT_SOME(pid);
+
+  // Set up the reaper to wait on the forked child.
+  Future<Option<int> > status = process::reap(pid.get());
+
+  // Continue in the parent.
+  ::close(pipes[0]);
+
+  // Isolate the forked child.
+  AWAIT_READY(isolator.get()->isolate(containerId, pid.get()));
+
+  // Now signal the child to continue.
+  int buf;
+  ASSERT_LT(0, ::write(pipes[1], &buf, sizeof(buf)));
+  ::close(pipes[1]);
+
+  // Wait up to 5 seconds for the child process to consume 256 MB of memory;
+  ResourceStatistics statistics;
+  Bytes threshold = Megabytes(256);
+  Duration waited = Duration::zero();
+  do {
+    Future<ResourceStatistics> usage = isolator.get()->usage(containerId);
+    AWAIT_READY(usage);
+
+    statistics = usage.get();
+
+    // If we meet our usage expectations, we're done!
+    if (statistics.mem_rss_bytes() >= threshold.bytes()) {
+      break;
+    }
+
+    os::sleep(Seconds(1));
+    waited += Seconds(1);
+  } while (waited < Seconds(5));
+
+  EXPECT_LE(threshold.bytes(), statistics.mem_rss_bytes());
+
+  // Ensure all processes are killed.
+  AWAIT_READY(launcher.get()->destroy(containerId));
+
+  // Make sure the child was reaped.
+  AWAIT_READY(status);
 
-  driver.stop();
-  driver.join();
+  // Let the isolator clean up.
+  AWAIT_READY(isolator.get()->cleanup(containerId));
 
-  this->Shutdown(); // Must shutdown before 'isolator' gets deallocated.
+  delete isolator.get();
+  delete launcher.get();
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/master_contender_detector_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_contender_detector_tests.cpp b/src/tests/master_contender_detector_tests.cpp
index b42574b..8da7420 100644
--- a/src/tests/master_contender_detector_tests.cpp
+++ b/src/tests/master_contender_detector_tests.cpp
@@ -51,7 +51,6 @@
 
 #include "slave/slave.hpp"
 
-#include "tests/isolator.hpp"
 #include "tests/mesos.hpp"
 #ifdef MESOS_HAS_JAVA
 #include "tests/zookeeper.hpp"

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index 9c14114..42c5a77 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -41,12 +41,12 @@
 #include "master/master.hpp"
 
 #include "slave/constants.hpp"
+#include "slave/containerizer/mesos_containerizer.hpp"
 #include "slave/gc.hpp"
 #include "slave/flags.hpp"
-#include "slave/process_isolator.hpp"
 #include "slave/slave.hpp"
 
-#include "tests/isolator.hpp"
+#include "tests/containerizer.hpp"
 #include "tests/mesos.hpp"
 
 #ifdef MESOS_HAS_JAVA
@@ -60,9 +60,9 @@ using namespace mesos::internal::tests;
 using mesos::internal::master::Master;
 
 using mesos::internal::slave::GarbageCollectorProcess;
-using mesos::internal::slave::Isolator;
-using mesos::internal::slave::ProcessIsolator;
 using mesos::internal::slave::Slave;
+using mesos::internal::slave::Containerizer;
+using mesos::internal::slave::MesosContainerizerProcess;
 
 using process::Clock;
 using process::Future;
@@ -89,9 +89,10 @@ TEST_F(MasterTest, TaskRunning)
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestingIsolator isolator(&exec);
 
-  Try<PID<Slave> > slave = StartSlave(&isolator);
+  TestContainerizer containerizer(&exec);
+
+  Try<PID<Slave> > slave = StartSlave(&containerizer);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -127,10 +128,12 @@ TEST_F(MasterTest, TaskRunning)
   EXPECT_CALL(exec, launchTask(_, _))
     .WillOnce(SendStatusUpdateFromTask(TASK_RUNNING));
 
-  Future<Nothing> resourcesChanged;
-  EXPECT_CALL(isolator,
-              resourcesChanged(_, _, Resources(offers.get()[0].resources())))
-    .WillOnce(FutureSatisfy(&resourcesChanged));
+  Future<Nothing> resourcesUpdated;
+  Future<Nothing> update;
+  EXPECT_CALL(containerizer,
+              update(_, Resources(offers.get()[0].resources())))
+    .WillOnce(DoAll(FutureSatisfy(&resourcesUpdated),
+                    Return(update)));
 
   Future<TaskStatus> status;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
@@ -141,7 +144,7 @@ TEST_F(MasterTest, TaskRunning)
   AWAIT_READY(status);
   EXPECT_EQ(TASK_RUNNING, status.get().state());
 
-  AWAIT_READY(resourcesChanged);
+  AWAIT_READY(resourcesUpdated);
 
   EXPECT_CALL(exec, shutdown(_))
     .Times(AtMost(1));
@@ -149,7 +152,7 @@ TEST_F(MasterTest, TaskRunning)
   driver.stop();
   driver.join();
 
-  Shutdown(); // Must shutdown before 'isolator' gets deallocated.
+  Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }
 
 
@@ -159,12 +162,13 @@ TEST_F(MasterTest, ShutdownFrameworkWhileTaskRunning)
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestingIsolator isolator(&exec);
+
+  TestContainerizer containerizer(&exec);
 
   slave::Flags flags = CreateSlaveFlags();
   flags.executor_shutdown_grace_period = Seconds(0);
 
-  Try<PID<Slave> > slave = StartSlave(&isolator, flags);
+  Try<PID<Slave> > slave = StartSlave(&containerizer, flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -200,10 +204,12 @@ TEST_F(MasterTest, ShutdownFrameworkWhileTaskRunning)
   EXPECT_CALL(exec, launchTask(_, _))
     .WillOnce(SendStatusUpdateFromTask(TASK_RUNNING));
 
-  Future<Nothing> resourcesChanged;
-  EXPECT_CALL(isolator,
-              resourcesChanged(_, _, Resources(offers.get()[0].resources())))
-    .WillOnce(FutureSatisfy(&resourcesChanged));
+  Future<Nothing> resourcesUpdated;
+  Future<Nothing> update;
+  EXPECT_CALL(containerizer,
+              update(_, Resources(offers.get()[0].resources())))
+    .WillOnce(DoAll(FutureSatisfy(&resourcesUpdated),
+                    Return(update)));
 
   Future<TaskStatus> status;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
@@ -214,7 +220,7 @@ TEST_F(MasterTest, ShutdownFrameworkWhileTaskRunning)
   AWAIT_READY(status);
   EXPECT_EQ(TASK_RUNNING, status.get().state());
 
-  AWAIT_READY(resourcesChanged);
+  AWAIT_READY(resourcesUpdated);
 
   EXPECT_CALL(exec, shutdown(_))
     .Times(AtMost(1));
@@ -222,7 +228,7 @@ TEST_F(MasterTest, ShutdownFrameworkWhileTaskRunning)
   driver.stop();
   driver.join();
 
-  Shutdown(); // Must shutdown before 'isolator' gets deallocated.
+  Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }
 
 
@@ -376,13 +382,14 @@ TEST_F(MasterTest, RecoverResources)
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestingIsolator isolator(&exec);
+
+  TestContainerizer containerizer(&exec);
 
   slave::Flags flags = CreateSlaveFlags();
   flags.resources = Option<string>(
       "cpus:2;mem:1024;disk:1024;ports:[1-10, 20-30]");
 
-  Try<PID<Slave> > slave = StartSlave(&isolator, flags);
+  Try<PID<Slave> > slave = StartSlave(&containerizer, flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -470,10 +477,7 @@ TEST_F(MasterTest, RecoverResources)
     .Times(AtMost(1));
 
   // Now kill the executor, scheduler should get an offer it's resources.
-  dispatch(isolator,
-           &Isolator::killExecutor,
-           offer.framework_id(),
-           executorInfo.executor_id());
+  containerizer.destroy(offer.framework_id(), executorInfo.executor_id());
 
   // TODO(benh): We can't do driver.reviveOffers() because we need to
   // wait for the killed executors resources to get aggregated! We
@@ -488,7 +492,7 @@ TEST_F(MasterTest, RecoverResources)
   driver.stop();
   driver.join();
 
-  Shutdown(); // Must shutdown before 'isolator' gets deallocated.
+  Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }
 
 
@@ -588,13 +592,13 @@ TEST_F(MasterTest, MultipleExecutors)
   MockExecutor exec1(executor1.executor_id());
   MockExecutor exec2(executor2.executor_id());
 
-  map<ExecutorID, Executor*> execs;
+  hashmap<ExecutorID, Executor*> execs;
   execs[executor1.executor_id()] = &exec1;
   execs[executor2.executor_id()] = &exec2;
 
-  TestingIsolator isolator(execs);
+  TestContainerizer containerizer(execs);
 
-  Try<PID<Slave> > slave = StartSlave(&isolator);
+  Try<PID<Slave> > slave = StartSlave(&containerizer);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -676,7 +680,7 @@ TEST_F(MasterTest, MultipleExecutors)
   driver.stop();
   driver.join();
 
-  Shutdown(); // Must shutdown before 'isolator' gets deallocated.
+  Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }
 
 
@@ -685,12 +689,15 @@ TEST_F(MasterTest, ShutdownUnregisteredExecutor)
   Try<PID<Master> > master = StartMaster();
   ASSERT_SOME(master);
 
-  ProcessIsolator isolator;
-
   // Need flags for 'executor_registration_timeout'.
   slave::Flags flags = CreateSlaveFlags();
+  // Set the isolation flag so we know a MesoContainerizer will be created.
+  flags.isolation = "posix/cpu,posix/mem";
 
-  Try<PID<Slave> > slave = StartSlave(&isolator);
+  Try<Containerizer*> containerizer = Containerizer::create(flags, false);
+  CHECK_SOME(containerizer);
+
+  Try<PID<Slave> > slave = StartSlave(containerizer.get());
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -740,14 +747,14 @@ TEST_F(MasterTest, ShutdownUnregisteredExecutor)
     .WillOnce(FutureArg<1>(&status));
 
   // Ensure that the slave times out and kills the executor.
-  Future<Nothing> killExecutor =
-    FUTURE_DISPATCH(_, &Isolator::killExecutor);
+  Future<Nothing> destroyExecutor =
+    FUTURE_DISPATCH(_, &MesosContainerizerProcess::destroy);
 
   Clock::advance(flags.executor_registration_timeout);
 
-  AWAIT_READY(killExecutor);
+  AWAIT_READY(destroyExecutor);
 
-  Clock::settle(); // Wait for ProcessIsolator::killExecutor to complete.
+  Clock::settle(); // Wait for Containerizer::destroy to complete.
 
   // Now advance time until the reaper reaps the executor.
   while (status.isPending()) {
@@ -763,7 +770,7 @@ TEST_F(MasterTest, ShutdownUnregisteredExecutor)
   driver.stop();
   driver.join();
 
-  Shutdown(); // Must shutdown before 'isolator' gets deallocated.
+  Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }
 
 
@@ -775,9 +782,10 @@ TEST_F(MasterTest, RemoveUnregisteredTerminatedExecutor)
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestingIsolator isolator(&exec);
 
-  Try<PID<Slave> > slave = StartSlave(&isolator);
+  TestContainerizer containerizer(&exec);
+
+  Try<PID<Slave> > slave = StartSlave(&containerizer);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -823,10 +831,7 @@ TEST_F(MasterTest, RemoveUnregisteredTerminatedExecutor)
     FUTURE_DISPATCH(_, &GarbageCollectorProcess::schedule);
 
   // Now kill the executor.
-  dispatch(isolator,
-           &Isolator::killExecutor,
-           offers.get()[0].framework_id(),
-           DEFAULT_EXECUTOR_ID);
+  containerizer.destroy(offers.get()[0].framework_id(), DEFAULT_EXECUTOR_ID);
 
   AWAIT_READY(status);
   EXPECT_EQ(TASK_LOST, status.get().state());
@@ -841,7 +846,7 @@ TEST_F(MasterTest, RemoveUnregisteredTerminatedExecutor)
   driver.stop();
   driver.join();
 
-  Shutdown(); // Must shutdown before 'isolator' gets deallocated.
+  Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }
 
 
@@ -1043,9 +1048,10 @@ TEST_F(MasterTest, ReconcileTaskTest)
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestingIsolator isolator(&exec);
 
-  Try<PID<Slave> > slave = StartSlave(&isolator);
+  TestContainerizer containerizer(&exec);
+
+  Try<PID<Slave> > slave = StartSlave(&containerizer);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -1104,7 +1110,7 @@ TEST_F(MasterTest, ReconcileTaskTest)
   driver.stop();
   driver.join();
 
-  Shutdown(); // Must shutdown before 'isolator' gets deallocated.
+  Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }
 
 
@@ -1120,7 +1126,7 @@ TEST_F(MasterTest, LaunchCombinedOfferTest)
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestingIsolator isolator(&exec);
+  TestContainerizer containerizer(&exec);
 
   // The CPU granularity is 1.0 which means that we need slaves with at least
   // 2 cpus for a combined offer.
@@ -1130,7 +1136,7 @@ TEST_F(MasterTest, LaunchCombinedOfferTest)
   slave::Flags flags = CreateSlaveFlags();
   flags.resources = Option<string>(stringify(fullSlave));
 
-  Try<PID<Slave> > slave = StartSlave(&isolator, flags);
+  Try<PID<Slave> > slave = StartSlave(&containerizer, flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -1248,7 +1254,7 @@ TEST_F(MasterTest, LaunchCombinedOfferTest)
   driver.stop();
   driver.join();
 
-  Shutdown(); // Must shutdown before 'isolator' gets deallocated.
+  Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }
 
 
@@ -1259,7 +1265,7 @@ TEST_F(MasterTest, LaunchAcrossSlavesTest)
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestingIsolator isolator(&exec);
+  TestContainerizer containerizer(&exec);
 
   // See LaunchCombinedOfferTest() for resource size motivation.
   Resources fullSlave = Resources::parse("cpus:2;mem:1024").get();
@@ -1268,7 +1274,7 @@ TEST_F(MasterTest, LaunchAcrossSlavesTest)
   slave::Flags flags = CreateSlaveFlags();
   flags.resources = Option<string>(stringify(fullSlave));
 
-  Try<PID<Slave> > slave1 = StartSlave(&isolator, flags);
+  Try<PID<Slave> > slave1 = StartSlave(&containerizer, flags);
   ASSERT_SOME(slave1);
 
   MockScheduler sched;
@@ -1294,7 +1300,7 @@ TEST_F(MasterTest, LaunchAcrossSlavesTest)
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers2));
 
-  Try<PID<Slave> > slave2 = StartSlave(&isolator, flags);
+  Try<PID<Slave> > slave2 = StartSlave(&containerizer, flags);
   ASSERT_SOME(slave2);
 
   AWAIT_READY(offers2);
@@ -1331,7 +1337,7 @@ TEST_F(MasterTest, LaunchAcrossSlavesTest)
   driver.stop();
   driver.join();
 
-  Shutdown(); // Must shutdown before 'isolator' gets deallocated.
+  Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }
 
 
@@ -1343,7 +1349,7 @@ TEST_F(MasterTest, LaunchDuplicateOfferTest)
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestingIsolator isolator(&exec);
+  TestContainerizer containerizer(&exec);
 
   // See LaunchCombinedOfferTest() for resource size motivation.
   Resources fullSlave = Resources::parse("cpus:2;mem:1024").get();
@@ -1351,7 +1357,7 @@ TEST_F(MasterTest, LaunchDuplicateOfferTest)
   slave::Flags flags = CreateSlaveFlags();
   flags.resources = Option<string>(stringify(fullSlave));
 
-  Try<PID<Slave> > slave = StartSlave(&isolator, flags);
+  Try<PID<Slave> > slave = StartSlave(&containerizer, flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -1403,7 +1409,7 @@ TEST_F(MasterTest, LaunchDuplicateOfferTest)
   driver.stop();
   driver.join();
 
-  Shutdown(); // Must shutdown before 'isolator' gets deallocated.
+  Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }