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:07 UTC

[01/11] git commit: Revise os::killtree semantics if process not found.

Updated Branches:
  refs/heads/master 7fc36c053 -> 0f3f8f35a


Revise os::killtree semantics if process not found.

Change semantics so it is not an error if the process is not found.

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


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

Branch: refs/heads/master
Commit: 413947c809daf944b135d291aa1ba2345bf683d7
Parents: 7fc36c0
Author: Ian Downes <ia...@gmail.com>
Authored: Tue Feb 11 16:32:55 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Tue Feb 11 16:33:43 2014 -0800

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/stout/include/stout/os/killtree.hpp | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/413947c8/3rdparty/libprocess/3rdparty/stout/include/stout/os/killtree.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/killtree.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/killtree.hpp
index c5917e7..1f45897 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os/killtree.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/killtree.hpp
@@ -63,7 +63,9 @@ inline Try<std::list<ProcessTree> > killtree(
   Result<Process> process = os::process(pid, processes.get());
 
   if (process.isNone()) {
-    return Error("Failed to find process " + stringify(pid));
+    // We do not consider it an error if the process is not present since it
+    // can exit at any time.
+    return std::list<ProcessTree>();
   }
 
   struct {


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

Posted by vi...@apache.org.
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.
 }
 
 


[08/11] git commit: Containerizer (part 1)

Posted by vi...@apache.org.
Containerizer (part 1)

The proposed Containerizer interface is to replace the existing
Isolator.

One ContainerizerProcess has been written:
MesosContainerizerProcess - implements containerizeration internally
using a Launcher and one or more Isolators (following review)

The intent is to also support a generic ExternalContainerizerProcess
that can delegate containerizeration by making external calls. Other
Containerizers could interface with specific external containerization
techniques such as Docker or LXC.

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


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

Branch: refs/heads/master
Commit: f90fe7641ea8f7066a6a1171a24ddaa8dc30e789
Parents: c578fe5
Author: Ian Downes <ia...@gmail.com>
Authored: Tue Feb 11 16:24:59 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Tue Feb 11 16:59:50 2014 -0800

----------------------------------------------------------------------
 include/mesos/mesos.proto                       |   25 +-
 src/Makefile.am                                 |   26 +-
 src/common/type_utils.hpp                       |   40 +
 src/launcher/fetcher.cpp                        |  242 +++
 src/launcher/launcher.cpp                       |  489 ------
 src/launcher/launcher.hpp                       |  125 --
 src/launcher/main.cpp                           |  100 --
 src/local/local.cpp                             |   23 +-
 src/slave/cgroups_isolator.cpp                  | 1412 ------------------
 src/slave/cgroups_isolator.hpp                  |  320 ----
 src/slave/containerizer/containerizer.cpp       |  269 ++++
 src/slave/containerizer/containerizer.hpp       |  137 ++
 src/slave/containerizer/mesos_containerizer.cpp |  907 +++++++++++
 src/slave/containerizer/mesos_containerizer.hpp |  208 +++
 src/slave/flags.hpp                             |   15 +-
 src/slave/http.cpp                              |    2 +-
 src/slave/isolator.cpp                          |   53 -
 src/slave/isolator.hpp                          |  110 --
 src/slave/main.cpp                              |   20 +-
 src/slave/monitor.cpp                           |  132 +-
 src/slave/monitor.hpp                           |   54 +-
 src/slave/paths.hpp                             |   32 +-
 src/slave/process_isolator.cpp                  |  516 -------
 src/slave/process_isolator.hpp                  |  117 --
 src/slave/slave.cpp                             |  487 +++---
 src/slave/slave.hpp                             |   32 +-
 src/slave/state.cpp                             |   45 +-
 src/slave/state.hpp                             |   10 +-
 src/slave/status_update_manager.cpp             |   33 +-
 src/slave/status_update_manager.hpp             |    8 +-
 30 files changed, 2243 insertions(+), 3746 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index 7079e03..69a4a60 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -83,6 +83,17 @@ message ExecutorID {
 
 
 /**
+ * A slave generated ID to distinguish a container. The ID must be unique
+ * between any active or completed containers on the slave. In particular,
+ * containers for different runs of the same (framework, executor) pair must be
+ * unique.
+ */
+message ContainerID {
+  required string value = 1;
+}
+
+
+/**
  * Describes a framework. If the user field is set to an empty string
  * Mesos will automagically set it to the current user. Note that the
  * ID is only available after a framework has registered, however, it
@@ -109,13 +120,13 @@ message FrameworkInfo {
 
 
 /**
- * Describes a command, executed via: '/bin/sh -c value'. Any uri's
- * specified are fetched before executing the command.
- * If executable field for an uri is set, executable file permission 
- * is set on the downloaded file. Also, if the downloaded file has ".tgz" 
- * extension it is extracted into the executor's working directory.
- * In addition, any environment variables are set before executing 
- * the command (so they can be used to "parameterize" your command).
+ * Describes a command, executed via: '/bin/sh -c value'. Any URIs specified
+ * are fetched before executing the command.  If the executable field for an
+ * uri is set, executable file permission is set on the downloaded file.
+ * Otherwise, if the downloaded file has a recognized archive extension
+ * (currently [compressed] tar and zip) it is extracted into the executor's
+ * working directory.  In addition, any environment variables are set before
+ * executing the command (so they can be used to "parameterize" your command).
  */
 message CommandInfo {
   message URI {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index c307068..9d39666 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -176,10 +176,9 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	slave/state.cpp							\
 	slave/slave.cpp							\
 	slave/http.cpp							\
-	slave/isolator.cpp						\
-	slave/process_isolator.cpp					\
+	slave/containerizer/containerizer.cpp				\
+	slave/containerizer/mesos_containerizer.cpp			\
 	slave/status_update_manager.cpp					\
-	launcher/launcher.cpp						\
 	exec/exec.cpp							\
 	common/lock.cpp							\
 	common/date_utils.cpp						\
@@ -204,11 +203,9 @@ pkginclude_HEADERS = $(top_srcdir)/include/mesos/executor.hpp	\
 nodist_pkginclude_HEADERS = ../include/mesos/mesos.hpp mesos.pb.h
 
 if OS_LINUX
-  libmesos_no_3rdparty_la_SOURCES += slave/cgroups_isolator.cpp
   libmesos_no_3rdparty_la_SOURCES += linux/cgroups.cpp
   libmesos_no_3rdparty_la_SOURCES += linux/fs.cpp
 else
-  EXTRA_DIST += slave/cgroups_isolator.cpp
   EXTRA_DIST += linux/cgroups.cpp
   EXTRA_DIST += linux/fs.cpp
 endif
@@ -220,7 +217,7 @@ libmesos_no_3rdparty_la_SOURCES += common/attributes.hpp		\
 	common/type_utils.hpp common/thread.hpp				\
 	examples/utils.hpp files/files.hpp				\
 	hdfs/hdfs.hpp							\
-	launcher/launcher.hpp linux/cgroups.hpp				\
+	linux/cgroups.hpp						\
 	linux/fs.hpp local/flags.hpp local/local.hpp			\
 	logging/flags.hpp logging/logging.hpp				\
 	master/allocator.hpp						\
@@ -232,17 +229,16 @@ libmesos_no_3rdparty_la_SOURCES += common/attributes.hpp		\
 	master/registrar.hpp						\
 	master/master.hpp master/sorter.hpp				\
 	messages/messages.hpp slave/constants.hpp			\
+	slave/containerizer/containerizer.hpp				\
+	slave/containerizer/mesos_containerizer.hpp			\
 	slave/flags.hpp slave/gc.hpp slave/monitor.hpp			\
-	slave/isolator.hpp						\
-	slave/cgroups_isolator.hpp					\
 	slave/paths.hpp slave/state.hpp					\
 	slave/status_update_manager.hpp					\
-	slave/process_isolator.hpp					\
 	slave/slave.hpp							\
 	tests/environment.hpp tests/script.hpp				\
 	tests/zookeeper.hpp tests/flags.hpp tests/utils.hpp		\
 	tests/cluster.hpp						\
-	tests/isolator.hpp						\
+	tests/containerizer.hpp						\
 	tests/mesos.hpp							\
 	tests/zookeeper_test_server.hpp zookeeper/authentication.hpp	\
 	zookeeper/contender.hpp						\
@@ -391,10 +387,10 @@ mesos_local_SOURCES = local/main.cpp
 mesos_local_CPPFLAGS = $(MESOS_CPPFLAGS)
 mesos_local_LDADD = libmesos.la
 
-pkglibexec_PROGRAMS += mesos-launcher
-mesos_launcher_SOURCES = launcher/main.cpp
-mesos_launcher_CPPFLAGS = $(MESOS_CPPFLAGS)
-mesos_launcher_LDADD = libmesos.la
+pkglibexec_PROGRAMS += mesos-fetcher
+mesos_fetcher_SOURCES = launcher/fetcher.cpp
+mesos_fetcher_CPPFLAGS = $(MESOS_CPPFLAGS)
+mesos_fetcher_LDADD = libmesos.la
 
 pkglibexec_PROGRAMS += mesos-executor
 mesos_executor_SOURCES = launcher/executor.cpp
@@ -838,6 +834,7 @@ mesos_tests_SOURCES =				\
   tests/allocator_tests.cpp			\
   tests/attributes_tests.cpp			\
   tests/authentication_tests.cpp		\
+  tests/containerizer.cpp			\
   tests/environment.cpp				\
   tests/examples_tests.cpp			\
   tests/exception_tests.cpp			\
@@ -878,7 +875,6 @@ 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/f90fe764/src/common/type_utils.hpp
----------------------------------------------------------------------
diff --git a/src/common/type_utils.hpp b/src/common/type_utils.hpp
index b8fc573..784a808 100644
--- a/src/common/type_utils.hpp
+++ b/src/common/type_utils.hpp
@@ -72,6 +72,14 @@ inline std::ostream& operator << (
 }
 
 
+inline std::ostream& operator << (
+    std::ostream& stream,
+    const ContainerID& containerId)
+{
+  return stream << containerId.value();
+}
+
+
 inline std::ostream& operator << (std::ostream& stream, const TaskState& state)
 {
   return stream << TaskState_descriptor()->FindValueByNumber(state)->name();
@@ -150,6 +158,18 @@ inline bool operator == (const ExecutorID& left, const ExecutorID& right)
 }
 
 
+inline bool operator == (const ContainerID& left, const ContainerID& right)
+{
+  return left.value() == right.value();
+}
+
+
+inline bool operator != (const ContainerID& left, const ContainerID& right)
+{
+  return left.value() != right.value();
+}
+
+
 inline bool operator == (const FrameworkID& left, const std::string& right)
 {
   return left.value() == right;
@@ -180,6 +200,12 @@ inline bool operator == (const ExecutorID& left, const std::string& right)
 }
 
 
+inline bool operator == (const ContainerID& left, const std::string& right)
+{
+  return left.value() == right;
+}
+
+
 inline bool operator < (const FrameworkID& left, const FrameworkID& right)
 {
   return left.value() < right.value();
@@ -210,6 +236,12 @@ inline bool operator < (const ExecutorID& left, const ExecutorID& right)
 }
 
 
+inline bool operator < (const ContainerID& left, const ContainerID& right)
+{
+  return left.value() < right.value();
+}
+
+
 inline bool operator == (const Environment& left, const Environment& right)
 {
   if (left.variables().size() != right.variables().size()) {
@@ -355,6 +387,14 @@ inline std::size_t hash_value(const ExecutorID& executorId)
 }
 
 
+inline std::size_t hash_value(const ContainerID& containerId)
+{
+  size_t seed = 0;
+  boost::hash_combine(seed, containerId.value());
+  return seed;
+}
+
+
 namespace internal {
 
 inline bool operator == (const Task& left, const Task& right)

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/launcher/fetcher.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/fetcher.cpp b/src/launcher/fetcher.cpp
new file mode 100644
index 0000000..9c9f07d
--- /dev/null
+++ b/src/launcher/fetcher.cpp
@@ -0,0 +1,242 @@
+/**
+ * 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 <string>
+
+#include <mesos/mesos.hpp>
+
+#include <stout/net.hpp>
+#include <stout/option.hpp>
+#include <stout/os.hpp>
+#include <stout/strings.hpp>
+
+#include "hdfs/hdfs.hpp"
+
+using namespace mesos;
+
+using std::string;
+
+// Try to extract filename into directory. If filename is recognized as an
+// archive it will be extracted and true returned; if not recognized then false
+// will be returned. An Error is returned if the extraction command fails.
+Try<Nothing> extract(const string& filename, const string& directory)
+{
+  string command;
+  // Extract any .tgz, tar.gz, tar.bz2 or zip files.
+  if (strings::endsWith(filename, ".tgz") ||
+      strings::endsWith(filename, ".tar.gz") ||
+      strings::endsWith(filename, ".tbz2") ||
+      strings::endsWith(filename, ".tar.bz2") ||
+      strings::endsWith(filename, ".txz") ||
+      strings::endsWith(filename, ".tar.xz")) {
+    command = "tar -C '" + directory + "' xJf";
+  } else if (strings::endsWith(filename, ".zip")) {
+    command = "unzip -d '" + directory + "'";
+  } else {
+    return Error("Could not extract file with unrecognized extension");
+  }
+
+  command += " '" + filename + "'";
+  int status = os::system(command);
+  if (status != 0) {
+    return Error("Failed to extract: command " + command +
+                 " exited with status: " + stringify(status));
+  }
+
+  LOG(INFO) << "Extracted resource '" << filename
+            << "' into '" << directory << "'";
+
+  return Nothing();
+}
+
+
+// Fetch URI into directory.
+Try<string> fetch(
+    const string& uri,
+    const string& directory)
+{
+  LOG(INFO) << "Fetching URI '" << uri << "'";
+
+  // Some checks to make sure using the URI value in shell commands
+  // is safe. TODO(benh): These should be pushed into the scheduler
+  // driver and reported to the user.
+  if (uri.find_first_of('\\') != string::npos ||
+      uri.find_first_of('\'') != string::npos ||
+      uri.find_first_of('\0') != string::npos) {
+    LOG(ERROR) << "URI contains illegal characters, refusing to fetch";
+    return Error("Illegal characters in URI");
+  }
+
+  // Grab the resource from HDFS if its path begins with hdfs:// or
+  // hftp:
+  // TODO(matei): Enforce some size limits on files we get from HDFS
+  if (strings::startsWith(uri, "hdfs://") ||
+      strings::startsWith(uri, "hftp://")) {
+    Try<string> base = os::basename(uri);
+    if (base.isError()) {
+      LOG(ERROR) << "Invalid basename for URI: " << base.error();
+      return Error("Invalid basename for URI");
+    }
+    string path = path::join(directory, base.get());
+
+    HDFS hdfs;
+
+    LOG(INFO) << "Downloading resource from '" << uri
+              << "' to '" << path << "'";
+    Try<Nothing> result = hdfs.copyToLocal(uri, path);
+    if (result.isError()) {
+      LOG(ERROR) << "HDFS copyToLocal failed: " << result.error();
+      return Error(result.error());
+    }
+
+    return path;
+  } else if (strings::startsWith(uri, "http://") ||
+             strings::startsWith(uri, "https://") ||
+             strings::startsWith(uri, "ftp://") ||
+             strings::startsWith(uri, "ftps://")) {
+    string path = uri.substr(uri.find("://") + 3);
+    if (path.find("/") == string::npos ||
+        path.size() <= path.find("/") + 1) {
+      LOG(ERROR) << "Malformed URL (missing path)";
+      return Error("Malformed URI");
+    }
+
+    path =  path::join(directory, path.substr(path.find_last_of("/") + 1));
+    LOG(INFO) << "Downloading '" << uri << "' to '" << path << "'";
+    Try<int> code = net::download(uri, path);
+    if (code.isError()) {
+      LOG(ERROR) << "Error downloading resource: " << code.error().c_str();
+      return Error("Fetch of URI failed (" + code.error() + ")");
+    } else if (code.get() != 200) {
+      LOG(ERROR) << "Error downloading resource, received HTTP/FTP return code "
+                 << code.get();
+      return Error("HTTP/FTP error (" + stringify(code.get()) + ")");
+    }
+
+    return path;
+  } else { // Copy the local resource.
+    string local = uri;
+    if (local.find_first_of("/") != 0) {
+      // We got a non-Hadoop and non-absolute path.
+      if (os::hasenv("MESOS_FRAMEWORKS_HOME")) {
+        local = path::join(os::getenv("MESOS_FRAMEWORKS_HOME"), local);
+        LOG(INFO) << "Prepended environment variable "
+                  << "MESOS_FRAMEWORKS_HOME to relative path, "
+                  << "making it: '" << local << "'";
+      } else {
+        LOG(ERROR) << "A relative path was passed for the resource but the "
+                   << "environment variable MESOS_FRAMEWORKS_HOME is not set. "
+                   << "Please either specify this config option "
+                   << "or avoid using a relative path";
+        return Error("Could not resolve relative URI");
+      }
+    }
+
+    Try<string> base = os::basename(local);
+    if (base.isError()) {
+      LOG(ERROR) << base.error();
+      return Error("Fetch of URI failed");
+    }
+
+    // Copy the resource to the directory.
+    string path = path::join(directory, base.get());
+    std::ostringstream command;
+    command << "cp '" << local << "' '" << path << "'";
+    LOG(INFO) << "Copying resource from '" << local
+              << "' to '" << directory << "'";
+
+    int status = os::system(command.str());
+    if (status != 0) {
+      LOG(ERROR) << "Failed to copy '" << local
+                 << "' : Exit status " << status;
+      return Error("Local copy failed");
+    }
+
+    return path;
+  }
+}
+
+
+int main(int argc, char* argv[])
+{
+  GOOGLE_PROTOBUF_VERIFY_VERSION;
+
+  CommandInfo commandInfo;
+  // Construct URIs from the encoded environment string.
+  const std::string& uris = os::getenv("MESOS_EXECUTOR_URIS");
+  foreach (const std::string& token, strings::tokenize(uris, " ")) {
+    // Delimiter between URI and execute permission.
+    size_t pos = token.rfind("+");
+    CHECK(pos != std::string::npos)
+      << "Invalid executor uri token in env " << token;
+
+    CommandInfo::URI uri;
+    uri.set_value(token.substr(0, pos));
+    uri.set_executable(token.substr(pos + 1) == "1");
+
+    commandInfo.add_uris()->MergeFrom(uri);
+  }
+
+  CHECK(os::hasenv("MESOS_WORK_DIRECTORY"))
+    << "Missing MESOS_WORK_DIRECTORY environment variable";
+  std::string directory = os::getenv("MESOS_WORK_DIRECTORY");
+
+  // We cannot use Some in the ternary expression because the compiler needs to
+  // be able to infer the type, thus the explicit Option<string>.
+  // TODO(idownes): Add an os::hasenv that returns an Option<string>.
+  Option<std::string> user = os::hasenv("MESOS_USER")
+    ? Option<std::string>(os::getenv("MESOS_USER")) // Explicit so it compiles.
+    : None();
+
+  // Fetch each URI to a local file, chmod, then chown if a user is provided.
+  foreach (const CommandInfo::URI& uri, commandInfo.uris()) {
+    // Fetch the URI to a local file.
+    Try<string> fetched = fetch(uri.value(), directory);
+    if (fetched.isError()) {
+      EXIT(1) << "Failed to fetch: " << uri.value();
+    }
+
+    // Chmod the fetched URI if it's executable, else assume it's an archive
+    // that should be extracted.
+    if (uri.executable()) {
+      bool chmodded = os::chmod(
+          fetched.get(), S_IRWXU | S_IRGRP | S_IXGRP | S_IROTH | S_IXOTH);
+      if (!chmodded) {
+        EXIT(1) << "Failed to chmod: " << fetched.get();
+      }
+    } else {
+      //TODO(idownes): Consider removing the archive once extracted.
+      // Try to extract the file if it's recognized as an archive.
+      Try<Nothing> extracted = extract(fetched.get(), directory);
+      if (extracted.isError()) {
+        EXIT(1) << "Failed to extract "
+                << fetched.get() << ":" << extracted.error();
+      }
+    }
+
+    // Recursively chown the directory if a user is provided.
+    if (user.isSome()) {
+      Try<Nothing> chowned = os::chown(user.get(), directory);
+      if (chowned.isError()) {
+        EXIT(1) << "Failed to chown " << directory << ": " << chowned.error();
+      }
+    }
+  }
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/launcher/launcher.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/launcher.cpp b/src/launcher/launcher.cpp
deleted file mode 100644
index d5ab667..0000000
--- a/src/launcher/launcher.cpp
+++ /dev/null
@@ -1,489 +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.
- */
-
-#include <dirent.h>
-#include <errno.h>
-#include <libgen.h>
-#include <pwd.h>
-#include <stdlib.h>
-#include <unistd.h>
-
-#include <iostream>
-#include <map>
-#include <sstream>
-
-#include <sys/stat.h>
-#include <sys/types.h>
-#include <sys/wait.h>
-
-#include <stout/fatal.hpp>
-#include <stout/foreach.hpp>
-#include <stout/net.hpp>
-#include <stout/nothing.hpp>
-#include <stout/os.hpp>
-#include <stout/path.hpp>
-
-#include "hdfs/hdfs.hpp"
-
-#include "launcher/launcher.hpp"
-
-#include "slave/flags.hpp"
-#include "slave/paths.hpp"
-#include "slave/state.hpp"
-
-using std::cerr;
-using std::cout;
-using std::endl;
-using std::map;
-using std::ostringstream;
-using std::string;
-
-namespace mesos {
-namespace internal {
-namespace launcher {
-
-ExecutorLauncher::ExecutorLauncher(
-    const SlaveID& _slaveId,
-    const FrameworkID& _frameworkId,
-    const ExecutorID& _executorId,
-    const UUID& _uuid,
-    const CommandInfo& _commandInfo,
-    const string& _user,
-    const string& _workDirectory,
-    const string& _slaveDirectory,
-    const string& _slavePid,
-    const string& _frameworksHome,
-    const string& _hadoopHome,
-    bool _redirectIO,
-    bool _shouldSwitchUser,
-    bool _checkpoint,
-    Duration _recoveryTimeout)
-  : slaveId(_slaveId),
-    frameworkId(_frameworkId),
-    executorId(_executorId),
-    uuid(_uuid),
-    commandInfo(_commandInfo),
-    user(_user),
-    workDirectory(_workDirectory),
-    slaveDirectory(_slaveDirectory),
-    slavePid(_slavePid),
-    frameworksHome(_frameworksHome),
-    hadoopHome(_hadoopHome),
-    redirectIO(_redirectIO),
-    shouldSwitchUser(_shouldSwitchUser),
-    checkpoint(_checkpoint),
-    recoveryTimeout(_recoveryTimeout) {}
-
-
-ExecutorLauncher::~ExecutorLauncher() {}
-
-
-// NOTE: We avoid fatalerror()s in this function because, we don't
-// want to kill the slave (in the case of cgroups isolator).
-int ExecutorLauncher::setup()
-{
-  // Checkpoint the forked pid, if necessary. The checkpointing must
-  // be done in the forked process (cgroups isolator) or execed
-  // launcher process (process isolator), because the slave process
-  // can die immediately after the isolator forks but before it would
-  // have a chance to write the pid to disk. That would result in an
-  // orphaned executor process unknown to the recovering slave.
-  if (checkpoint) {
-    const string& path = slave::paths::getForkedPidPath(
-        slave::paths::getMetaRootDir(slaveDirectory),
-        slaveId,
-        frameworkId,
-        executorId,
-        uuid);
-    cout << "Checkpointing executor's forked pid " << getpid()
-         << " to '" << path <<  "'" << endl;
-
-    Try<Nothing> checkpoint =
-      slave::state::checkpoint(path, stringify(getpid()));
-
-    if (checkpoint.isError()) {
-      cerr << "Failed to checkpoint executor's forked pid to '"
-           << path << "': " << checkpoint.error();
-      return -1;
-    }
-  }
-
-  const string& cwd = os::getcwd();
-
-  // TODO(benh): Do this in the slave?
-  if (shouldSwitchUser) {
-    Try<Nothing> chown = os::chown(user, workDirectory);
-
-    if (chown.isError()) {
-      cerr << "Failed to change ownership of the executor work directory "
-           << workDirectory << " to user " << user << ": " << chown.error()
-           << endl;
-      return -1;
-    }
-  }
-
-  // Enter working directory.
-  if (!os::chdir(workDirectory)) {
-    cerr << "Failed to chdir into executor work directory" << endl;
-    return -1;
-  }
-
-  // Redirect output to files in working dir if required.
-  // TODO(bmahler): It would be best if instead of closing stderr /
-  // stdout and redirecting, we instead always output to stderr /
-  // stdout. Also tee'ing their output into the work directory files
-  // when redirection is desired.
-  if (redirectIO) {
-    if (freopen("stdout", "w", stdout) == NULL) {
-      fatalerror("freopen failed");
-    }
-    if (freopen("stderr", "w", stderr) == NULL) {
-      fatalerror("freopen failed");
-    }
-  }
-
-  if (fetchExecutors() < 0) {
-    cerr << "Failed to fetch executors" << endl;
-    return -1;
-  }
-
-  // Go back to previous directory.
-  if (!os::chdir(cwd)) {
-    cerr << "Failed to chdir (back) into slave directory" << endl;
-    return -1;
-  }
-
-  return 0;
-}
-
-
-int ExecutorLauncher::launch()
-{
-  // Enter working directory.
-  if (os::chdir(workDirectory) < 0) {
-    fatalerror("Failed to chdir into the executor work directory");
-  }
-
-  if (shouldSwitchUser) {
-    switchUser();
-  }
-
-  setupEnvironment();
-
-  const string& command = commandInfo.value();
-
-  // Execute the command (via '/bin/sh -c command').
-  execl("/bin/sh", "sh", "-c", command.c_str(), (char*) NULL);
-
-  // If we get here, the execv call failed.
-  fatalerror("Could not execute '/bin/sh -c %s'", command.c_str());
-
-  return -1; // Silence end of non-void function warning.
-}
-
-
-int ExecutorLauncher::run()
-{
-  int ret = setup();
-  if (ret < 0) {
-    return ret;
-  }
-  return launch();
-}
-
-
-// Download the executor's files and optionally set executable permissions
-// if requested.
-int ExecutorLauncher::fetchExecutors()
-{
-  cout << "Fetching resources into '" << workDirectory << "'" << endl;
-
-  foreach(const CommandInfo::URI& uri, commandInfo.uris()) {
-    string resource = uri.value();
-    bool executable = uri.has_executable() && uri.executable();
-
-    cout << "Fetching resource '" << resource << "'" << endl;
-
-    // Some checks to make sure using the URI value in shell commands
-    // is safe. TODO(benh): These should be pushed into the scheduler
-    // driver and reported to the user.
-    if (resource.find_first_of('\\') != string::npos ||
-        resource.find_first_of('\'') != string::npos ||
-        resource.find_first_of('\0') != string::npos) {
-      cerr << "Illegal characters in URI" << endl;
-      return -1;
-    }
-
-    // Grab the resource from HDFS if its path begins with hdfs:// or
-    // htfp://. TODO(matei): Enforce some size limits on files we get
-    // from HDFS
-    if (resource.find("hdfs://") == 0 || resource.find("hftp://") == 0) {
-      HDFS hdfs(path::join(hadoopHome, "bin/hadoop"));
-
-      Try<std::string> basename = os::basename(resource);
-      if (basename.isError()) {
-        cerr << basename.error() << endl;
-        return -1;
-      }
-
-      string localFile = path::join(".", basename.get());
-
-      Try<Nothing> copy = hdfs.copyToLocal(resource, localFile);
-
-      if (copy.isError()) {
-        cerr << "Failed to copy from HDFS: " << copy.error() << endl;
-        return -1;
-      }
-
-      resource = localFile;
-    } else if (resource.find("http://") == 0
-               || resource.find("https://") == 0
-               || resource.find("ftp://") == 0
-               || resource.find("ftps://") == 0) {
-      string path = resource.substr(resource.find("://") + 3);
-      if (path.find("/") == string::npos) {
-        cerr << "Malformed URL (missing path)" << endl;
-        return -1;
-      }
-
-      if (path.size() <= path.find("/") + 1) {
-        cerr << "Malformed URL (missing path)" << endl;
-        return -1;
-      }
-
-      path =  path::join(".", path.substr(path.find_last_of("/") + 1));
-      cout << "Downloading '" << resource << "' to '" << path << "'" << endl;
-      Try<int> code = net::download(resource, path);
-      if (code.isError()) {
-        cerr << "Error downloading resource: " << code.error().c_str() << endl;
-        return -1;
-      } else if (code.get() != 200) {
-        cerr << "Error downloading resource, received HTTP/FTP return code "
-             << code.get() << endl;
-        return -1;
-      }
-      resource = path;
-    } else { // Copy the local resource.
-      if (resource.find_first_of("/") != 0) {
-        // We got a non-Hadoop and non-absolute path.
-        if (frameworksHome != "") {
-          resource = path::join(frameworksHome, resource);
-          cout << "Prepended configuration option frameworks_home to resource "
-               << "path, making it: '" << resource << "'" << endl;
-        } else {
-          cerr << "A relative path was passed for the resource, but "
-               << "the configuration option frameworks_home is not set. "
-               << "Please either specify this config option "
-               << "or avoid using a relative path" << endl;
-          return -1;
-        }
-      }
-
-      // Copy the resource to the current working directory.
-      ostringstream command;
-      command << "cp '" << resource << "' .";
-      cout << "Copying resource from '" << resource << "' to ." << endl;
-
-      int status = os::system(command.str());
-      if (status != 0) {
-        cerr << "Failed to copy '" << resource
-             << "' : Exit status " << status << endl;
-        return -1;
-      }
-
-      Try<std::string> base = os::basename(resource);
-      if (base.isError()) {
-        cerr << base.error() << endl;
-        return -1;
-      }
-
-      resource = path::join(".", base.get());
-    }
-
-    if (shouldSwitchUser) {
-      Try<Nothing> chown = os::chown(user, resource);
-
-      if (chown.isError()) {
-        cerr << "Failed to chown '" << resource << "' to user " << user << ": "
-             << chown.error() << endl;
-        return -1;
-      }
-    }
-
-    if (executable &&
-        !os::chmod(resource, S_IRWXU | S_IRGRP | S_IXGRP | S_IROTH | S_IXOTH)) {
-      cerr << "Failed to chmod '" << resource << "'" << endl;
-      return -1;
-    }
-
-    // Extract any .tgz, tar.gz, tar.bz2 or zip files.
-    if (strings::endsWith(resource, ".tgz") ||
-        strings::endsWith(resource, ".tar.gz")) {
-      string command = "tar xzf '" + resource + "'";
-      cout << "Extracting resource: " << command << endl;
-      int code = os::system(command);
-      if (code != 0) {
-        cerr << "Failed to extract resource: tar exit code " << code << endl;
-        return -1;
-      }
-    } else if (strings::endsWith(resource, ".tbz2") ||
-               strings::endsWith(resource, ".tar.bz2")) {
-      string command = "tar xjf '" + resource + "'";
-      cout << "Extracting resource: " << command << endl;
-      int code = os::system(command);
-      if (code != 0) {
-        cerr << "Failed to extract resource: tar exit code " << code << endl;
-        return -1;
-      }
-    } else if (strings::endsWith(resource, ".txz") ||
-               strings::endsWith(resource, ".tar.xz")) {
-      // If you want to use XZ on Mac OS, you can try the packages here:
-      // http://macpkg.sourceforge.net/
-      string command = "tar xJf '" + resource + "'";
-      cout << "Extracting resource: " << command << endl;
-      int code = os::system(command);
-      if (code != 0) {
-        cerr << "Failed to extract resource: tar exit code " << code << endl;
-        return -1;
-      }
-    } else if (strings::endsWith(resource, ".zip")) {
-      string command = "unzip '" + resource + "'";
-      cout << "Extracting resource: " << command << endl;
-      int code = os::system(command);
-      if (code != 0) {
-        cerr << "Failed to extract resource: unzip exit code " << code << endl;
-        return -1;
-      }
-    }
-  }
-
-  // Recursively chown the work directory, since extraction may have occurred.
-  if (shouldSwitchUser) {
-    Try<Nothing> chown = os::chown(user, ".");
-
-    if (chown.isError()) {
-      cerr << "Failed to recursively chown the work directory "
-           << workDirectory << " to user " << user << ": " << chown.error()
-           << endl;
-      return -1;
-    }
-  }
-
-  return 0;
-}
-
-
-void ExecutorLauncher::switchUser()
-{
-  if (!os::su(user)) {
-    fatal("Failed to switch to user %s for executor %s of framework %s",
-          user.c_str(), executorId.value().c_str(), frameworkId.value().c_str());
-  }
-}
-
-
-// Set up environment variables for launching a framework's executor.
-void ExecutorLauncher::setupEnvironment()
-{
-  foreachpair (const string& key, const string& value, getEnvironment()) {
-    os::setenv(key, value);
-  }
-}
-
-
-map<string, string> ExecutorLauncher::getEnvironment()
-{
-  map<string, string> env;
-
-  // Set LIBPROCESS_PORT so that we bind to a random free port (since
-  // this might have been set via --port option). We do this before
-  // the environment variables below in case it is included.
-  env["LIBPROCESS_PORT"] = "0";
-
-  // Also add MESOS_NATIVE_LIBRARY if it's not already present (and
-  // like above, we do this before the environment variables below in
-  // case the framework wants to override).
-  if (!os::hasenv("MESOS_NATIVE_LIBRARY")) {
-    string path =
-#ifdef __APPLE__
-      LIBDIR "/libmesos-" VERSION ".dylib";
-#else
-      LIBDIR "/libmesos-" VERSION ".so";
-#endif
-    if (os::exists(path)) {
-      env["MESOS_NATIVE_LIBRARY"] = path;
-    }
-  }
-
-  // Set up the environment as specified in the ExecutorInfo.
-  if (commandInfo.has_environment()) {
-    foreach (const Environment::Variable& variable,
-             commandInfo.environment().variables()) {
-      env[variable.name()] = variable.value();
-    }
-  }
-
-  // Set Mesos environment variables for slave ID, framework ID, etc.
-  env["MESOS_DIRECTORY"] = workDirectory;
-  env["MESOS_SLAVE_PID"] = slavePid;
-  env["MESOS_SLAVE_ID"] = slaveId.value();
-  env["MESOS_FRAMEWORK_ID"] = frameworkId.value();
-  env["MESOS_EXECUTOR_ID"] = executorId.value();
-  env["MESOS_EXECUTOR_UUID"] = uuid.toString();
-  env["MESOS_CHECKPOINT"] = checkpoint ? "1" : "0";
-
-  if (checkpoint) {
-    env["MESOS_RECOVERY_TIMEOUT"] = stringify(recoveryTimeout);
-  }
-
-  return env;
-}
-
-
-// Get Mesos environment variables that launcher/main.cpp will
-// pass as arguments to an ExecutorLauncher there.
-map<string, string> ExecutorLauncher::getLauncherEnvironment()
-{
-  map<string, string> env = getEnvironment();
-
-  string uris = "";
-  foreach (const CommandInfo::URI& uri, commandInfo.uris()) {
-   uris += uri.value() + "+" +
-           (uri.has_executable() && uri.executable() ? "1" : "0");
-   uris += " ";
-  }
-
-  // Remove extra space at the end.
-  if (uris.size() > 0) {
-    uris = strings::trim(uris);
-  }
-
-  env["MESOS_EXECUTOR_URIS"] = uris;
-  env["MESOS_COMMAND"] = commandInfo.value();
-  env["MESOS_USER"] = user;
-  env["MESOS_SLAVE_DIRECTORY"] = slaveDirectory;
-  env["MESOS_HADOOP_HOME"] = hadoopHome;
-  env["MESOS_REDIRECT_IO"] = redirectIO ? "1" : "0";
-  env["MESOS_SWITCH_USER"] = shouldSwitchUser ? "1" : "0";
-
-  return env;
-}
-
-} // namespace launcher {
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/launcher/launcher.hpp
----------------------------------------------------------------------
diff --git a/src/launcher/launcher.hpp b/src/launcher/launcher.hpp
deleted file mode 100644
index 104fe81..0000000
--- a/src/launcher/launcher.hpp
+++ /dev/null
@@ -1,125 +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 __LAUNCHER_HPP__
-#define __LAUNCHER_HPP__
-
-#include <map>
-#include <string>
-
-#include <mesos/mesos.hpp>
-
-#include <stout/duration.hpp>
-#include <stout/uuid.hpp>
-
-#include "slave/flags.hpp"
-
-namespace mesos {
-namespace internal {
-namespace launcher {
-
-// This class sets up the environment for an executor and then exec()'s it.
-// It can either be used after a fork() in the slave process, or run as a
-// standalone program (with the main function in launcher_main.cpp).
-//
-// The environment is initialized through for steps:
-// 1) A work directory for the framework is created by createWorkingDirectory().
-// 2) The executor is fetched off HDFS if necessary by fetchExecutor().
-// 3) Environment variables are set by setupEnvironment().
-// 4) We switch to the framework's user in switchUser().
-//
-// Isolators that wish to override the default behaviour can subclass
-// Launcher and override some of the methods to perform extra actions.
-class ExecutorLauncher {
-public:
-  ExecutorLauncher(
-      const SlaveID& slaveId,
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId,
-      const UUID& uuid,
-      const CommandInfo& commandInfo,
-      const std::string& user,
-      const std::string& workDirectory,
-      const std::string& slaveWorkDirectory,
-      const std::string& slavePid,
-      const std::string& frameworksHome,
-      const std::string& hadoopHome,
-      bool redirectIO,
-      bool shouldSwitchUser,
-      bool checkpoint,
-      Duration recoveryTimeout);
-
-  virtual ~ExecutorLauncher();
-
-  // Initialize the working directory and fetch the executor.
-  virtual int setup();
-
-  // Launches the downloaded executor.
-  virtual int launch();
-
-  // Convenience function that calls setup() and then launch().
-  virtual int run();
-
-  // Return a map of environment variables for exec'ing a
-  // launch_main.cpp (mesos-launcher binary) process. This is used
-  // by isolators that cannot exec the user's executor directly
-  // (e.g., due to potential deadlocks in forked process).
-  virtual std::map<std::string, std::string> getLauncherEnvironment();
-
-protected:
-  // Download the required files for the executor from the given set of URIs.
-  // Optionally, it will set the executable file permissions for the files.
-  // This method is expected to place files in the workDirectory.
-  virtual int fetchExecutors();
-
-  // Return a map of environment variables for launching a
-  // framework's executor.
-  virtual std::map<std::string, std::string> getEnvironment();
-
-  // Set up environment variables for launching a
-  // framework's executor.
-  virtual void setupEnvironment();
-
-  // Switch to a framework's user in preparation for exec()'ing its executor.
-  virtual void switchUser();
-
-protected:
-  const SlaveID slaveId;
-  const FrameworkID frameworkId;
-  const ExecutorID executorId;
-  const UUID uuid;
-  const CommandInfo commandInfo;
-  const std::string user;
-  const std::string workDirectory;
-  const std::string slaveDirectory;
-  const std::string slavePid;
-  const std::string frameworksHome;
-  const std::string hadoopHome;
-  const bool redirectIO;   // Whether to redirect stdout and stderr to files.
-  const bool shouldSwitchUser; // Whether to setuid to framework's user.
-  const bool checkpoint; // Whether the framework enabled checkpointing.
-
-  // Executor suicide timeout for slave recovery.
-  const Duration recoveryTimeout;
-};
-
-} // namespace launcher {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __LAUNCHER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/launcher/main.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/main.cpp b/src/launcher/main.cpp
deleted file mode 100644
index de64609..0000000
--- a/src/launcher/main.cpp
+++ /dev/null
@@ -1,100 +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.
- */
-
-#include <string>
-
-#include <mesos/mesos.hpp>
-
-#include <stout/duration.hpp>
-#include <stout/strings.hpp>
-#include <stout/os.hpp>
-
-#include "launcher/launcher.hpp"
-
-using namespace mesos;
-using namespace mesos::internal; // For 'utils'.
-
-using std::string;
-
-
-int main(int argc, char** argv)
-{
-  GOOGLE_PROTOBUF_VERIFY_VERSION;
-
-  SlaveID slaveId;
-  slaveId.set_value(os::getenv("MESOS_SLAVE_ID"));
-
-  FrameworkID frameworkId;
-  frameworkId.set_value(os::getenv("MESOS_FRAMEWORK_ID"));
-
-  ExecutorID executorId;
-  executorId.set_value(os::getenv("MESOS_EXECUTOR_ID"));
-
-  CommandInfo commandInfo;
-  commandInfo.set_value(os::getenv("MESOS_COMMAND"));
-
-  // Construct URIs from the encoded environment string.
-  const std::string& uris = os::getenv("MESOS_EXECUTOR_URIS");
-  foreach (const std::string& token, strings::tokenize(uris, " ")) {
-    size_t pos = token.rfind("+"); // Delim between uri and exec permission.
-    CHECK(pos != std::string::npos) << "Invalid executor uri token in env "
-                                    << token;
-
-    CommandInfo::URI uri;
-    uri.set_value(token.substr(0, pos));
-    uri.set_executable(token.substr(pos + 1) == "1");
-
-    commandInfo.add_uris()->MergeFrom(uri);
-  }
-
-  bool checkpoint = os::getenv("MESOS_CHECKPOINT", false) == "1";
-
-  Duration recoveryTimeout = slave::RECOVERY_TIMEOUT;
-
-  // Get the recovery timeout if checkpointing is enabled.
-  if (checkpoint) {
-    string value = os::getenv("MESOS_RECOVERY_TIMEOUT", false);
-
-    if (!value.empty()) {
-      Try<Duration> _recoveryTimeout = Duration::parse(value);
-
-      CHECK_SOME(_recoveryTimeout)
-        << "Cannot parse MESOS_RECOVERY_TIMEOUT '" + value + "'";
-
-      recoveryTimeout = _recoveryTimeout.get();
-    }
-  }
-
-  return mesos::internal::launcher::ExecutorLauncher(
-      slaveId,
-      frameworkId,
-      executorId,
-      UUID::fromString(os::getenv("MESOS_EXECUTOR_UUID")),
-      commandInfo,
-      os::getenv("MESOS_USER"),
-      os::getenv("MESOS_DIRECTORY"),
-      os::getenv("MESOS_SLAVE_DIRECTORY"),
-      os::getenv("MESOS_SLAVE_PID"),
-      os::getenv("MESOS_FRAMEWORKS_HOME", false),
-      os::getenv("MESOS_HADOOP_HOME"),
-      os::getenv("MESOS_REDIRECT_IO") == "1",
-      os::getenv("MESOS_SWITCH_USER") == "1",
-      checkpoint,
-      recoveryTimeout)
-    .run();
-}

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/local/local.cpp
----------------------------------------------------------------------
diff --git a/src/local/local.cpp b/src/local/local.cpp
index e650de9..5112391 100644
--- a/src/local/local.cpp
+++ b/src/local/local.cpp
@@ -40,7 +40,7 @@
 #include "master/master.hpp"
 #include "master/registrar.hpp"
 
-#include "slave/process_isolator.hpp"
+#include "slave/containerizer/containerizer.hpp"
 #include "slave/slave.hpp"
 
 #include "state/leveldb.hpp"
@@ -57,9 +57,8 @@ using mesos::internal::master::allocator::HierarchicalDRFAllocatorProcess;
 using mesos::internal::master::Master;
 using mesos::internal::master::Registrar;
 
+using mesos::internal::slave::Containerizer;
 using mesos::internal::slave::Slave;
-using mesos::internal::slave::Isolator;
-using mesos::internal::slave::ProcessIsolator;
 
 using process::PID;
 using process::UPID;
@@ -80,7 +79,7 @@ static state::Storage* storage = NULL;
 static state::protobuf::State* state = NULL;
 static Registrar* registrar = NULL;
 static Master* master = NULL;
-static map<Isolator*, Slave*> slaves;
+static map<Containerizer*, Slave*> slaves;
 static StandaloneMasterDetector* detector = NULL;
 static MasterContender* contender = NULL;
 static Files* files = NULL;
@@ -142,9 +141,6 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
   vector<UPID> pids;
 
   for (int i = 0; i < flags.num_slaves; i++) {
-    // TODO(benh): Create a local isolator?
-    ProcessIsolator* isolator = new ProcessIsolator();
-
     slave::Flags flags;
     Try<Nothing> load = flags.load("MESOS_");
     if (load.isError()) {
@@ -152,13 +148,18 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
               << "slave flags from the environment: " << load.error();
     }
 
+    Try<Containerizer*> containerizer = Containerizer::create(flags, true);
+    if (containerizer.isError()) {
+      EXIT(1) << "Failed to create a containerizer: " << containerizer.error();
+    }
+
     // Use a different work directory for each slave.
     flags.work_dir = path::join(flags.work_dir, stringify(i));
 
     // NOTE: At this point detector is already initialized by the
     // Master.
-    Slave* slave = new Slave(flags, true, detector, isolator, files);
-    slaves[isolator] = slave;
+    Slave* slave = new Slave(flags, detector, containerizer.get(), files);
+    slaves[containerizer.get()] = slave;
     pids.push_back(process::spawn(slave));
   }
 
@@ -182,10 +183,10 @@ void shutdown()
     // isolator, we can't delete the isolator until we have stopped
     // the slave.
 
-    foreachpair (Isolator* isolator, Slave* slave, slaves) {
+    foreachpair (Containerizer* containerizer, Slave* slave, slaves) {
       process::terminate(slave->self());
       process::wait(slave->self());
-      delete isolator;
+      delete containerizer;
       delete slave;
     }
 


[04/11] git commit: Containerizer - launchers (part 2).

Posted by vi...@apache.org.
Containerizer - launchers (part 2).

Launcher interface and MesosLauncher to support MesosContainerizers.

Launchers handle the lifecycle of the executor process
(and descendants).

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


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

Branch: refs/heads/master
Commit: 71c6b5c050df91083b6308eabb3abf02dca6b129
Parents: f90fe76
Author: Ian Downes <ia...@gmail.com>
Authored: Tue Feb 11 16:27:51 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Tue Feb 11 16:59:50 2014 -0800

----------------------------------------------------------------------
 src/Makefile.am                              |   4 +
 src/linux/cgroups.cpp                        | 115 +++++++++
 src/linux/cgroups.hpp                        |  13 +
 src/slave/containerizer/cgroups_launcher.cpp | 274 ++++++++++++++++++++++
 src/slave/containerizer/cgroups_launcher.hpp |  64 +++++
 src/slave/containerizer/launcher.cpp         | 159 +++++++++++++
 src/slave/containerizer/launcher.hpp         |  88 +++++++
 7 files changed, 717 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/71c6b5c0/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 9d39666..a4b5a52 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -177,6 +177,7 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	slave/slave.cpp							\
 	slave/http.cpp							\
 	slave/containerizer/containerizer.cpp				\
+	slave/containerizer/launcher.cpp				\
 	slave/containerizer/mesos_containerizer.cpp			\
 	slave/status_update_manager.cpp					\
 	exec/exec.cpp							\
@@ -204,6 +205,7 @@ nodist_pkginclude_HEADERS = ../include/mesos/mesos.hpp mesos.pb.h
 
 if OS_LINUX
   libmesos_no_3rdparty_la_SOURCES += linux/cgroups.cpp
+  libmesos_no_3rdparty_la_SOURCES += slave/container/cgroups_launcher.cpp
   libmesos_no_3rdparty_la_SOURCES += linux/fs.cpp
 else
   EXTRA_DIST += linux/cgroups.cpp
@@ -229,7 +231,9 @@ libmesos_no_3rdparty_la_SOURCES += common/attributes.hpp		\
 	master/registrar.hpp						\
 	master/master.hpp master/sorter.hpp				\
 	messages/messages.hpp slave/constants.hpp			\
+	slave/containerizer/cgroups_launcher.hpp			\
 	slave/containerizer/containerizer.hpp				\
+	slave/containerizer/launcher.hpp				\
 	slave/containerizer/mesos_containerizer.hpp			\
 	slave/flags.hpp slave/gc.hpp slave/monitor.hpp			\
 	slave/paths.hpp slave/state.hpp					\

http://git-wip-us.apache.org/repos/asf/mesos/blob/71c6b5c0/src/linux/cgroups.cpp
----------------------------------------------------------------------
diff --git a/src/linux/cgroups.cpp b/src/linux/cgroups.cpp
index 19ab1f3..8ac2599 100644
--- a/src/linux/cgroups.cpp
+++ b/src/linux/cgroups.cpp
@@ -405,6 +405,121 @@ static Try<Nothing> write(
 } // namespace internal {
 
 
+Try<string> prepare(
+    const std::string& baseHierarchy,
+    const std::string& subsystem,
+    const std::string& cgroup)
+{
+  // Construct the hierarchy for this subsystem.
+  std::string hierarchy = path::join(baseHierarchy, subsystem);
+
+  // Ensure cgroups are enabled in the kernel.
+  if (!cgroups::enabled()) {
+    return Error("No cgroups support detected in this kernel");
+  }
+
+  // Ensure we have root permissions.
+  if (geteuid() != 0) {
+    return Error("Using cgroups requires root permissions");
+  }
+
+  // Check if the hierarchy is already mounted, and if not, mount it.
+  Try<bool> mounted = cgroups::mounted(hierarchy);
+
+  if (mounted.isError()) {
+    return Error("Failed to determine if " + hierarchy +
+                 " is already mounted: " + mounted.error());
+  }
+
+  if (mounted.get()) {
+    // Make sure that desired subsystem is attached to the already
+    // mounted hierarchy.
+    Try<std::set<std::string> > attached = cgroups::subsystems(hierarchy);
+    if (attached.isError()) {
+      return Error(string("Failed to determine the attached subsystems") +
+                   "for the cgroup hierarchy at " + hierarchy + ": " +
+                   attached.error());
+    }
+
+    if (attached.get().count(subsystem) == 0) {
+      return Error("The cgroups hierarchy at " + hierarchy +
+                   " can not be used because it does not have the '" +
+                   subsystem + "' subsystem attached");
+    }
+
+    if (attached.get().size() > 1) {
+      return Error("The " + subsystem + " subsystem is co-mounted at " +
+                   hierarchy + " with other subsytems");
+    }
+  } else {
+    // Attempt to mount the hierarchy ourselves.
+    if (os::exists(hierarchy)) {
+      // The path specified by the given hierarchy already exists in
+      // the file system. We try to remove it if it is an empty
+      // directory. This will helps us better deal with slave restarts
+      // since we won't need to manually remove the directory.
+      Try<Nothing> rmdir = os::rmdir(hierarchy, false);
+      if (rmdir.isError()) {
+        return Error("Failed to mount cgroups hierarchy at '" + hierarchy +
+                     "' because we could not remove the existing directory: " +
+                     rmdir.error());
+      }
+    }
+
+    // Mount the subsystem.
+    Try<Nothing> mount = cgroups::mount(hierarchy, subsystem);
+    if (mount.isError()) {
+      return Error("Failed to mount cgroups hierarchy at '" + hierarchy +
+                   "': " + mount.error());
+    }
+  }
+
+  // Create the cgroup if it doesn't exist.
+  Try<bool> exists = cgroups::exists(hierarchy, cgroup);
+  if (exists.isError()) {
+    return Error("Failed to check existence of root cgroup " +
+                 path::join(hierarchy, cgroup) +
+                 ": " + exists.error());
+  }
+
+  if (!exists.get()) {
+    // No cgroup exists, create it.
+    Try<Nothing> create = cgroups::create(hierarchy, cgroup);
+    if (create.isError()) {
+      return Error("Failed to create root cgroup " +
+                   path::join(hierarchy, cgroup) +
+                   ": " + create.error());
+    }
+  }
+
+  const string& testCgroup = path::join(cgroup, "test");
+  // Create a nested test cgroup if it doesn't exist.
+  exists = cgroups::exists(hierarchy, testCgroup);
+  if (exists.isError()) {
+    return Error("Failed to check existence nested of test cgroup " +
+                 path::join(hierarchy, testCgroup) +
+                 ": " + exists.error());
+  }
+
+  if (!exists.get()) {
+    // Make sure this kernel supports creating nested cgroups.
+    Try<Nothing> create = cgroups::create(hierarchy, testCgroup);
+    if (create.isError()) {
+      return Error(string("Failed to create a nested 'test' cgroup.") +
+                   " Your kernel might be too old to use the" +
+                   " cgroups isolator: " + create.error());
+    }
+  }
+
+  // Remove the nested 'test' cgroup.
+  Try<Nothing> remove = cgroups::remove(hierarchy, testCgroup);
+  if (remove.isError()) {
+    return Error("Failed to remove the nested test cgroup: " + remove.error());
+  }
+
+  return hierarchy;
+}
+
 // Returns some error string if either (a) hierarchy is not mounted,
 // (b) cgroup does not exist, or (c) control file does not exist.
 static Option<Error> verify(

http://git-wip-us.apache.org/repos/asf/mesos/blob/71c6b5c0/src/linux/cgroups.hpp
----------------------------------------------------------------------
diff --git a/src/linux/cgroups.hpp b/src/linux/cgroups.hpp
index bf0d173..2f4ab25 100644
--- a/src/linux/cgroups.hpp
+++ b/src/linux/cgroups.hpp
@@ -58,6 +58,19 @@ const unsigned int EMPTY_WATCHER_RETRIES = 50;
 // Control   -  A control file in a cgroup (e.g. tasks, cpu.shares).
 
 
+// TODO(idownes): Rework all functions in this file to better support
+// separately mounted subsystems.
+
+// Prepare a hierarchy which has the specified subsystem (and only that
+// subsystem) mounted and also has the specified cgroup created. Returns the
+// hierarchy. Checks are made to ensure that cgroups are supported and that
+// nested cgroups can be created.
+Try<std::string> prepare(
+    const std::string& baseHierarchy,
+    const std::string& subsystem,
+    const std::string& cgroup);
+
+
 // Check whether cgroups module is enabled on the current machine.
 // @return  True if cgroups module is enabled.
 //          False if cgroups module is not available.

http://git-wip-us.apache.org/repos/asf/mesos/blob/71c6b5c0/src/slave/containerizer/cgroups_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/cgroups_launcher.cpp b/src/slave/containerizer/cgroups_launcher.cpp
new file mode 100644
index 0000000..0921d63
--- /dev/null
+++ b/src/slave/containerizer/cgroups_launcher.cpp
@@ -0,0 +1,274 @@
+/**
+ * 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 <unistd.h>
+
+#include <vector>
+
+#include <stout/hashset.hpp>
+#include <stout/path.hpp>
+#include <stout/unreachable.hpp>
+
+#include "linux/cgroups.hpp"
+
+#include "mesos/resources.hpp"
+
+#include "slave/containerizer/cgroups_launcher.hpp"
+
+using namespace process;
+
+using std::list;
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+using state::RunState;
+
+CgroupsLauncher::CgroupsLauncher(const Flags& _flags, const string& _hierarchy)
+  : flags(_flags),
+    hierarchy(_hierarchy) {}
+
+
+Try<Launcher*> CgroupsLauncher::create(const Flags& flags)
+{
+  Try<string> hierarchy = cgroups::prepare(
+      flags.cgroups_hierarchy, "freezer", flags.cgroups_root);
+
+  if (hierarchy.isError()) {
+    return Error("Failed to create cgroups launcher: " + hierarchy.error());
+  }
+
+  LOG(INFO) << "Using " << hierarchy.get()
+            << " as the freezer hierarchy for the cgroups launcher";
+
+  return new CgroupsLauncher(flags, hierarchy.get());
+}
+
+
+Try<Nothing> CgroupsLauncher::recover(const std::list<state::RunState>& states)
+{
+  hashset<string> cgroups;
+
+  foreach (const RunState& state, states) {
+    if (state.id.isNone()) {
+      return Error("ContainerID is required to recover");
+    }
+    const ContainerID& containerId = state.id.get();
+
+    Try<bool> exists = cgroups::exists(hierarchy, cgroup(containerId));
+
+    if (!exists.get()) {
+      // This may occur if the freezer cgroup was destroyed but the slave dies
+      // before noticing this.
+      // The containerizer will monitor the container's pid and notice that it
+      // has exited, triggering destruction of the container.
+      LOG(INFO) << "Couldn't find freezer cgroup for container " << containerId;
+      continue;
+    }
+
+    if (state.forkedPid.isNone()) {
+      return Error("Executor pid is required to recover container " +
+                   stringify(containerId));
+    }
+    pid_t pid = state.forkedPid.get();
+
+    if (pids.containsValue(pid)) {
+      // This should (almost) never occur. There is the possibility that a new
+      // executor is launched with the same pid as one that just exited (highly
+      // unlikely) and the slave dies after the new executor is launched but
+      // before it hears about the termination of the earlier executor (also
+      // unlikely). Regardless, the launcher can't do anything sensible so this
+      // is considered an error.
+      return Error("Detected duplicate pid " + stringify(pid) +
+                   " for container " + stringify(containerId));
+    }
+
+    pids.put(containerId, pid);
+
+    cgroups.insert(cgroup(containerId));
+  }
+
+  Try<vector<string> > orphans = cgroups::get(hierarchy, flags.cgroups_root);
+  if (orphans.isError()) {
+    return Error(orphans.error());
+  }
+
+  foreach (const string& orphan, orphans.get()) {
+    if (!cgroups.contains(orphan)) {
+      LOG(INFO) << "Removing orphaned cgroup"
+                << " '" << path::join("freezer", orphan) << "'";
+      cgroups::destroy(hierarchy, orphan);
+    }
+  }
+
+  return Nothing();
+}
+
+
+Try<pid_t> CgroupsLauncher::fork(
+    const ContainerID& containerId,
+    const lambda::function<int()>& inChild)
+{
+  // Create a freezer cgroup for this container if necessary.
+  Try<bool> exists = cgroups::exists(hierarchy, cgroup(containerId));
+
+  if (exists.isError()) {
+    return Error("Failed to create freezer cgroup: " + exists.error());
+  }
+
+  if (!exists.get()) {
+    Try<Nothing> created = cgroups::create(hierarchy, cgroup(containerId));
+
+    if (created.isError()) {
+      LOG(ERROR) << "Failed to create freezer cgroup for container '"
+                 << containerId << "': " << created.error();
+      return Error("Failed to contain process: " + created.error());
+    }
+  }
+
+  // Additional processes forked will be put into the same process group and
+  // session.
+  Option<pid_t> pgid = pids.get(containerId);
+
+  // Use a pipe to block the child until it's been moved into the freezer
+  // cgroup.
+  int pipes[2];
+  // We assume this should not fail under reasonable conditions so we use CHECK.
+  CHECK(pipe(pipes) == 0);
+
+  pid_t pid;
+
+  if ((pid = ::fork()) == -1) {
+    return ErrnoError("Failed to fork");
+  }
+
+  if (pid > 0) {
+    // In parent.
+    os::close(pipes[0]);
+
+    // Move the child into the freezer cgroup. Any grandchildren will also be
+    // contained in the cgroup.
+    Try<Nothing> assign = cgroups::assign(hierarchy, cgroup(containerId), pid);
+
+    if (assign.isError()) {
+      LOG(ERROR) << "Failed to assign process " << pid
+                 << " of container '" << containerId << "'"
+                 << " to its freezer cgroup: " << assign.error();
+      kill(pid, SIGKILL);
+      return Error("Failed to contain process");
+    }
+
+    // Now that we've contained the child we can signal it to continue by
+    // writing to the pipe.
+    int buf;
+    ssize_t len;
+    while ((len = write(pipes[1], &buf, sizeof(buf))) == -1 && errno == EINTR);
+
+    if (len != sizeof(buf)) {
+      // Ensure the child is killed.
+      kill(pid, SIGKILL);
+      os::close(pipes[1]);
+      return Error("Failed to synchronize child process");
+    }
+    os::close(pipes[1]);
+
+    // Store the pid (session id and process group id) if this is the first
+    // process forked for this container.
+    if (!pids.contains(containerId)) {
+      pids.put(containerId, pid);
+    }
+
+    return pid;
+  } else {
+    // In child.
+    os::close(pipes[1]);
+
+    // Move to a previously created process group (and session) if available,
+    // else create a new session and process group. Even though we track
+    // processes using cgroups we need to move to a different session so we're
+    // independent from the slave's session (otherwise children will receive
+    // SIGHUP if the slave exits).
+    // TODO(idownes): perror is not listed as async-signal-safe and should be
+    // reimplemented safely.
+    if (pgid.isSome() && (setpgid(0, pgid.get()) == -1)) {
+      perror("Failed to put child into process group");
+      os::close(pipes[0]);
+      _exit(1);
+    } else if (setsid() == -1) {
+      perror("Failed to put child in a new session");
+      os::close(pipes[0]);
+      _exit(1);
+    }
+
+    // Do a blocking read on the pipe until the parent signals us to continue.
+    int buf;
+    int len;
+    while ((len = read(pipes[0], &buf, sizeof(buf))) == -1 && errno == EINTR);
+
+    if (len != sizeof(buf)) {
+      const char* message = "Failed to synchronize with parent";
+      // Ignore the return value from write() to silence compiler warning.
+      while (write(STDERR_FILENO, message, strlen(message)) == -1 &&
+          errno == EINTR);
+      os::close(pipes[0]);
+      _exit(1);
+    }
+
+    os::close(pipes[0]);
+
+    // This function should exec() and therefore not return.
+    inChild();
+
+    return UNREACHABLE();
+  }
+}
+
+
+Future<Nothing> _destroy(
+    const ContainerID& containerId,
+    process::Future<bool> destroyed)
+{
+  if (destroyed.isFailed()) {
+    LOG(ERROR) << "Failed to destroy freezer cgroup for '"
+               << containerId << "': " << destroyed.failure();
+    return Failure("Failed to destroy launcher: " + destroyed.failure());
+  }
+  return Nothing();
+}
+
+
+Future<Nothing> CgroupsLauncher::destroy(const ContainerID& containerId)
+{
+  pids.erase(containerId);
+
+  return cgroups::destroy(hierarchy, cgroup(containerId))
+    .then(lambda::bind(&_destroy, containerId, lambda::_1));
+}
+
+
+string CgroupsLauncher::cgroup(const ContainerID& containerId)
+{
+  return path::join(flags.cgroups_root, containerId.value());
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/71c6b5c0/src/slave/containerizer/cgroups_launcher.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/cgroups_launcher.hpp b/src/slave/containerizer/cgroups_launcher.hpp
new file mode 100644
index 0000000..db61107
--- /dev/null
+++ b/src/slave/containerizer/cgroups_launcher.hpp
@@ -0,0 +1,64 @@
+/**
+ * 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 __CGROUPS_LAUNCHER_HPP__
+#define __CGROUPS_LAUNCHER_HPP__
+
+#include "slave/containerizer/launcher.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// Launcher for Linux systems with cgroups. Uses a freezer cgroup to track
+// pids.
+class CgroupsLauncher : public Launcher
+{
+public:
+  static Try<Launcher*> create(const Flags& flags);
+
+  virtual ~CgroupsLauncher() {}
+
+  virtual Try<Nothing> recover(const std::list<state::RunState>& states);
+
+  virtual Try<pid_t> fork(
+      const ContainerID& containerId,
+      const lambda::function<int()>& inChild);
+
+  virtual process::Future<Nothing> destroy(const ContainerID& containerId);
+
+private:
+  CgroupsLauncher(const Flags& flags, const std::string& hierarchy);
+
+  static const std::string subsystem;
+  const Flags flags;
+  const std::string hierarchy;
+
+  std::string cgroup(const ContainerID& containerId);
+
+  // The 'pid' is the process id of the first process and also the process
+  // group id and session id.
+  hashmap<ContainerID, pid_t> pids;
+};
+
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __CGROUPS_LAUNCHER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/71c6b5c0/src/slave/containerizer/launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/launcher.cpp b/src/slave/containerizer/launcher.cpp
new file mode 100644
index 0000000..ddfa91c
--- /dev/null
+++ b/src/slave/containerizer/launcher.cpp
@@ -0,0 +1,159 @@
+/**
+ * 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 <unistd.h>
+
+#include <process/collect.hpp>
+#include <process/delay.hpp>
+#include <process/process.hpp>
+#include <process/reap.hpp>
+
+#include <stout/unreachable.hpp>
+
+#include "mesos/resources.hpp"
+
+#include "slave/containerizer/launcher.hpp"
+
+using namespace process;
+
+using std::list;
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+using state::RunState;
+
+Try<Launcher*> PosixLauncher::create(const Flags& flags)
+{
+  return new PosixLauncher();
+}
+
+
+Try<Nothing> PosixLauncher::recover(const list<RunState>& states)
+{
+  foreach (const RunState& state, states) {
+    if (state.id.isNone()) {
+      return Error("ContainerID is required to recover");
+    }
+
+    const ContainerID& containerId = state.id.get();
+
+    if (state.forkedPid.isNone()) {
+      return Error("Executor pid is required to recover container " +
+                   stringify(containerId));
+    }
+    pid_t pid = state.forkedPid.get();
+
+    if (pids.containsValue(pid)) {
+      // This should (almost) never occur. There is the possibility that a new
+      // executor is launched with the same pid as one that just exited (highly
+      // unlikely) and the slave dies after the new executor is launched but
+      // before it hears about the termination of the earlier executor (also
+      // unlikely). Regardless, the launcher can't do anything sensible so this
+      // is considered an error.
+      return Error("Detected duplicate pid " + stringify(pid) +
+                   " for container " + stringify(containerId));
+    }
+
+    pids.put(containerId, pid);
+  }
+
+  return Nothing();
+}
+
+
+Try<pid_t> PosixLauncher::fork(
+    const ContainerID& containerId,
+    const lambda::function<int()>& inChild)
+{
+  if (pids.contains(containerId)) {
+    return Error("Process has already been forked for container " +
+                 stringify(containerId));
+  }
+
+  pid_t pid;
+
+  if ((pid = ::fork()) == -1) {
+    return ErrnoError("Failed to fork");
+  }
+
+  if (pid > 0) {
+    // In parent.
+    LOG(INFO) << "Forked child with pid '" << pid
+              << "' for container '" << containerId << "'";
+    // Store the pid (session id and process group id).
+    pids.put(containerId, pid);
+
+    return pid;
+  } else {
+    // In child.
+    // POSIX guarantees a forked child's pid does not match any existing
+    // process group id so only a single setsid() is required and the session
+    // id will be the pid.
+    // TODO(idownes): perror is not listed as async-signal-safe and should be
+    // reimplemented safely.
+    if (setsid() == -1) {
+      perror("Failed to put child in a new session");
+      _exit(1);
+    }
+
+    // This function should exec() and therefore not return.
+    inChild();
+
+    return UNREACHABLE();
+  }
+}
+
+
+Future<Nothing> _destroy(const Future<Option<int> >& future)
+{
+  if (future.isReady()) {
+    return Nothing();
+  } else {
+    return Failure("Failed to kill all processes: " +
+                   (future.isFailed() ? future.failure() : "unknown error"));
+  }
+}
+
+
+Future<Nothing> PosixLauncher::destroy(const ContainerID& containerId)
+{
+  if (!pids.contains(containerId)) {
+    return Failure("Unknown container " + containerId.value());
+  }
+
+  pid_t pid = pids.get(containerId).get();
+
+  // Kill all processes in the session and process group.
+  Try<list<os::ProcessTree> > trees =
+    os::killtree(pid, SIGKILL, true, true);
+
+  pids.erase(containerId);
+
+  // The child process may not have been waited on yet so we'll delay
+  // completing destroy until we're sure it has been reaped.
+  return process::reap(pid)
+    .then(lambda::bind(&_destroy, lambda::_1));
+}
+
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/71c6b5c0/src/slave/containerizer/launcher.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/launcher.hpp b/src/slave/containerizer/launcher.hpp
new file mode 100644
index 0000000..dee526f
--- /dev/null
+++ b/src/slave/containerizer/launcher.hpp
@@ -0,0 +1,88 @@
+/**
+ * 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 __LAUNCHER_HPP__
+#define __LAUNCHER_HPP__
+
+#include <list>
+
+#include <process/future.hpp>
+
+#include <stout/lambda.hpp>
+#include <stout/try.hpp>
+
+#include "slave/flags.hpp"
+#include "slave/state.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class Launcher
+{
+public:
+  virtual ~Launcher() {}
+
+  // Recover the necessary state for each container listed in state.
+  virtual Try<Nothing> recover(const std::list<state::RunState>& states) = 0;
+
+  // Fork a new process in the containerized context. The child will call the
+  // specified function and the parent will return the child's pid.
+  // NOTE: The function must be async-signal safe and should exec as soon as
+  // possible.
+  virtual Try<pid_t> fork(
+      const ContainerID& containerId,
+      const lambda::function<int()>&) = 0;
+
+  // Kill all processes in the containerized context.
+  virtual process::Future<Nothing> destroy(const ContainerID& containerId) = 0;
+};
+
+
+// Launcher suitable for any POSIX compliant system. Uses process groups and
+// sessions to track processes in a container. POSIX states that process groups
+// cannot migrate between sessions so all processes for a container will be
+// contained in a session.
+class PosixLauncher : public Launcher
+{
+public:
+  static Try<Launcher*> create(const Flags& flags);
+
+  virtual ~PosixLauncher() {}
+
+  virtual Try<Nothing> recover(const std::list<state::RunState>& states);
+
+  virtual Try<pid_t> fork(
+      const ContainerID& containerId,
+      const lambda::function<int()>& inChild);
+
+  virtual process::Future<Nothing> destroy(const ContainerID& containerId);
+
+private:
+  PosixLauncher() {}
+
+  // The 'pid' is the process id of the first process and also the process
+  // group id and session id.
+  hashmap<ContainerID, pid_t> pids;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __LAUNCHER_HPP__


[07/11] Containerizer (part 1)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/cgroups_isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/cgroups_isolator.cpp b/src/slave/cgroups_isolator.cpp
deleted file mode 100644
index ef7dd68..0000000
--- a/src/slave/cgroups_isolator.cpp
+++ /dev/null
@@ -1,1412 +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.
- */
-
-#include <math.h> // For floor.
-#include <signal.h>
-#include <unistd.h>
-
-#include <sys/file.h> // For flock.
-#include <sys/types.h>
-
-#include <algorithm>
-#include <set>
-#include <sstream>
-#include <string>
-#include <vector>
-
-#include <process/clock.hpp>
-#include <process/defer.hpp>
-#include <process/dispatch.hpp>
-#include <process/reap.hpp>
-
-#include <stout/bytes.hpp>
-#include <stout/check.hpp>
-#include <stout/duration.hpp>
-#include <stout/error.hpp>
-#include <stout/exit.hpp>
-#include <stout/foreach.hpp>
-#include <stout/hashmap.hpp>
-#include <stout/hashset.hpp>
-#include <stout/lambda.hpp>
-#include <stout/none.hpp>
-#include <stout/nothing.hpp>
-#include <stout/numify.hpp>
-#include <stout/option.hpp>
-#include <stout/os.hpp>
-#include <stout/path.hpp>
-#include <stout/proc.hpp>
-#include <stout/stringify.hpp>
-#include <stout/strings.hpp>
-#include <stout/uuid.hpp>
-
-#include "linux/cgroups.hpp"
-
-#include "slave/cgroups_isolator.hpp"
-#include "slave/state.hpp"
-
-using process::defer;
-using process::Failure;
-using process::Future;
-
-using std::list;
-using std::map;
-using std::set;
-using std::string;
-using std::ostringstream;
-using std::vector;
-
-namespace mesos {
-namespace internal {
-namespace slave {
-
-using state::SlaveState;
-using state::FrameworkState;
-using state::ExecutorState;
-using state::RunState;
-
-// CPU subsystem constants.
-const size_t CPU_SHARES_PER_CPU = 1024;
-const size_t MIN_CPU_SHARES = 10;
-const Duration CPU_CFS_PERIOD = Milliseconds(100); // Linux default.
-const Duration MIN_CPU_CFS_QUOTA = Milliseconds(1);
-
-// Memory subsystem constants.
-const Bytes MIN_MEMORY = Megabytes(32);
-
-
-// This is an approximate double precision equality check.
-// It only considers up to 0.001 precision.
-// This is used so that we can enforce correct arithmetic on "millicpu" units.
-// TODO(bmahler): Banish this to hell when we expose individual cpus as a
-// resource to frameworks, so that we can enforce having no fractions.
-bool almostEqual(double d1, double d2) {
-  return (d1 <= (d2 + 0.001)) && (d1 >= (d2 - 0.001));
-}
-
-
-map<proc::CPU, double> Cpuset::grow(
-    double delta,
-    const map<proc::CPU, double>& usage)
-{
-  // The technique used here is to allocate as much as possible to
-  // each cpu that has availability, until we've allocated the delta.
-  // Note that we examine the cpus in the same order every time, which
-  // means we don't yet consider locality.
-  map<proc::CPU, double> allocation;
-  foreachpair (const proc::CPU& cpu, double used, usage) {
-    // Are we done allocating?
-    if (almostEqual(delta, 0.0)) {
-      break;
-    }
-
-    // Allocate as much as possible to this CPU.
-    if (!almostEqual(used, 1.0)) {
-      double free = 1.0 - used;
-      double allocated = std::min(delta, free);
-      allocation[cpu] = allocated;
-      delta -= allocated;
-      cpus[cpu] += allocated;
-    }
-  }
-
-  CHECK(almostEqual(delta, 0.0))
-    << "Failed to grow the cpuset by " << delta << " cpus\n"
-    << "  cpus: " << stringify(cpus) << "\n"
-    << "  usage: " << stringify(usage);
-
-  return allocation;
-}
-
-
-map<proc::CPU, double> Cpuset::shrink(double delta)
-{
-  // The technique used here is to free as much as possible from the
-  // least allocated cpu. This means we'll avoid fragmenting as we're
-  // constantly trying to remove cpus belonging to this Cpuset.
-  map<proc::CPU, double> deallocation;
-  while (!almostEqual(delta, 0.0)) {
-    // Find the CPU to which we have the least allocated.
-    Option<proc::CPU> least;
-    foreachpair (const proc::CPU& cpu, double used, cpus) {
-      if (least.isNone() || used <= cpus[least.get()]) {
-        least = cpu;
-      }
-    }
-
-    CHECK(least.isSome())
-      << "Failed to shrink the cpuset by " << delta << " cpus\n"
-      << "  cpus: " << stringify(cpus);
-
-    // Deallocate as much as possible from the least allocated CPU.
-    double used = cpus[least.get()];
-    double deallocated = std::min(used, delta);
-    deallocation[least.get()] = deallocated;
-    delta -= deallocated;
-    cpus[least.get()] -= deallocated;
-
-    // Ensure this Cpuset never contains unallocated CPUs.
-    if (almostEqual(cpus[least.get()], 0.0)) {
-      cpus.erase(least.get());
-    }
-  }
-
-  return deallocation;
-}
-
-
-double Cpuset::usage() const
-{
-  double total = 0.0;
-  foreachvalue (double used, cpus) {
-    total += used;
-  }
-  return total;
-}
-
-
-std::ostream& operator << (std::ostream& out, const Cpuset& cpuset)
-{
-  vector<unsigned int> cpus;
-  foreachpair (const proc::CPU& cpu, double used, cpuset.cpus) {
-    CHECK(!almostEqual(used, 0.0));
-    cpus.push_back(cpu.id);
-  }
-  std::sort(cpus.begin(), cpus.end());
-
-  return out << strings::join(",", cpus);
-}
-
-
-CgroupsIsolator::CgroupsIsolator()
-  : ProcessBase(ID::generate("cgroups-isolator")),
-    local(false),
-    initialized(false),
-    lockFile(None()) {}
-
-
-void CgroupsIsolator::initialize(
-    const Flags& _flags,
-    const Resources& _resources,
-    bool _local,
-    const PID<Slave>& _slave)
-{
-  flags = _flags;
-  local = _local;
-  slave = _slave;
-
-  // Make sure that cgroups is enabled by the kernel.
-  if (!cgroups::enabled()) {
-    EXIT(1) << "No cgroups support detected in this kernel";
-  }
-
-  // Make sure that we have root permissions.
-  if (geteuid() != 0) {
-    EXIT(1) << "Using cgroups requires root permissions";
-  }
-
-  // Configure cgroups hierarchy root path.
-  hierarchy = flags.cgroups_hierarchy;
-
-  LOG(INFO) << "Using " << hierarchy << " as cgroups hierarchy root";
-
-  // Determine desired subsystems.
-  foreach (const string& subsystem,
-           strings::tokenize(flags.cgroups_subsystems, ",")) {
-    // TODO(benh): Implement a 'sets::union' that takes a vector or
-    // set rather than looping here!
-    subsystems.insert(subsystem);
-  }
-
-  // Regardless of whether or not it was desired, we require the
-  // 'freezer' subsystem in order to destroy a cgroup.
-  subsystems.insert("freezer");
-
-  // We require the 'cpuacct' subsystem to perform resource monitoring.
-  subsystems.insert("cpuacct");
-
-  // Check if the hierarchy is already mounted, and if not, mount it.
-  Try<bool> mounted = cgroups::mounted(hierarchy);
-
-  if (mounted.isError()) {
-    LOG(FATAL) << "Failed to determine if " << hierarchy
-               << " is already mounted: " << mounted.error();
-  } else if (mounted.get()) {
-    // Make sure that all the desired subsystems are attached to the
-    // already mounted hierarchy.
-    Try<set<string> > attached = cgroups::subsystems(hierarchy);
-
-    if (attached.isError()) {
-      LOG(FATAL) << "Failed to determine the attached subsystems "
-                 << "for the cgroup hierarchy at " << hierarchy << ": "
-                 << attached.error();
-    }
-    foreach (const string& subsystem, subsystems) {
-      if (attached.get().count(subsystem) == 0) {
-        EXIT(1) << "The cgroups hierarchy at " << hierarchy
-                << " can not be used because it does not have the '"
-                << subsystem << "' subsystem attached";
-      }
-    }
-  } else {
-    // Attempt to mount the hierarchy ourselves.
-    if (os::exists(hierarchy)) {
-      // The path specified by the given hierarchy already exists in
-      // the file system. We try to remove it if it is an empty
-      // directory. This will helps us better deal with slave restarts
-      // since we won't need to manually remove the directory.
-      Try<Nothing> rmdir = os::rmdir(hierarchy, false);
-      if (rmdir.isError()) {
-        EXIT(1) << "Failed to mount cgroups hierarchy at '" << hierarchy
-                << "' because we could not remove existing directory"
-                << ": " << rmdir.error();
-      }
-    }
-
-    // Mount the cgroups hierarchy.
-    Try<Nothing> mount = cgroups::mount(
-        hierarchy, strings::join(",", subsystems));
-
-    if (mount.isError()) {
-      EXIT(1) << "Failed to mount cgroups hierarchy at '" << hierarchy
-              << "': " << mount.error();
-    }
-  }
-
-  // Create the root cgroup if it doesn't exist.
-  Try<bool> exists = cgroups::exists(hierarchy, flags.cgroups_root);
-  CHECK_SOME(exists)
-    << "Failed to determine if '"<< flags.cgroups_root << "' cgroup "
-    << "already exists in the hierarchy at '" << hierarchy << "'";
-
-  if (!exists.get()) {
-    // No root cgroup exists, create it.
-    Try<Nothing> create = cgroups::create(hierarchy, flags.cgroups_root);
-    CHECK_SOME(create)
-      << "Failed to create the '" << flags.cgroups_root << "' cgroup";
-  }
-
-  // Create the nested test cgroup if it doesn't exist.
-  exists = cgroups::exists(
-      hierarchy, path::join(flags.cgroups_root, "test"));
-  CHECK_SOME(exists)
-    << "Failed to determine if '"<< flags.cgroups_root << "/test'"
-    << " nested cgroup already exists in the hierarchy at '"
-    << hierarchy << "'";
-
-  if (!exists.get()) {
-    // Make sure this kernel supports creating nested cgroups.
-    Try<Nothing> create =
-      cgroups::create(hierarchy, path::join(flags.cgroups_root, "test"));
-
-    if (create.isError()) {
-      EXIT(1) << "Failed to create a nested 'test' cgroup. Your kernel "
-        << "might be too old to use the cgroups isolator: "
-        << create.error();
-    }
-  }
-
-  // Remove the nested 'test' cgroup.
-  Try<Nothing> remove =
-    cgroups::remove(hierarchy, path::join(flags.cgroups_root, "test"));
-
-  CHECK_SOME(remove) << "Failed to remove the nested 'test' cgroup";
-
-  // Try and put an _advisory_ file lock on the tasks' file of our
-  // root cgroup to check and see if another slave is already running.
-  Try<int> open =
-    os::open(path::join(hierarchy, flags.cgroups_root, "tasks"), O_RDONLY);
-
-  CHECK_SOME(open);
-
-  lockFile = open.get();
-  Try<Nothing> cloexec = os::cloexec(lockFile.get());
-  CHECK_SOME(cloexec);
-  if (flock(lockFile.get(), LOCK_EX | LOCK_NB) != 0) {
-    EXIT(1) << "Another mesos-slave appears to be running!";
-  }
-
-  // Make sure the kernel supports OOM controls.
-  exists = cgroups::exists(
-      hierarchy, flags.cgroups_root, "memory.oom_control");
-
-  CHECK_SOME(exists)
-    << "Failed to determine if 'memory.oom_control' control exists";
-
-  if (!exists.get()) {
-    EXIT(1) << "Failed to find 'memory.oom_control', your kernel "
-            << "might be too old to use the cgroups isolator";
-  }
-
-  // Make sure the memcg OOM-killer is not disabled.
-  // The Mesos OOM handler, as implemented, is not capable of handling
-  // the oom condition by itself safely given the limitations Linux
-  // imposes on this code path.
-  Try<Nothing> write = cgroups::write(
-      hierarchy, flags.cgroups_root, "memory.oom_control", "0");
-
-  CHECK_SOME(write) << "Failed to update memory.oom_control";
-
-  if (subsystems.contains("cpu") && subsystems.contains("cpuset")) {
-    EXIT(1) << "The use of both 'cpu' and 'cpuset' subsystems is not allowed.\n"
-            << "Please use only one of:\n"
-            << "  cpu:    When willing to share cpus for higher efficiency.\n"
-            << "  cpuset: When cpu pinning is desired.";
-  }
-
-  // Configure resource changed handlers. We only add handlers for
-  // resources that have the appropriate subsystems attached.
-  if (subsystems.contains("cpu")) {
-    handlers["cpus"] = &CgroupsIsolator::cpusChanged;
-  }
-
-  if (subsystems.contains("cpuset")) {
-    // TODO(bmahler): Consider making a cgroups primitive helper to perform
-    // cgroups list format -> list of ints / strings conversion.
-    hashset<unsigned int> cgroupCpus;
-    Try<string> cpuset =
-      cgroups::read(hierarchy, flags.cgroups_root, "cpuset.cpus");
-
-    CHECK_SOME(cpuset) << "Failed to read cpuset.cpus";
-    cpuset = strings::trim(cpuset.get());
-
-    // Parse from "0-2,7,12-14" to a set(0,1,2,7,12,13,14).
-    foreach (string range, strings::tokenize(cpuset.get(), ",")) {
-      range = strings::trim(range);
-
-      if (strings::contains(range, "-")) {
-        // Case startId-endId (e.g. 0-2 in 0-2,7,12-14).
-        vector<string> startEnd = strings::split(range, "-");
-        CHECK(startEnd.size() == 2)
-          << "Failed to parse cpu range '" << range
-          << "' from cpuset.cpus '" << cpuset.get() << "'";
-
-        Try<unsigned int> start =
-          numify<unsigned int>(strings::trim(startEnd[0]));
-        Try<unsigned int> end =
-          numify<unsigned int>(strings::trim(startEnd[1]));
-
-        CHECK(start.isSome() && end.isSome())
-          << "Failed to parse cpu range '" << range
-          << "' from cpuset.cpus '" << cpuset.get() << "'";
-
-        for (unsigned int i = start.get(); i <= end.get(); i++) {
-          cgroupCpus.insert(i);
-        }
-      } else {
-        // Case id (e.g. 7 in 0-2,7,12-14).
-        Try<unsigned int> cpuId = numify<unsigned int>(range);
-
-        CHECK_SOME(cpuId)
-          << "Failed to parse cpu '" << range << "' from cpuset.cpus '"
-          << cpuset.get()  << "'";
-
-        cgroupCpus.insert(cpuId.get());
-      }
-    }
-
-    Value::Scalar none;
-    Value::Scalar cpusResource = _resources.get("cpus", none);
-    if (cpusResource.value() > cgroupCpus.size()) {
-      EXIT(1) << "You have specified " << cpusResource.value() << " cpus, but "
-              << "this is more than allowed by the cgroup cpuset.cpus: "
-              << cpuset.get();
-    }
-
-    // Initialize our cpu allocations.
-    Try<list<proc::CPU> > cpus = proc::cpus();
-
-    CHECK_SOME(cpus) << "Failed to extract CPUs from /proc/cpuinfo";
-
-    foreach (const proc::CPU& cpu, cpus.get()) {
-      if (this->cpus.size() >= cpusResource.value()) {
-        break;
-      }
-
-      if (cgroupCpus.contains(cpu.id)) {
-        LOG(INFO) << "Initializing cpu allocation for " << cpu;
-        this->cpus[cpu] = 0.0;
-      }
-    }
-
-    handlers["cpus"] = &CgroupsIsolator::cpusetChanged;
-  }
-
-  if (subsystems.contains("memory")) {
-    handlers["mem"] = &CgroupsIsolator::memChanged;
-  }
-
-  // Add handlers for optional subsystem features.
-  if (flags.cgroups_enable_cfs) {
-    // Verify dependent subsystem is present and kernel supports CFS controls.
-    if (!subsystems.contains("cpu")) {
-      EXIT(1) << "The 'cfs' cgroups feature flag is dependent on the 'cpu' "
-              << "subsystem.\n"
-              << "Please enable the cpu subsystem to use the cfs feature.";
-    }
-
-    exists = cgroups::exists(hierarchy, flags.cgroups_root, "cpu.cfs_quota_us");
-
-    CHECK_SOME(exists)
-      << "Failed to determine if 'cpu.cfs_quota_us' control exists";
-
-    if (!exists.get()) {
-      EXIT(1) << "Failed to find 'cpu.cfs_quota_us'. Your kernel "
-              << "might be too old to use the CFS cgroups feature";
-    }
-
-    // Make "cfsChanged" the cpu resource handler.
-    // TODO(tdmackey): Allow multiple handlers per resource.
-    handlers["cpus"] = &CgroupsIsolator::cfsChanged;
-  }
-
-  initialized = true;
-}
-
-
-void CgroupsIsolator::finalize()
-{
-  // Unlock the advisory file.
-  CHECK_SOME(lockFile) << "Uninitialized file descriptor!";
-  if (flock(lockFile.get(), LOCK_UN) != 0) {
-    PLOG(FATAL)
-      << "Failed to unlock advisory lock file '"
-      << path::join(hierarchy, flags.cgroups_root, "tasks") << "'";
-  }
-
-  Try<Nothing> close = os::close(lockFile.get());
-  if (close.isError()) {
-    LOG(ERROR) << "Failed to close advisory lock file '"
-               << path::join(hierarchy, flags.cgroups_root, "tasks")
-               << "': " << close.error();
-  }
-}
-
-
-void CgroupsIsolator::launchExecutor(
-    const SlaveID& slaveId,
-    const FrameworkID& frameworkId,
-    const FrameworkInfo& frameworkInfo,
-    const ExecutorInfo& executorInfo,
-    const UUID& uuid,
-    const string& directory,
-    const Resources& resources)
-{
-  CHECK(initialized) << "Cannot launch executors before initialization";
-
-  const ExecutorID& executorId = executorInfo.executor_id();
-
-  // Register the cgroup information.
-  CgroupInfo* info =
-    registerCgroupInfo(frameworkId, executorId, uuid, None(), flags);
-
-  LOG(INFO) << "Launching " << executorId
-            << " (" << executorInfo.command().value() << ")"
-            << " in " << directory
-            << " with resources " << resources
-            << " for framework " << frameworkId
-            << " in cgroup " << info->name();
-
-  // Create a new cgroup for the executor.
-  Try<Nothing> create = cgroups::create(hierarchy, info->name());
-
-  if (create.isError()) {
-    LOG(FATAL) << "Failed to create cgroup for executor " << executorId
-               << " of framework " << frameworkId
-               << ": " << create.error();
-  }
-
-  // Setup the initial resource constraints.
-  resourcesChanged(frameworkId, executorId, resources);
-
-  // Start listening on OOM events.
-  oomListen(frameworkId, executorId);
-
-  // Use pipes to determine which child has successfully changed session.
-  int pipes[2];
-  if (pipe(pipes) < 0) {
-    PLOG(FATAL) << "Failed to create a pipe";
-  }
-
-  // Set the FD_CLOEXEC flags on these pipes
-  Try<Nothing> cloexec = os::cloexec(pipes[0]);
-  CHECK_SOME(cloexec) << "Error setting FD_CLOEXEC on pipe[0]";
-
-  cloexec = os::cloexec(pipes[1]);
-  CHECK_SOME(cloexec) << "Error setting FD_CLOEXEC on pipe[1]";
-
-  // Launch the executor using fork-exec.
-  pid_t pid;
-  if ((pid = ::fork()) == -1) {
-    LOG(FATAL) << "Failed to fork to launch new executor";
-  }
-
-  if (pid > 0) {
-    os::close(pipes[1]);
-
-    // Get the child's pid via the pipe.
-    if (read(pipes[0], &pid, sizeof(pid)) == -1) {
-      PLOG(FATAL) << "Failed to get child PID from pipe";
-    }
-
-    os::close(pipes[0]);
-
-    // In parent process.
-    LOG(INFO) << "Forked executor at = " << pid;
-
-    // Store the pid of the leading process of the executor.
-    info->pid = pid;
-
-    process::reap(pid)
-      .onAny(defer(PID<CgroupsIsolator>(this),
-                   &CgroupsIsolator::reaped,
-                   pid,
-                   lambda::_1));
-
-    // Tell the slave this executor has started.
-    dispatch(slave,
-             &Slave::executorStarted,
-             frameworkId,
-             executorId,
-             pid);
-  } else {
-    // In child process, we make cleanup easier by putting process
-    // into it's own session. DO NOT USE GLOG!
-    os::close(pipes[0]);
-
-    // NOTE: We setsid() in a loop because setsid() might fail if another
-    // process has the same process group id as the calling process.
-    while ((pid = setsid()) == -1) {
-      perror("Could not put executor in its own session");
-
-      std::cout << "Forking another process and retrying ..." << std::endl;
-
-      if ((pid = fork()) == -1) {
-        perror("Failed to fork to launch executor");
-        abort();
-      }
-
-      if (pid > 0) {
-        // In parent process.
-        exit(0);
-      }
-    }
-
-    if (write(pipes[1], &pid, sizeof(pid)) != sizeof(pid)) {
-      perror("Failed to write PID on pipe");
-      abort();
-    }
-
-    os::close(pipes[1]);
-
-    launcher::ExecutorLauncher launcher(
-        slaveId,
-        frameworkId,
-        executorInfo.executor_id(),
-        uuid,
-        executorInfo.command(),
-        frameworkInfo.user(),
-        directory,
-        flags.work_dir,
-        slave,
-        flags.frameworks_home,
-        flags.hadoop_home,
-        !local,
-        flags.switch_user,
-        frameworkInfo.checkpoint(),
-        flags.recovery_timeout);
-
-    // First fetch the executor.
-    if (launcher.setup() < 0) {
-      EXIT(1) << "Failed to setup executor '" << executorId
-              << "' for framework " << frameworkId;
-    }
-
-    // Put self into the newly created cgroup.
-    // Note that the memory used for setting up the executor
-    // (launcher.setup()) is charged to the slave's cgroup and
-    // not to the executor's cgroup. When we assign the executor
-    // to the its own cgroup, below, its memory charge will start
-    // at 0. For more details, refer to
-    // http://www.kernel.org/doc/Documentation/cgroups/memory.txt
-    Try<Nothing> assign = cgroups::assign(hierarchy, info->name(), ::getpid());
-
-    if (assign.isError()) {
-      EXIT(1) << "Failed to assign executor '" << executorId
-              << "' of framework " << frameworkId
-              << " to its own cgroup '" << path::join(hierarchy, info->name())
-              << "' : " << assign.error();
-    }
-
-    // Now launch the executor (this function should not return).
-    launcher.launch();
-  }
-}
-
-
-void CgroupsIsolator::killExecutor(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId)
-{
-  CHECK(initialized) << "Cannot kill executors before initialization";
-
-  CgroupInfo* info = findCgroupInfo(frameworkId, executorId);
-  if (info == NULL || info->killed) {
-    LOG(ERROR) << "Asked to kill an unknown/killed executor!";
-    return;
-  }
-
-  LOG(INFO) << "Killing executor " << executorId
-            << " of framework " << frameworkId;
-
-  // Stop the OOM listener if needed.
-  if (info->oomNotifier.isPending()) {
-    info->oomNotifier.discard();
-  }
-
-  info->killed = true;
-
-  // Destroy the cgroup that is associated with the executor. Here, we
-  // don't wait for it to succeed as we don't want to block the
-  // isolator. Instead, we register a callback which will be invoked
-  // when its result is ready.
-  cgroups::destroy(hierarchy, info->name())
-    .onAny(defer(PID<CgroupsIsolator>(this),
-                 &CgroupsIsolator::_killExecutor,
-                 info,
-                 lambda::_1));
-}
-
-
-void CgroupsIsolator::resourcesChanged(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId,
-    const Resources& resources)
-{
-  CHECK(initialized) << "Cannot change resources before initialization";
-
-  CgroupInfo* info = findCgroupInfo(frameworkId, executorId);
-  if (info == NULL || info->killed) {
-    LOG(INFO) << "Asked to update resources for an unknown/killed executor";
-    return;
-  }
-
-  if (info->resources == resources) {
-    // This could happen when 'resourcesChanged()' is called after
-    // launching the first task since the executor includes the
-    // resources for the first task.
-    return;
-  }
-
-  info->resources = resources;
-
-  LOG(INFO) << "Changing cgroup controls for executor " << executorId
-            << " of framework " << frameworkId
-            << " with resources " << resources;
-
-  // For each resource, invoke the corresponding handler.
-  foreach (const Resource& resource, resources) {
-    if (handlers.contains(resource.name())) {
-      Try<Nothing> result = (this->*handlers[resource.name()])(info, resource);
-
-      if (result.isError()) {
-        LOG(ERROR) << result.error();
-      }
-    }
-  }
-}
-
-
-Future<ResourceStatistics> CgroupsIsolator::usage(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId)
-{
-  if (!infos.contains(frameworkId) ||
-      !infos[frameworkId].contains(executorId) ||
-      infos[frameworkId][executorId]->killed) {
-    return Failure("Unknown or killed executor");
-  }
-
-  // Get the number of clock ticks, used for cpu accounting.
-  static long ticks = sysconf(_SC_CLK_TCK);
-
-  PCHECK(ticks > 0) << "Failed to get sysconf(_SC_CLK_TCK)";
-
-  CgroupInfo* info = infos[frameworkId][executorId];
-  CHECK_NOTNULL(info);
-
-  ResourceStatistics result;
-  result.set_timestamp(Clock::now().secs());
-
-  // Set the resource allocations.
-  Option<Bytes> mem = info->resources.mem();
-  if (mem.isSome()) {
-    result.set_mem_limit_bytes(mem.get().bytes());
-  }
-
-  Option<double> cpus = info->resources.cpus();
-  if (cpus.isSome()) {
-    result.set_cpus_limit(cpus.get());
-  }
-
-  Try<hashmap<string, uint64_t> > stat =
-    cgroups::stat(hierarchy, info->name(), "cpuacct.stat");
-
-  if (stat.isError()) {
-    return Failure("Failed to read cpuacct.stat: " + stat.error());
-  }
-
-  // TODO(bmahler): Add namespacing to cgroups to enforce the expected
-  // structure, e.g., cgroups::cpuacct::stat.
-  if (stat.get().contains("user") && stat.get().contains("system")) {
-    result.set_cpus_user_time_secs(
-        (double) stat.get()["user"] / (double) ticks);
-    result.set_cpus_system_time_secs(
-        (double) stat.get()["system"] / (double) ticks);
-  }
-
-  // The rss from memory.stat is wrong in two dimensions:
-  //   1. It does not include child cgroups.
-  //   2. It does not include any file backed pages.
-  Try<Bytes> usage = cgroups::memory::usage_in_bytes(hierarchy, info->name());
-  if (usage.isError()) {
-    return Failure("Failed to parse memory.usage_in_bytes: " + usage.error());
-  }
-
-  // TODO(bmahler): Add namespacing to cgroups to enforce the expected
-  // structure, e.g, cgroups::memory::stat.
-  result.set_mem_rss_bytes(usage.get().bytes());
-
-  stat = cgroups::stat(hierarchy, info->name(), "memory.stat");
-  if (stat.isError()) {
-    return Failure("Failed to read memory.stat: " + stat.error());
-  }
-
-  if (stat.get().contains("total_cache")) {
-    result.set_mem_file_bytes(stat.get()["total_cache"]);
-  }
-
-  if (stat.get().contains("total_rss")) {
-    result.set_mem_anon_bytes(stat.get()["total_rss"]);
-  }
-
-  if (stat.get().contains("total_mapped_file")) {
-    result.set_mem_mapped_file_bytes(stat.get()["total_mapped_file"]);
-  }
-
-  // Add the cpu.stat information.
-  stat = cgroups::stat(hierarchy, info->name(), "cpu.stat");
-
-  if (stat.isError()) {
-    return Failure("Failed to read cpu.stat: " + stat.error());
-  }
-
-  if (stat.get().contains("nr_periods")) {
-    result.set_cpus_nr_periods(
-        (uint32_t) stat.get()["nr_periods"]);
-  }
-
-  if (stat.get().contains("nr_throttled")) {
-    result.set_cpus_nr_throttled(
-        (uint32_t) stat.get()["nr_throttled"]);
-  }
-
-  if (stat.get().contains("throttled_time")) {
-    result.set_cpus_throttled_time_secs(
-        Nanoseconds(stat.get()["throttled_time"]).secs());
-  }
-
-  return result;
-}
-
-
-Future<Nothing> CgroupsIsolator::recover(
-    const Option<SlaveState>& state)
-{
-  LOG(INFO) << "Recovering isolator";
-
-  hashset<std::string> cgroups; // Recovered cgroups.
-
-  if (state.isSome()) {
-    foreachvalue (const FrameworkState& framework, state.get().frameworks) {
-      foreachvalue (const ExecutorState& executor, framework.executors) {
-        LOG(INFO) << "Recovering executor '" << executor.id
-                  << "' of framework " << framework.id;
-
-        if (executor.info.isNone()) {
-          LOG(WARNING) << "Skipping recovery of executor '" << executor.id
-                       << "' of framework " << framework.id
-                       << " because its info cannot be recovered";
-          continue;
-        }
-
-        if (executor.latest.isNone()) {
-          LOG(WARNING) << "Skipping recovery of executor '" << executor.id
-                       << "' of framework " << framework.id
-                       << " because its latest run cannot be recovered";
-          continue;
-        }
-
-        // We are only interested in the latest run of the executor!
-        const UUID& uuid = executor.latest.get();
-        CHECK(executor.runs.contains(uuid));
-        const RunState& run = executor.runs.get(uuid).get();
-
-        if (run.completed) {
-          VLOG(1) << "Skipping recovery of executor '" << executor.id
-                  << "' of framework " << framework.id
-                  << " because its latest run " << uuid << " is completed";
-          continue;
-        }
-
-        // TODO(vinod): Currently, we assume that the cgroups
-        // information (e.g., hierarchy, root) used while recovering
-        // is same as the one that was used by the previous slave
-        // while checkpointing. Instead, we should checkpoint the
-        // cgroups information.
-        CgroupInfo* info = registerCgroupInfo(
-            framework.id, executor.id, uuid, run.forkedPid, flags);
-
-        // If the cgroup has already been removed inform the slave.
-        Try<bool> exists = cgroups::exists(hierarchy, info->name());
-        CHECK_SOME(exists) << "Failed to find the existence of cgroup "
-                           << info->name();
-        if (!exists.get()) {
-          dispatch(slave,
-                   &Slave::executorTerminated,
-                   info->frameworkId,
-                   info->executorId,
-                   info->status,
-                   info->destroyed,
-                   info->message);
-
-          unregisterCgroupInfo(framework.id, executor.id);
-
-          continue;
-        }
-
-        cgroups.insert(info->name());
-
-        // Add the pid to the reaper to monitor exit status.
-        if (run.forkedPid.isSome()) {
-          process::reap(run.forkedPid.get())
-            .onAny(defer(PID<CgroupsIsolator>(this),
-                         &CgroupsIsolator::reaped,
-                         run.forkedPid.get(),
-                         lambda::_1));
-        }
-
-        // Start listening for OOMs. If the executor OOMed while the
-        // slave was down or recovering, the cgroup will already be
-        // under_oom, resulting in immediate notification.
-        // TODO(bmahler): I've been unable to find documentation
-        // guaranteeing this, but the kernel source indicates they
-        // notify if already under_oom.
-        if (subsystems.contains("memory")) {
-          oomListen(framework.id, executor.id);
-        }
-      }
-    }
-  }
-
-  // Cleanup any orphaned cgroups that are not going to be recovered (this
-  // should be safe because we've been able to acquire the file lock).
-  Try<vector<string> > orphans = cgroups::get(hierarchy, flags.cgroups_root);
-  if (orphans.isError()) {
-    return Failure(orphans.error());
-  }
-
-  foreach (const string& orphan, orphans.get()) {
-    if (!cgroups.contains(orphan)) {
-      LOG(INFO) << "Removing orphaned cgroup '" << orphan << "'";
-      cgroups::destroy(hierarchy, orphan)
-        .onAny(defer(PID<CgroupsIsolator>(this),
-               &CgroupsIsolator::_destroy,
-               orphan,
-               lambda::_1));
-    }
-  }
-
-  return Nothing();
-}
-
-
-void CgroupsIsolator::reaped(pid_t pid, const Future<Option<int> >& status)
-{
-  CgroupInfo* info = findCgroupInfo(pid);
-  if (info != NULL) {
-    FrameworkID frameworkId = info->frameworkId;
-    ExecutorID executorId = info->executorId;
-
-    if (!status.isReady()) {
-      LOG(ERROR) << "Failed to get the status for executor " << executorId
-                 << " of framework " << frameworkId << ": "
-                 << (status.isFailed() ? status.failure() : "discarded");
-      return;
-    }
-
-    if (status.get().isSome()) {
-      int _status = status.get().get();
-      LOG(INFO) << "Executor '" << executorId
-                << "' of framework " << frameworkId
-                << (WIFEXITED(_status) ? " has exited with status "
-                                       : " has terminated with signal ")
-                << (WIFEXITED(_status) ? stringify(WEXITSTATUS(_status))
-                                       : strsignal(WTERMSIG(_status)));
-    } else {
-      LOG(WARNING) << "Executor '" << executorId
-                   << "' of framework " << frameworkId
-                   << " terminated with unknown status";
-    }
-
-    // Set the exit status, so that '_killExecutor()' can send it to the slave.
-    info->status = status.get();
-
-    if (!info->killed) {
-      killExecutor(frameworkId, executorId);
-    }
-  }
-}
-
-
-Try<Nothing> CgroupsIsolator::cpusChanged(
-    CgroupInfo* info,
-    const Resource& resource)
-{
-  CHECK(resource.name() == "cpus");
-
-  if (resource.type() != Value::SCALAR) {
-    return Error("Expecting resource 'cpus' to be a scalar");
-  }
-
-  double cpus = resource.scalar().value();
-  size_t shares =
-    std::max((size_t) (CPU_SHARES_PER_CPU * cpus), MIN_CPU_SHARES);
-
-  Try<Nothing> write = cgroups::write(
-      hierarchy, info->name(), "cpu.shares", stringify(shares));
-
-  if (write.isError()) {
-    return Error("Failed to update 'cpu.shares': " + write.error());
-  }
-
-  LOG(INFO) << "Updated 'cpu.shares' to " << shares
-            << " for executor " << info->executorId
-            << " of framework " << info->frameworkId;
-
-  return Nothing();
-}
-
-
-Try<Nothing> CgroupsIsolator::cpusetChanged(
-    CgroupInfo* info,
-    const Resource& resource)
-{
-  CHECK_NOTNULL(info->cpuset);
-  CHECK(resource.name() == "cpus");
-  CHECK(resource.type() == Value::SCALAR);
-
-  double delta = resource.scalar().value() - info->cpuset->usage();
-
-  if (delta < 0) {
-    map<proc::CPU, double> deallocated = info->cpuset->shrink(fabs(delta));
-    foreachpair (const proc::CPU& cpu, double freed, deallocated) {
-      cpus[cpu] -= freed;
-      CHECK(cpus[cpu] > -0.001); // Check approximately >= 0.
-    }
-  } else {
-    map<proc::CPU, double> allocated = info->cpuset->grow(delta, cpus);
-    foreachpair (const proc::CPU& cpu, double used, allocated) {
-      cpus[cpu] += used;
-      CHECK(cpus[cpu] < 1.001); // Check approximately <= 1.
-    }
-  }
-
-  Try<Nothing> write = cgroups::write(
-      hierarchy, info->name(), "cpuset.cpus", stringify(*(info->cpuset)));
-
-  if (write.isError()) {
-    return Error("Failed to update 'cpuset.cpus': " + write.error());
-  }
-
-  LOG(INFO) << "Updated 'cpuset.cpus' to " << *(info->cpuset)
-            << " for executor " << info->executorId
-            << " of framework " << info->frameworkId;
-
-  return Nothing();
-}
-
-
-Try<Nothing> CgroupsIsolator::cfsChanged(
-    CgroupInfo* info,
-    const Resource& resource)
-{
-  CHECK(resource.name() == "cpus");
-  CHECK(resource.type() == Value::SCALAR);
-
-  Try<Nothing> write = cgroups::write(
-      hierarchy,
-      info->name(),
-      "cpu.cfs_period_us",
-      stringify(CPU_CFS_PERIOD.us()));
-
-  if (write.isError()) {
-    return Error("Failed to update 'cpu.cfs_period_us': " + write.error());
-  }
-
-  double cpus = resource.scalar().value();
-  size_t quota = static_cast<size_t>(
-    std::max(CPU_CFS_PERIOD.us() * cpus, MIN_CPU_CFS_QUOTA.us()));
-
-  write = cgroups::write(
-      hierarchy, info->name(), "cpu.cfs_quota_us", stringify(quota));
-
-  if (write.isError()) {
-    return Error("Failed to update 'cpu.cfs_quota_us': " + write.error());
-  }
-
-  LOG(INFO) << "Updated 'cpu.cfs_period_us' to " << CPU_CFS_PERIOD.us()
-            << " and 'cpu.cfs_quota_us' to " << quota
-            << " for executor " << info->executorId
-            << " of framework " << info->frameworkId;
-
-  // Set cpu.shares as well.
-  // TODO(tdmackey): Allow multiple handlers per resource.
-  cpusChanged(info, resource);
-
-  return Nothing();
-}
-
-
-Try<Nothing> CgroupsIsolator::memChanged(
-    CgroupInfo* info,
-    const Resource& resource)
-{
-  CHECK(resource.name() == "mem");
-
-  if (resource.type() != Value::SCALAR) {
-    return Error("Expecting resource 'mem' to be a scalar");
-  }
-
-  Bytes mem = Bytes((uint64_t) resource.scalar().value() * 1024LL * 1024LL);
-  Bytes limit = std::max(mem, MIN_MEMORY);
-
-  // Always set the soft limit.
-  Try<Nothing> write =
-    cgroups::memory::soft_limit_in_bytes(hierarchy, info->name(), limit);
-
-  if (write.isError()) {
-    return Error("Failed to set 'memory.soft_limit_in_bytes': "
-        + write.error());
-  }
-
-  LOG(INFO) << "Updated 'memory.soft_limit_in_bytes' to " << limit
-            << " for executor " << info->executorId
-            << " of framework " << info->frameworkId;
-
-  // Read the existing limit.
-  Try<Bytes> currentLimit =
-    cgroups::memory::limit_in_bytes(hierarchy, info->name());
-
-  if (currentLimit.isError()) {
-    return Error(
-        "Failed to read 'memory.limit_in_bytes': " + currentLimit.error());
-  }
-
-  // Determine whether to set the hard limit. If this is the first
-  // time (info->pid.isNone()), or we're raising the existing limit,
-  // then we can update the hard limit safely. Otherwise, if we need
-  // to decrease 'memory.limit_in_bytes' we may induce an OOM if too
-  // much memory is in use. As a result, we only update the soft
-  // limit when the memory reservation is being reduced. This is
-  // probably okay if the machine has available resources.
-  // TODO(benh): Introduce a MemoryWatcherProcess which monitors the
-  // discrepancy between usage and soft limit and introduces a
-  // "manual oom" if necessary.
-  if (info->pid.isNone() || limit > currentLimit.get()) {
-    write = cgroups::memory::limit_in_bytes(hierarchy, info->name(), limit);
-
-    if (write.isError()) {
-      return Error("Failed to set 'memory.limit_in_bytes': " + write.error());
-    }
-
-    LOG(INFO) << "Updated 'memory.limit_in_bytes' to " << limit
-              << " for executor " << info->executorId
-              << " of framework " << info->frameworkId;
-  }
-
-  return Nothing();
-}
-
-
-void CgroupsIsolator::oomListen(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId)
-{
-  CgroupInfo* info = findCgroupInfo(frameworkId, executorId);
-  CHECK(info != NULL) << "Cgroup info is not registered";
-
-  info->oomNotifier =
-    cgroups::listen(hierarchy, info->name(), "memory.oom_control");
-
-  // If the listening fails immediately, something very wrong happened.
-  // Therefore, we report a fatal error here.
-  if (info->oomNotifier.isFailed()) {
-    LOG(FATAL) << "Failed to listen for OOM events for executor " << executorId
-               << " of framework " << frameworkId
-               << ": "<< info->oomNotifier.failure();
-  }
-
-  LOG(INFO) << "Started listening for OOM events for executor " << executorId
-            << " of framework " << frameworkId;
-
-  CHECK_SOME(info->uuid);
-  info->oomNotifier.onAny(
-      defer(PID<CgroupsIsolator>(this),
-            &CgroupsIsolator::oomWaited,
-            frameworkId,
-            executorId,
-            info->uuid.get(),
-            lambda::_1));
-}
-
-
-void CgroupsIsolator::oomWaited(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId,
-    const UUID& uuid,
-    const Future<uint64_t>& future)
-{
-  LOG(INFO) << "OOM notifier is triggered for executor "
-            << executorId << " of framework " << frameworkId
-            << " with uuid " << uuid;
-
-  if (future.isDiscarded()) {
-    LOG(INFO) << "Discarded OOM notifier for executor "
-              << executorId << " of framework " << frameworkId
-              << " with uuid " << uuid;
-  } else if (future.isFailed()) {
-    LOG(ERROR) << "Listening on OOM events failed for executor "
-               << executorId << " of framework " << frameworkId
-               << " with uuid " << uuid << ": " << future.failure();
-  } else {
-    // Out-of-memory event happened, call the handler.
-    oom(frameworkId, executorId, uuid);
-  }
-}
-
-
-void CgroupsIsolator::oom(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId,
-    const UUID& uuid)
-{
-  CgroupInfo* info = findCgroupInfo(frameworkId, executorId);
-  if (info == NULL) {
-    // It is likely that processExited is executed before this function (e.g.
-    // The kill and OOM events happen at the same time, and the process exit
-    // event arrives first.) Therefore, we should not report a fatal error here.
-    LOG(INFO) << "OOM detected for an already terminated executor";
-    return;
-  }
-
-  // We can also ignore an OOM event that we are late to process for a
-  // previous instance of an executor.
-  CHECK_SOME(info->uuid);
-  if (uuid != info->uuid.get()) {
-    LOG(INFO) << "OOM detected for a previous executor instance";
-    return;
-  }
-
-  // It's possible for an executor to OOM right as it was being
-  // killed, ignore this case.
-  if (info->killed) {
-    LOG(INFO) << "OOM detected for an already killed executor";
-    return;
-  }
-
-  LOG(INFO) << "OOM detected for executor " << executorId
-            << " of framework " << frameworkId
-            << " with uuid " << uuid;
-
-  // Construct a "message" string to describe why the isolator
-  // destroyed the executor's cgroup (in order to assist in debugging).
-  ostringstream message;
-  message << "Memory limit exceeded: ";
-
-  // Output the requested memory limit.
-  Try<Bytes> limit = cgroups::memory::limit_in_bytes(hierarchy, info->name());
-
-  if (limit.isError()) {
-    LOG(ERROR) << "Failed to read 'memory.limit_in_bytes': " << limit.error();
-  } else {
-    message << "Requested: " << limit.get() << " ";
-  }
-
-  // Output the maximum memory usage.
-  Try<Bytes> usage = cgroups::memory::max_usage_in_bytes(hierarchy, info->name());
-
-  if (usage.isError()) {
-    LOG(ERROR) << "Failed to read 'memory.max_usage_in_bytes': " << usage.error();
-  } else {
-    message << "Maximum Used: " << usage.get() << "\n";
-  }
-
-  // Output 'memory.stat' of the cgroup to help with debugging.
-  // NOTE: With Kernel OOM-killer enabled these stats may not reflect
-  // memory state at time of OOM.
-  Try<string> read = cgroups::read(hierarchy, info->name(), "memory.stat");
-  if (read.isError()) {
-    LOG(ERROR) << "Failed to read 'memory.stat': " << read.error();
-  } else {
-    message << "\nMEMORY STATISTICS: \n" << read.get() << "\n";
-  }
-
-  LOG(INFO) << strings::trim(message.str()); // Trim the extra '\n' at the end.
-
-  info->destroyed = true;
-  info->message = message.str();
-
-  killExecutor(frameworkId, executorId);
-}
-
-
-void CgroupsIsolator::_destroy(
-    const string& cgroup,
-    const Future<bool>& future)
-{
-  CHECK(initialized) << "Cannot destroy cgroups before initialization";
-
-  if (future.isReady()) {
-    LOG(INFO) << "Successfully destroyed cgroup " << cgroup;
-  } else {
-    LOG(FATAL) << "Failed to destroy cgroup " << cgroup
-               << ": " << future.failure();
-  }
-}
-
-
-void CgroupsIsolator::_killExecutor(
-    CgroupInfo* info,
-    const Future<bool>& future)
-{
-  CHECK(initialized) << "Cannot kill executors before initialization";
-
-  CHECK_NOTNULL(info);
-
-  if (future.isReady()) {
-    LOG(INFO) << "Successfully destroyed cgroup " << info->name();
-
-    CHECK(info->killed)
-      << "Unexpectedly alive executor " << info->executorId
-      << " of framework " << info->frameworkId;
-
-    // NOTE: The exit status of the executor might not be set if this
-    // function is called before 'processTerminated()' is called.
-    // TODO(vinod): When reaper returns a future instead of issuing a callback,
-    // wait for that future to be ready and grab the exit status.
-    dispatch(slave,
-             &Slave::executorTerminated,
-             info->frameworkId,
-             info->executorId,
-             info->status,
-             info->destroyed,
-             info->message);
-
-    // We make a copy here because 'info' will be deleted when we unregister.
-    unregisterCgroupInfo(
-        utils::copy(info->frameworkId),
-        utils::copy(info->executorId));
-  } else {
-    LOG(FATAL) << "Failed to destroy cgroup " << info->name()
-               << ": " << future.failure();
-  }
-}
-
-
-CgroupsIsolator::CgroupInfo* CgroupsIsolator::registerCgroupInfo(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId,
-    const UUID& uuid,
-    const Option<pid_t>& pid,
-    const Flags& flags)
-{
-  CgroupInfo* info = new CgroupInfo();
-  info->frameworkId = frameworkId;
-  info->executorId = executorId;
-  info->uuid = uuid;
-  info->pid = pid;
-  info->killed = false;
-  info->destroyed = false;
-  info->status = -1;
-  info->message = "";
-  info->flags = flags;
-  if (subsystems.contains("cpuset")) {
-    info->cpuset = new Cpuset();
-  } else {
-    info->cpuset = NULL;
-  }
-  infos[frameworkId][executorId] = info;
-  return info;
-}
-
-
-void CgroupsIsolator::unregisterCgroupInfo(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId)
-{
-  if (infos.contains(frameworkId)) {
-    if (infos[frameworkId].contains(executorId)) {
-      delete infos[frameworkId][executorId];
-      infos[frameworkId].erase(executorId);
-      if (infos[frameworkId].empty()) {
-        infos.erase(frameworkId);
-      }
-    }
-  }
-}
-
-
-CgroupsIsolator::CgroupInfo* CgroupsIsolator::findCgroupInfo(
-    pid_t pid)
-{
-  foreachkey (const FrameworkID& frameworkId, infos) {
-    foreachvalue (CgroupInfo* info, infos[frameworkId]) {
-      if (info->pid.isSome() && info->pid.get() == pid) {
-        return info;
-      }
-    }
-  }
-  return NULL;
-}
-
-
-CgroupsIsolator::CgroupInfo* CgroupsIsolator::findCgroupInfo(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId)
-{
-  if (infos.find(frameworkId) != infos.end()) {
-    if (infos[frameworkId].find(executorId) != infos[frameworkId].end()) {
-      return infos[frameworkId][executorId];
-    }
-  }
-  return NULL;
-}
-
-} // namespace mesos {
-} // namespace internal {
-} // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/cgroups_isolator.hpp
----------------------------------------------------------------------
diff --git a/src/slave/cgroups_isolator.hpp b/src/slave/cgroups_isolator.hpp
deleted file mode 100644
index 1a66dc6..0000000
--- a/src/slave/cgroups_isolator.hpp
+++ /dev/null
@@ -1,320 +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 __CGROUPS_ISOLATOR_HPP__
-#define __CGROUPS_ISOLATOR_HPP__
-
-#include <unistd.h>
-
-#include <map>
-#include <sstream>
-#include <string>
-
-#include <process/future.hpp>
-#include <process/pid.hpp>
-
-#include <stout/hashmap.hpp>
-#include <stout/hashset.hpp>
-#include <stout/lambda.hpp>
-#include <stout/nothing.hpp>
-#include <stout/option.hpp>
-#include <stout/path.hpp>
-#include <stout/proc.hpp>
-#include <stout/uuid.hpp>
-
-#include "launcher/launcher.hpp"
-
-#include "slave/flags.hpp"
-#include "slave/isolator.hpp"
-#include "slave/slave.hpp"
-
-namespace mesos {
-namespace internal {
-namespace slave {
-
-// TODO(bmahler): Migrate this into it's own file, along with moving
-// all cgroups code inside of a 'cgroups' directory.
-class Cpuset
-{
-public:
-  // Grows this cpu set by the provided delta.
-  // @param   delta   Amount of cpus to grow by.
-  // @param   usage   Cpu usage, as allocated by the cgroups isolator.
-  // @return  The new cpu allocations made by this Cpuset.
-  std::map<proc::CPU, double> grow(
-      double delta,
-      const std::map<proc::CPU, double>& usage);
-
-  // Shrinks this cpu set by the provided delta.
-  // @param   delta   Amount of cpus to shrink by.
-  // @return  The new cpu deallocations made by this Cpuset.
-  std::map<proc::CPU, double> shrink(double delta);
-
-  // @return The total cpu usage across all the cpus in this Cpuset.
-  double usage() const;
-
-  friend std::ostream& operator << (std::ostream& out, const Cpuset& cpuset);
-
-private:
-  std::map<proc::CPU, double> cpus; // CPU id -> % allocated.
-};
-
-
-class CgroupsIsolator : public Isolator
-{
-public:
-  CgroupsIsolator();
-
-  virtual void initialize(
-      const Flags& flags,
-      const Resources& resources,
-      bool local,
-      const process::PID<Slave>& slave);
-
-  virtual void finalize();
-
-  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);
-
-  virtual void killExecutor(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId);
-
-  virtual void resourcesChanged(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId,
-      const Resources& resources);
-
-  virtual process::Future<ResourceStatistics> usage(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId);
-
-  virtual process::Future<Nothing> recover(
-      const Option<state::SlaveState>& state);
-
-private:
-  // No copying, no assigning.
-  CgroupsIsolator(const CgroupsIsolator&);
-  CgroupsIsolator& operator = (const CgroupsIsolator&);
-
-  void reaped(pid_t pid, const Future<Option<int> >& status);
-
-  // The cgroup information for each live executor.
-  struct CgroupInfo
-  {
-    ~CgroupInfo()
-    {
-      if (cpuset != NULL) {
-        delete cpuset;
-        cpuset = NULL;
-      }
-    }
-
-    // Returns the canonicalized name of the cgroup in the filesystem.
-    std::string name() const
-    {
-      CHECK_SOME(uuid);
-      std::ostringstream out;
-      out << "framework_" << frameworkId
-          << "_executor_" << executorId
-          << "_tag_" << uuid.get();
-      return path::join(flags.cgroups_root, out.str());
-    }
-
-    FrameworkID frameworkId;
-    ExecutorID executorId;
-
-    // The UUID to distinguish between different launches of the same
-    // executor (which have the same frameworkId and executorId).
-    Option<UUID> uuid;
-
-    // PID of the forked process of the executor.
-    Option<pid_t> pid;
-
-    bool killed; // True if "killing" has been initiated via 'killExecutor()'.
-
-    // Indicates if this executor has been destroyed by the isolator.
-    // NOTE: An executor may have terminated due to reasons
-    // other than destruction by the isolator (e.g. killed by
-    // slave, exited, etc.).
-    bool destroyed;
-
-    std::string message; // The reason behind the destruction.
-
-    Option<int> status; // Exit status of the executor.
-
-    Flags flags; // Slave flags.
-
-    Resources resources; // Resources allocated to the cgroup.
-
-    // Used to cancel the OOM listening.
-    process::Future<uint64_t> oomNotifier;
-
-    // CPUs allocated if using 'cpuset' subsystem.
-    Cpuset* cpuset;
-  };
-
-  // The callback which will be invoked when "cpus" resource has changed.
-  // @param   info          The Cgroup information.
-  // @param   resources     The handle for the resources.
-  // @return  Whether the operation succeeds.
-  Try<Nothing> cpusChanged(
-      CgroupInfo* info,
-      const Resource& resource);
-
-  // The callback which will be invoked when "cpus" resource has changed.
-  // This is only invoked when we are using the cpuset subsystem.
-  // @param   info          The Cgroup information.
-  // @param   resources     The handle for the resources.
-  // @return  Whether the operation succeeds.
-  Try<Nothing> cpusetChanged(
-      CgroupInfo* info,
-      const Resource& resource);
-
-  // The callback which will be invoked when "cpus" resource has changed,
-  // and the cfs cgroups feature flag is enabled..
-  // @param   info          The Cgroup information.
-  // @param   resources     The handle for the resources.
-  // @return  Whether the operation succeeds.
-  Try<Nothing> cfsChanged(
-      CgroupInfo* info,
-      const Resource& resource);
-
-  // The callback which will be invoked when "mem" resource has changed.
-  // @param   info          The Cgroup information.
-  // @param   resources     The handle for the resources.
-  // @return  Whether the operation succeeds.
-  Try<Nothing> memChanged(
-      CgroupInfo* info,
-      const Resource& resource);
-
-  // Start listening on OOM events. This function will create an eventfd and
-  // start polling on it.
-  // @param   frameworkId   The id of the given framework.
-  // @param   executorId    The id of the given executor.
-  void oomListen(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId);
-
-  // This function is invoked when the polling on eventfd has a result.
-  // @param   frameworkId   The id of the given framework.
-  // @param   executorId    The id of the given executor.
-  // @param   uuid          The uuid of the given executor.
-  void oomWaited(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId,
-      const UUID& uuid,
-      const process::Future<uint64_t>& future);
-
-  // This function is invoked when the OOM event happens.
-  // @param   frameworkId   The id of the given framework.
-  // @param   executorId    The id of the given executor.
-  // @param   uuid          The uuid of the given executor.
-  void oom(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId,
-      const UUID& uuid);
-
-  // This callback is invoked when destroy cgroup has a result.
-  // @param   info        The information of cgroup that is being destroyed.
-  // @param   future      The future describing the destroy process.
-  void _killExecutor(
-      CgroupInfo* info,
-      const process::Future<bool>& future);
-
-  // This callback is invoked when destroying orphaned cgroups from the
-  // previous slave execution.
-  // @param   cgroup        The cgroup that is being destroyed.
-  // @param   future        The future describing the destroy process.
-  void _destroy(
-      const std::string& cgroup,
-      const process::Future<bool>& future);
-
-  // Register a cgroup in the isolator.
-  // @param   frameworkId   The id of the given framework.
-  // @param   executorId    The id of the given executor.
-  // @param   uuid          The uuid of the given executor run.
-  // @param   pid           The executor pid.
-  // @param   flags         The slave flags.
-  // @return  A pointer to the cgroup info registered.
-  CgroupInfo* registerCgroupInfo(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId,
-      const UUID& uuid,
-      const Option<pid_t>& pid,
-      const Flags& flags);
-
-  // Unregister a cgroup in the isolator.
-  // @param   frameworkId   The id of the given framework.
-  // @param   executorId    The id of the given executor.
-  void unregisterCgroupInfo(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId);
-
-  // Find a registered cgroup by the PID of the leading process.
-  // @param   pid           The PID of the leading process in the cgroup.
-  // @return  A pointer to the cgroup info if found, NULL otherwise.
-  CgroupInfo* findCgroupInfo(pid_t pid);
-
-  // Find a registered cgroup by the frameworkId and the executorId.
-  // @param   frameworkId   The id of the given framework.
-  // @param   executorId    The id of the given executor.
-  // @return  A pointer to the cgroup info if found, NULL otherwise.
-  CgroupInfo* findCgroupInfo(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId);
-
-  Flags flags;
-  bool local;
-  process::PID<Slave> slave;
-  bool initialized;
-
-  // File descriptor to 'mesos/tasks' file in the cgroup on which we place
-  // an advisory lock.
-  Option<int> lockFile;
-
-  // The cgroup information for each live executor.
-  hashmap<FrameworkID, hashmap<ExecutorID, CgroupInfo*> > infos;
-
-  // The path to the cgroups hierarchy root.
-  std::string hierarchy;
-
-  // The cgroups subsystems being used.
-  hashset<std::string> subsystems;
-
-  // Allocated cpus (if using cpuset subsystem).
-  std::map<proc::CPU, double> cpus;
-
-  // Handlers for each resource name, used for resource changes.
-  hashmap<std::string,
-          Try<Nothing>(CgroupsIsolator::*)(
-              CgroupInfo*,
-              const Resource&)> handlers;
-};
-
-} // namespace mesos {
-} // namespace internal {
-} // namespace slave {
-
-#endif // __CGROUPS_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/containerizer/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.cpp b/src/slave/containerizer/containerizer.cpp
new file mode 100644
index 0000000..d0a1023
--- /dev/null
+++ b/src/slave/containerizer/containerizer.cpp
@@ -0,0 +1,269 @@
+/**
+ * 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 <map>
+#include <vector>
+
+#include <process/dispatch.hpp>
+#include <process/owned.hpp>
+
+#include <stout/fs.hpp>
+#include <stout/hashmap.hpp>
+#include <stout/net.hpp>
+#include <stout/stringify.hpp>
+#include <stout/uuid.hpp>
+
+#include "slave/flags.hpp"
+#include "slave/slave.hpp"
+
+#ifdef __linux__
+#include "slave/containerizer/cgroups_launcher.hpp"
+#endif // __linux__
+#include "slave/containerizer/containerizer.hpp"
+#include "slave/containerizer/isolator.hpp"
+#include "slave/containerizer/launcher.hpp"
+#include "slave/containerizer/mesos_containerizer.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__
+
+using std::map;
+using std::string;
+using std::vector;
+
+using namespace process;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// TODO(idownes): Move this to the Containerizer interface to complete
+// the delegation of containerization, i.e., external containerizers should be
+// able to report the resources they can isolate.
+Try<Resources> Containerizer::resources(const Flags& flags)
+{
+  Try<Resources> parsed = Resources::parse(
+      flags.resources.get(""), flags.default_role);
+
+  if (parsed.isError()) {
+    return Error(parsed.error());
+  }
+
+  Resources resources = parsed.get();
+
+  // CPU resource.
+  if (!resources.cpus().isSome()) {
+    // No CPU specified so probe OS or resort to DEFAULT_CPUS.
+    double cpus;
+    Try<long> cpus_ = os::cpus();
+    if (!cpus_.isSome()) {
+      LOG(WARNING) << "Failed to auto-detect the number of cpus to use: '"
+                   << cpus_.error()
+                   << "'; defaulting to " << DEFAULT_CPUS;
+      cpus = DEFAULT_CPUS;
+    } else {
+      cpus = cpus_.get();
+    }
+
+    resources += Resources::parse(
+        "cpus",
+        stringify(cpus),
+        flags.default_role).get();
+  }
+
+  // Memory resource.
+  if (!resources.mem().isSome()) {
+    // No memory specified so probe OS or resort to DEFAULT_MEM.
+    Bytes mem;
+    Try<os::Memory> mem_ = os::memory();
+    if (mem_.isError()) {
+      LOG(WARNING) << "Failed to auto-detect the size of main memory: '"
+                    << mem_.error()
+                    << "' ; defaulting to DEFAULT_MEM";
+      mem = DEFAULT_MEM;
+    } else {
+      mem = mem_.get().total;
+
+      // Leave 1 GB free if we have more than 1 GB, otherwise, use all!
+      // TODO(benh): Have better default scheme (e.g., % of mem not greater
+      // than 1 GB?)
+      if (mem > Gigabytes(1)) {
+        mem = mem - Gigabytes(1);
+      }
+    }
+
+    resources += Resources::parse(
+        "mem",
+        stringify(mem.megabytes()),
+        flags.default_role).get();
+  }
+
+  // Disk resource.
+  if (!resources.disk().isSome()) {
+    // No disk specified so probe OS or resort to DEFAULT_DISK.
+    Bytes disk;
+
+    // NOTE: We calculate disk size of the file system on
+    // which the slave work directory is mounted.
+    Try<Bytes> disk_ = fs::size(flags.work_dir);
+    if (!disk_.isSome()) {
+      LOG(WARNING) << "Failed to auto-detect the disk space: '"
+                   << disk_.error()
+                   << "' ; defaulting to " << DEFAULT_DISK;
+      disk = DEFAULT_DISK;
+    } else {
+      disk = disk_.get();
+      // Leave 5 GB free if we have more than 10 GB, otherwise, use all!
+      // TODO(benh): Have better default scheme (e.g., % of disk not
+      // greater than 10 GB?)
+      if (disk > Gigabytes(10)) {
+        disk = disk - Gigabytes(5);
+      }
+    }
+
+    resources += Resources::parse(
+        "disk",
+        stringify(disk.megabytes()),
+        flags.default_role).get();
+  }
+
+  // Network resource.
+  if (!resources.ports().isSome()) {
+    // No ports specified so resort to DEFAULT_PORTS.
+    resources += Resources::parse(
+        "ports",
+        stringify(DEFAULT_PORTS),
+        flags.default_role).get();
+  }
+
+  return resources;
+}
+
+
+Try<Containerizer*> Containerizer::create(
+    const Flags& flags,
+    bool local)
+{
+  string isolation;
+  if (flags.isolation == "process") {
+    LOG(WARNING) << "The 'process' isolation flag is deprecated, "
+                 << "please update your flags to"
+                 << " '--isolation=posix/cpu,posix/mem'.";
+    isolation = "posix/cpu,posix/mem";
+  } else if (flags.isolation == "cgroups") {
+    LOG(WARNING) << "The 'cgroups' isolation flag is deprecated, "
+                 << "please update your flags to"
+                 << " '--isolation=cgroups/cpu,cgroups/mem'.";
+    isolation = "cgroups/cpu,cgroups/mem";
+  } else {
+    isolation = flags.isolation;
+  }
+
+  LOG(INFO) << "Using isolation: " << isolation;
+
+  // Create a MesosContainerizerProcess using isolators and a launcher.
+  hashmap<std::string, Try<Isolator*> (*)(const Flags&)> creators;
+
+  creators["posix/cpu"]   = &PosixCpuIsolatorProcess::create;
+  creators["posix/mem"]   = &PosixMemIsolatorProcess::create;
+#ifdef __linux__
+  creators["cgroups/cpu"] = &CgroupsCpushareIsolatorProcess::create;
+  creators["cgroups/mem"] = &CgroupsMemIsolatorProcess::create;
+#endif // __linux__
+
+  vector<Owned<Isolator> > isolators;
+
+  foreach (const string& type, strings::split(isolation, ",")) {
+    if (creators.contains(type)) {
+      Try<Isolator*> isolator = creators[type](flags);
+      if (isolator.isError()) {
+        return Error(
+            "Could not create isolator " + type + ": " + isolator.error());
+      } else {
+        isolators.push_back(Owned<Isolator>(isolator.get()));
+      }
+    } else {
+      return Error("Unknown or unsupported isolator: " + type);
+    }
+  }
+
+#ifdef __linux__
+  // Use cgroups on Linux if any cgroups isolators are used.
+  Try<Launcher*> launcher = strings::contains(isolation, "cgroups")
+    ? CgroupsLauncher::create(flags) : PosixLauncher::create(flags);
+#else
+  Try<Launcher*> launcher = PosixLauncher::create(flags);
+#endif // __linux__
+  if (launcher.isError()) {
+    return Error("Failed to create launcher: " + launcher.error());
+  }
+
+  return new MesosContainerizer(
+      flags, local, Owned<Launcher>(launcher.get()), isolators);
+}
+
+
+map<string, string> executorEnvironment(
+    const ExecutorInfo& executorInfo,
+    const string& directory,
+    const SlaveID& slaveId,
+    const PID<Slave>& slavePid,
+    bool checkpoint,
+    const Duration& recoveryTimeout)
+{
+  map<string, string> env;
+  // Set LIBPROCESS_PORT so that we bind to a random free port (since
+  // this might have been set via --port option). We do this before
+  // the environment variables below in case it is included.
+  env["LIBPROCESS_PORT"] = "0";
+
+  // Also add MESOS_NATIVE_LIBRARY if it's not already present (and
+  // like above, we do this before the environment variables below in
+  // case the framework wants to override).
+  if (!os::hasenv("MESOS_NATIVE_LIBRARY")) {
+    string path =
+#ifdef __APPLE__
+      LIBDIR "/libmesos-" VERSION ".dylib";
+#else
+      LIBDIR "/libmesos-" VERSION ".so";
+#endif
+    if (os::exists(path)) {
+      env["MESOS_NATIVE_LIBRARY"] = path;
+    }
+  }
+
+  env["MESOS_FRAMEWORK_ID"] = executorInfo.framework_id().value();
+  env["MESOS_EXECUTOR_ID"] = executorInfo.executor_id().value();
+  env["MESOS_DIRECTORY"] = directory;
+  env["MESOS_SLAVE_ID"] = slaveId.value();
+  env["MESOS_SLAVE_PID"] = stringify(slavePid);
+  env["MESOS_CHECKPOINT"] = checkpoint ? "1" : "0";
+
+  if (checkpoint) {
+    env["MESOS_RECOVERY_TIMEOUT"] = stringify(recoveryTimeout);
+  }
+
+  return env;
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/containerizer/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
new file mode 100644
index 0000000..d9ae326
--- /dev/null
+++ b/src/slave/containerizer/containerizer.hpp
@@ -0,0 +1,137 @@
+/**
+ * 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 __CONTAINERIZER_HPP__
+#define __CONTAINERIZER_HPP__
+
+#include <map>
+
+#include <mesos/mesos.hpp>
+#include <mesos/resources.hpp>
+
+#include <process/future.hpp>
+#include <process/owned.hpp>
+#include <process/process.hpp>
+
+#include <stout/duration.hpp>
+#include <stout/option.hpp>
+#include <stout/try.hpp>
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// Forward declaration.
+class Slave;
+class Flags;
+
+namespace state {
+// Forward declaration.
+struct SlaveState;
+} // namespace state {
+
+// An abstraction of a Containerizer that will contain an executor and its
+// tasks.
+class Containerizer
+{
+public:
+  // Information about a container termination.
+  struct Termination
+  {
+    Termination(
+        const Option<int>& _status,
+        bool _killed,
+        const std::string& _message)
+      : status(_status),
+        killed(_killed),
+        message(_message) {}
+
+    // Exit status of the executor.
+    const Option<int> status;
+
+    // A container may be killed if it exceeds its resources; this will be
+    // indicated by killed=true and described by the message string.
+    const bool killed;
+    const std::string message;
+  };
+
+  // Attempts to create a containerizer as specified by 'isolation' in flags.
+  static Try<Containerizer*> create(const Flags& flags, bool local);
+
+  // Determine slave resources from flags, probing the system or querying a
+  // delegate.
+  // TODO(idownes): Consider making this non-static and moving to containerizer
+  // implementations to enable a containerizer to best determine the resources,
+  // particularly if containerizeration is delegated.
+  static Try<Resources> resources(const Flags& flags);
+
+  virtual ~Containerizer() {}
+
+  // Recover all containerized executors specified in state. Any containerized
+  // executors present on the system but not included in state (or state is
+  // None) will be terminated and cleaned up.
+  virtual process::Future<Nothing> recover(
+      const Option<state::SlaveState>& state) = 0;
+
+  // Launch a containerized executor.
+  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>& slavePid,
+      bool checkpoint) = 0;
+
+  // Update the resources for a container.
+  virtual process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources) = 0;
+
+  // Get resource usage statistics on the container.
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId) = 0;
+
+  // Wait on the container's 'Termination'. If the executor terminates, the
+  // containerizer should also destroy the containerized context. The future
+  // may be failed if an error occurs during termination of the executor or
+  // destruction of the container.
+  virtual process::Future<Termination> wait(const ContainerID& containerId) = 0;
+
+  // Destroy a running container, killing all processes and releasing all
+  // resources.
+  // NOTE: Containerizers will automatically destroy containers on executor
+  // termination and manual destruction is not necessary. See wait().
+  virtual void destroy(const ContainerID& containerId) = 0;
+};
+
+
+// Executor environment variables returned as (name, value) map.
+std::map<std::string, std::string> executorEnvironment(
+    const ExecutorInfo& executorInfo,
+    const std::string& directory,
+    const SlaveID& slaveId,
+    const process::PID<Slave>& slavePid,
+    bool checkpoint,
+    const Duration& recoveryTimeout);
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __CONTAINERIZER_HPP__


[09/11] git commit: Containerizer - cgroup isolators (part 4).

Posted by vi...@apache.org.
Containerizer - cgroup isolators (part 4).

CPU and Memory isolators using Linux cgroups.

This code was written by Chi Zhang <cz...@twitter.com> and based on
the original cgroups_isolator.

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


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

Branch: refs/heads/master
Commit: d0db1ba1ce2cb40d52a1c0f0ad8095c3d7f883be
Parents: d5266b8
Author: Ian Downes <ia...@gmail.com>
Authored: Tue Feb 11 16:30:01 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Tue Feb 11 16:59:51 2014 -0800

----------------------------------------------------------------------
 src/Makefile.am                                 |   6 +-
 .../isolators/cgroups/cpushare.cpp              | 453 +++++++++++++++++
 .../isolators/cgroups/cpushare.hpp              | 104 ++++
 .../containerizer/isolators/cgroups/mem.cpp     | 498 +++++++++++++++++++
 .../containerizer/isolators/cgroups/mem.hpp     | 116 +++++
 5 files changed, 1176 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d0db1ba1/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index d9cb9e9..8a6a976 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -206,7 +206,9 @@ nodist_pkginclude_HEADERS = ../include/mesos/mesos.hpp mesos.pb.h
 
 if OS_LINUX
   libmesos_no_3rdparty_la_SOURCES += linux/cgroups.cpp
-  libmesos_no_3rdparty_la_SOURCES += slave/container/cgroups_launcher.cpp
+  libmesos_no_3rdparty_la_SOURCES += slave/containerizer/isolators/cgroups/cpushare.cpp
+  libmesos_no_3rdparty_la_SOURCES += slave/containerizer/isolators/cgroups/mem.cpp
+  libmesos_no_3rdparty_la_SOURCES += slave/containerizer/cgroups_launcher.cpp
   libmesos_no_3rdparty_la_SOURCES += linux/fs.cpp
 else
   EXTRA_DIST += linux/cgroups.cpp
@@ -235,6 +237,8 @@ libmesos_no_3rdparty_la_SOURCES += common/attributes.hpp		\
 	slave/containerizer/cgroups_launcher.hpp			\
 	slave/containerizer/containerizer.hpp				\
 	slave/containerizer/isolator.hpp				\
+	slave/containerizer/isolators/cgroups/cpushare.hpp		\
+	slave/containerizer/isolators/cgroups/mem.hpp			\
 	slave/containerizer/isolators/posix.hpp				\
 	slave/containerizer/launcher.hpp				\
 	slave/containerizer/mesos_containerizer.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/d0db1ba1/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
new file mode 100644
index 0000000..989d384
--- /dev/null
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
@@ -0,0 +1,453 @@
+/**
+ * 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 <vector>
+
+#include <mesos/values.hpp>
+#include <mesos/resources.hpp>
+
+#include <process/collect.hpp>
+#include <process/defer.hpp>
+#include <process/pid.hpp>
+
+#include <stout/bytes.hpp>
+#include <stout/check.hpp>
+#include <stout/error.hpp>
+#include <stout/foreach.hpp>
+#include <stout/hashmap.hpp>
+#include <stout/hashset.hpp>
+#include <stout/nothing.hpp>
+#include <stout/stringify.hpp>
+#include <stout/try.hpp>
+
+#include "common/type_utils.hpp"
+
+#include "linux/cgroups.hpp"
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/isolators/cgroups/cpushare.hpp"
+
+using namespace process;
+
+using std::list;
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// CPU subsystem constants.
+const size_t CPU_SHARES_PER_CPU = 1024;
+const size_t MIN_CPU_SHARES = 10;
+const Duration CPU_CFS_PERIOD = Milliseconds(100); // Linux default.
+const Duration MIN_CPU_CFS_QUOTA = Milliseconds(1);
+
+
+CgroupsCpushareIsolatorProcess::CgroupsCpushareIsolatorProcess(
+    const Flags& _flags,
+    const hashmap<string, string>& _hierarchies)
+  : flags(_flags), hierarchies(_hierarchies) {}
+
+
+CgroupsCpushareIsolatorProcess::~CgroupsCpushareIsolatorProcess() {}
+
+
+Try<Isolator*> CgroupsCpushareIsolatorProcess::create(
+    const Flags& flags)
+{
+  hashmap<string, string> hierarchies;
+
+  vector<string> subsystems;
+  subsystems.push_back("cpu");
+  subsystems.push_back("cpuacct");
+
+  foreach (const string& subsystem, subsystems) {
+    Try<string> hierarchy = cgroups::prepare(
+        flags.cgroups_hierarchy, subsystem, flags.cgroups_root);
+
+    if (hierarchy.isError()) {
+      return Error("Failed to create isolator: " + hierarchy.error());
+    }
+
+    hierarchies[subsystem] = hierarchy.get();
+  }
+
+  if (flags.cgroups_enable_cfs) {
+    Try<bool> exists = cgroups::exists(
+        hierarchies["cpu"], flags.cgroups_root, "cpu.cfs_quota_us");
+
+    if (exists.isError() || !exists.get()) {
+      return Error("Failed to find 'cpu.cfs_quota_us'. Your kernel "
+                   "might be too old to use the CFS cgroups feature.");
+    }
+  }
+
+  process::Owned<IsolatorProcess> process(
+      new CgroupsCpushareIsolatorProcess(flags, hierarchies));
+
+  return new Isolator(process);
+}
+
+
+Future<Nothing> CgroupsCpushareIsolatorProcess::recover(
+    const list<state::RunState>& states)
+{
+  hashset<string> cgroups;
+
+  foreach (const state::RunState& state, states) {
+    if (!state.id.isSome()) {
+      foreachvalue (Info* info, infos) {
+        delete info;
+      }
+      infos.clear();
+      return Failure("ContainerID is required to recover");
+    }
+
+    const ContainerID& containerId = state.id.get();
+
+    Info* info = new Info(
+        containerId, path::join(flags.cgroups_root, containerId.value()));
+    CHECK_NOTNULL(info);
+
+    Try<bool> exists = cgroups::exists(hierarchies["cpu"], info->cgroup);
+    if (exists.isError()) {
+      delete info;
+      foreachvalue (Info* info, infos) {
+        delete info;
+      }
+      infos.clear();
+      return Failure("Failed to check cgroup for container '" +
+                     stringify(containerId) + "'");
+    }
+
+    if (!exists.get()) {
+      // This may occur if the executor has exited and the isolator has
+      // destroyed the cgroup but the slave dies before noticing this. This
+      // will be detected when the containerizer tries to monitor the
+      // executor's pid.
+      LOG(WARNING) << "Couldn't find cgroup for container " << containerId;
+      continue;
+    }
+
+    infos[containerId] = info;
+    cgroups.insert(info->cgroup);
+  }
+
+  // Remove orphans in the cpu hierarchy.
+  Try<vector<string> > orphans = cgroups::get(
+      hierarchies["cpu"], flags.cgroups_root);
+  if (orphans.isError()) {
+    foreachvalue (Info* info, infos) {
+      delete info;
+    }
+    infos.clear();
+    return Failure(orphans.error());
+  }
+
+  foreach (const string& orphan, orphans.get()) {
+    if (!cgroups.contains(orphan)) {
+      LOG(INFO) << "Removing orphaned cgroup"
+                << " '" << path::join("cpu", orphan) << "'";
+      cgroups::destroy(hierarchies["cpu"], orphan);
+    }
+  }
+
+  // Remove orphans in the cpuacct hierarchy.
+  orphans = cgroups::get(hierarchies["cpuacct"], flags.cgroups_root);
+  if (orphans.isError()) {
+    foreachvalue (Info* info, infos) {
+      delete info;
+    }
+    infos.clear();
+    return Failure(orphans.error());
+  }
+
+  foreach (const string& orphan, orphans.get()) {
+    if (!cgroups.contains(orphan)) {
+      LOG(INFO) << "Removing orphaned cgroup"
+                << " '" << path::join("cpuacct", orphan) << "'";
+      cgroups::destroy(hierarchies["cpuacct"], orphan);
+    }
+  }
+
+  return Nothing();
+}
+
+
+Future<Nothing> CgroupsCpushareIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo)
+{
+  if (infos.contains(containerId)) {
+    return Failure("Container has already been prepared");
+  }
+
+  Info* info = new Info(
+      containerId, path::join(flags.cgroups_root, containerId.value()));
+
+  infos[containerId] = CHECK_NOTNULL(info);
+
+  // Create a 'cpu' cgroup for this container.
+  Try<bool> exists = cgroups::exists(hierarchies["cpu"], info->cgroup);
+
+  if (exists.isError()) {
+    return Failure("Failed to prepare isolator: " + exists.error());
+  }
+
+  if (exists.get()) {
+    return Failure("Failed to prepare isolator: cgroup already exists");
+  }
+
+  if (!exists.get()) {
+    Try<Nothing> create = cgroups::create(hierarchies["cpu"], info->cgroup);
+    if (create.isError()) {
+      return Failure("Failed to prepare isolator: " + create.error());
+    }
+  }
+
+  // Create a 'cpuacct' cgroup for this container.
+  exists = cgroups::exists(hierarchies["cpuacct"], info->cgroup);
+
+  if (exists.isError()) {
+    return Failure("Failed to prepare isolator: " + exists.error());
+  }
+
+  if (exists.get()) {
+    return Failure("Failed to prepare isolator: cgroup already exists");
+  }
+
+  if (!exists.get()) {
+    Try<Nothing> create = cgroups::create(hierarchies["cpuacct"], info->cgroup);
+    if (create.isError()) {
+      return Failure("Failed to prepare isolator: " + create.error());
+    }
+  }
+
+  return update(containerId, executorInfo.resources());
+}
+
+
+Future<Option<CommandInfo> > CgroupsCpushareIsolatorProcess::isolate(
+    const ContainerID& containerId,
+    pid_t pid)
+{
+  if (!infos.contains(containerId)) {
+    return Failure("Unknown container");
+  }
+
+  Info* info = CHECK_NOTNULL(infos[containerId]);
+
+  CHECK(info->pid.isNone());
+  info->pid = pid;
+
+  Try<Nothing> assign = cgroups::assign(hierarchies["cpu"], info->cgroup, pid);
+  if (assign.isError()) {
+    LOG(ERROR) << "Failed to assign container '" << info->containerId
+               << " to its own cgroup '"
+               << path::join(hierarchies["cpu"], info->cgroup)
+               << "' : " << assign.error();
+    return Failure("Failed to isolate container: " + assign.error());
+  }
+
+  assign = cgroups::assign(hierarchies["cpuacct"], info->cgroup, pid);
+  if (assign.isError()) {
+    LOG(ERROR) << "Failed to assign container '" << info->containerId
+               << " to its own cgroup '"
+               << path::join(hierarchies["cpuacct"], info->cgroup)
+               << "' : " << assign.error();
+    return Failure("Failed to isolate container: " + assign.error());
+  }
+
+  return None();
+}
+
+
+Future<Limitation> CgroupsCpushareIsolatorProcess::watch(
+    const ContainerID& containerId)
+{
+  if (!infos.contains(containerId)) {
+    return Failure("Unknown container");
+  }
+
+  CHECK_NOTNULL(infos[containerId]);
+
+  return infos[containerId]->limitation.future();
+}
+
+
+Future<Nothing> CgroupsCpushareIsolatorProcess::update(
+    const ContainerID& containerId,
+    const Resources& resources)
+{
+  if (resources.cpus().isNone()) {
+    return Failure("No cpus resource given");
+  }
+
+  if (!infos.contains(containerId)) {
+    return Failure("Unknown container");
+  }
+
+  Info* info = CHECK_NOTNULL(infos[containerId]);
+
+  double cpus = resources.cpus().get();
+
+  // Always set cpu.shares.
+  size_t shares =
+    std::max((size_t) (CPU_SHARES_PER_CPU * cpus), MIN_CPU_SHARES);
+
+  Try<Nothing> write = cgroups::write(
+      hierarchies["cpu"], info->cgroup, "cpu.shares", stringify(shares));
+
+  if (write.isError()) {
+    return Failure("Failed to update 'cpu.shares': " + write.error());
+  }
+
+  LOG(INFO) << "Updated 'cpu.shares' to " << shares
+            << " (cpus " << cpus << ")"
+            << " for container " << containerId;
+
+  // Set cfs quota if enabled.
+  if (flags.cgroups_enable_cfs) {
+    write = cgroups::write(
+        hierarchies["cpu"],
+        info->cgroup,
+        "cpu.cfs_period_us",
+        stringify(CPU_CFS_PERIOD.us()));
+    if (write.isError()) {
+      return Failure("Failed to update 'cpu.cfs_period_us': " + write.error());
+    }
+
+    Duration desired = Microseconds(
+        static_cast<int64_t>(CPU_CFS_PERIOD.us() * cpus));
+    Duration quota = std::max(desired, MIN_CPU_CFS_QUOTA);
+
+    write = cgroups::write(
+        hierarchies["cpu"],
+        info->cgroup,
+        "cpu.cfs_quota_us",
+        stringify(quota.us()));
+    if (write.isError()) {
+      return Failure("Failed to update 'cpu.cfs_quota_us': " + write.error());
+    }
+
+    LOG(INFO) << "Updated 'cpu.cfs_period_us' to " << CPU_CFS_PERIOD
+              << " and 'cpu.cfs_quota_us' to " << quota
+              << " (cpus " << cpus << ")"
+              << " for container " << containerId;
+  }
+
+  return Nothing();
+}
+
+
+Future<ResourceStatistics> CgroupsCpushareIsolatorProcess::usage(
+    const ContainerID& containerId)
+{
+  if (!infos.contains(containerId)) {
+    return Failure("Unknown container");
+  }
+
+  Info* info = CHECK_NOTNULL(infos[containerId]);
+
+  ResourceStatistics result;
+
+  // Get the number of clock ticks, used for cpu accounting.
+  static long ticks = sysconf(_SC_CLK_TCK);
+
+  PCHECK(ticks > 0) << "Failed to get sysconf(_SC_CLK_TCK)";
+
+  // Add the cpuacct.stat information.
+  Try<hashmap<string, uint64_t> > stat =
+    cgroups::stat(hierarchies["cpuacct"], info->cgroup, "cpuacct.stat");
+
+  if (stat.isError()) {
+    return Failure("Failed to read cpuacct.stat: " + stat.error());
+  }
+
+  // TODO(bmahler): Add namespacing to cgroups to enforce the expected
+  // structure, e.g., cgroups::cpuacct::stat.
+  if (stat.get().contains("user") && stat.get().contains("system")) {
+    result.set_cpus_user_time_secs(
+        (double) stat.get()["user"] / (double) ticks);
+    result.set_cpus_system_time_secs(
+        (double) stat.get()["system"] / (double) ticks);
+  }
+
+  // Add the cpu.stat information.
+  stat = cgroups::stat(hierarchies["cpu"], info->cgroup, "cpu.stat");
+
+  if (stat.isError()) {
+    return Failure("Failed to read cpu.stat: " + stat.error());
+  }
+
+  if (stat.get().contains("nr_periods")) {
+    result.set_cpus_nr_periods(
+        (uint32_t) stat.get()["nr_periods"]);
+  }
+
+  if (stat.get().contains("nr_throttled")) {
+    result.set_cpus_nr_throttled(
+        (uint32_t) stat.get()["nr_throttled"]);
+  }
+
+  if (stat.get().contains("throttled_time")) {
+    result.set_cpus_throttled_time_secs(
+        Nanoseconds(stat.get()["throttled_time"]).secs());
+  }
+
+  return result;
+}
+
+
+Future<Nothing> CgroupsCpushareIsolatorProcess::cleanup(
+    const ContainerID& containerId)
+{
+  if (!infos.contains(containerId)) {
+    return Failure("Unknown container");
+  }
+
+  Info* info = CHECK_NOTNULL(infos[containerId]);
+
+  list<Future<bool> > futures;
+  futures.push_back(cgroups::destroy(hierarchies["cpu"], info->cgroup));
+  futures.push_back(cgroups::destroy(hierarchies["cpuacct"], info->cgroup));
+
+  return collect(futures)
+    .then(defer(PID<CgroupsCpushareIsolatorProcess>(this),
+                &CgroupsCpushareIsolatorProcess::_cleanup,
+                containerId));
+}
+
+
+Future<Nothing> CgroupsCpushareIsolatorProcess::_cleanup(
+    const ContainerID& containerId)
+{
+  CHECK(infos.contains(containerId));
+
+  delete infos[containerId];
+  infos.erase(containerId);
+
+  return Nothing();
+}
+
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/d0db1ba1/src/slave/containerizer/isolators/cgroups/cpushare.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/cpushare.hpp b/src/slave/containerizer/isolators/cgroups/cpushare.hpp
new file mode 100644
index 0000000..49cc5bc
--- /dev/null
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.hpp
@@ -0,0 +1,104 @@
+/**
+ * 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 __CPUSHARE_ISOLATOR_HPP__
+#define __CPUSHARE_ISOLATOR_HPP__
+
+#include <mesos/resources.hpp>
+
+#include <process/future.hpp>
+
+#include <stout/hashmap.hpp>
+#include <stout/try.hpp>
+
+#include "slave/containerizer/isolator.hpp"
+
+#include "slave/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+
+// Use the Linux cpu cgroup controller for cpu isolation which uses the
+// Completely Fair Scheduler (CFS).
+// - cpushare implements proportionally weighted scheduling.
+// - cfs implements hard quota based scheduling.
+class CgroupsCpushareIsolatorProcess : public IsolatorProcess
+{
+public:
+  static Try<Isolator*> create(const Flags& flags);
+
+  virtual ~CgroupsCpushareIsolatorProcess();
+
+  virtual process::Future<Nothing> recover(
+      const std::list<state::RunState>& states);
+
+  virtual process::Future<Nothing> prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo);
+
+  virtual process::Future<Option<CommandInfo> > isolate(
+      const ContainerID& containerId,
+      pid_t pid);
+
+  virtual process::Future<Limitation> watch(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources);
+
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> cleanup(
+      const ContainerID& containerId);
+
+private:
+  CgroupsCpushareIsolatorProcess(
+      const Flags& flags,
+      const hashmap<std::string, std::string>& hierarchies);
+
+  virtual process::Future<Nothing> _cleanup(const ContainerID& containerId);
+
+  struct Info
+  {
+    Info(const ContainerID& _containerId, const std::string& _cgroup)
+      : containerId(_containerId), cgroup(_cgroup) {}
+
+    const ContainerID containerId;
+    const std::string cgroup;
+    Option<pid_t> pid;
+
+    process::Promise<Limitation> limitation;
+  };
+
+  const Flags flags;
+
+  // Map from subsystem to hierarchy.
+  hashmap<std::string, std::string> hierarchies;
+
+  hashmap<ContainerID, Info*> infos;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __CPUSHARE_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/d0db1ba1/src/slave/containerizer/isolators/cgroups/mem.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/mem.cpp b/src/slave/containerizer/isolators/cgroups/mem.cpp
new file mode 100644
index 0000000..a01e114
--- /dev/null
+++ b/src/slave/containerizer/isolators/cgroups/mem.cpp
@@ -0,0 +1,498 @@
+/**
+ * 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 <vector>
+
+#include <mesos/resources.hpp>
+#include <mesos/values.hpp>
+
+#include <process/collect.hpp>
+#include <process/defer.hpp>
+#include <process/pid.hpp>
+
+#include <stout/bytes.hpp>
+#include <stout/check.hpp>
+#include <stout/error.hpp>
+#include <stout/foreach.hpp>
+#include <stout/hashmap.hpp>
+#include <stout/hashset.hpp>
+#include <stout/lambda.hpp>
+#include <stout/nothing.hpp>
+#include <stout/stringify.hpp>
+#include <stout/try.hpp>
+
+#include "common/type_utils.hpp"
+
+#include "linux/cgroups.hpp"
+
+#include "slave/containerizer/isolators/cgroups/mem.hpp"
+
+using namespace process;
+
+using std::list;
+using std::ostringstream;
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// Memory subsystem constants.
+const Bytes MIN_MEMORY = Megabytes(32);
+
+
+CgroupsMemIsolatorProcess::CgroupsMemIsolatorProcess(
+    const Flags& _flags,
+    const string& _hierarchy)
+  : flags(_flags), hierarchy(_hierarchy) {}
+
+
+CgroupsMemIsolatorProcess::~CgroupsMemIsolatorProcess() {}
+
+
+Try<Isolator*> CgroupsMemIsolatorProcess::create(const Flags& flags)
+{
+  Try<string> hierarchy = cgroups::prepare(
+      flags.cgroups_hierarchy, "memory", flags.cgroups_root);
+
+  if (hierarchy.isError()) {
+    return Error("Failed to create memory cgroup: " + hierarchy.error());
+  }
+
+  // Make sure the kernel supports OOM controls.
+  Try<bool> exists = cgroups::exists(
+      hierarchy.get(), flags.cgroups_root, "memory.oom_control");
+  if (exists.isError() || !exists.get()) {
+    return Error("Failed to determine if 'memory.oom_control' control exists");
+  }
+
+  // Make sure the kernel OOM-killer is enabled.
+  // The Mesos OOM handler, as implemented, is not capable of handling
+  // the oom condition by itself safely given the limitations Linux
+  // imposes on this code path.
+  Try<Nothing> write = cgroups::write(
+      hierarchy.get(), flags.cgroups_root, "memory.oom_control", "0");
+  if (write.isError()) {
+    return Error("Failed to update memory.oom_control");
+  }
+
+  process::Owned<IsolatorProcess> process(
+      new CgroupsMemIsolatorProcess(flags, hierarchy.get()));
+
+  return new Isolator(process);
+}
+
+
+Future<Nothing> CgroupsMemIsolatorProcess::recover(
+    const list<state::RunState>& states)
+{
+  hashset<string> cgroups;
+
+  foreach (const state::RunState& state, states) {
+    if (state.id.isNone()) {
+      foreachvalue (Info* info, infos) {
+        delete info;
+      }
+      infos.clear();
+      return Failure("ContainerID is required to recover");
+    }
+
+    const ContainerID& containerId = state.id.get();
+
+    Info* info = new Info(
+        containerId, path::join(flags.cgroups_root, containerId.value()));
+    CHECK_NOTNULL(info);
+
+    Try<bool> exists = cgroups::exists(hierarchy, info->cgroup);
+    if (exists.isError()) {
+      delete info;
+      foreachvalue (Info* info, infos) {
+        delete info;
+      }
+      infos.clear();
+      return Failure("Failed to check cgroup for container '" +
+                     stringify(containerId) + "'");
+    }
+
+    if (!exists.get()) {
+      VLOG(1) << "Couldn't find cgroup for container " << containerId;
+      // This may occur if the executor has exiting and the isolator has
+      // destroyed the cgroup but the slave dies before noticing this. This
+      // will be detected when the containerizer tries to monitor the
+      // executor's pid.
+      continue;
+    }
+
+    infos[containerId] = info;
+    cgroups.insert(info->cgroup);
+
+    oomListen(containerId);
+  }
+
+  Try<vector<string> > orphans = cgroups::get(
+      hierarchy, flags.cgroups_root);
+  if (orphans.isError()) {
+    foreachvalue (Info* info, infos) {
+      delete info;
+    }
+    infos.clear();
+    return Failure(orphans.error());
+  }
+
+  foreach (const string& orphan, orphans.get()) {
+    if (!cgroups.contains(orphan)) {
+      LOG(INFO) << "Removing orphaned cgroup '" << orphan << "'";
+      cgroups::destroy(hierarchy, orphan);
+    }
+  }
+
+  return Nothing();
+}
+
+
+Future<Nothing> CgroupsMemIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo)
+{
+  if (infos.contains(containerId)) {
+    return Failure("Container has already been prepared");
+  }
+
+  Info* info = new Info(
+      containerId, path::join(flags.cgroups_root, containerId.value()));
+
+  infos[containerId] = CHECK_NOTNULL(info);
+
+  // Create a cgroup for this container.
+  Try<bool> exists = cgroups::exists(hierarchy, info->cgroup);
+
+  if (exists.isError()) {
+    return Failure("Failed to prepare isolator: " + exists.error());
+  }
+
+  if (exists.get()) {
+    return Failure("Failed to prepare isolator: cgroup already exists");
+  }
+
+  if (!exists.get()) {
+    Try<Nothing> create = cgroups::create(hierarchy, info->cgroup);
+    if (create.isError()) {
+      return Failure("Failed to prepare isolator: " + create.error());
+    }
+  }
+
+  oomListen(containerId);
+
+  return update(containerId, executorInfo.resources());
+}
+
+
+Future<Option<CommandInfo> > CgroupsMemIsolatorProcess::isolate(
+    const ContainerID& containerId,
+    pid_t pid)
+{
+  if (!infos.contains(containerId)) {
+    return Failure("Unknown container");
+  }
+
+  Info* info = CHECK_NOTNULL(infos[containerId]);
+
+  CHECK(info->pid.isNone());
+  info->pid = pid;
+
+  Try<Nothing> assign = cgroups::assign(hierarchy, info->cgroup, pid);
+  if (assign.isError()) {
+    return Failure("Failed to assign container '" +
+                   stringify(info->containerId) + "' to its own cgroup '" +
+                   path::join(hierarchy, info->cgroup) +
+                   "' : " + assign.error());
+  }
+
+  return None();
+}
+
+
+Future<Limitation> CgroupsMemIsolatorProcess::watch(
+    const ContainerID& containerId)
+{
+  if (!infos.contains(containerId)) {
+    return Failure("Unknown container");
+  }
+
+  CHECK_NOTNULL(infos[containerId]);
+
+  return infos[containerId]->limitation.future();
+}
+
+
+Future<Nothing> CgroupsMemIsolatorProcess::update(
+    const ContainerID& containerId,
+    const Resources& resources)
+{
+  if (resources.mem().isNone()) {
+    return Failure("No memory resource given");
+  }
+
+  if (!infos.contains(containerId)) {
+    return Failure("Unknown container");
+  }
+
+  Info* info = CHECK_NOTNULL(infos[containerId]);
+
+  // New limit.
+  Bytes mem = resources.mem().get();
+  Bytes limit = std::max(mem, MIN_MEMORY);
+
+  // Always set the soft limit.
+  Try<Nothing> write =
+    cgroups::memory::soft_limit_in_bytes(hierarchy, info->cgroup, limit);
+
+  if (write.isError()) {
+    return Failure("Failed to set 'memory.soft_limit_in_bytes': "
+        + write.error());
+  }
+
+  LOG(INFO) << "Updated 'memory.soft_limit_in_bytes' to " << limit
+            << " for container " << containerId;
+
+  // Read the existing limit.
+  Try<Bytes> currentLimit =
+    cgroups::memory::limit_in_bytes(hierarchy, info->cgroup);
+
+  if (currentLimit.isError()) {
+    return Failure(
+        "Failed to read 'memory.limit_in_bytes': " + currentLimit.error());
+  }
+
+  // Determine whether to set the hard limit. If this is the first
+  // time (info->pid.isNone()), or we're raising the existing limit,
+  // then we can update the hard limit safely. Otherwise, if we need
+  // to decrease 'memory.limit_in_bytes' we may induce an OOM if too
+  // much memory is in use. As a result, we only update the soft
+  // limit when the memory reservation is being reduced. This is
+  // probably okay if the machine has available resources.
+  // TODO(benh): Introduce a MemoryWatcherProcess which monitors the
+  // discrepancy between usage and soft limit and introduces a "manual oom" if
+  // necessary.
+  if (info->pid.isNone() || limit > currentLimit.get()) {
+    write = cgroups::memory::limit_in_bytes(hierarchy, info->cgroup, limit);
+
+    if (write.isError()) {
+      return Failure("Failed to set 'memory.limit_in_bytes': " +
+                     write.error());
+    }
+
+    LOG(INFO) << "Updated 'memory.limit_in_bytes' to " << limit
+              << " for container " << containerId;
+  }
+
+  return Nothing();
+}
+
+
+Future<ResourceStatistics> CgroupsMemIsolatorProcess::usage(
+    const ContainerID& containerId)
+{
+  if (!infos.contains(containerId)) {
+    return Failure("Unknown container");
+  }
+
+  Info* info = CHECK_NOTNULL(infos[containerId]);
+
+  ResourceStatistics result;
+
+  // The rss from memory.stat is wrong in two dimensions:
+  //   1. It does not include child cgroups.
+  //   2. It does not include any file backed pages.
+  Try<Bytes> usage = cgroups::memory::usage_in_bytes(hierarchy, info->cgroup);
+  if (usage.isError()) {
+    return Failure("Failed to parse memory.usage_in_bytes: " + usage.error());
+  }
+
+  // TODO(bmahler): Add namespacing to cgroups to enforce the expected
+  // structure, e.g, cgroups::memory::stat.
+  result.set_mem_rss_bytes(usage.get().bytes());
+
+  Try<hashmap<string, uint64_t> > stat =
+    cgroups::stat(hierarchy, info->cgroup, "memory.stat");
+
+  if (stat.isError()) {
+    return Failure("Failed to read memory.stat: " + stat.error());
+  }
+
+  if (stat.get().contains("total_cache")) {
+    result.set_mem_file_bytes(stat.get()["total_cache"]);
+  }
+
+  if (stat.get().contains("total_rss")) {
+    result.set_mem_anon_bytes(stat.get()["total_rss"]);
+  }
+
+  if (stat.get().contains("total_mapped_file")) {
+    result.set_mem_mapped_file_bytes(stat.get()["total_mapped_file"]);
+  }
+
+  return result;
+}
+
+
+Future<Nothing> CgroupsMemIsolatorProcess::cleanup(
+    const ContainerID& containerId)
+{
+  if (!infos.contains(containerId)) {
+    return Failure("Unknown container");
+  }
+
+  Info* info = CHECK_NOTNULL(infos[containerId]);
+
+  if (info->oomNotifier.isPending()) {
+    info->oomNotifier.discard();
+  }
+
+  return cgroups::destroy(hierarchy, info->cgroup)
+    .then(defer(PID<CgroupsMemIsolatorProcess>(this),
+                &CgroupsMemIsolatorProcess::_cleanup,
+                containerId));
+}
+
+
+Future<Nothing> CgroupsMemIsolatorProcess::_cleanup(
+    const ContainerID& containerId)
+{
+  CHECK(infos.contains(containerId));
+
+  delete infos[containerId];
+  infos.erase(containerId);
+
+  return Nothing();
+}
+
+
+void CgroupsMemIsolatorProcess::oomListen(
+    const ContainerID& containerId)
+{
+  CHECK(infos.contains(containerId));
+  Info* info = CHECK_NOTNULL(infos[containerId]);
+
+  info->oomNotifier =
+    cgroups::listen(hierarchy, info->cgroup, "memory.oom_control");
+
+  // If the listening fails immediately, something very wrong
+  // happened.  Therefore, we report a fatal error here.
+  if (info->oomNotifier.isFailed()) {
+    LOG(FATAL) << "Failed to listen for OOM events for container "
+               << containerId << ": "
+               << info->oomNotifier.failure();
+  }
+
+  LOG(INFO) << "Started listening for OOM events for container "
+            << containerId;
+
+  info->oomNotifier.onReady(defer(
+      PID<CgroupsMemIsolatorProcess>(this),
+      &CgroupsMemIsolatorProcess::oomWaited,
+      containerId,
+      lambda::_1));
+}
+
+
+void CgroupsMemIsolatorProcess::oomWaited(
+    const ContainerID& containerId,
+    const Future<uint64_t>& future)
+{
+  LOG(INFO) << "OOM notifier is triggered for container "
+            << containerId;
+
+  if (future.isDiscarded()) {
+    LOG(INFO) << "Discarded OOM notifier for container "
+              << containerId;
+  } else if (future.isFailed()) {
+    LOG(ERROR) << "Listening on OOM events failed for container "
+               << containerId << ": " << future.failure();
+  } else {
+    // Out-of-memory event happened, call the handler.
+    oom(containerId);
+  }
+}
+
+
+void CgroupsMemIsolatorProcess::oom(const ContainerID& containerId)
+{
+  if (!infos.contains(containerId)) {
+    // It is likely that process exited is executed before this
+    // function (e.g.  The kill and OOM events happen at the same
+    // time, and the process exit event arrives first.) Therefore, we
+    // should not report a fatal error here.
+    LOG(INFO) << "OOM detected for an already terminated executor";
+    return;
+  }
+
+  Info* info = CHECK_NOTNULL(infos[containerId]);
+
+  LOG(INFO) << "OOM detected for container " << containerId;
+
+  // Construct a "message" string to describe why the isolator
+  // destroyed the executor's cgroup (in order to assist in
+  // debugging).
+  ostringstream message;
+  message << "Memory limit exceeded: ";
+
+  // Output the requested memory limit.
+  Try<Bytes> limit = cgroups::memory::limit_in_bytes(hierarchy, info->cgroup);
+
+  if (limit.isError()) {
+    LOG(ERROR) << "Failed to read 'memory.limit_in_bytes': " << limit.error();
+  } else {
+    message << "Requested: " << limit.get() << " ";
+  }
+
+  // Output the maximum memory usage.
+  Try<Bytes> usage = cgroups::memory::max_usage_in_bytes(
+      hierarchy, info->cgroup);
+
+  if (usage.isError()) {
+    LOG(ERROR) << "Failed to read 'memory.max_usage_in_bytes': "
+               << usage.error();
+  } else {
+    message << "Maximum Used: " << usage.get() << "\n";
+  }
+
+  // Output 'memory.stat' of the cgroup to help with debugging.
+  // NOTE: With Kernel OOM-killer enabled these stats may not reflect
+  // memory state at time of OOM.
+  Try<string> read = cgroups::read(hierarchy, info->cgroup, "memory.stat");
+  if (read.isError()) {
+    LOG(ERROR) << "Failed to read 'memory.stat': " << read.error();
+  } else {
+    message << "\nMEMORY STATISTICS: \n" << read.get() << "\n";
+  }
+
+  LOG(INFO) << strings::trim(message.str()); // Trim the extra '\n' at the end.
+
+  Resource mem = Resources::parse(
+      "mem",
+      stringify(usage.isSome() ? usage.get().bytes() : 0),
+      "*").get();
+
+  info->limitation.set(Limitation(mem, message.str()));
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/d0db1ba1/src/slave/containerizer/isolators/cgroups/mem.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/mem.hpp b/src/slave/containerizer/isolators/cgroups/mem.hpp
new file mode 100644
index 0000000..aabb4df
--- /dev/null
+++ b/src/slave/containerizer/isolators/cgroups/mem.hpp
@@ -0,0 +1,116 @@
+/**
+ * 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 __MEM_ISOLATOR_HPP__
+#define __MEM_ISOLATOR_HPP__
+
+#include <mesos/resources.hpp>
+
+#include <process/future.hpp>
+
+#include <stout/nothing.hpp>
+#include <stout/try.hpp>
+
+#include "mesos/resources.hpp"
+
+#include "slave/containerizer/isolator.hpp"
+
+#include "slave/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+
+class CgroupsMemIsolatorProcess : public IsolatorProcess
+{
+public:
+  static Try<Isolator*> create(const Flags& flags);
+
+  virtual ~CgroupsMemIsolatorProcess();
+
+  virtual process::Future<Nothing> recover(
+      const std::list<state::RunState>& states);
+
+  virtual process::Future<Nothing> prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo);
+
+  virtual process::Future<Option<CommandInfo> > isolate(
+      const ContainerID& containerId,
+      pid_t pid);
+
+  virtual process::Future<Limitation> watch(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources);
+
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> cleanup(
+      const ContainerID& containerId);
+
+private:
+  CgroupsMemIsolatorProcess(const Flags& flags, const std::string& hierarchy);
+
+  virtual process::Future<Nothing> _cleanup(const ContainerID& containerId);
+
+  struct Info
+  {
+    Info(const ContainerID& _containerId, const std::string& _cgroup)
+      : containerId(_containerId), cgroup(_cgroup) {}
+
+    const ContainerID containerId;
+    const std::string cgroup;
+    Option<pid_t> pid;
+
+    process::Promise<Limitation> limitation;
+
+    // Used to cancel the OOM listening.
+    process::Future<uint64_t> oomNotifier;
+  };
+
+  // Start listening on OOM events. This function will create an
+  // eventfd and start polling on it.
+  void oomListen(const ContainerID& containerId);
+
+  // This function is invoked when the polling on eventfd has a
+  // result.
+  void oomWaited(
+      const ContainerID& containerId,
+      const process::Future<uint64_t>& future);
+
+  // This function is invoked when the OOM event happens.
+  void oom(const ContainerID& containerId);
+
+  const Flags flags;
+
+  // The path to the cgroups subsystem hierarchy root.
+  const std::string hierarchy;
+
+  hashmap<ContainerID, Info*> infos;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MEM_ISOLATOR_HPP__


[06/11] Containerizer (part 1)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/containerizer/mesos_containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos_containerizer.cpp b/src/slave/containerizer/mesos_containerizer.cpp
new file mode 100644
index 0000000..c61739b
--- /dev/null
+++ b/src/slave/containerizer/mesos_containerizer.cpp
@@ -0,0 +1,907 @@
+/**
+ * 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 <sstream>
+
+#include <process/collect.hpp>
+#include <process/defer.hpp>
+#include <process/io.hpp>
+#include <process/reap.hpp>
+#include <process/subprocess.hpp>
+
+#include <stout/fatal.hpp>
+#include <stout/os.hpp>
+#include <stout/unreachable.hpp>
+
+#include "slave/paths.hpp"
+#include "slave/slave.hpp"
+
+#include "slave/containerizer/mesos_containerizer.hpp"
+
+using std::list;
+using std::map;
+using std::string;
+using std::vector;
+
+using namespace process;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+using state::SlaveState;
+using state::FrameworkState;
+using state::ExecutorState;
+using state::RunState;
+
+// Local function declaration/definitions.
+Future<Nothing> _nothing() { return Nothing(); }
+
+
+MesosContainerizer::MesosContainerizer(
+    const Flags& flags,
+    bool local,
+    const Owned<Launcher>& launcher,
+    const vector<Owned<Isolator> >& isolators)
+{
+  process = new MesosContainerizerProcess(
+      flags, local, launcher, isolators);
+  spawn(process);
+}
+
+
+MesosContainerizer::~MesosContainerizer()
+{
+  terminate(process);
+  process::wait(process);
+  delete process;
+}
+
+
+Future<Nothing> MesosContainerizer::recover(const Option<state::SlaveState>& state)
+{
+  return dispatch(process, &MesosContainerizerProcess::recover, state);
+}
+
+
+Future<Nothing> MesosContainerizer::launch(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo,
+    const string& directory,
+    const Option<string>& user,
+    const SlaveID& slaveId,
+    const PID<Slave>& slavePid,
+    bool checkpoint)
+{
+  return dispatch(process,
+                  &MesosContainerizerProcess::launch,
+                  containerId,
+                  executorInfo,
+                  directory,
+                  user,
+                  slaveId,
+                  slavePid,
+                  checkpoint);
+}
+
+
+Future<Nothing> MesosContainerizer::update(
+    const ContainerID& containerId,
+    const Resources& resources)
+{
+  return dispatch(process,
+                  &MesosContainerizerProcess::update,
+                  containerId,
+                  resources);
+}
+
+
+Future<ResourceStatistics> MesosContainerizer::usage(
+    const ContainerID& containerId)
+{
+  return dispatch(process, &MesosContainerizerProcess::usage, containerId);
+}
+
+
+Future<Containerizer::Termination> MesosContainerizer::wait(
+    const ContainerID& containerId)
+{
+  return dispatch(process, &MesosContainerizerProcess::wait, containerId);
+}
+
+
+void MesosContainerizer::destroy(const ContainerID& containerId)
+{
+  dispatch(process, &MesosContainerizerProcess::destroy, containerId);
+}
+
+
+Future<Nothing> MesosContainerizerProcess::recover(
+    const Option<state::SlaveState>& state)
+{
+  LOG(INFO) << "Recovering containerizer";
+
+  // Gather the executor run states that we will attempt to recover.
+  list<RunState> recoverable;
+  if (state.isSome()) {
+    foreachvalue (const FrameworkState& framework, state.get().frameworks) {
+      foreachvalue (const ExecutorState& executor, framework.executors) {
+        if (executor.info.isNone()) {
+          LOG(WARNING) << "Skipping recovery of executor '" << executor.id
+                       << "' of framework " << framework.id
+                       << " because its info could not be recovered";
+          continue;
+        }
+
+        if (executor.latest.isNone()) {
+          LOG(WARNING) << "Skipping recovery of executor '" << executor.id
+                       << "' of framework " << framework.id
+                       << " because its latest run could not be recovered";
+          continue;
+        }
+
+        // We are only interested in the latest run of the executor!
+        const ContainerID& containerId = executor.latest.get();
+        CHECK(executor.runs.contains(containerId));
+        const RunState& run = executor.runs.get(containerId).get();
+
+        // We need the pid so the reaper can monitor the executor so skip this
+        // executor if it's not present. This is not an error because the slave
+        // will try to wait on the container which will return a failed
+        // Termination and everything will get cleaned up.
+        if (!run.forkedPid.isSome()) {
+          continue;
+        }
+
+        if (run.completed) {
+          VLOG(1) << "Skipping recovery of executor '" << executor.id
+                  << "' of framework " << framework.id
+                  << " because its latest run "
+                  << containerId << " is completed";
+          continue;
+        }
+
+        LOG(INFO) << "Recovering container '" << containerId
+                  << "' for executor '" << executor.id
+                  << "' of framework " << framework.id;
+
+        recoverable.push_back(run);
+      }
+    }
+  }
+
+  // Try to recover the launcher first.
+  Try<Nothing> recover = launcher->recover(recoverable);
+  if (recover.isError()) {
+    return Failure(recover.error());
+  }
+
+  // Then recover the isolators.
+  list<Future<Nothing> > futures;
+  foreach (const Owned<Isolator>& isolator, isolators) {
+    futures.push_back(isolator->recover(recoverable));
+  }
+
+  // If all isolators recover then continue.
+  return collect(futures)
+    .then(defer(self(), &Self::_recover, recoverable));
+}
+
+
+Future<Nothing> MesosContainerizerProcess::_recover(
+    const list<RunState>& recovered)
+{
+  foreach (const RunState& run, recovered) {
+    CHECK_SOME(run.id);
+    const ContainerID& containerId = run.id.get();
+
+    Owned<Promise<Containerizer::Termination> > promise(
+        new Promise<Containerizer::Termination>());
+    promises.put(containerId, promise);
+
+    CHECK_SOME(run.forkedPid);
+    Future<Option<int > > status = process::reap(run.forkedPid.get());
+    statuses[containerId] = status;
+    status.onAny(defer(self(), &Self::exited, containerId));
+
+    foreach (const Owned<Isolator>& isolator, isolators) {
+      isolator->watch(containerId)
+        .onAny(defer(self(), &Self::limited, containerId, lambda::_1));
+    }
+  }
+
+  return Nothing();
+}
+
+
+// Log the message and then exit(1) in an async-signal-safe manner.
+// TODO(idownes): Move this into stout, possibly replacing its fatal(), and
+// support multiple messages to write out.
+void asyncSafeFatal(const char* message)
+{
+  // Ignore the return value from write() to silence compiler warning.
+  (void) write(STDERR_FILENO, message, strlen(message));
+  _exit(1);
+}
+
+
+// This function is executed by the forked child and should be
+// async-signal-safe.
+// TODO(idownes): Several functions used here are not actually
+// async-signal-safe:
+// 1) os::close, os::chown and os::chdir concatenate strings on error
+// 2) os::setenv uses ::setenv that is not listed as safe
+// 3) freopen is not listed as safe
+// These can all be corrected and also we could write better error messages
+// with multiple writes in an improved asyncSafeFatal.
+int execute(
+    const CommandInfo& command,
+    const string& directory,
+    const Option<string>& user,
+    const map<string, string>& env,
+    bool redirectIO,
+    int pipeRead,
+    int pipeWrite)
+{
+  // Do a blocking read on the pipe until the parent signals us to continue.
+  os::close(pipeWrite);
+  int buf;
+  ssize_t len;
+  while ((len = read(pipeRead, &buf, sizeof(buf))) == -1 && errno == EINTR);
+
+  if (len != sizeof(buf)) {
+    os::close(pipeRead);
+    asyncSafeFatal("Failed to synchronize with parent");
+  }
+  os::close(pipeRead);
+
+  // Chown the work directory if a user is provided.
+  if (user.isSome()) {
+    Try<Nothing> chown = os::chown(user.get(), directory);
+    if (chown.isError()) {
+      asyncSafeFatal("Failed to chown work directory");
+    }
+  }
+
+  // Change user if provided.
+  if (user.isSome() && !os::su(user.get())) {
+    asyncSafeFatal("Failed to change user");
+  }
+
+  // Enter working directory.
+  if (os::chdir(directory) < 0) {
+    asyncSafeFatal("Failed to chdir into work directory");
+  }
+
+  // First set up any additional environment variables.
+  // TODO(idownes): setenv is not async-signal-safe. Environment variables
+  // could instead be set using execle.
+  foreachpair (const string& key, const string& value, env) {
+    os::setenv(key, value);
+  }
+
+  // Then set up environment variables from CommandInfo.
+  foreach(const Environment::Variable& variable,
+      command.environment().variables()) {
+    os::setenv(variable.name(), variable.value());
+  }
+
+  // Redirect output to files in working dir if required. We append because
+  // others (e.g., mesos-fetcher) may have already logged to the files.
+  // TODO(bmahler): It would be best if instead of closing stderr /
+  // stdout and redirecting, we instead always output to stderr /
+  // stdout. Also tee'ing their output into the work directory files
+  // when redirection is desired.
+  // TODO(idownes): freopen is not async-signal-safe. Could use dup2 and open
+  // directly.
+  if (redirectIO) {
+    if (freopen("stdout", "a", stdout) == NULL) {
+      asyncSafeFatal("freopen failed");
+    }
+    if (freopen("stderr", "a", stderr) == NULL) {
+      asyncSafeFatal("freopen failed");
+    }
+  }
+
+  // Execute the command (via '/bin/sh -c command').
+  execl("/bin/sh", "sh", "-c", command.value().c_str(), (char*) NULL);
+
+  // If we get here, the execv call failed.
+  asyncSafeFatal("Failed to execute command");
+
+  // Silence end of non-void function warning.
+  return UNREACHABLE();
+}
+
+
+// Launching an executor involves the following steps:
+// 1. Prepare the container. First call prepare on each isolator and then
+//    fetch the executor into the container sandbox.
+// 2. Fork the executor. The forked child is blocked from exec'ing until it has
+//    been isolated.
+// 3. Isolate the executor. Call isolate with the pid for each isolator.
+// 4. Exec the executor. The forked child is signalled to continue and exec the
+//    executor.
+Future<Nothing> MesosContainerizerProcess::launch(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo,
+    const string& directory,
+    const Option<string>& user,
+    const SlaveID& slaveId,
+    const PID<Slave>& slavePid,
+    bool checkpoint)
+{
+  if (promises.contains(containerId)) {
+    LOG(ERROR) << "Cannot start already running container '"
+               << containerId << "'";
+    return Failure("Container already started");
+  }
+
+  Owned<Promise<Containerizer::Termination> > promise(
+      new Promise<Containerizer::Termination>());
+  promises.put(containerId, promise);
+
+  // Store the resources for usage().
+  resources.put(containerId, executorInfo.resources());
+
+  LOG(INFO) << "Starting container '" << containerId
+            << "' for executor '" << executorInfo.executor_id()
+            << "' of framework '" << executorInfo.framework_id() << "'";
+
+  // Prepare additional environment variables for the executor.
+  const map<string, string>& env = executorEnvironment(
+      executorInfo,
+      directory,
+      slaveId,
+      slavePid,
+      checkpoint,
+      flags.recovery_timeout);
+
+  // Use a pipe to block the child until it's been isolated.
+  // The parent will close its read end after the child is forked, and the
+  // write end afer the child is signalled to exec.
+  // TODO(idownes): Ensure the pipe's file descriptors are closed even if some
+  // stage of the executor launch fails.
+  int pipes[2];
+  // We assume this should not fail under reasonable conditions so we use CHECK.
+  CHECK(pipe(pipes) == 0);
+
+  // Prepare a function for the forked child to exec() the executor.
+  lambda::function<int()> inChild = lambda::bind(
+      &execute,
+      executorInfo.command(),
+      directory,
+      user,
+      env,
+      !local,
+      pipes[0],
+      pipes[1]);
+
+  return prepare(containerId, executorInfo, directory, user)
+    .then(defer(self(),
+                &Self::fork,
+                containerId,
+                executorInfo,
+                inChild,
+                slaveId,
+                checkpoint,
+                pipes[0]))
+    .then(defer(self(),
+                &Self::isolate,
+                containerId,
+                lambda::_1))
+    .then(defer(self(),
+                &Self::exec,
+                containerId,
+                pipes[1]))
+    .onAny(lambda::bind(&os::close, pipes[0]))
+    .onAny(lambda::bind(&os::close, pipes[1]))
+    .onFailed(defer(self(),
+                    &Self::destroy,
+                    containerId));
+}
+
+
+Future<Nothing> MesosContainerizerProcess::prepare(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo,
+    const string& directory,
+    const Option<string>& user)
+{
+  // Start preparing all isolators (in parallel).
+  list<Future<Nothing> > futures;
+  foreach (const Owned<Isolator>& isolator, isolators) {
+    futures.push_back(isolator->prepare(containerId, executorInfo));
+  }
+
+  // Wait for all isolators to complete preparations then fetch the executor.
+  return collect(futures)
+    .then(defer(
+          self(),
+          &Self::fetch,
+          containerId,
+          executorInfo.command(),
+          directory,
+          user));
+}
+
+
+Future<Nothing> _fetch(
+    const ContainerID& containerId,
+    const Option<int>& status)
+{
+  if (status.isNone() || (status.get() != 0)) {
+    return Failure("Failed to fetch URIs for container '" +
+                   stringify(containerId) + "': exit status " +
+                   (status.isNone() ? "none" : stringify(status.get())));
+  }
+
+  return Nothing();
+}
+
+
+Future<Nothing> MesosContainerizerProcess::fetch(
+    const ContainerID& containerId,
+    const CommandInfo& commandInfo,
+    const string& directory,
+    const Option<string>& user)
+{
+  // Determine path for mesos-fetcher.
+  Result<string> realpath = os::realpath(
+      path::join(flags.launcher_dir, "mesos-fetcher"));
+
+  if (!realpath.isSome()) {
+    LOG(ERROR) << "Failed to determine the canonical path "
+                << "for the mesos-fetcher '"
+                << path::join(flags.launcher_dir, "mesos-fetcher")
+                << "': "
+                << (realpath.isError() ? realpath.error()
+                                       : "No such file or directory");
+    return Failure("Could not fetch URIs: failed to find mesos-fetcher");
+  }
+
+  // Prepare the environment variables to pass to mesos-fetcher.
+  string uris = "";
+  foreach (const CommandInfo::URI& uri, commandInfo.uris()) {
+    uris += uri.value() + "+" +
+            (uri.has_executable() && uri.executable() ? "1" : "0");
+    uris += " ";
+  }
+  // Remove extra space at the end.
+  uris = strings::trim(uris);
+
+  // Use /usr/bin/env to set the environment variables for the fetcher
+  // subprocess because we cannot pollute the slave's environment.
+  // TODO(idownes): Remove this once Subprocess accepts environment variables.
+  string command = "/usr/bin/env";
+  command += " MESOS_EXECUTOR_URIS=" + uris;
+  command += " MESOS_WORK_DIRECTORY=" + directory;
+  if (user.isSome()) {
+    command += " MESOS_USER=" + user.get();
+  }
+  command += " MESOS_FRAMEWORKS_HOME=" + flags.frameworks_home;
+  command += " HADOOP_HOME=" + flags.hadoop_home;
+
+  // Now the actual mesos-fetcher command.
+  command += " " + realpath.get();
+
+  LOG(INFO) << "Fetching URIs for container '" << containerId
+            << "' using command '" << command << "'";
+
+  Try<Subprocess> fetcher = subprocess(command);
+  if (fetcher.isError()) {
+    return Failure("Failed to execute mesos-fetcher: " + fetcher.error());
+  }
+
+  // Redirect output (stdout and stderr) from the fetcher to log files in the
+  // executor work directory, chown'ing them if a user is specified.
+  Try<int> out = os::open(
+      path::join(directory, "stdout"),
+      O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK,
+      S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
+
+  if (out.isError()) {
+    return Failure("Failed to redirect stdout: " + out.error());
+  }
+
+  if (user.isSome()) {
+    Try<Nothing> chown = os::chown(user.get(), path::join(directory, "stdout"));
+    if (chown.isError()) {
+      os::close(out.get());
+      return Failure("Failed to redirect stdout:" + chown.error());
+    }
+  }
+
+  Try<Nothing> nonblock = os::nonblock(fetcher.get().out());
+  if (nonblock.isError()) {
+    os::close(out.get());
+    return Failure("Failed to redirect stdout:" + nonblock.error());
+  }
+
+  io::splice(fetcher.get().out(), out.get())
+    .onAny(lambda::bind(&os::close, out.get()));
+
+  // Repeat for stderr.
+  Try<int> err = os::open(
+      path::join(directory, "stderr"),
+      O_WRONLY | O_CREAT | O_TRUNC | O_NONBLOCK,
+      S_IRUSR | S_IWUSR | S_IRGRP | S_IRWXO);
+
+  if (err.isError()) {
+    os::close(out.get());
+    return Failure("Failed to redirect stderr:" + err.error());
+  }
+
+  if (user.isSome()) {
+    Try<Nothing> chown = os::chown(user.get(), path::join(directory, "stderr"));
+    if (chown.isError()) {
+      os::close(out.get());
+      os::close(err.get());
+      return Failure("Failed to redirect stderr:" + chown.error());
+    }
+  }
+
+  nonblock = os::nonblock(fetcher.get().err());
+  if (nonblock.isError()) {
+    os::close(out.get());
+    os::close(err.get());
+    return Failure("Failed to redirect stderr:" + nonblock.error());
+  }
+
+  io::splice(fetcher.get().err(), err.get())
+    .onAny(lambda::bind(&os::close, err.get()));
+
+  return fetcher.get().status()
+    .then(lambda::bind(&_fetch, containerId, lambda::_1));
+}
+
+
+Future<pid_t> MesosContainerizerProcess::fork(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo,
+    lambda::function<int()>& inChild,
+    const SlaveID& slaveId,
+    bool checkpoint,
+    int pipeRead)
+{
+  Try<pid_t> forked = launcher->fork(containerId, inChild);
+
+  if (forked.isError()) {
+    return Failure("Failed to fork executor: " + forked.error());
+  }
+  pid_t pid = forked.get();
+
+  // Checkpoint the executor's pid if requested.
+  if (checkpoint) {
+    const string& path = slave::paths::getForkedPidPath(
+        slave::paths::getMetaRootDir(flags.work_dir),
+        slaveId,
+        executorInfo.framework_id(),
+        executorInfo.executor_id(),
+        containerId);
+
+    LOG(INFO) << "Checkpointing executor's forked pid " << pid
+              << " to '" << path <<  "'";
+
+    Try<Nothing> checkpointed =
+      slave::state::checkpoint(path, stringify(pid));
+
+    if (checkpointed.isError()) {
+      LOG(ERROR) << "Failed to checkpoint executor's forked pid to '"
+                 << path << "': " << checkpointed.error();
+
+      return Failure("Could not checkpoint executor's pid");
+    }
+  }
+
+  // Monitor the executor's pid. We keep the future because we'll refer to it
+  // again during container destroy.
+  Future<Option<int> > status = process::reap(pid);
+  statuses.put(containerId, status);
+  status.onAny(defer(self(), &Self::exited, containerId));
+
+  return pid;
+}
+
+
+Future<Nothing> MesosContainerizerProcess::isolate(
+    const ContainerID& containerId,
+    pid_t _pid)
+{
+  // Set up callbacks for isolator limitations.
+  foreach (const Owned<Isolator>& isolator, isolators) {
+    isolator->watch(containerId)
+      .onAny(defer(self(), &Self::limited, containerId, lambda::_1));
+  }
+
+  // Isolate the executor with each isolator and get optional additional
+  // commands to be run in the containerized context.
+  list<Future<Option<CommandInfo> > > futures;
+  foreach (const Owned<Isolator>& isolator, isolators) {
+    futures.push_back(isolator->isolate(containerId, _pid));
+  }
+
+  // Wait for all isolators to complete then run additional commands.
+  return collect(futures)
+    .then(defer(self(), &Self::_isolate, containerId, lambda::_1));
+}
+
+
+Future<Nothing> MesosContainerizerProcess::_isolate(
+    const ContainerID& containerId,
+    const list<Option<CommandInfo> >& commands)
+{
+  // TODO(idownes): Implement execution of additional isolation commands.
+  foreach (const Option<CommandInfo>& command, commands) {
+    if (command.isSome()) {
+      LOG(WARNING) << "Additional isolation commands not implemented";
+    }
+  }
+
+  return Nothing();
+}
+
+
+Future<Nothing> MesosContainerizerProcess::exec(
+    const ContainerID& containerId,
+    int pipeWrite)
+{
+  CHECK(promises.contains(containerId));
+
+  // Now that we've contained the child we can signal it to continue by
+  // writing to the pipe.
+  int buf;
+  ssize_t len;
+  while ((len = write(pipeWrite, &buf, sizeof(buf))) == -1 && errno == EINTR);
+
+  if (len != sizeof(buf)) {
+    return Failure("Failed to synchronize child process: " +
+                   string(strerror(errno)));
+  }
+
+  return Nothing();
+}
+
+
+Future<Containerizer::Termination> MesosContainerizerProcess::wait(
+    const ContainerID& containerId)
+{
+  if (!promises.contains(containerId)) {
+    return Failure("Unknown container: " + stringify(containerId));
+  }
+
+  return promises[containerId]->future();
+}
+
+
+Future<Nothing> MesosContainerizerProcess::update(
+    const ContainerID& containerId,
+    const Resources& _resources)
+{
+  if (!resources.contains(containerId)) {
+    return Failure("Unknown container: " + stringify(containerId));
+  }
+
+  // Store the resources for usage().
+  resources.put(containerId, _resources);
+
+  // Update each isolator.
+  list<Future<Nothing> > futures;
+  foreach (const Owned<Isolator>& isolator, isolators) {
+    futures.push_back(isolator->update(containerId, _resources));
+  }
+
+  // Wait for all isolators to complete.
+  return collect(futures)
+    .then(lambda::bind(_nothing));
+}
+
+
+// Resources are used to set the limit fields in the statistics but are
+// optional because they aren't known after recovery until/unless update() is
+// called.
+Future<ResourceStatistics> _usage(
+    const ContainerID& containerId,
+    const Option<Resources>& resources,
+    const list<Future<ResourceStatistics> >& statistics)
+{
+  ResourceStatistics result;
+
+  // Set the timestamp now we have all statistics.
+  result.set_timestamp(Clock::now().secs());
+
+  foreach (const Future<ResourceStatistics>& statistic, statistics) {
+    if (statistic.isReady()) {
+      result.MergeFrom(statistic.get());
+    } else {
+      LOG(WARNING) << "Skipping resource statistic for container "
+                   << containerId << " because: "
+                   << (statistic.isFailed() ? statistic.failure()
+                                            : "discarded");
+    }
+  }
+
+  if (resources.isSome()) {
+    // Set the resource allocations.
+    Option<Bytes> mem = resources.get().mem();
+    if (mem.isSome()) {
+      result.set_mem_limit_bytes(mem.get().bytes());
+    }
+
+    Option<double> cpus = resources.get().cpus();
+    if (cpus.isSome()) {
+      result.set_cpus_limit(cpus.get());
+    }
+  }
+
+  return result;
+}
+
+
+Future<ResourceStatistics> MesosContainerizerProcess::usage(
+    const ContainerID& containerId)
+{
+  if (!promises.contains(containerId)) {
+    return Failure("Unknown container: " + stringify(containerId));
+  }
+
+  list<Future<ResourceStatistics> > futures;
+  foreach (const Owned<Isolator>& isolator, isolators) {
+    futures.push_back(isolator->usage(containerId));
+  }
+
+  // Use await() here so we can return partial usage statistics.
+  // TODO(idownes): After recovery resources won't be known until after an
+  // update() because they aren't part of the SlaveState.
+  return await(futures)
+    .then(lambda::bind(
+          _usage, containerId, resources.get(containerId), lambda::_1));
+}
+
+
+void MesosContainerizerProcess::destroy(const ContainerID& containerId)
+{
+  if (!promises.contains(containerId)) {
+    LOG(WARNING) << "Ignoring destroy of unknown container: " << containerId;
+    return;
+  }
+
+  if (destroying.contains(containerId)) {
+    // Destroy has already been initiated.
+    return;
+  }
+  destroying.insert(containerId);
+
+  LOG(INFO) << "Destroying container '" << containerId << "'";
+
+  if (statuses.contains(containerId)) {
+    // Kill all processes then continue destruction.
+    launcher->destroy(containerId)
+      .onAny(defer(self(), &Self::_destroy, containerId, lambda::_1));
+  } else {
+    // The executor never forked so no processes to kill, go straight to
+    // __destroy() with status = None().
+    __destroy(containerId, None());
+  }
+}
+
+
+void MesosContainerizerProcess::_destroy(
+    const ContainerID& containerId,
+    const Future<Nothing>& future)
+{
+  // Something has gone wrong and the launcher wasn't able to kill all the
+  // processes in the container. We cannot clean up the isolators because they
+  // may require that all processes have exited so just return the failure to
+  // the slave.
+  // TODO(idownes): This is a pretty bad state to be in but we should consider
+  // cleaning up here.
+  if (!future.isReady()) {
+    promises[containerId]->fail(
+        "Failed to destroy container: " +
+        future.isFailed() ? future.failure() : "discarded future");
+    return;
+  }
+
+  // We've successfully killed all processes in the container so get the exit
+  // status of the executor when it's ready (it may already be) and continue
+  // the destroy.
+  statuses.get(containerId).get()
+    .onAny(defer(self(), &Self::__destroy, containerId, lambda::_1));
+}
+
+
+void MesosContainerizerProcess::__destroy(
+    const ContainerID& containerId,
+    const Future<Option<int > >& status)
+{
+  // A container is 'killed' if any isolator limited it.
+  // Note: We may not see a limitation in time for it to be registered. This
+  // could occur if the limitation (e.g., an OOM) killed the executor and we
+  // triggered destroy() off the executor exit.
+  bool killed = false;
+  string message;
+  if (limitations.contains(containerId)) {
+    killed = true;
+    foreach (const Limitation& limitation, limitations.get(containerId)) {
+      message += limitation.message;
+    }
+    message = strings::trim(message);
+  } else {
+    message = "Executor terminated";
+  }
+
+  // We can now clean up all isolators.
+  foreach (const Owned<Isolator>& isolator, isolators) {
+    isolator->cleanup(containerId);
+  }
+
+  promises[containerId]->set(Containerizer::Termination(
+        status.isReady() ? status.get() : None(),
+        killed,
+        message));
+
+  promises.erase(containerId);
+  statuses.erase(containerId);
+  limitations.erase(containerId);
+  resources.erase(containerId);
+  destroying.erase(containerId);
+}
+
+
+void MesosContainerizerProcess::exited(const ContainerID& containerId)
+{
+  if (!promises.contains(containerId)) {
+    return;
+  }
+
+  LOG(INFO) << "Executor for container '" << containerId << "' has exited";
+
+  // The executor has exited so destroy the container.
+  destroy(containerId);
+}
+
+
+void MesosContainerizerProcess::limited(
+    const ContainerID& containerId,
+    const Future<Limitation>& future)
+{
+  if (!promises.contains(containerId)) {
+    return;
+  }
+
+  if (future.isReady()) {
+    LOG(INFO) << "Container " << containerId << " has reached its limit for"
+              << " resource " << future.get().resource
+              << " and will be terminated";
+    limitations.put(containerId, future.get());
+  } else {
+    // TODO(idownes): A discarded future will not be an error when isolators
+    // discard their promises after cleanup.
+    LOG(ERROR) << "Error in a resource limitation for container "
+               << containerId << ": " << (future.isFailed() ? future.failure()
+                                                            : "discarded");
+  }
+
+  // The container has been affected by the limitation so destroy it.
+  destroy(containerId);
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/containerizer/mesos_containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos_containerizer.hpp b/src/slave/containerizer/mesos_containerizer.hpp
new file mode 100644
index 0000000..f302c34
--- /dev/null
+++ b/src/slave/containerizer/mesos_containerizer.hpp
@@ -0,0 +1,208 @@
+/**
+ * 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 __MESOS_CONTAINERIZER_HPP__
+#define __MESOS_CONTAINERIZER_HPP__
+
+#include <list>
+#include <vector>
+
+#include <stout/hashmap.hpp>
+#include <stout/lambda.hpp>
+#include <stout/multihashmap.hpp>
+
+#include "slave/containerizer/containerizer.hpp"
+#include "slave/containerizer/isolator.hpp"
+#include "slave/containerizer/launcher.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// Forward declaration.
+class MesosContainerizerProcess;
+
+class MesosContainerizer : public Containerizer
+{
+public:
+  MesosContainerizer(
+      const Flags& flags,
+      bool local,
+      const process::Owned<Launcher>& launcher,
+      const std::vector<process::Owned<Isolator> >& isolators);
+
+  virtual ~MesosContainerizer();
+
+  virtual process::Future<Nothing> recover(
+      const Option<state::SlaveState>& state);
+
+  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>& slavePid,
+      bool checkpoint);
+
+  virtual process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources);
+
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId);
+
+  virtual process::Future<Containerizer::Termination> wait(
+      const ContainerID& containerId);
+
+  virtual void destroy(const ContainerID& containerId);
+
+private:
+  MesosContainerizerProcess* process;
+};
+
+
+class MesosContainerizerProcess : public process::Process<MesosContainerizerProcess>
+{
+public:
+  MesosContainerizerProcess(
+      const Flags& _flags,
+      bool _local,
+      const process::Owned<Launcher>& _launcher,
+      const std::vector<process::Owned<Isolator> >& _isolators)
+    : flags(_flags),
+      local(_local),
+      launcher(_launcher),
+      isolators(_isolators) {}
+
+  virtual ~MesosContainerizerProcess() {}
+
+  process::Future<Nothing> recover(
+      const Option<state::SlaveState>& state);
+
+  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>& slavePid,
+      bool checkpoint);
+
+  process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources);
+
+  process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId);
+
+  process::Future<Containerizer::Termination> wait(
+      const ContainerID& containerId);
+
+  void destroy(const ContainerID& containerId);
+
+private:
+  process::Future<Nothing> _recover(
+      const std::list<state::RunState>& recovered);
+
+  process::Future<Nothing> prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo,
+      const std::string& directory,
+      const Option<std::string>& user);
+
+  process::Future<Nothing> fetch(
+      const ContainerID& containerId,
+      const CommandInfo& commandInfo,
+      const std::string& directory,
+      const Option<std::string>& user);
+
+  process::Future<pid_t> fork(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo,
+      lambda::function<int()>& inChild,
+      const SlaveID& slaveId,
+      bool checkpoint,
+      int pipeRead);
+
+  process::Future<Nothing> isolate(
+      const ContainerID& containerId,
+      pid_t _pid);
+
+  process::Future<Nothing> _isolate(
+      const ContainerID& containerId,
+      const std::list<Option<CommandInfo> >& commands);
+
+  process::Future<Nothing> exec(
+      const ContainerID& containerId,
+      int pipeWrite);
+
+  // Continues 'destroy()' once all processes have been killed by the launcher.
+  void _destroy(
+      const ContainerID& containerId,
+      const process::Future<Nothing>& future);
+
+  // Continues (and completes) '_destroy()' once we get the exit status of the
+  // executor.
+  void __destroy(
+      const ContainerID& containerId,
+      const process::Future<Option<int > >& status);
+
+  // Call back for when an isolator limits a container and impacts the
+  // processes. This will trigger container destruction.
+  void limited(
+      const ContainerID& containerId,
+      const process::Future<Limitation>& future);
+
+  // Call back for when the executor exits. This will trigger container
+  // destroy.
+  void exited(const ContainerID& containerId);
+
+  const Flags flags;
+  const bool local;
+  const process::Owned<Launcher> launcher;
+  const std::vector<process::Owned<Isolator> > isolators;
+
+  // TODO(idownes): Consider putting these per-container variables into a
+  // struct.
+  // Promises for futures returned from wait().
+  hashmap<ContainerID,
+    process::Owned<process::Promise<Containerizer::Termination> > > promises;
+
+  // We need to keep track of the future exit status for each executor because
+  // we'll only get a single notification when the executor exits.
+  hashmap<ContainerID, process::Future<Option<int> > > statuses;
+
+  // We keep track of any limitations received from each isolator so we can
+  // determine the cause of an executor termination.
+  multihashmap<ContainerID, Limitation> limitations;
+
+  // We keep track of the resources for each container so we can set the
+  // ResourceStatistics limits in usage().
+  hashmap<ContainerID, Resources> resources;
+
+  // Set of containers that are in process of being destroyed.
+  hashset<ContainerID> destroying;
+};
+
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MESOS_CONTAINERIZER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 827b2d0..e4d98a5 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -49,6 +49,12 @@ public:
         "Total consumable resources per slave, in\n"
         "the form 'name(role):value;name(role):value...'.");
 
+    add(&Flags::isolation,
+        "isolation",
+        "Isolation mechanisms to use, e.g., 'posix/cpu,posix/mem'\n"
+        "or 'cgroups/cpu,cgroups/mem'.",
+        "posix/cpu,posix/mem");
+
     add(&Flags::default_role,
         "default_role",
         "Any resources in the --resources flag that\n"
@@ -167,7 +173,7 @@ public:
     add(&Flags::cgroups_hierarchy,
         "cgroups_hierarchy",
         "The path to the cgroups hierarchy root\n",
-        "/cgroup");
+        "/sys/fs/cgroup");
 
     add(&Flags::cgroups_root,
         "cgroups_root",
@@ -176,8 +182,8 @@ public:
 
     add(&Flags::cgroups_subsystems,
         "cgroups_subsystems",
-        "List of subsystems to enable (e.g., 'cpu,freezer')\n",
-        "cpu,memory,freezer");
+        "This flag has been deprecated and is no longer used,\n"
+        "please update your flags");
 
     add(&Flags::cgroups_enable_cfs,
         "cgroups_enable_cfs",
@@ -189,6 +195,7 @@ public:
 
   Option<std::string> hostname;
   Option<std::string> resources;
+  std::string isolation;
   std::string default_role;
   Option<std::string> attributes;
   std::string work_dir;
@@ -208,7 +215,7 @@ public:
 #ifdef __linux__
   std::string cgroups_hierarchy;
   std::string cgroups_root;
-  std::string cgroups_subsystems;
+  Option<std::string> cgroups_subsystems;
   bool cgroups_enable_cfs;
 #endif
 };

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index c4f598f..7c4cfba 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -201,7 +201,7 @@ JSON::Object model(const Executor& executor)
   object.values["id"] = executor.id.value();
   object.values["name"] = executor.info.name();
   object.values["source"] = executor.info.source();
-  object.values["uuid"] = executor.uuid.toString();
+  object.values["container"] = executor.containerId.value();
   object.values["directory"] = executor.directory;
   object.values["resources"] = model(executor.resources);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/isolator.cpp b/src/slave/isolator.cpp
deleted file mode 100644
index c9643cf..0000000
--- a/src/slave/isolator.cpp
+++ /dev/null
@@ -1,53 +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.
- */
-
-#include "isolator.hpp"
-#include "process_isolator.hpp"
-#ifdef __linux__
-#include "cgroups_isolator.hpp"
-#endif
-
-
-namespace mesos {
-namespace internal {
-namespace slave {
-
-Isolator* Isolator::create(const std::string &type)
-{
-  if (type == "process") {
-    return new ProcessIsolator();
-#ifdef __linux__
-  } else if (type == "cgroups") {
-    return new CgroupsIsolator();
-#endif
-  }
-
-  return NULL;
-}
-
-
-void Isolator::destroy(Isolator* isolator)
-{
-  if (isolator != NULL) {
-    delete isolator;
-  }
-}
-
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/isolator.hpp
----------------------------------------------------------------------
diff --git a/src/slave/isolator.hpp b/src/slave/isolator.hpp
deleted file mode 100644
index 9634535..0000000
--- a/src/slave/isolator.hpp
+++ /dev/null
@@ -1,110 +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 __ISOLATOR_HPP__
-#define __ISOLATOR_HPP__
-
-#include <unistd.h>
-
-#include <string>
-
-#include <mesos/mesos.hpp>
-#include <mesos/resources.hpp>
-
-#include <process/future.hpp>
-#include <process/process.hpp>
-
-#include <stout/hashmap.hpp>
-#include <stout/nothing.hpp>
-#include <stout/option.hpp>
-#include <stout/uuid.hpp>
-
-#include "slave/flags.hpp"
-
-namespace mesos {
-namespace internal {
-namespace slave {
-namespace state {
-
-struct SlaveState; // Forward declaration.
-
-} // namespace state {
-
-// Forward declaration.
-class Slave;
-
-
-class Isolator : public process::Process<Isolator>
-{
-public:
-  static Isolator* create(const std::string& type);
-  static void destroy(Isolator* isolator);
-
-  virtual ~Isolator() {}
-
-  // Explicitely use 'initialize' since we're overloading below.
-  using process::ProcessBase::initialize;
-
-  // Called during slave initialization.
-  virtual void initialize(
-      const Flags& flags,
-      const Resources& resources,
-      bool local,
-      const process::PID<Slave>& slave) = 0;
-
-  // Called by the slave to launch an executor for a given framework.
-  // If 'checkpoint' is true, the isolator is expected to checkpoint
-  // the executor pid to the 'path'.
-  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) = 0;
-
-  // Terminate a framework's executor, if it is still running.
-  // The executor is expected to be gone after this method exits.
-  virtual void killExecutor(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId) = 0;
-
-  // Update the resource limits for a given framework. This method will
-  // be called only after an executor for the framework is started.
-  virtual void resourcesChanged(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId,
-      const Resources& resources) = 0;
-
-  // Returns the resource usage for the isolator.
-  virtual process::Future<ResourceStatistics> usage(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId) = 0;
-
-  // Recover executors.
-  virtual process::Future<Nothing> recover(
-      const Option<state::SlaveState>& state) = 0;
-};
-
-
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/main.cpp
----------------------------------------------------------------------
diff --git a/src/slave/main.cpp b/src/slave/main.cpp
index 44020af..8aba4ed 100644
--- a/src/slave/main.cpp
+++ b/src/slave/main.cpp
@@ -67,12 +67,6 @@ int main(int argc, char** argv)
   uint16_t port;
   flags.add(&port, "port", "Port to listen on", SlaveInfo().port());
 
-  string isolation;
-  flags.add(&isolation,
-            "isolation",
-            "Isolation mechanism, may be one of: process, cgroups",
-            "process");
-
   Option<string> master;
   flags.add(&master,
             "master",
@@ -127,11 +121,10 @@ int main(int argc, char** argv)
     LOG(INFO) << "Git SHA: " << build::GIT_SHA.get();
   }
 
-  LOG(INFO) << "Creating \"" << isolation << "\" isolator";
-
-  Isolator* isolator = Isolator::create(isolation);
-  if (isolator == NULL) {
-    EXIT(1) << "Unrecognized isolation type: " << isolation;
+  Try<Containerizer*> containerizer = Containerizer::create(flags, false);
+  if (containerizer.isError()) {
+    EXIT(1) << "Failed to create a containerizer: "
+            << containerizer.error();
   }
 
   Try<MasterDetector*> detector = MasterDetector::create(master.get());
@@ -142,14 +135,15 @@ int main(int argc, char** argv)
   LOG(INFO) << "Starting Mesos slave";
 
   Files files;
-  Slave* slave = new Slave(flags, false,  detector.get(), isolator, &files);
+  Slave* slave = new Slave(flags, detector.get(), containerizer.get(), &files);
   process::spawn(slave);
 
   process::wait(slave->self());
   delete slave;
 
   delete detector.get();
-  Isolator::destroy(isolator);
+
+  delete containerizer.get();
 
   return 0;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/monitor.cpp
----------------------------------------------------------------------
diff --git a/src/slave/monitor.cpp b/src/slave/monitor.cpp
index bb3723e..1c02986 100644
--- a/src/slave/monitor.cpp
+++ b/src/slave/monitor.cpp
@@ -36,7 +36,7 @@
 #include <stout/lambda.hpp>
 #include <stout/protobuf.hpp>
 
-#include "slave/isolator.hpp"
+#include "slave/containerizer/containerizer.hpp"
 #include "slave/monitor.hpp"
 
 using namespace process;
@@ -60,123 +60,116 @@ const size_t MONITORING_TIME_SERIES_CAPACITY = 1000;
 const size_t MONITORING_ARCHIVED_TIME_SERIES = 25;
 
 
-Future<Nothing> ResourceMonitorProcess::watch(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId,
+Future<Nothing> ResourceMonitorProcess::start(
+    const ContainerID& containerId,
     const ExecutorInfo& executorInfo,
     const Duration& interval)
 {
-  if (executors.contains(frameworkId) &&
-      executors[frameworkId].contains(executorId)) {
-    return Failure("Already watched");
+  if (monitored.contains(containerId)) {
+    return Failure("Already monitored");
   }
 
-  executors[frameworkId][executorId] =
+  monitored[containerId] =
       MonitoringInfo(executorInfo,
                      MONITORING_TIME_SERIES_WINDOW,
                      MONITORING_TIME_SERIES_CAPACITY);
 
   // Schedule the resource collection.
-  delay(interval, self(), &Self::collect, frameworkId, executorId, interval);
+  delay(interval, self(), &Self::collect, containerId, interval);
 
   return Nothing();
 }
 
 
-Future<Nothing> ResourceMonitorProcess::unwatch(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId)
+Future<Nothing> ResourceMonitorProcess::stop(
+    const ContainerID& containerId)
 {
-  if (!executors.contains(frameworkId) ||
-      !executors[frameworkId].contains(executorId)) {
-    return Failure("Not watched");
+  if (!monitored.contains(containerId)) {
+    return Failure("Not monitored");
   }
 
   // Add the monitoring information to the archive.
-  archive.push_back(executors[frameworkId][executorId]);
-  executors[frameworkId].erase(executorId);
-
-  if (executors[frameworkId].empty()) {
-    executors.erase(frameworkId);
-  }
+  archive.push_back(monitored[containerId]);
+  monitored.erase(containerId);
 
   return Nothing();
 }
 
 
 void ResourceMonitorProcess::collect(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId,
+    const ContainerID& containerId,
     const Duration& interval)
 {
-  // Has the executor been unwatched?
-  if (!executors.contains(frameworkId) ||
-      !executors[frameworkId].contains(executorId)) {
+  // Has monitoring stopped?
+  if (!monitored.contains(containerId)) {
     return;
   }
 
-  dispatch(isolator, &Isolator::usage, frameworkId, executorId)
+  containerizer->usage(containerId)
     .onAny(defer(self(),
                  &Self::_collect,
                  lambda::_1,
-                 frameworkId,
-                 executorId,
+                 containerId,
                  interval));
 }
 
 
 void ResourceMonitorProcess::_collect(
     const Future<ResourceStatistics>& statistics,
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId,
+    const ContainerID& containerId,
     const Duration& interval)
 {
-  // Has the executor been unwatched?
-  if (!executors.contains(frameworkId) ||
-      !executors[frameworkId].contains(executorId)) {
+  // Has monitoring been stopped?
+  if (!monitored.contains(containerId)) {
     return;
   }
 
+  const ExecutorID& executorId =
+    monitored[containerId].executorInfo.executor_id();
+  const FrameworkID& frameworkId =
+    monitored[containerId].executorInfo.framework_id();
+
   if (statistics.isDiscarded()) {
-    // Note that the isolator may have terminated, causing pending
-    // dispatches to be deleted.
-    VLOG(1) << "Ignoring discarded future collecting resource usage for "
-            << "executor '" << executorId
+    VLOG(1) << "Ignoring discarded future collecting resource usage for"
+            << " container '" << containerId
+            << "' for executor '" << executorId
             << "' of framework '" << frameworkId << "'";
   } else if (statistics.isFailed()) {
-    // TODO(bmahler): Have the Isolators discard the result when the
+    // TODO(bmahler): Have the Containerizer discard the result when the
     // executor was killed or completed.
     VLOG(1)
-      << "Failed to collect resource usage for executor '" << executorId
+      << "Failed to collect resource usage for"
+      << " container '" << containerId
+      << "' for executor '" << executorId
       << "' of framework '" << frameworkId << "': " << statistics.failure();
   } else {
     Try<Time> time = Time::create(statistics.get().timestamp());
 
     if (time.isError()) {
       LOG(ERROR) << "Invalid timestamp " << statistics.get().timestamp()
-                 << " for executor '" << executorId
+                 << " for container '" << containerId
+                 << "' for executor '" << executorId
                  << "' of framework '" << frameworkId << ": " << time.error();
     } else {
       // Add the statistics to the time series.
-      executors[frameworkId][executorId].statistics.set(
+      monitored[containerId].statistics.set(
           statistics.get(), time.get());
     }
   }
 
   // Schedule the next collection.
-  delay(interval, self(), &Self::collect, frameworkId, executorId, interval);
+  delay(interval, self(), &Self::collect, containerId, interval);
 }
 
 
 ResourceMonitorProcess::Usage ResourceMonitorProcess::usage(
-    const FrameworkID& frameworkId,
+    const ContainerID& containerId,
     const ExecutorInfo& executorInfo)
 {
   Usage usage;
-  usage.frameworkId = frameworkId;
+  usage.containerId = containerId;
   usage.executorInfo = executorInfo;
-  usage.statistics = dispatch(
-      isolator, &Isolator::usage, frameworkId, executorInfo.executor_id());
+  usage.statistics = containerizer->usage(containerId);
 
   return usage;
 }
@@ -196,12 +189,12 @@ Future<http::Response> ResourceMonitorProcess::_statistics(
   list<Usage> usages;
   list<Future<ResourceStatistics> > futures;
 
-  foreachkey (const FrameworkID& frameworkId, executors) {
-    foreachvalue (const MonitoringInfo& info, executors[frameworkId]) {
-      // TODO(bmahler): Consider a batch usage API on the Isolator.
-      usages.push_back(usage(frameworkId, info.executorInfo));
-      futures.push_back(usages.back().statistics);
-    }
+  foreachpair (const ContainerID& containerId,
+               const MonitoringInfo& info,
+               monitored) {
+    // TODO(bmahler): Consider a batch usage API on the Containerizer.
+    usages.push_back(usage(containerId, info.executorInfo));
+    futures.push_back(usages.back().statistics);
   }
 
   return process::await(futures)
@@ -217,9 +210,10 @@ Future<http::Response> ResourceMonitorProcess::__statistics(
 
   foreach (const Usage& usage, usages) {
     if (usage.statistics.isFailed()) {
-      LOG(WARNING) << "Failed to get resource usage for executor "
-                   << usage.executorInfo.executor_id()
-                   << " of framework " << usage.frameworkId
+      LOG(WARNING) << "Failed to get resource usage for "
+                   << " container " << usage.containerId
+                   << " for executor " << usage.executorInfo.executor_id()
+                   << " of framework " << usage.executorInfo.framework_id()
                    << ": " << usage.statistics.failure();
       continue;
     } else if (usage.statistics.isDiscarded()) {
@@ -227,7 +221,7 @@ Future<http::Response> ResourceMonitorProcess::__statistics(
     }
 
     JSON::Object entry;
-    entry.values["framework_id"] = usage.frameworkId.value();
+    entry.values["framework_id"] = usage.executorInfo.framework_id().value();
     entry.values["executor_id"] = usage.executorInfo.executor_id().value();
     entry.values["executor_name"] = usage.executorInfo.name();
     entry.values["source"] = usage.executorInfo.source();
@@ -246,7 +240,7 @@ const string ResourceMonitorProcess::STATISTICS_HELP = HELP(
     USAGE(
         "/statistics.json"),
     DESCRIPTION(
-        "Returns the current resource consumption data for executors",
+        "Returns the current resource consumption data for containers",
         "running under this slave.",
         "",
         "Example:",
@@ -276,9 +270,9 @@ const string ResourceMonitorProcess::STATISTICS_HELP = HELP(
         "```"));
 
 
-ResourceMonitor::ResourceMonitor(Isolator* isolator)
+ResourceMonitor::ResourceMonitor(Containerizer* containerizer)
 {
-  process = new ResourceMonitorProcess(isolator);
+  process = new ResourceMonitorProcess(containerizer);
   spawn(process);
 }
 
@@ -291,28 +285,24 @@ ResourceMonitor::~ResourceMonitor()
 }
 
 
-Future<Nothing> ResourceMonitor::watch(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId,
+Future<Nothing> ResourceMonitor::start(
+    const ContainerID& containerId,
     const ExecutorInfo& executorInfo,
     const Duration& interval)
 {
   return dispatch(
       process,
-      &ResourceMonitorProcess::watch,
-      frameworkId,
-      executorId,
+      &ResourceMonitorProcess::start,
+      containerId,
       executorInfo,
       interval);
 }
 
 
-Future<Nothing> ResourceMonitor::unwatch(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId)
+Future<Nothing> ResourceMonitor::stop(
+    const ContainerID& containerId)
 {
-  return dispatch(
-      process, &ResourceMonitorProcess::unwatch, frameworkId, executorId);
+  return dispatch(process, &ResourceMonitorProcess::stop, containerId);
 }
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/monitor.hpp
----------------------------------------------------------------------
diff --git a/src/slave/monitor.hpp b/src/slave/monitor.hpp
index b677410..c042bc1 100644
--- a/src/slave/monitor.hpp
+++ b/src/slave/monitor.hpp
@@ -19,6 +19,7 @@
 #ifndef __SLAVE_MONITOR_HPP__
 #define __SLAVE_MONITOR_HPP__
 
+#include <map>
 #include <string>
 
 #include <boost/circular_buffer.hpp>
@@ -43,7 +44,7 @@ namespace internal {
 namespace slave {
 
 // Forward declarations.
-class Isolator;
+class Containerizer;
 class ResourceMonitorProcess;
 
 
@@ -54,7 +55,7 @@ const extern size_t MONITORING_TIME_SERIES_CAPACITY;
 const extern size_t MONITORING_ARCHIVED_TIME_SERIES;
 
 
-// Provides resource monitoring for executors. Resource usage time
+// Provides resource monitoring for containers. Resource usage time
 // series are stored using the Statistics module. Usage information
 // is also exported via a JSON endpoint.
 // TODO(bmahler): Forward usage information to the master.
@@ -68,22 +69,20 @@ const extern size_t MONITORING_ARCHIVED_TIME_SERIES;
 class ResourceMonitor
 {
 public:
-  ResourceMonitor(Isolator* isolator);
+  ResourceMonitor(Containerizer* containerizer);
   ~ResourceMonitor();
 
-  // Starts monitoring resources for the given executor.
-  // Returns a failure if the executor is already being watched.
-  process::Future<Nothing> watch(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId,
+  // Starts monitoring resources for the given container.
+  // Returns a failure if the container is already being watched.
+  process::Future<Nothing> start(
+      const ContainerID& containerId,
       const ExecutorInfo& executorInfo,
       const Duration& interval);
 
-  // Stops monitoring resources for the given executor.
-  // Returns a failure if the executor is unknown to the monitor.
-  process::Future<Nothing> unwatch(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId);
+  // Stops monitoring resources for the given container.
+  // Returns a failure if the container is unknown to the monitor.
+  process::Future<Nothing> stop(
+      const ContainerID& containerId);
 
 private:
   ResourceMonitorProcess* process;
@@ -93,23 +92,21 @@ private:
 class ResourceMonitorProcess : public process::Process<ResourceMonitorProcess>
 {
 public:
-  ResourceMonitorProcess(Isolator* _isolator)
+  ResourceMonitorProcess(Containerizer* _containerizer)
     : ProcessBase("monitor"),
-      isolator(_isolator),
+      containerizer(_containerizer),
       limiter(2, Seconds(1)), // 2 permits per second.
       archive(MONITORING_ARCHIVED_TIME_SERIES) {}
 
   virtual ~ResourceMonitorProcess() {}
 
-  process::Future<Nothing> watch(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId,
+  process::Future<Nothing> start(
+      const ContainerID& containerId,
       const ExecutorInfo& executorInfo,
       const Duration& interval);
 
-  process::Future<Nothing> unwatch(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId);
+  process::Future<Nothing> stop(
+      const ContainerID& containerId);
 
 protected:
   virtual void initialize()
@@ -125,26 +122,24 @@ protected:
 
 private:
   void collect(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId,
+      const ContainerID& containerId,
       const Duration& interval);
   void _collect(
       const process::Future<ResourceStatistics>& statistics,
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId,
+      const ContainerID& containerId,
       const Duration& interval);
 
   // This is a convenience struct for bundling usage information.
   struct Usage
   {
-    FrameworkID frameworkId;
+    ContainerID containerId;
     ExecutorInfo executorInfo;
     process::Future<ResourceStatistics> statistics;
   };
 
   // Helper for returning the usage for a particular executor.
   Usage usage(
-      const FrameworkID& frameworkId,
+      const ContainerID& containerId,
       const ExecutorInfo& executorInfo);
 
   // HTTP Endpoints.
@@ -159,7 +154,7 @@ private:
 
   static const std::string STATISTICS_HELP;
 
-  Isolator* isolator;
+  Containerizer* containerizer;
 
   // Used to rate limit the statistics.json endpoint.
   process::RateLimiter limiter;
@@ -178,7 +173,8 @@ private:
     process::TimeSeries<ResourceStatistics> statistics;
   };
 
-  hashmap<FrameworkID, hashmap<ExecutorID, MonitoringInfo> > executors;
+  // The monitoring info is stored for each monitored container.
+  hashmap<ContainerID, MonitoringInfo> monitored;
 
   // Fixed-size history of monitoring information.
   boost::circular_buffer<MonitoringInfo> archive;

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/paths.hpp b/src/slave/paths.hpp
index 70ee0f3..41bb73d 100644
--- a/src/slave/paths.hpp
+++ b/src/slave/paths.hpp
@@ -192,7 +192,7 @@ inline std::string getExecutorRunPath(
     const SlaveID& slaveId,
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const UUID& executorUUID)
+    const ContainerID& containerId)
 {
   return strings::format(
       EXECUTOR_RUN_PATH,
@@ -200,7 +200,7 @@ inline std::string getExecutorRunPath(
       slaveId,
       frameworkId,
       executorId,
-      executorUUID.toString()).get();
+      containerId).get();
 }
 
 
@@ -209,7 +209,7 @@ inline std::string getExecutorSentinelPath(
     const SlaveID& slaveId,
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const UUID& executorUUID)
+    const ContainerID& containerId)
 {
   return strings::format(
       EXECUTOR_SENTINEL_PATH,
@@ -217,7 +217,7 @@ inline std::string getExecutorSentinelPath(
       slaveId,
       frameworkId,
       executorId,
-      executorUUID.toString()).get();
+      containerId).get();
 }
 
 
@@ -241,7 +241,7 @@ inline std::string getLibprocessPidPath(
     const SlaveID& slaveId,
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const UUID& executorUUID)
+    const ContainerID& containerId)
 {
   return strings::format(
       LIBPROCESS_PID_PATH,
@@ -249,7 +249,7 @@ inline std::string getLibprocessPidPath(
       slaveId,
       frameworkId,
       executorId,
-      executorUUID.toString()).get();
+      containerId).get();
 }
 
 
@@ -258,7 +258,7 @@ inline std::string getForkedPidPath(
     const SlaveID& slaveId,
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const UUID& executorUUID)
+    const ContainerID& containerId)
 {
   return strings::format(
       FORKED_PID_PATH,
@@ -266,7 +266,7 @@ inline std::string getForkedPidPath(
       slaveId,
       frameworkId,
       executorId,
-      executorUUID.toString()).get();
+      containerId).get();
 }
 
 
@@ -275,7 +275,7 @@ inline std::string getTaskPath(
     const SlaveID& slaveId,
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const UUID& executorUUID,
+    const ContainerID& containerId,
     const TaskID& taskId)
 {
   return strings::format(
@@ -284,7 +284,7 @@ inline std::string getTaskPath(
       slaveId,
       frameworkId,
       executorId,
-      executorUUID.toString(),
+      containerId,
       taskId).get();
 }
 
@@ -294,7 +294,7 @@ inline std::string getTaskInfoPath(
     const SlaveID& slaveId,
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const UUID& executorUUID,
+    const ContainerID& containerId,
     const TaskID& taskId)
 {
   return strings::format(
@@ -303,7 +303,7 @@ inline std::string getTaskInfoPath(
       slaveId,
       frameworkId,
       executorId,
-      executorUUID.toString(),
+      containerId,
       taskId).get();
 }
 
@@ -313,7 +313,7 @@ inline std::string getTaskUpdatesPath(
     const SlaveID& slaveId,
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const UUID& executorUUID,
+    const ContainerID& containerId,
     const TaskID& taskId)
 {
   return strings::format(
@@ -322,7 +322,7 @@ inline std::string getTaskUpdatesPath(
       slaveId,
       frameworkId,
       executorId,
-      executorUUID.toString(),
+      containerId,
       taskId).get();
 }
 
@@ -332,10 +332,10 @@ inline std::string createExecutorDirectory(
     const SlaveID& slaveId,
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const UUID& executorUUID)
+    const ContainerID& containerId)
 {
   std::string directory =
-    getExecutorRunPath(rootDir, slaveId, frameworkId, executorId, executorUUID);
+    getExecutorRunPath(rootDir, slaveId, frameworkId, executorId, containerId);
 
   Try<Nothing> mkdir = os::mkdir(directory);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/process_isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/process_isolator.cpp b/src/slave/process_isolator.cpp
deleted file mode 100644
index 09cb996..0000000
--- a/src/slave/process_isolator.cpp
+++ /dev/null
@@ -1,516 +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.
- */
-
-#include <errno.h>
-#include <signal.h>
-#include <stdio.h> // For perror.
-#include <string.h>
-
-#include <list>
-#include <map>
-#include <set>
-
-#include <process/clock.hpp>
-#include <process/defer.hpp>
-#include <process/dispatch.hpp>
-#include <process/id.hpp>
-#include <process/reap.hpp>
-
-#include <stout/check.hpp>
-#include <stout/exit.hpp>
-#include <stout/foreach.hpp>
-#include <stout/lambda.hpp>
-#include <stout/nothing.hpp>
-#include <stout/option.hpp>
-#include <stout/os.hpp>
-#include <stout/uuid.hpp>
-
-#include "common/type_utils.hpp"
-
-#include "slave/flags.hpp"
-#include "slave/process_isolator.hpp"
-#include "slave/state.hpp"
-
-using namespace process;
-
-using std::map;
-using std::set;
-using std::string;
-
-using process::defer;
-using process::wait; // Necessary on some OS's to disambiguate.
-
-namespace mesos {
-namespace internal {
-namespace slave {
-
-using launcher::ExecutorLauncher;
-
-using state::SlaveState;
-using state::FrameworkState;
-using state::ExecutorState;
-using state::RunState;
-
-ProcessIsolator::ProcessIsolator()
-  : ProcessBase(ID::generate("process-isolator")),
-    local(false),
-    initialized(false) {}
-
-
-void ProcessIsolator::initialize(
-    const Flags& _flags,
-    const Resources& _,
-    bool _local,
-    const PID<Slave>& _slave)
-{
-  flags = _flags;
-  local = _local;
-  slave = _slave;
-
-  initialized = true;
-}
-
-
-void ProcessIsolator::launchExecutor(
-    const SlaveID& slaveId,
-    const FrameworkID& frameworkId,
-    const FrameworkInfo& frameworkInfo,
-    const ExecutorInfo& executorInfo,
-    const UUID& uuid,
-    const string& directory,
-    const Resources& resources)
-{
-  CHECK(initialized) << "Cannot launch executors before initialization!";
-
-  const ExecutorID& executorId = executorInfo.executor_id();
-
-  LOG(INFO) << "Launching " << executorId
-            << " (" << executorInfo.command().value() << ")"
-            << " in " << directory
-            << " with resources " << resources
-            << "' for framework " << frameworkId;
-
-  ProcessInfo* info = new ProcessInfo(frameworkId, executorId);
-
-  infos[frameworkId][executorId] = info;
-
-  // Use pipes to determine which child has successfully changed session.
-  int pipes[2];
-  if (pipe(pipes) < 0) {
-    PLOG(FATAL) << "Failed to create a pipe";
-  }
-
-  // Set the FD_CLOEXEC flags on these pipes
-  Try<Nothing> cloexec = os::cloexec(pipes[0]);
-  CHECK_SOME(cloexec) << "Error setting FD_CLOEXEC on pipe[0]";
-
-  cloexec = os::cloexec(pipes[1]);
-  CHECK_SOME(cloexec) << "Error setting FD_CLOEXEC on pipe[1]";
-
-  // Create the ExecutorLauncher instance before the fork for the
-  // child process to use.
-  ExecutorLauncher launcher(
-      slaveId,
-      frameworkId,
-      executorInfo.executor_id(),
-      uuid,
-      executorInfo.command(),
-      frameworkInfo.user(),
-      directory,
-      flags.work_dir,
-      slave,
-      flags.frameworks_home,
-      flags.hadoop_home,
-      !local,
-      flags.switch_user,
-      frameworkInfo.checkpoint(),
-      flags.recovery_timeout);
-
-  // We get the environment map for launching mesos-launcher before
-  // the fork, because we have seen deadlock issues with ostringstream
-  // in the forked process before it calls exec.
-  map<string, string> env = launcher.getLauncherEnvironment();
-
-  pid_t pid;
-  if ((pid = fork()) == -1) {
-    PLOG(FATAL) << "Failed to fork to launch new executor";
-  }
-
-  if (pid > 0) {
-    os::close(pipes[1]);
-
-    // Get the child's pid via the pipe.
-    if (read(pipes[0], &pid, sizeof(pid)) == -1) {
-      PLOG(FATAL) << "Failed to get child PID from pipe";
-    }
-
-    os::close(pipes[0]);
-
-    // In parent process.
-    LOG(INFO) << "Forked executor at " << pid;
-
-    // Record the pid (should also be the pgid since we setsid below).
-    infos[frameworkId][executorId]->pid = pid;
-
-    process::reap(pid)
-      .onAny(defer(PID<ProcessIsolator>(this),
-                   &ProcessIsolator::reaped,
-                   pid,
-                   lambda::_1));
-
-    // Tell the slave this executor has started.
-    dispatch(slave, &Slave::executorStarted, frameworkId, executorId, pid);
-  } else {
-    // In child process, we make cleanup easier by putting process
-    // into it's own session. DO NOT USE GLOG!
-    os::close(pipes[0]);
-
-    // NOTE: We setsid() in a loop because setsid() might fail if another
-    // process has the same process group id as the calling process.
-    while ((pid = setsid()) == -1) {
-      perror("Could not put executor in its own session");
-
-      std::cout << "Forking another process and retrying ..." << std::endl;
-
-      if ((pid = fork()) == -1) {
-        perror("Failed to fork to launch executor");
-        abort();
-      }
-
-      if (pid > 0) {
-        // In parent process.
-        exit(0);
-      }
-    }
-
-    if (write(pipes[1], &pid, sizeof(pid)) != sizeof(pid)) {
-      perror("Failed to write PID on pipe");
-      abort();
-    }
-
-    os::close(pipes[1]);
-
-    // Setup the environment for launcher.
-    foreachpair (const string& key, const string& value, env) {
-      os::setenv(key, value);
-    }
-
-    const char** args = (const char**) new char*[2];
-
-    // Determine path for mesos-launcher.
-    Result<string> realpath = os::realpath(
-        path::join(flags.launcher_dir, "mesos-launcher"));
-
-    if (!realpath.isSome()) {
-      EXIT(1) << "Failed to determine the canonical path "
-              << "for the mesos-launcher '"
-              << path::join(flags.launcher_dir, "mesos-launcher")
-              << "': "
-              << (realpath.isError()
-                  ? realpath.error()
-                  : "No such file or directory");
-    }
-
-    // Grab a copy of the path so that we can reliably use 'c_str()'.
-    const string& path = realpath.get();
-
-    args[0] = path.c_str();
-    args[1] = NULL;
-
-    // Execute the mesos-launcher!
-    execvp(args[0], (char* const*) args);
-
-    // If we get here, the execvp call failed.
-    perror("Failed to execvp the mesos-launcher");
-    abort();
-  }
-}
-
-// NOTE: This function can be called by the isolator itself or by the
-// slave if it doesn't hear about an executor exit after it sends a
-// shutdown message.
-void ProcessIsolator::killExecutor(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId)
-{
-  CHECK(initialized) << "Cannot kill executors before initialization!";
-
-  if (!infos.contains(frameworkId) ||
-      !infos[frameworkId].contains(executorId) ||
-      infos[frameworkId][executorId]->killed) {
-    LOG(ERROR) << "Asked to kill an unknown/killed executor! " << executorId;
-    return;
-  }
-
-  const Option<pid_t>& pid = infos[frameworkId][executorId]->pid;
-
-  if (pid.isSome()) {
-    // TODO(vinod): Call killtree on the pid of the actual executor process
-    // that is running the tasks (stored in the local storage by the
-    // executor module).
-    Try<std::list<os::ProcessTree> > trees =
-      os::killtree(pid.get(), SIGKILL, true, true);
-
-    if (trees.isError()) {
-      LOG(WARNING) << "Failed to kill the process tree rooted at pid "
-                   << pid.get() << ": " << trees.error();
-    } else {
-      LOG(INFO) << "Killed the following process trees:\n"
-                << stringify(trees.get());
-    }
-
-    // Also kill all processes that belong to the process group of the executor.
-    // This is valuable in situations where the top level executor process
-    // exited and hence killtree is unable to kill any spawned orphans.
-    // NOTE: This assumes that the process group id of the executor process is
-    // same as its pid (which is expected to be the case with setsid()).
-    // TODO(vinod): Also (recursively) kill processes belonging to the
-    // same session, but have a different process group id.
-    if (killpg(pid.get(), SIGKILL) == -1 && errno != ESRCH) {
-      PLOG(WARNING) << "Failed to kill process group " << pid.get();
-    }
-
-    infos[frameworkId][executorId]->killed = true;
-  }
-}
-
-
-void ProcessIsolator::resourcesChanged(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId,
-    const Resources& resources)
-{
-  CHECK(initialized) << "Cannot do resourcesChanged before initialization!";
-
-  if (!infos.contains(frameworkId) ||
-      !infos[frameworkId].contains(executorId) ||
-      infos[frameworkId][executorId]->killed) {
-    LOG(INFO) << "Asked to update resources for an unknown/killed executor '"
-              << executorId << "' of framework " << frameworkId;
-    return;
-  }
-
-  ProcessInfo* info = CHECK_NOTNULL(infos[frameworkId][executorId]);
-
-  info->resources = resources;
-
-  // Do nothing; subclasses may override this.
-}
-
-
-Future<Nothing> ProcessIsolator::recover(
-    const Option<SlaveState>& state)
-{
-  LOG(INFO) << "Recovering isolator";
-
-  if (state.isNone()) {
-    return Nothing();
-  }
-
-  foreachvalue (const FrameworkState& framework, state.get().frameworks) {
-    foreachvalue (const ExecutorState& executor, framework.executors) {
-      LOG(INFO) << "Recovering executor '" << executor.id
-                << "' of framework " << framework.id;
-
-      if (executor.info.isNone()) {
-        LOG(WARNING) << "Skipping recovery of executor '" << executor.id
-                     << "' of framework " << framework.id
-                     << " because its info cannot be recovered";
-        continue;
-      }
-
-      if (executor.latest.isNone()) {
-        LOG(WARNING) << "Skipping recovery of executor '" << executor.id
-                     << "' of framework " << framework.id
-                     << " because its latest run cannot be recovered";
-        continue;
-      }
-
-      // We are only interested in the latest run of the executor!
-      const UUID& uuid = executor.latest.get();
-      CHECK(executor.runs.contains(uuid));
-      const RunState& run  = executor.runs.get(uuid).get();
-
-      if (run.completed) {
-        VLOG(1) << "Skipping recovery of executor '" << executor.id
-                << "' of framework " << framework.id
-                << " because its latest run " << uuid << " is completed";
-        continue;
-      }
-
-      ProcessInfo* info =
-        new ProcessInfo(framework.id, executor.id, run.forkedPid);
-
-      infos[framework.id][executor.id] = info;
-
-      // Add the pid to the reaper to monitor exit status.
-      if (run.forkedPid.isSome()) {
-        process::reap(run.forkedPid.get())
-          .onAny(defer(PID<ProcessIsolator>(this),
-                       &ProcessIsolator::reaped,
-                       run.forkedPid.get(),
-                       lambda::_1));
-      }
-    }
-  }
-
-  return Nothing();
-}
-
-
-Future<ResourceStatistics> ProcessIsolator::usage(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId)
-{
-  if (!infos.contains(frameworkId) ||
-      !infos[frameworkId].contains(executorId) ||
-      infos[frameworkId][executorId]->killed) {
-    return Failure("Unknown/killed executor");
-  }
-
-  ProcessInfo* info = infos[frameworkId][executorId];
-  CHECK_NOTNULL(info);
-
-  ResourceStatistics result;
-
-  result.set_timestamp(Clock::now().secs());
-
-  // Set the resource allocations.
-  const Option<Bytes>& mem = info->resources.mem();
-  if (mem.isSome()) {
-    result.set_mem_limit_bytes(mem.get().bytes());
-  }
-
-  const Option<double>& cpus = info->resources.cpus();
-  if (cpus.isSome()) {
-    result.set_cpus_limit(cpus.get());
-  }
-
-  CHECK_SOME(info->pid);
-
-  Result<os::Process> process = os::process(info->pid.get());
-
-  if (!process.isSome()) {
-    return Failure(
-        process.isError() ? process.error() : "Process does not exist");
-  }
-
-  result.set_timestamp(Clock::now().secs());
-
-  if (process.get().rss.isSome()) {
-    result.set_mem_rss_bytes(process.get().rss.get().bytes());
-  }
-
-  // We only show utime and stime when both are available, otherwise
-  // we're exposing a partial view of the CPU times.
-  if (process.get().utime.isSome() && process.get().stime.isSome()) {
-    result.set_cpus_user_time_secs(process.get().utime.get().secs());
-    result.set_cpus_system_time_secs(process.get().stime.get().secs());
-  }
-
-  // Now aggregate all descendant process usage statistics.
-  const Try<set<pid_t> >& children = os::children(info->pid.get(), true);
-
-  if (children.isError()) {
-    return Failure(
-        "Failed to get children of " + stringify(info->pid.get()) + ": " +
-        children.error());
-  }
-
-  // Aggregate the usage of all child processes.
-  foreach (pid_t child, children.get()) {
-    process = os::process(child);
-
-    // Skip processes that disappear.
-    if (process.isNone()) {
-      continue;
-    }
-
-    if (process.isError()) {
-      LOG(WARNING) << "Failed to get status of descendant process " << child
-                   << " of parent " << info->pid.get() << ": "
-                   << process.error();
-      continue;
-    }
-
-    if (process.get().rss.isSome()) {
-      result.set_mem_rss_bytes(
-          result.mem_rss_bytes() + process.get().rss.get().bytes());
-    }
-
-    // We only show utime and stime when both are available, otherwise
-    // we're exposing a partial view of the CPU times.
-    if (process.get().utime.isSome() && process.get().stime.isSome()) {
-      result.set_cpus_user_time_secs(
-          result.cpus_user_time_secs() + process.get().utime.get().secs());
-      result.set_cpus_system_time_secs(
-          result.cpus_system_time_secs() + process.get().stime.get().secs());
-    }
-  }
-
-  return result;
-}
-
-
-void ProcessIsolator::reaped(pid_t pid, const Future<Option<int> >& status)
-{
-  foreachkey (const FrameworkID& frameworkId, infos) {
-    foreachkey (const ExecutorID& executorId, infos[frameworkId]) {
-      ProcessInfo* info = infos[frameworkId][executorId];
-
-      if (info->pid.isSome() && info->pid.get() == pid) {
-        if (!status.isReady()) {
-          LOG(ERROR) << "Failed to get the status for executor '" << executorId
-                     << "' of framework " << frameworkId << ": "
-                     << (status.isFailed() ? status.failure() : "discarded");
-          return;
-        }
-
-        LOG(INFO) << "Telling slave of terminated executor '" << executorId
-                  << "' of framework " << frameworkId;
-
-        dispatch(slave,
-                 &Slave::executorTerminated,
-                 frameworkId,
-                 executorId,
-                 status.get(),
-                 false,
-                 "Executor terminated");
-
-        if (!info->killed) {
-          // Try and cleanup after the executor.
-          killExecutor(frameworkId, executorId);
-        }
-
-        if (infos[frameworkId].size() == 1) {
-          infos.erase(frameworkId);
-        } else {
-          infos[frameworkId].erase(executorId);
-        }
-        delete info;
-
-        return;
-      }
-    }
-  }
-}
-
-
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/process_isolator.hpp
----------------------------------------------------------------------
diff --git a/src/slave/process_isolator.hpp b/src/slave/process_isolator.hpp
deleted file mode 100644
index bc52f33..0000000
--- a/src/slave/process_isolator.hpp
+++ /dev/null
@@ -1,117 +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 __PROCESS_ISOLATOR_HPP__
-#define __PROCESS_ISOLATOR_HPP__
-
-#include <string>
-
-#include <sys/types.h>
-
-#include <process/future.hpp>
-
-#include <stout/hashmap.hpp>
-#include <stout/nothing.hpp>
-#include <stout/option.hpp>
-#include <stout/uuid.hpp>
-
-#include "launcher/launcher.hpp"
-
-#include "slave/flags.hpp"
-#include "slave/isolator.hpp"
-#include "slave/slave.hpp"
-
-namespace mesos {
-namespace internal {
-namespace slave {
-
-class ProcessIsolator : public Isolator
-{
-public:
-  ProcessIsolator();
-
-  virtual void initialize(
-      const Flags& flags,
-      const Resources& resources,
-      bool local,
-      const process::PID<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);
-
-  virtual void killExecutor(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId);
-
-  virtual void resourcesChanged(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId,
-      const Resources& resources);
-
-  virtual process::Future<ResourceStatistics> usage(
-      const FrameworkID& frameworkId,
-      const ExecutorID& executorId);
-
-  virtual process::Future<Nothing> recover(
-      const Option<state::SlaveState>& state);
-
-
-private:
-  // No copying, no assigning.
-  ProcessIsolator(const ProcessIsolator&);
-  ProcessIsolator& operator = (const ProcessIsolator&);
-
-  struct ProcessInfo
-  {
-    ProcessInfo(const FrameworkID& _frameworkId,
-                const ExecutorID& _executorId,
-                const Option<pid_t>& _pid = None(),
-                bool _killed = false)
-      : frameworkId(_frameworkId),
-        executorId(_executorId),
-        pid(_pid),
-        killed(_killed) {}
-
-    FrameworkID frameworkId;
-    ExecutorID executorId;
-    Option<pid_t> pid; // PID of the forked executor process.
-    bool killed; // True if "killing" has been initiated via 'killExecutor'.
-    Resources resources; // Resources allocated to the process tree.
-  };
-
-  // TODO(benh): Make variables const by passing them via constructor.
-  Flags flags;
-  bool local;
-  process::PID<Slave> slave;
-  bool initialized;
-  hashmap<FrameworkID, hashmap<ExecutorID, ProcessInfo*> > infos;
-
-  void reaped(pid_t pid, const Future<Option<int> >& status);
-};
-
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __PROCESS_ISOLATOR_HPP__


[05/11] Containerizer (part 1)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 213df86..8ad955a 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -22,6 +22,7 @@
 #include <algorithm>
 #include <iomanip>
 #include <list>
+#include <map>
 #include <sstream>
 #include <string>
 #include <vector>
@@ -61,6 +62,7 @@
 #include "slave/status_update_manager.hpp"
 
 using std::list;
+using std::map;
 using std::string;
 using std::vector;
 
@@ -73,20 +75,18 @@ namespace slave {
 using namespace state;
 
 Slave::Slave(const slave::Flags& _flags,
-             bool _local,
              MasterDetector* _detector,
-             Isolator* _isolator,
+             Containerizer* _containerizer,
              Files* _files)
   : ProcessBase(ID::generate("slave")),
     state(RECOVERING),
     http(*this),
     flags(_flags),
-    local(_local),
     completedFrameworks(MAX_COMPLETED_FRAMEWORKS),
     detector(_detector),
-    isolator(_isolator),
+    containerizer(_containerizer),
     files(_files),
-    monitor(_isolator),
+    monitor(containerizer),
     statusUpdateManager(new StatusUpdateManager()),
     metaDir(paths::getMetaRootDir(flags.work_dir)),
     recoveryErrors(0) {}
@@ -115,101 +115,11 @@ void Slave::initialize()
   CHECK_SOME(os::mkdir(flags.work_dir))
     << "Failed to create slave work directory '" << flags.work_dir << "'";
 
-  // Properly set up resources.
-  // TODO(benh): Move this computation into Flags as the "default".
-  // TODO(vinod): Move some of this computation into Resources.
-  Try<Resources> parse = Resources::parse(
-      flags.resources.isSome() ? flags.resources.get() : "",
-      flags.default_role);
-  CHECK_SOME(parse);
-  resources = parse.get();
-
-  if (!resources.cpus().isSome()) {
-    double cpus;
-
-    Try<long> cpus_ = os::cpus();
-    if (!cpus_.isSome()) {
-      LOG(WARNING) << "Failed to auto-detect the number of cpus to use: '"
-                   << cpus_.error()
-                   << "' ; defaulting to " << DEFAULT_CPUS;
-      cpus = DEFAULT_CPUS;
-    } else {
-      cpus = cpus_.get();
-    }
-
-    Resource r = Resources::parse(
-        "cpus",
-        stringify(cpus),
-        flags.default_role).get();
-    resources += r;
-  }
-
-
-  if (!resources.mem().isSome()) {
-    Bytes mem;
-
-    Try<os::Memory> mem_ = os::memory();
-    if (!mem_.isSome()) {
-      LOG(WARNING) << "Failed to auto-detect the size of main memory: '"
-                   << mem_.error()
-                   << "' ; defaulting to " << DEFAULT_MEM;
-      mem = DEFAULT_MEM;
-    } else {
-      mem = mem_.get().total;
-
-      // Leave 1 GB free if we have more than 1 GB, otherwise, use all!
-      // TODO(benh): Have better default scheme (e.g., % of mem not
-      // greater than 1 GB?)
-      if (mem > Gigabytes(1)) {
-        mem = mem - Gigabytes(1);
-      }
-    }
-
-    Resource r = Resources::parse(
-        "mem",
-        stringify(mem.megabytes()),
-        flags.default_role).get();
-    resources += r;
+  Try<Resources> _resources = Containerizer::resources(flags);
+  if (_resources.isError()) {
+    EXIT(1) << "Failed to determine slave resources: " << _resources.error();
   }
-
-  if (!resources.disk().isSome()) {
-    Bytes disk;
-
-    // NOTE: We calculate disk size of the file system on
-    // which the slave work directory is mounted.
-    Try<Bytes> disk_ = fs::size(flags.work_dir);
-    if (!disk_.isSome()) {
-      LOG(WARNING) << "Failed to auto-detect the disk space: '"
-                   << disk_.error()
-                   << "' ; defaulting to " << DEFAULT_DISK;
-      disk = DEFAULT_DISK;
-    } else {
-      disk = disk_.get();
-
-      // Leave 5 GB free if we have more than 10 GB, otherwise, use all!
-      // TODO(benh): Have better default scheme (e.g., % of disk not
-      // greater than 10 GB?)
-      if (disk > Gigabytes(10)) {
-        disk = disk - Gigabytes(5);
-      }
-    }
-
-    Resource r = Resources::parse(
-        "disk",
-        stringify(disk.megabytes()),
-        flags.default_role).get();
-    resources += r;
-  }
-
-  if (!resources.ports().isSome()) {
-    Resource r = Resources::parse(
-        "ports",
-        stringify(DEFAULT_PORTS),
-        flags.default_role).get();
-    resources += r;
-  }
-
-  LOG(INFO) << "Slave resources: " << resources;
+  LOG(INFO) << "Slave resources: " << _resources.get();
 
   if (flags.attributes.isSome()) {
     attributes = Attributes::parse(flags.attributes.get());
@@ -233,8 +143,8 @@ void Slave::initialize()
   // Initialize slave info.
   info.set_hostname(hostname);
   info.set_port(self().port);
-  info.mutable_resources()->MergeFrom(resources);
-  info.mutable_attributes()->MergeFrom(attributes);
+  info.mutable_resources()->CopyFrom(_resources.get());
+  info.mutable_attributes()->CopyFrom(attributes);
   info.set_checkpoint(flags.checkpoint);
 
   LOG(INFO) << "Slave hostname: " << info.hostname();
@@ -246,23 +156,6 @@ void Slave::initialize()
   // executor on an older release).
   // TODO(benh): Remove this after the deprecation cycle.
   info.set_webui_hostname(hostname);
-
-  // Spawn and initialize the isolator.
-  // TODO(benh): Seems like the isolator should really be
-  // spawned before being passed to the slave.
-  spawn(isolator);
-
-  // TODO(vinod): Also pass SlaveID here. Currently it is tricky
-  // because SlaveID is only known either after recovery (if previous
-  // state exists) or after the slave registers with the master. We
-  // cannot delay initialize until after (re-)registration because
-  // during recovery (but before re-registration), the isolator needs
-  // to be initialized before accepting any messages
-  // (e.g., killExecutor) from the slave.
-  dispatch(isolator, &Isolator::initialize, flags, resources, local, self());
-
-  // TODO(vinod): Also pass SlaveID here. The reason that this is
-  // tricky is due to similar reasons described in the above comment.
   statusUpdateManager->initialize(flags, self());
 
   // Start disk monitoring.
@@ -420,10 +313,6 @@ void Slave::finalize()
       CHECK_SOME(os::rm(paths::getLatestSlavePath(metaDir)));
     }
   }
-
-  // Stop the isolator.
-  terminate(isolator);
-  wait(isolator);
 }
 
 
@@ -971,11 +860,8 @@ void Slave::_runTask(
       // Update the resources.
       // TODO(Charles Reiss): The isolator is not guaranteed to update
       // the resources before the executor acts on its RunTaskMessage.
-      dispatch(isolator,
-               &Isolator::resourcesChanged,
-               framework->id,
-               executor->id,
-               executor->resources);
+      // TODO(idownes): Wait until this completes.
+      containerizer->update(executor->containerId, executor->resources);
 
       LOG(INFO) << "Sending task '" << task.task_id()
                 << "' to executor '" << executorId
@@ -1089,8 +975,7 @@ void Slave::killTask(
         LOG(WARNING) << "Killing the unregistered executor '" << executor->id
                      << "' of framework " << framework->id
                      << " because it has no tasks";
-        dispatch(
-            isolator, &Isolator::killExecutor, framework->id, executor->id);
+        containerizer->destroy(executor->containerId);
       }
       break;
     }
@@ -1511,7 +1396,7 @@ void Slave::registerExecutor(
             info.id(),
             executor->frameworkId,
             executor->id,
-            executor->uuid);
+            executor->containerId);
 
         LOG(INFO) << "Checkpointing executor pid '"
                   << executor->pid << "' to '" << path << "'";
@@ -1531,11 +1416,8 @@ void Slave::registerExecutor(
       // TODO(Charles Reiss): We don't actually have a guarantee
       // that this will be delivered or (where necessary) acted on
       // before the executor gets its RunTaskMessages.
-      dispatch(isolator,
-               &Isolator::resourcesChanged,
-               framework->id,
-               executor->id,
-               executor->resources);
+      // TODO(idownes): Wait until this completes.
+      containerizer->update(executor->containerId, executor->resources);
 
       // Tell executor it's registered and give it any queued tasks.
       ExecutorRegisteredMessage message;
@@ -1651,12 +1533,9 @@ void Slave::reregisterExecutor(
         statusUpdate(update, executor->pid);
       }
 
-      // Tell the isolator to update the resources.
-      dispatch(isolator,
-               &Isolator::resourcesChanged,
-               frameworkId,
-               executorId,
-               executor->resources);
+      // Tell the containerizer to update the resources.
+      // TODO(idownes): Wait until this completes.
+      containerizer->update(executor->containerId, executor->resources);
 
       hashmap<TaskID, TaskInfo> unackedTasks;
       foreach (const TaskInfo& task, tasks) {
@@ -1733,8 +1612,7 @@ void Slave::reregisterExecutorTimeout()
 
           executor->state = Executor::TERMINATING;
 
-          dispatch(
-              isolator, &Isolator::killExecutor, framework->id, executor->id);
+          containerizer->destroy(executor->containerId);
           break;
         default:
           LOG(FATAL) << "Executor '" << executor->id
@@ -1843,16 +1721,17 @@ void Slave::statusUpdate(const StatusUpdate& update, const UPID& pid)
     executor->terminateTask(status.task_id(), status.state());
 
     // Tell the isolator to update the resources.
-    dispatch(isolator,
-             &Isolator::resourcesChanged,
-             framework->id,
-             executor->id,
-             executor->resources);
+    // TODO(idownes): Wait until this completes.
+    containerizer->update(executor->containerId, executor->resources);
   }
 
   if (executor->checkpoint) {
     // Ask the status update manager to checkpoint and reliably send the update.
-    statusUpdateManager->update(update, info.id(), executor->id, executor->uuid)
+    statusUpdateManager->update(
+        update,
+        info.id(),
+        executor->id,
+        executor->containerId)
       .onAny(defer(self(),
                    &Slave::_statusUpdate,
                    lambda::_1,
@@ -2032,12 +1911,6 @@ ExecutorInfo Slave::getExecutorInfo(
           "'; exit 1");
     }
 
-    // TODO(benh): Set some resources for the executor so that a task
-    // doesn't end up getting killed because the amount of resources
-    // of the executor went over those allocated. Note that this might
-    // mean that the number of resources on the machine will actually
-    // be slightly oversubscribed, so we'll need to reevaluate with
-    // respect to resources that can't be oversubscribed.
     return executor;
   }
 
@@ -2045,23 +1918,41 @@ ExecutorInfo Slave::getExecutorInfo(
 }
 
 
-void _watch(
-    const Future<Nothing>& watch,
+void _monitor(
+    const Future<Nothing>& monitor,
     const FrameworkID& frameworkId,
-    const ExecutorID& executorId);
-
+    const ExecutorID& executorId,
+    const ContainerID& containerId)
+{
+  if (!monitor.isReady()) {
+    LOG(ERROR) << "Failed to monitor container '" << containerId
+               << "' for executor '" << executorId
+               << "' of framework '" << frameworkId
+               << ":" << (monitor.isFailed() ? monitor.failure() : "discarded");
+  }
+}
 
-// N.B. When the slave is running in "local" mode then the pid is
-// uninteresting (and possibly could cause bugs).
 void Slave::executorStarted(
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    pid_t pid)
+    const ContainerID& containerId,
+    const Future<Nothing>& future)
 {
+  if (!future.isReady()) {
+    // The containerizer will clean up if the launch fails we'll just log this
+    // and leave the executor registration to timeout.
+    LOG(ERROR) << "Container '" << containerId
+               << "' for executor '" << executorId
+               << "' of framework '" << frameworkId
+               << "' failed to start: "
+               << (future.isFailed() ? future.failure() : " future discarded");
+    return;
+  }
+
   Framework* framework = getFramework(frameworkId);
   if (framework == NULL) {
-    LOG(WARNING) << "Framework " << frameworkId
-                 << " for executor '" << executorId
+    LOG(WARNING) << "Framework '" << frameworkId
+                 << "' for executor '" << executorId
                  << "' is no longer valid";
     return;
   }
@@ -2074,61 +1965,52 @@ void Slave::executorStarted(
     LOG(WARNING) << "Killing executor '" << executorId
                  << "' of framework '" << frameworkId
                  << "' because the framework is terminating";
-    dispatch(isolator, &Isolator::killExecutor, frameworkId, executorId);
+    containerizer->destroy(containerId);
     return;
   }
 
   Executor* executor = framework->getExecutor(executorId);
   if (executor == NULL) {
     LOG(WARNING) << "Killing unknown executor '" << executorId
-                 << "' of framework " << frameworkId;
-    dispatch(isolator, &Isolator::killExecutor, frameworkId, executorId);
+                 << "' of framework '" << frameworkId << "'";
+    containerizer->destroy(containerId);
     return;
   }
 
   switch (executor->state) {
     case Executor::TERMINATING:
       LOG(WARNING) << "Killing executor '" << executorId
-                   << "' of framework " << frameworkId
-                   << " because the executor is terminating";
-      dispatch(isolator, &Isolator::killExecutor, frameworkId, executorId);
+                   << "' of framework '" << frameworkId
+                   << "' because the executor is terminating";
+      containerizer->destroy(containerId);
       break;
     case Executor::REGISTERING:
     case Executor::RUNNING:
-      LOG(INFO) << "Monitoring executor " << executorId
-                << " of framework " << frameworkId
-                << " forked at pid " << pid;
-      monitor.watch(
-          frameworkId,
-          executorId,
+      LOG(INFO) << "Monitoring executor '" << executorId
+                << "' of framework '" << frameworkId
+                << "' in container '" << containerId << "'";
+      // Start monitoring the container's resources.
+      monitor.start(
+          containerId,
           executor->info,
           flags.resource_monitoring_interval)
-        .onAny(lambda::bind(_watch, lambda::_1, frameworkId, executorId));
+        .onAny(lambda::bind(_monitor,
+                            lambda::_1,
+                            frameworkId,
+                            executorId,
+                            containerId));
       break;
     case Executor::TERMINATED:
     default:
       LOG(FATAL) << " Executor '" << executorId
-                 << "' of framework " << frameworkId
-                 << "is in unexpected state " << executor->state;
+                 << "' of framework '" << frameworkId
+                 << "' is in an unexpected state " << executor->state;
       break;
   }
 }
 
 
-void _watch(
-    const Future<Nothing>& watch,
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId)
-{
-  if (!watch.isReady()) {
-    LOG(ERROR) << "Failed to watch executor " << executorId
-               << " of framework " << frameworkId
-               << ": " << (watch.isFailed() ? watch.failure() : "discarded");
-  }
-}
-
-
-void _unwatch(
+void _unmonitor(
     const Future<Nothing>& watch,
     const FrameworkID& frameworkId,
     const ExecutorID& executorId);
@@ -2138,19 +2020,30 @@ void _unwatch(
 void Slave::executorTerminated(
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const Option<int>& status_,
-    bool destroyed,
-    const string& message)
+    const Future<Containerizer::Termination>& termination)
 {
   int status;
-  if (status_.isNone()) {
+  // A termination failure indicates the containerizer could not destroy a
+  // container.
+  // TODO(idownes): This is a serious error so consider aborting the slave if
+  // this occurs.
+  if (!termination.isReady()) {
+    LOG(ERROR) << "Termination of executor '" << executorId
+               << "' of framework '" << frameworkId
+               << "' failed: "
+               << (termination.isFailed()
+                   ? termination.failure()
+                   : "discarded");
+    // Set a special status for failure.
+    status = -1;
+  } else if (termination.get().status.isNone()) {
     LOG(INFO) << "Executor '" << executorId
               << "' of framework " << frameworkId
               << " has terminated with unknown status";
     // Set a special status for None.
     status = -1;
   } else {
-    status = status_.get();
+    status = termination.get().status.get();
     LOG(INFO) << "Executor '" << executorId
               << "' of framework " << frameworkId
               << (WIFEXITED(status)
@@ -2187,12 +2080,12 @@ void Slave::executorTerminated(
     case Executor::TERMINATING: {
       executor->state = Executor::TERMINATED;
 
-      // Stop monitoring this executor.
-      monitor.unwatch(frameworkId, executorId)
-        .onAny(lambda::bind(_unwatch, lambda::_1, frameworkId, executorId));
+      // Stop monitoring the executor's container.
+      monitor.stop(executor->containerId)
+        .onAny(lambda::bind(_unmonitor, lambda::_1, frameworkId, executorId));
 
       // Transition all live tasks to TASK_LOST/TASK_FAILED.
-      // If the isolator destroyed the executor (e.g., due to OOM event)
+      // If the containerizer killed  the executor (e.g., due to OOM event)
       // or if this is a command executor, we send TASK_FAILED status updates
       // instead of TASK_LOST.
       // NOTE: We don't send updates if the framework is terminating
@@ -2209,7 +2102,8 @@ void Slave::executorTerminated(
         foreach (Task* task, executor->launchedTasks.values()) {
           if (!protobuf::isTerminalState(task->state())) {
             mesos::TaskState taskState;
-            if (destroyed || executor->commandExecutor) {
+            if ((termination.isReady() && termination.get().killed) ||
+                 executor->commandExecutor) {
               taskState = TASK_FAILED;
             } else {
               taskState = TASK_LOST;
@@ -2219,7 +2113,8 @@ void Slave::executorTerminated(
                 info.id(),
                 task->task_id(),
                 taskState,
-                message,
+                termination.isReady() ? termination.get().message :
+                                        "Abnormal executor termination",
                 executorId),
                 UPID());
           }
@@ -2230,7 +2125,8 @@ void Slave::executorTerminated(
         // supports it.
         foreach (const TaskInfo& task, executor->queuedTasks.values()) {
           mesos::TaskState taskState;
-          if (destroyed || executor->commandExecutor) {
+          if ((termination.isReady() && termination.get().killed) ||
+               executor->commandExecutor) {
             taskState = TASK_FAILED;
           } else {
             taskState = TASK_LOST;
@@ -2240,7 +2136,8 @@ void Slave::executorTerminated(
               info.id(),
               task.task_id(),
               taskState,
-              message,
+              termination.isReady() ? termination.get().message :
+                                      "Abnormal executor termination",
               executorId),
               UPID());
         }
@@ -2305,7 +2202,7 @@ void Slave::removeExecutor(Framework* framework, Executor* executor)
   // is completed.
   if (executor->checkpoint) {
     const string& path = paths::getExecutorSentinelPath(
-        metaDir, info.id(), framework->id, executor->id, executor->uuid);
+        metaDir, info.id(), framework->id, executor->id, executor->containerId);
     CHECK_SOME(os::touch(path));
   }
 
@@ -2314,7 +2211,11 @@ void Slave::removeExecutor(Framework* framework, Executor* executor)
 
   // Schedule the executor run work directory to get garbage collected.
   const string& path = paths::getExecutorRunPath(
-      flags.work_dir, info.id(), framework->id, executor->id, executor->uuid);
+      flags.work_dir,
+      info.id(),
+      framework->id,
+      executor->id,
+      executor->containerId);
 
   os::utime(path); // Update the modification time.
   garbageCollect(path)
@@ -2333,7 +2234,7 @@ void Slave::removeExecutor(Framework* framework, Executor* executor)
   if (executor->checkpoint) {
     // Schedule the executor run meta directory to get garbage collected.
     const string& path = paths::getExecutorRunPath(
-        metaDir, info.id(), framework->id, executor->id, executor->uuid);
+        metaDir, info.id(), framework->id, executor->id, executor->containerId);
 
     os::utime(path); // Update the modification time.
     garbageCollect(path);
@@ -2410,15 +2311,15 @@ void Slave::removeFramework(Framework* framework)
 }
 
 
-void _unwatch(
-    const Future<Nothing>& unwatch,
+void _unmonitor(
+    const Future<Nothing>& unmonitor,
     const FrameworkID& frameworkId,
     const ExecutorID& executorId)
 {
-  if (!unwatch.isReady()) {
-    LOG(ERROR) << "Failed to unwatch executor " << executorId
+  if (!unmonitor.isReady()) {
+    LOG(ERROR) << "Failed to unmonitor container for executor " << executorId
                << " of framework " << frameworkId << ": "
-               << (unwatch.isFailed() ? unwatch.failure() : "discarded");
+               << (unmonitor.isFailed() ? unmonitor.failure() : "discarded");
   }
 }
 
@@ -2452,14 +2353,14 @@ void Slave::shutdownExecutor(Framework* framework, Executor* executor)
         &Slave::shutdownExecutorTimeout,
         framework->id,
         executor->id,
-        executor->uuid);
+        executor->containerId);
 }
 
 
 void Slave::shutdownExecutorTimeout(
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const UUID& uuid)
+    const ContainerID& containerId)
 {
   Framework* framework = getFramework(frameworkId);
   if (framework == NULL) {
@@ -2481,12 +2382,12 @@ void Slave::shutdownExecutorTimeout(
     return;
   }
 
-  if (executor->uuid != uuid ) { // Make sure this timeout is valid.
+  if (executor->containerId != containerId) { // Make sure this timeout is valid.
     LOG(INFO) << "A new executor '" << executorId
               << "' of framework " << frameworkId
-              << " with run " << executor->uuid
+              << " with run " << executor->containerId
               << " seems to be active. Ignoring the shutdown timeout"
-              << " for the old executor run " << uuid;
+              << " for the old executor run " << containerId;
     return;
   }
 
@@ -2500,7 +2401,7 @@ void Slave::shutdownExecutorTimeout(
       LOG(INFO) << "Killing executor '" << executor->id
                 << "' of framework " << framework->id;
 
-      dispatch(isolator, &Isolator::killExecutor, framework->id, executor->id);
+      containerizer->destroy(executor->containerId);
       break;
     default:
       LOG(FATAL) << "Executor '" << executor->id
@@ -2514,7 +2415,7 @@ void Slave::shutdownExecutorTimeout(
 void Slave::registerExecutorTimeout(
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const UUID& uuid)
+    const ContainerID& containerId)
 {
   Framework* framework = getFramework(frameworkId);
   if (framework == NULL) {
@@ -2543,12 +2444,12 @@ void Slave::registerExecutorTimeout(
     return;
   }
 
-  if (executor->uuid != uuid ) {
+  if (executor->containerId != containerId ) {
     LOG(INFO) << "A new executor '" << executorId
               << "' of framework " << frameworkId
-              << " with run " << executor->uuid
+              << " with run " << executor->containerId
               << " seems to be active. Ignoring the registration timeout"
-              << " for the old executor run " << uuid;
+              << " for the old executor run " << containerId;
     return;
   }
 
@@ -2567,7 +2468,7 @@ void Slave::registerExecutorTimeout(
       executor->state = Executor::TERMINATING;
 
       // Immediately kill the executor.
-      dispatch(isolator, &Isolator::killExecutor, framework->id, executor->id);
+      containerizer->destroy(executor->containerId);
       break;
     default:
       LOG(FATAL) << "Executor '" << executor->id
@@ -2670,7 +2571,14 @@ Future<Nothing> Slave::recover(const Result<SlaveState>& _state)
   }
 
   return statusUpdateManager->recover(metaDir, state)
-    .then(defer(isolator, &Isolator::recover, state));
+    .then(defer(self(), &Slave::_recoverContainerizer, state));
+}
+
+
+Future<Nothing> Slave::_recoverContainerizer(
+    const Option<state::SlaveState>& state)
+{
+  return containerizer->recover(state);
 }
 
 
@@ -2678,22 +2586,25 @@ Future<Nothing> Slave::_recover()
 {
   foreachvalue (Framework* framework, frameworks) {
     foreachvalue (Executor* executor, framework->executors) {
-      // If the executor is already terminating/terminated don't
-      // bother reconnecting or killing it. This could happen if
-      // the recovered isolator sent a 'ExecutorTerminated' message
-      // before the slave is here.
-      if (executor->state == Executor::TERMINATING ||
-          executor->state == Executor::TERMINATED) {
-        continue;
-      }
-
       // Monitor the executor.
-      monitor.watch(
-          framework->id,
-          executor->id,
+      monitor.start(
+          executor->containerId,
           executor->info,
           flags.resource_monitoring_interval)
-        .onAny(lambda::bind(_watch, lambda::_1, framework->id, executor->id));
+        .onAny(lambda::bind(_monitor,
+                            lambda::_1,
+                            framework->id,
+                            executor->id,
+                            executor->containerId));
+
+      // Set up callback for executor termination.
+      containerizer->wait(executor->containerId)
+        .onAny(defer(self(),
+                     &Self::executorTerminated,
+                     framework->id,
+                     executor->id,
+                     lambda::_1));
+
 
       if (flags.recover == "reconnect") {
         if (executor->pid) {
@@ -2722,8 +2633,7 @@ Future<Nothing> Slave::_recover()
                     << "' of framework " << framework->id
                     << " because no libprocess PID was found";
 
-          dispatch(
-              isolator, &Isolator::killExecutor, framework->id, executor->id);
+          containerizer->destroy(executor->containerId);
         }
       }
     }
@@ -2924,12 +2834,12 @@ Executor* Framework::launchExecutor(
     const ExecutorInfo& executorInfo,
     const TaskInfo& taskInfo)
 {
-  // We create a UUID for the new executor. The UUID uniquely
-  // identifies this new instance of the executor across executors
-  // sharing the same executorID that may have previously run. It
-  // also provides a means for the executor to have a unique
-  // directory.
-  UUID uuid = UUID::random();
+  // Generate an ID for the executor's container.
+  // TODO(idownes) This should be done by the containerizer but we need the
+  // ContainerID to create the executor's directory and to set up monitoring.
+  // Fix this when 'launchExecutor()' is handled asynchronously.
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
 
   // Create a directory for the executor.
   const string& directory = paths::createExecutorDirectory(
@@ -2937,10 +2847,10 @@ Executor* Framework::launchExecutor(
       slave->info.id(),
       id,
       executorInfo.executor_id(),
-      uuid);
+      containerId);
 
   Executor* executor = new Executor(
-      slave, id, executorInfo, uuid, directory, info.checkpoint());
+      slave, id, executorInfo, containerId, directory, info.checkpoint());
 
   CHECK(!executors.contains(executorInfo.executor_id()))
     << "Unknown executor " << executorInfo.executor_id();
@@ -2953,20 +2863,37 @@ Executor* Framework::launchExecutor(
                  lambda::_1,
                  executor->directory));
 
-  // Tell the isolator to launch the executor.
-  // NOTE: We include the task's resources when launching the
-  // executor so that the isolator has non-zero resources to
-  // work with when the executor has no resources. This should
-  // be revisited after MESOS-600.
-  dispatch(slave->isolator,
-           &Isolator::launchExecutor,
-           slave->info.id(),
-           id,
-           info,
-           executor->info,
-           executor->uuid,
-           executor->directory,
-           executor->resources + taskInfo.resources());
+  // Tell the containerizer to launch the executor.
+  // NOTE: We modify the ExecutorInfo to include the task's
+  // resources when launching the executor so that the containerizer
+  // has non-zero resources to work with when the executor has
+  // no resources. This should be revisited after MESOS-600.
+  ExecutorInfo executorInfo_ = executor->info;
+  executorInfo_.mutable_resources()->MergeFrom(taskInfo.resources());
+
+  // Launch the container.
+  slave->containerizer->launch(
+      containerId,
+      executorInfo_, // modified to include the task's resources
+      executor->directory,
+      slave->flags.switch_user ? Option<string>(info.user()) : None(),
+      slave->info.id(),
+      slave->self(),
+      info.checkpoint())
+    .onAny(defer(slave,
+                 &Slave::executorStarted,
+                 id,
+                 executor->id,
+                 containerId,
+                 lambda::_1));
+
+  // Set up callback for executor termination.
+  slave->containerizer->wait(containerId)
+    .onAny(defer(slave,
+                 &Slave::executorTerminated,
+                 id,
+                 executor->id,
+                 lambda::_1));
 
   // Make sure the executor registers within the given timeout.
   delay(slave->flags.executor_registration_timeout,
@@ -2974,7 +2901,7 @@ Executor* Framework::launchExecutor(
         &Slave::registerExecutorTimeout,
         id,
         executor->id,
-        executor->uuid);
+        containerId);
 
   return executor;
 }
@@ -3043,11 +2970,11 @@ void Framework::recoverExecutor(const ExecutorState& state)
   // NOTE: We don't schedule the top level executor work and meta
   // directories for GC here, because they will be scheduled when
   // the latest executor run terminates.
-  const UUID& uuid = state.latest.get();
+  const ContainerID& latest = state.latest.get();
   foreachvalue (const RunState& run, state.runs) {
     CHECK_SOME(run.id);
-    const UUID& runId = run.id.get();
-    if (uuid != runId) {
+    const ContainerID& runId = run.id.get();
+    if (latest != runId) {
       // GC the executor run's work directory.
       // TODO(vinod): Expose this directory to webui by recovering the
       // tasks and doing a 'files->attach()'.
@@ -3060,18 +2987,18 @@ void Framework::recoverExecutor(const ExecutorState& state)
     }
   }
 
-  CHECK(state.runs.contains(uuid))
-    << "Cannot find latest run " << uuid << " for executor " << state.id
+  CHECK(state.runs.contains(latest))
+    << "Cannot find latest run " << latest << " for executor " << state.id
     << " of framework " << id;
 
-  const RunState& run = state.runs.get(uuid).get();
+  const RunState& run = state.runs.get(latest).get();
 
   // Create executor.
   const string& directory = paths::getExecutorRunPath(
-      slave->flags.work_dir, slave->info.id(), id, state.id, uuid);
+      slave->flags.work_dir, slave->info.id(), id, state.id, latest);
 
   Executor* executor = new Executor(
-      slave, id, state.info.get(), uuid, directory, info.checkpoint());
+      slave, id, state.info.get(), latest, directory, info.checkpoint());
 
   // Recover the libprocess PID if possible.
   if (run.libprocessPid.isSome()) {
@@ -3109,7 +3036,7 @@ void Framework::recoverExecutor(const ExecutorState& state)
     executor->state = Executor::TERMINATED;
 
     CHECK_SOME(run.id);
-    const UUID& runId = run.id.get();
+    const ContainerID& runId = run.id.get();
 
     // GC the executor run's work directory.
     const string& path = paths::getExecutorRunPath(
@@ -3142,7 +3069,7 @@ Executor::Executor(
     Slave* _slave,
     const FrameworkID& _frameworkId,
     const ExecutorInfo& _info,
-    const UUID& _uuid,
+    const ContainerID& _containerId,
     const string& _directory,
     bool _checkpoint)
   : state(REGISTERING),
@@ -3150,7 +3077,7 @@ Executor::Executor(
     id(_info.executor_id()),
     info(_info),
     frameworkId(_frameworkId),
-    uuid(_uuid),
+    containerId(_containerId),
     directory(_directory),
     checkpoint(_checkpoint),
     commandExecutor(strings::contains(
@@ -3161,6 +3088,7 @@ Executor::Executor(
     completedTasks(MAX_COMPLETED_TASKS_PER_EXECUTOR)
 {
   CHECK_NOTNULL(slave);
+
   if (checkpoint && slave->state != slave->RECOVERING) {
     // Checkpoint the executor info.
     const string& path = paths::getExecutorInfoPath(
@@ -3172,7 +3100,7 @@ Executor::Executor(
     // Create the meta executor directory.
     // NOTE: This creates the 'latest' symlink in the meta directory.
     paths::createExecutorDirectory(
-        slave->metaDir, slave->info.id(), frameworkId, id, uuid);
+        slave->metaDir, slave->info.id(), frameworkId, id, containerId);
   }
 }
 
@@ -3252,7 +3180,12 @@ void Executor::checkpointTask(const TaskInfo& task)
 
     const Task& t = protobuf::createTask(task, TASK_STAGING, id, frameworkId);
     const string& path = paths::getTaskInfoPath(
-        slave->metaDir, slave->info.id(), frameworkId, id, uuid, t.task_id());
+        slave->metaDir,
+        slave->info.id(),
+        frameworkId,
+        id,
+        containerId,
+        t.task_id());
 
     LOG(INFO) << "Checkpointing TaskInfo to '" << path << "'";
     CHECK_SOME(state::checkpoint(path, t));

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 2ddadb4..d82d4e9 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -46,9 +46,9 @@
 #include "master/detector.hpp"
 
 #include "slave/constants.hpp"
+#include "slave/containerizer/containerizer.hpp"
 #include "slave/flags.hpp"
 #include "slave/gc.hpp"
-#include "slave/isolator.hpp"
 #include "slave/monitor.hpp"
 #include "slave/paths.hpp"
 #include "slave/state.hpp"
@@ -79,9 +79,8 @@ class Slave : public ProtobufProcess<Slave>
 {
 public:
   Slave(const Flags& flags,
-        bool local,
         MasterDetector* detector,
-        Isolator* isolator,
+        Containerizer* containerizer,
         Files* files);
 
   virtual ~Slave();
@@ -179,14 +178,13 @@ public:
   void executorStarted(
       const FrameworkID& frameworkId,
       const ExecutorID& executorId,
-      pid_t pid);
+      const ContainerID& containerId,
+      const Future<Nothing>& future);
 
   void executorTerminated(
       const FrameworkID& frameworkId,
       const ExecutorID& executorId,
-      const Option<int>& status,
-      bool destroyed,
-      const std::string& message);
+      const Future<Containerizer::Termination>& termination);
 
   // NOTE: Pulled these to public to make it visible for testing.
   // TODO(vinod): Make tests friends to this class instead.
@@ -241,13 +239,13 @@ public:
   void shutdownExecutorTimeout(
       const FrameworkID& frameworkId,
       const ExecutorID& executorId,
-      const UUID& uuid);
+      const ContainerID& containerId);
 
   // Shuts down the executor if it did not register yet.
   void registerExecutorTimeout(
       const FrameworkID& frameworkId,
       const ExecutorID& executorId,
-      const UUID& uuid);
+      const ContainerID& containerId);
 
   // Cleans up all un-reregistered executors during recovery.
   void reregisterExecutorTimeout();
@@ -262,11 +260,16 @@ public:
   // Recovers the slave, status update manager and isolator.
   Future<Nothing> recover(const Result<state::SlaveState>& state);
 
-  // This is called after 'recoveR()'. If 'flags.reconnect' is
+  // This is called after 'recover()'. If 'flags.reconnect' is
   // 'reconnect', the slave attempts to reconnect to any old live
   // executors. Otherwise, the slave attempts to shutdown/kill them.
   Future<Nothing> _recover();
 
+  // This is a helper to call recover() on the containerizer at the end of
+  // recover() and before __recover().
+  // TODO(idownes): Remove this when we support defers to objects.
+  Future<Nothing> _recoverContainerizer(const Option<state::SlaveState>& state);
+
   // This is called when recovery finishes.
   void __recover(const Future<Nothing>& future);
 
@@ -316,8 +319,6 @@ private:
 
   const Flags flags;
 
-  bool local;
-
   SlaveInfo info;
 
   Option<UPID> master;
@@ -331,7 +332,8 @@ private:
 
   MasterDetector* detector;
 
-  Isolator* isolator;
+  Containerizer* containerizer;
+
   Files* files;
 
   // Statistics (initialized in Slave::initialize).
@@ -369,7 +371,7 @@ struct Executor
       Slave* slave,
       const FrameworkID& frameworkId,
       const ExecutorInfo& info,
-      const UUID& uuid,
+      const ContainerID& containerId,
       const std::string& directory,
       bool checkpoint);
 
@@ -402,7 +404,7 @@ struct Executor
 
   const FrameworkID frameworkId;
 
-  const UUID uuid; // Distinguishes executor instances with same ExecutorID.
+  const ContainerID containerId;
 
   const std::string directory;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/state.cpp
----------------------------------------------------------------------
diff --git a/src/slave/state.cpp b/src/slave/state.cpp
index 6c382cd..9af6c5b 100644
--- a/src/slave/state.cpp
+++ b/src/slave/state.cpp
@@ -336,21 +336,25 @@ Try<ExecutorState> ExecutorState::recover(
              : "No such file or directory"));
       }
 
-      // Store the UUID of the latest executor run.
-      state.latest = UUID::fromString(os::basename(latest.get()).get());
+      // Store the ContainerID of the latest executor run.
+      ContainerID containerId;
+      containerId.set_value(os::basename(latest.get()).get());
+      state.latest = containerId;
     } else {
-      const UUID& uuid = UUID::fromString(os::basename(path).get());
+      ContainerID containerId;
+      containerId.set_value(os::basename(path).get());
 
       const Try<RunState>& run = RunState::recover(
-          rootDir, slaveId, frameworkId, executorId, uuid, strict);
+          rootDir, slaveId, frameworkId, executorId, containerId, strict);
 
       if (run.isError()) {
-        return Error("Failed to recover run " + uuid.toString() +
-                     " of executor '" + executorId.value() +
-                     "': " + run.error());
+        return Error(
+            "Failed to recover run " + containerId.value() +
+            " of executor '" + executorId.value() +
+            "': " + run.error());
       }
 
-      state.runs[uuid] = run.get();
+      state.runs[containerId] = run.get();
       state.errors += run.get().errors;
     }
   }
@@ -373,11 +377,11 @@ Try<RunState> RunState::recover(
     const SlaveID& slaveId,
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const UUID& uuid,
+    const ContainerID& containerId,
     bool strict)
 {
   RunState state;
-  state.id = uuid;
+  state.id = containerId;
   string message;
 
   // Find the tasks.
@@ -387,12 +391,13 @@ Try<RunState> RunState::recover(
       slaveId,
       frameworkId,
       executorId,
-      uuid.toString(),
+      containerId,
       "*").get());
 
   if (tasks.isError()) {
-    return Error("Failed to find tasks for executor run " + uuid.toString() +
-                 ": " + tasks.error());
+    return Error(
+        "Failed to find tasks for executor run " + containerId.value() +
+        ": " + tasks.error());
   }
 
   // Recover tasks.
@@ -401,7 +406,7 @@ Try<RunState> RunState::recover(
     taskId.set_value(os::basename(path).get());
 
     const Try<TaskState>& task = TaskState::recover(
-        rootDir, slaveId, frameworkId, executorId, uuid, taskId, strict);
+        rootDir, slaveId, frameworkId, executorId, containerId, taskId, strict);
 
     if (task.isError()) {
       return Error(
@@ -414,7 +419,7 @@ Try<RunState> RunState::recover(
 
   // Read the forked pid.
   string path = paths::getForkedPidPath(
-      rootDir, slaveId, frameworkId, executorId, uuid);
+      rootDir, slaveId, frameworkId, executorId, containerId);
   if (!os::exists(path)) {
     // This could happen if the slave died before the isolator
     // checkpointed the forked pid.
@@ -454,7 +459,7 @@ Try<RunState> RunState::recover(
 
   // Read the libprocess pid.
   path = paths::getLibprocessPidPath(
-      rootDir, slaveId, frameworkId, executorId, uuid);
+      rootDir, slaveId, frameworkId, executorId, containerId);
 
   if (!os::exists(path)) {
     // This could happen if the slave died before the executor
@@ -490,7 +495,7 @@ Try<RunState> RunState::recover(
 
   // See if the sentinel file exists.
   path = paths::getExecutorSentinelPath(
-      rootDir, slaveId, frameworkId, executorId, uuid);
+      rootDir, slaveId, frameworkId, executorId, containerId);
 
   state.completed = os::exists(path);
 
@@ -503,7 +508,7 @@ Try<TaskState> TaskState::recover(
     const SlaveID& slaveId,
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const UUID& uuid,
+    const ContainerID& containerId,
     const TaskID& taskId,
     bool strict)
 {
@@ -513,7 +518,7 @@ Try<TaskState> TaskState::recover(
 
   // Read the task info.
   string path = paths::getTaskInfoPath(
-      rootDir, slaveId, frameworkId, executorId, uuid, taskId);
+      rootDir, slaveId, frameworkId, executorId, containerId, taskId);
   if (!os::exists(path)) {
     // This could happen if the slave died after creating the task
     // directory but before it checkpointed the task info.
@@ -546,7 +551,7 @@ Try<TaskState> TaskState::recover(
 
   // Read the status updates.
   path = paths::getTaskUpdatesPath(
-      rootDir, slaveId, frameworkId, executorId, uuid, taskId);
+      rootDir, slaveId, frameworkId, executorId, containerId, taskId);
   if (!os::exists(path)) {
     // This could happen if the slave died before it checkpointed
     // any status updates for this task.

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/state.hpp
----------------------------------------------------------------------
diff --git a/src/slave/state.hpp b/src/slave/state.hpp
index 78b20ff..22f569d 100644
--- a/src/slave/state.hpp
+++ b/src/slave/state.hpp
@@ -126,8 +126,8 @@ struct ExecutorState
 
   ExecutorID id;
   Option<ExecutorInfo> info;
-  Option<UUID> latest;
-  hashmap<UUID, RunState> runs;
+  Option<ContainerID> latest;
+  hashmap<ContainerID, RunState> runs;
   unsigned int errors;
 };
 
@@ -141,10 +141,10 @@ struct RunState
       const SlaveID& slaveId,
       const FrameworkID& frameworkId,
       const ExecutorID& executorId,
-      const UUID& uuid,
+      const ContainerID& containerId,
       bool strict);
 
-  Option<UUID> id;
+  Option<ContainerID> id;
   hashmap<TaskID, TaskState> tasks;
   Option<pid_t> forkedPid;
   Option<process::UPID> libprocessPid;
@@ -162,7 +162,7 @@ struct TaskState
       const SlaveID& slaveId,
       const FrameworkID& frameworkId,
       const ExecutorID& executorId,
-      const UUID& uuid,
+      const ContainerID& containerId,
       const TaskID& taskId,
       bool strict);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/status_update_manager.cpp
----------------------------------------------------------------------
diff --git a/src/slave/status_update_manager.cpp b/src/slave/status_update_manager.cpp
index 03f5eaf..a88bb18 100644
--- a/src/slave/status_update_manager.cpp
+++ b/src/slave/status_update_manager.cpp
@@ -71,7 +71,7 @@ public:
       const StatusUpdate& update,
       const SlaveID& slaveId,
       const ExecutorID& executorId,
-      const UUID& uuid);
+      const ContainerID& containerId);
 
   Future<Nothing> update(
       const StatusUpdate& update,
@@ -99,7 +99,7 @@ private:
       const SlaveID& slaveId,
       bool checkpoint,
       const Option<ExecutorID>& executorId,
-      const Option<UUID>& uuid);
+      const Option<ContainerID>& containerId);
 
   // Status update timeout.
   void timeout(const Duration& duration);
@@ -120,7 +120,7 @@ private:
       const SlaveID& slaveId,
       bool checkpoint,
       const Option<ExecutorID>& executorId,
-      const Option<UUID>& uuid);
+      const Option<ContainerID>& containerId);
 
   StatusUpdateStream* getStatusUpdateStream(
       const TaskID& taskId,
@@ -213,15 +213,16 @@ Future<Nothing> StatusUpdateManagerProcess::recover(
       }
 
       // We are only interested in the latest run of the executor!
-      const UUID& uuid = executor.latest.get();
-      CHECK(executor.runs.contains(uuid));
-      const RunState& run  = executor.runs.get(uuid).get();
+      const ContainerID& latest = executor.latest.get();
+      CHECK(executor.runs.contains(latest));
+      const RunState& run  = executor.runs.get(latest).get();
 
       if (run.completed) {
         VLOG(1) << "Skipping recovering updates of"
                 << " executor '" << executor.id
                 << "' of framework " << framework.id
-                << " because its latest run " << uuid << " is completed";
+                << " because its latest run " << latest.value()
+                << " is completed";
         continue;
       }
 
@@ -238,7 +239,7 @@ Future<Nothing> StatusUpdateManagerProcess::recover(
 
         // Create a new status update stream.
         StatusUpdateStream* stream = createStatusUpdateStream(
-            task.id, framework.id, state.get().id, true, executor.id, uuid);
+            task.id, framework.id, state.get().id, true, executor.id, latest);
 
         // Replay the stream.
         Try<Nothing> replay = stream->replay(task.updates, task.acks);
@@ -287,9 +288,9 @@ Future<Nothing> StatusUpdateManagerProcess::update(
     const StatusUpdate& update,
     const SlaveID& slaveId,
     const ExecutorID& executorId,
-    const UUID& uuid)
+    const ContainerID& containerId)
 {
-  return _update(update, slaveId, true, executorId, uuid);
+  return _update(update, slaveId, true, executorId, containerId);
 }
 
 
@@ -306,7 +307,7 @@ Future<Nothing> StatusUpdateManagerProcess::_update(
     const SlaveID& slaveId,
     bool checkpoint,
     const Option<ExecutorID>& executorId,
-    const Option<UUID>& uuid)
+    const Option<ContainerID>& containerId)
 {
   const TaskID& taskId = update.status().task_id();
   const FrameworkID& frameworkId = update.framework_id();
@@ -318,7 +319,7 @@ Future<Nothing> StatusUpdateManagerProcess::_update(
   StatusUpdateStream* stream = getStatusUpdateStream(taskId, frameworkId);
   if (stream == NULL) {
     stream = createStatusUpdateStream(
-        taskId, frameworkId, slaveId, checkpoint, executorId, uuid);
+        taskId, frameworkId, slaveId, checkpoint, executorId, containerId);
   }
 
   // Verify that we didn't get a non-checkpointable update for a
@@ -488,13 +489,13 @@ StatusUpdateStream* StatusUpdateManagerProcess::createStatusUpdateStream(
     const SlaveID& slaveId,
     bool checkpoint,
     const Option<ExecutorID>& executorId,
-    const Option<UUID>& uuid)
+    const Option<ContainerID>& containerId)
 {
   VLOG(1) << "Creating StatusUpdate stream for task " << taskId
           << " of framework " << frameworkId;
 
   StatusUpdateStream* stream = new StatusUpdateStream(
-      taskId, frameworkId, slaveId, flags, checkpoint, executorId, uuid);
+      taskId, frameworkId, slaveId, flags, checkpoint, executorId, containerId);
 
   streams[frameworkId][taskId] = stream;
   return stream;
@@ -569,7 +570,7 @@ Future<Nothing> StatusUpdateManager::update(
     const StatusUpdate& update,
     const SlaveID& slaveId,
     const ExecutorID& executorId,
-    const UUID& uuid)
+    const ContainerID& containerId)
 {
   return dispatch(
       process,
@@ -577,7 +578,7 @@ Future<Nothing> StatusUpdateManager::update(
       update,
       slaveId,
       executorId,
-      uuid);
+      containerId);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f90fe764/src/slave/status_update_manager.hpp
----------------------------------------------------------------------
diff --git a/src/slave/status_update_manager.hpp b/src/slave/status_update_manager.hpp
index 06ea465..d29e269 100644
--- a/src/slave/status_update_manager.hpp
+++ b/src/slave/status_update_manager.hpp
@@ -92,7 +92,7 @@ public:
       const StatusUpdate& update,
       const SlaveID& slaveId,
       const ExecutorID& executorId,
-      const UUID& uuid);
+      const ContainerID& containerId);
 
   // Retries the update to the master (as long as the slave is
   // alive), but does not checkpoint the update.
@@ -148,7 +148,7 @@ struct StatusUpdateStream
                      const Flags& _flags,
                      bool _checkpoint,
                      const Option<ExecutorID>& executorId,
-                     const Option<UUID>& uuid)
+                     const Option<ContainerID>& containerId)
     : checkpoint(_checkpoint),
       terminated(false),
       taskId(_taskId),
@@ -159,14 +159,14 @@ struct StatusUpdateStream
   {
     if (checkpoint) {
       CHECK_SOME(executorId);
-      CHECK_SOME(uuid);
+      CHECK_SOME(containerId);
 
       path = paths::getTaskUpdatesPath(
           paths::getMetaRootDir(flags.work_dir),
           slaveId,
           frameworkId,
           executorId.get(),
-          uuid.get(),
+          containerId.get(),
           taskId);
 
       // Create the base updates directory, if it doesn't exist.


[10/11] Containerizer - updated tests (part 5).

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index 1b1b4cc..8c0b006 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -3,14 +3,18 @@
 #include <stout/os.hpp>
 #include <stout/path.hpp>
 #include <stout/result.hpp>
+#include <stout/uuid.hpp>
 
 #ifdef __linux__
 #include "linux/cgroups.hpp"
 #endif
 
+#include "slave/containerizer/containerizer.hpp"
+#include "slave/containerizer/mesos_containerizer.hpp"
+
+#include "tests/containerizer.hpp"
 #include "tests/environment.hpp"
 #include "tests/flags.hpp"
-#include "tests/isolator.hpp"
 #include "tests/mesos.hpp"
 
 using namespace process;
@@ -120,16 +124,16 @@ Try<process::PID<master::Master> > MesosTest::StartMaster(
 Try<process::PID<slave::Slave> > MesosTest::StartSlave(
     const Option<slave::Flags>& flags)
 {
-  TestingIsolator* isolator = new TestingIsolator();
+  slave::Containerizer* containerizer = new TestContainerizer();
 
-  Try<process::PID<slave::Slave> > pid = StartSlave(isolator, flags);
+  Try<process::PID<slave::Slave> > pid = StartSlave(containerizer, flags);
 
   if (pid.isError()) {
-    delete isolator;
+    delete containerizer;
     return pid;
   }
 
-  isolators[pid.get()] = isolator;
+  containerizers[pid.get()] = containerizer;
 
   return pid;
 }
@@ -139,37 +143,39 @@ Try<process::PID<slave::Slave> > MesosTest::StartSlave(
     MockExecutor* executor,
     const Option<slave::Flags>& flags)
 {
-  TestingIsolator* isolator = new TestingIsolator(executor);
-    
-  Try<process::PID<slave::Slave> > pid = StartSlave(isolator, flags);
+  slave::Containerizer* containerizer = new TestContainerizer(executor);
+
+  Try<process::PID<slave::Slave> > pid = StartSlave(containerizer, flags);
 
   if (pid.isError()) {
-    delete isolator;
+    delete containerizer;
     return pid;
   }
 
-  isolators[pid.get()] = isolator;
+  containerizers[pid.get()] = containerizer;
 
   return pid;
 }
 
 
 Try<process::PID<slave::Slave> > MesosTest::StartSlave(
-    slave::Isolator* isolator,
+    slave::Containerizer* containerizer,
     const Option<slave::Flags>& flags)
 {
   return cluster.slaves.start(
-      isolator, flags.isNone() ? CreateSlaveFlags() : flags.get());
+      containerizer, flags.isNone() ? CreateSlaveFlags() : flags.get());
 }
 
 
 Try<process::PID<slave::Slave> > MesosTest::StartSlave(
-    slave::Isolator* isolator,
+    slave::Containerizer* containerizer,
     Owned<MasterDetector> detector,
     const Option<slave::Flags>& flags)
 {
   return cluster.slaves.start(
-      isolator, detector, flags.isNone() ? CreateSlaveFlags() : flags.get());
+      containerizer,
+      detector,
+      flags.isNone() ? CreateSlaveFlags() : flags.get());
 }
 
 
@@ -187,17 +193,19 @@ Try<PID<slave::Slave> > MesosTest::StartSlave(
     Owned<MasterDetector> detector,
     const Option<slave::Flags>& flags)
 {
-  TestingIsolator* isolator = new TestingIsolator(executor);
+  slave::Containerizer* containerizer = new TestContainerizer(executor);
 
   Try<process::PID<slave::Slave> > pid = cluster.slaves.start(
-      isolator, detector, flags.isNone() ? CreateSlaveFlags() : flags.get());
+      containerizer,
+      detector,
+      flags.isNone() ? CreateSlaveFlags() : flags.get());
 
   if (pid.isError()) {
-    delete isolator;
+    delete containerizer;
     return pid;
   }
 
-  isolators[pid.get()] = isolator;
+  containerizers[pid.get()] = containerizer;
 
   return pid;
 }
@@ -212,10 +220,10 @@ void MesosTest::Stop(const process::PID<master::Master>& pid)
 void MesosTest::Stop(const process::PID<slave::Slave>& pid, bool shutdown)
 {
   cluster.slaves.stop(pid, shutdown);
-  if (isolators.count(pid) > 0) {
-    TestingIsolator* isolator = isolators[pid];
-    isolators.erase(pid);
-    delete isolator;
+  if (containerizers.count(pid) > 0) {
+    slave::Containerizer* containerizer = containerizers[pid];
+    containerizers.erase(pid);
+    delete containerizer;
   }
 }
 
@@ -237,10 +245,10 @@ void MesosTest::ShutdownSlaves()
 {
   cluster.slaves.shutdown();
 
-  foreachvalue (TestingIsolator* isolator, isolators) {
-    delete isolator;
+  foreachvalue (slave::Containerizer* containerizer, containerizers) {
+    delete containerizer;
   }
-  isolators.clear();
+  containerizers.clear();
 }
 
 
@@ -251,71 +259,114 @@ void MesosTest::TearDown()
 }
 
 
-#ifdef __linux__
-void IsolatorTest<slave::CgroupsIsolator>::SetUpTestCase()
+slave::Flags ContainerizerTest<slave::MesosContainerizer>::CreateSlaveFlags()
 {
-  // Clean up the testing hierarchy, in case it wasn't cleaned up
-  // properly from previous tests.
-  AWAIT_READY(cgroups::cleanup(TEST_CGROUPS_HIERARCHY));
-}
+  slave::Flags flags = MesosTest::CreateSlaveFlags();
 
+#ifdef __linux__
+  flags.isolation = "cgroups/cpu,cgroups/mem";
+  flags.cgroups_hierarchy = baseHierarchy;
+  flags.cgroups_root = TEST_CGROUPS_ROOT + "_" + UUID::random().toString();
+#else
+  flags.isolation = "posix/cpu,posix/mem";
+#endif
 
-void IsolatorTest<slave::CgroupsIsolator>::TearDownTestCase()
-{
-  AWAIT_READY(cgroups::cleanup(TEST_CGROUPS_HIERARCHY));
+  return flags;
 }
 
 
-slave::Flags IsolatorTest<slave::CgroupsIsolator>::CreateSlaveFlags()
+#ifdef __linux__
+void ContainerizerTest<slave::MesosContainerizer>::SetUpTestCase()
 {
-  slave::Flags flags = MesosTest::CreateSlaveFlags();
-
-  flags.cgroups_hierarchy = hierarchy;
+  // Clean up any testing hierarchies.
+  Try<std::set<std::string> > hierarchies = cgroups::hierarchies();
+  ASSERT_SOME(hierarchies);
+  foreach (const std::string& hierarchy, hierarchies.get()) {
+    if (strings::startsWith(hierarchy, TEST_CGROUPS_HIERARCHY)) {
+      AWAIT_READY(cgroups::cleanup(hierarchy));
+    }
+  }
+}
 
-  // TODO(benh): Create a different cgroups root for each slave.
-  flags.cgroups_root = TEST_CGROUPS_ROOT;
 
-  return flags;
+void ContainerizerTest<slave::MesosContainerizer>::TearDownTestCase()
+{
+  // Clean up any testing hierarchies.
+  Try<std::set<std::string> > hierarchies = cgroups::hierarchies();
+  ASSERT_SOME(hierarchies);
+  foreach (const std::string& hierarchy, hierarchies.get()) {
+    if (strings::startsWith(hierarchy, TEST_CGROUPS_HIERARCHY)) {
+      AWAIT_READY(cgroups::cleanup(hierarchy));
+    }
+  }
 }
 
 
-void IsolatorTest<slave::CgroupsIsolator>::SetUp()
+void ContainerizerTest<slave::MesosContainerizer>::SetUp()
 {
   MesosTest::SetUp();
 
-  const std::string subsystems = "cpu,cpuacct,memory,freezer";
-  Result<std::string> hierarchy_ = cgroups::hierarchy(subsystems);
-  ASSERT_FALSE(hierarchy_.isError());
-  if (hierarchy_.isNone()) {
-    // Try to mount a hierarchy for testing.
-    ASSERT_SOME(cgroups::mount(TEST_CGROUPS_HIERARCHY, subsystems))
-      << "-------------------------------------------------------------\n"
-      << "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();
+  subsystems.insert("cpu");
+  subsystems.insert("cpuacct");
+  subsystems.insert("memory");
+  subsystems.insert("freezer");
+
+  foreach (const std::string& subsystem, 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"
+        << "-------------------------------------------------------------";
+    }
   }
 }
 
 
-void IsolatorTest<slave::CgroupsIsolator>::TearDown()
+void ContainerizerTest<slave::MesosContainerizer>::TearDown()
 {
   MesosTest::TearDown();
 
-  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, 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 (strings::startsWith(cgroup, TEST_CGROUPS_ROOT)) {
+        AWAIT_READY(cgroups::destroy(hierarchy, cgroup));
+      }
+    }
   }
 }
 #endif // __linux__

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index d7bdaee..018d4ff 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -48,10 +48,8 @@
 #include "master/hierarchical_allocator_process.hpp"
 #include "master/master.hpp"
 
-#ifdef __linux__
-#include "slave/cgroups_isolator.hpp"
-#endif
-#include "slave/isolator.hpp"
+#include "slave/containerizer/containerizer.hpp"
+#include "slave/containerizer/mesos_containerizer.hpp"
 #include "slave/slave.hpp"
 
 #include "tests/cluster.hpp"
@@ -62,7 +60,6 @@ namespace tests {
 
 // Forward declarations.
 class MockExecutor;
-class TestingIsolator;
 
 
 class MesosTest : public ::testing::Test
@@ -108,14 +105,14 @@ protected:
       MockExecutor* executor,
       const Option<slave::Flags>& flags = None());
 
-  // Starts a slave with the specified isolator and flags.
+  // Starts a slave with the specified containerizer and flags.
   virtual Try<process::PID<slave::Slave> > StartSlave(
-      slave::Isolator* isolator,
+      slave::Containerizer* containerizer,
       const Option<slave::Flags>& flags = None());
 
-  // Starts a slave with the specified isolator, detector and flags.
+  // Starts a slave with the specified containerizer, detector and flags.
   virtual Try<process::PID<slave::Slave> > StartSlave(
-      slave::Isolator* isolator,
+      slave::Containerizer* containerizer,
       process::Owned<MasterDetector> detector,
       const Option<slave::Flags>& flags = None());
 
@@ -151,15 +148,13 @@ protected:
 
   Cluster cluster;
 
-  // TestingIsolator(s) created during test that we need to cleanup.
-  std::map<process::PID<slave::Slave>, TestingIsolator*> isolators;
+  // Containerizer(s) created during test that we need to cleanup.
+  std::map<process::PID<slave::Slave>, slave::Containerizer*> containerizers;
 };
 
 
-
 template <typename T>
-class IsolatorTest : public MesosTest {};
-
+class ContainerizerTest : public MesosTest {};
 
 #ifdef __linux__
 // Cgroups hierarchy used by the cgroups related tests.
@@ -170,7 +165,7 @@ const static std::string TEST_CGROUPS_ROOT = "mesos_test";
 
 
 template <>
-class IsolatorTest<slave::CgroupsIsolator> : public MesosTest
+class ContainerizerTest<slave::MesosContainerizer> : public MesosTest
 {
 public:
   static void SetUpTestCase();
@@ -182,7 +177,21 @@ protected:
   virtual void TearDown();
 
 private:
-  std::string hierarchy;
+  // Base hierarchy for separately mounted cgroup controllers, e.g., if the
+  // base hierachy is /sys/fs/cgroup then each controller will be mounted to
+  // /sys/fs/cgroup/{controller}/.
+  std::string baseHierarchy;
+
+  // Set of cgroup subsystems used by the cgroups related tests.
+  hashset<std::string> subsystems;
+
+};
+#else
+template<>
+class ContainerizerTest<slave::MesosContainerizer> : public MesosTest
+{
+protected:
+  virtual slave::Flags CreateSlaveFlags();
 };
 #endif // __linux__
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/monitor_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/monitor_tests.cpp b/src/tests/monitor_tests.cpp
index 7988c90..4b950e1 100644
--- a/src/tests/monitor_tests.cpp
+++ b/src/tests/monitor_tests.cpp
@@ -25,6 +25,7 @@
 
 #include <process/clock.hpp>
 #include <process/future.hpp>
+#include <process/gmock.hpp>
 #include <process/gtest.hpp>
 #include <process/http.hpp>
 #include <process/pid.hpp>
@@ -35,7 +36,7 @@
 #include "slave/constants.hpp"
 #include "slave/monitor.hpp"
 
-#include "tests/isolator.hpp"
+#include "tests/containerizer.hpp"
 
 using namespace mesos;
 using namespace mesos::internal;
@@ -65,6 +66,9 @@ TEST(MonitorTest, Collection)
   ExecutorID executorId;
   executorId.set_value("executor");
 
+  ContainerID containerId;
+  containerId.set_value("container");
+
   ExecutorInfo executorInfo;
   executorInfo.mutable_executor_id()->CopyFrom(executorId);
   executorInfo.mutable_framework_id()->CopyFrom(frameworkId);
@@ -95,12 +99,10 @@ TEST(MonitorTest, Collection)
   statistics3.set_timestamp(
       statistics3.timestamp() + slave::RESOURCE_MONITORING_INTERVAL.secs());
 
-  TestingIsolator isolator;
-
-  process::spawn(isolator);
+  TestContainerizer containerizer;
 
   Future<Nothing> usage1, usage2, usage3;
-  EXPECT_CALL(isolator, usage(frameworkId, executorId))
+  EXPECT_CALL(containerizer, usage(containerId))
     .WillOnce(DoAll(FutureSatisfy(&usage1),
                     Return(statistics1)))
     .WillOnce(DoAll(FutureSatisfy(&usage2),
@@ -108,20 +110,19 @@ TEST(MonitorTest, Collection)
     .WillOnce(DoAll(FutureSatisfy(&usage3),
                     Return(statistics3)));
 
-  slave::ResourceMonitor monitor(&isolator);
+  slave::ResourceMonitor monitor(&containerizer);
 
   // We pause the clock first in order to make sure that we can
   // advance time below to force the 'delay' in
-  // ResourceMonitorProcess::watch to execute.
+  // ResourceMonitorProcess::start to execute.
   process::Clock::pause();
 
-  monitor.watch(
-      frameworkId,
-      executorId,
+  monitor.start(
+      containerId,
       executorInfo,
       slave::RESOURCE_MONITORING_INTERVAL);
 
-  // Now wait for ResouorceMonitorProcess::watch to finish so we can
+  // Now wait for ResouorceMonitorProcess::start to finish so we can
   // advance time to cause collection to begin.
   process::Clock::settle();
 
@@ -130,7 +131,7 @@ TEST(MonitorTest, Collection)
 
   AWAIT_READY(usage1);
 
-  // Wait until the isolator has finished returning the statistics.
+  // Wait until the containerizer has finished returning the statistics.
   process::Clock::settle();
 
   // Expect a second collection to occur after the interval.
@@ -139,7 +140,7 @@ TEST(MonitorTest, Collection)
 
   AWAIT_READY(usage2);
 
-  // Wait until the isolator has finished returning the statistics.
+  // Wait until the containerizer has finished returning the statistics.
   process::Clock::settle();
 
   // Expect a third collection to occur after the interval.
@@ -148,17 +149,17 @@ TEST(MonitorTest, Collection)
 
   AWAIT_READY(usage3);
 
-  // Wait until the isolator has finished returning the statistics.
+  // Wait until the containerize has finished returning the statistics.
   process::Clock::settle();
 
   // Ensure the monitor stops polling the isolator.
-  monitor.unwatch(frameworkId, executorId);
+  monitor.stop(containerId);
 
-  // Wait until ResourceMonitorProcess::unwatch has completed.
+  // Wait until ResourceMonitorProcess::stop has completed.
   process::Clock::settle();
 
-  // This time, Isolator::usage should not get called.
-  EXPECT_CALL(isolator, usage(frameworkId, executorId))
+  // This time, Containerizer::usage should not get called.
+  EXPECT_CALL(containerizer, usage(containerId))
     .Times(0);
 
   process::Clock::advance(slave::RESOURCE_MONITORING_INTERVAL);
@@ -174,6 +175,9 @@ TEST(MonitorTest, Statistics)
   ExecutorID executorId;
   executorId.set_value("executor");
 
+  ContainerID containerId;
+  containerId.set_value("container");
+
   ExecutorInfo executorInfo;
   executorInfo.mutable_executor_id()->CopyFrom(executorId);
   executorInfo.mutable_framework_id()->CopyFrom(frameworkId);
@@ -194,24 +198,21 @@ TEST(MonitorTest, Statistics)
   statistics.set_mem_limit_bytes(2048);
   statistics.set_timestamp(0);
 
-  TestingIsolator isolator;
-
-  process::spawn(isolator);
+  TestContainerizer containerizer;
 
   Future<Nothing> usage;
-  EXPECT_CALL(isolator, usage(frameworkId, executorId))
+  EXPECT_CALL(containerizer, usage(containerId))
     .WillOnce(DoAll(FutureSatisfy(&usage),
                     Return(statistics)));
 
-  slave::ResourceMonitor monitor(&isolator);
+  slave::ResourceMonitor monitor(&containerizer);
 
   // We pause the clock first to ensure unexpected collections
   // are avoided.
   process::Clock::pause();
 
-  monitor.watch(
-      frameworkId,
-      executorId,
+  monitor.start(
+      containerId,
       executorInfo,
       slave::RESOURCE_MONITORING_INTERVAL);
 
@@ -274,13 +275,13 @@ TEST(MonitorTest, Statistics)
       response);
 
   // Ensure the monitor stops polling the isolator.
-  monitor.unwatch(frameworkId, executorId);
+  monitor.stop(containerId);
 
-  // Wait until ResourceMonitorProcess::unwatch has completed.
+  // Wait until ResourceMonitorProcess::stop has completed.
   process::Clock::settle();
 
-  // This time, Isolator::usage should not get called.
-  EXPECT_CALL(isolator, usage(frameworkId, executorId))
+  // This time, Containerizer::usage should not get called.
+  EXPECT_CALL(containerizer, usage(containerId))
     .Times(0);
 
   response = process::http::get(upid, "statistics.json");

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/paths_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/paths_tests.cpp b/src/tests/paths_tests.cpp
index 40c644c..8e11264 100644
--- a/src/tests/paths_tests.cpp
+++ b/src/tests/paths_tests.cpp
@@ -42,12 +42,12 @@ class PathsTest : public ::testing::Test
 {
 public:
   PathsTest()
-    : uuid(UUID::random())
   {
     slaveId.set_value("slave1");
     frameworkId.set_value("framework1");
     executorId.set_value("executor1");
     taskId.set_value("task1");
+    containerId.set_value(UUID::random().toString());
 
     Try<string> path = os::mkdtemp();
     CHECK_SOME(path) << "Failed to mkdtemp";
@@ -64,7 +64,7 @@ protected:
   FrameworkID frameworkId;
   ExecutorID executorId;
   TaskID taskId;
-  UUID uuid;
+  ContainerID containerId;
   string rootDir;
 };
 
@@ -72,12 +72,12 @@ protected:
 TEST_F(PathsTest, CreateExecutorDirectory)
 {
   const string& result = paths::createExecutorDirectory(
-      rootDir, slaveId, frameworkId, executorId, uuid);
+      rootDir, slaveId, frameworkId, executorId, containerId);
 
   // Expected directory layout.
   string dir = rootDir + "/slaves/" + slaveId.value() + "/frameworks/"
                + frameworkId.value() + "/executors/" + executorId.value()
-               + "/runs/" + uuid.toString();
+               + "/runs/" + containerId.value();
 
   ASSERT_EQ(dir, result);
 }
@@ -97,13 +97,13 @@ TEST_F(PathsTest, format)
   ASSERT_EQ(dir, paths::getExecutorPath(
       rootDir, slaveId, frameworkId, executorId));
 
-  dir += "/runs/" + uuid.toString();
+  dir += "/runs/" + containerId.value();
   ASSERT_EQ(dir, paths::getExecutorRunPath(
-      rootDir, slaveId, frameworkId, executorId, uuid));
+      rootDir, slaveId, frameworkId, executorId, containerId));
 
   dir += "/tasks/" + taskId.value();
   ASSERT_EQ(dir, paths::getTaskPath(
-      rootDir, slaveId, frameworkId, executorId, uuid, taskId));
+      rootDir, slaveId, frameworkId, executorId, containerId, taskId));
 }
 
 } // namespace paths {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index 4779509..0fc256b 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -39,22 +39,16 @@
 
 #include "common/protobuf_utils.hpp"
 
-#ifdef __linux__
-#include "linux/cgroups.hpp"
-#endif
-
 #include "master/detector.hpp"
 #include "master/master.hpp"
 
 #include "slave/gc.hpp"
-#ifdef __linux__
-#include "slave/cgroups_isolator.hpp"
-#endif
 #include "slave/paths.hpp"
-#include "slave/process_isolator.hpp"
 #include "slave/slave.hpp"
 #include "slave/state.hpp"
 
+#include "slave/containerizer/containerizer.hpp"
+
 #include "messages/messages.hpp"
 
 #include "tests/mesos.hpp"
@@ -70,10 +64,7 @@ using namespace process;
 using mesos::internal::master::Master;
 
 using mesos::internal::slave::GarbageCollectorProcess;
-#ifdef __linux__
-using mesos::internal::slave::CgroupsIsolator;
-#endif
-using mesos::internal::slave::ProcessIsolator;
+using mesos::internal::slave::Containerizer;
 
 using std::map;
 using std::string;
@@ -115,14 +106,13 @@ TEST_F(SlaveStateTest, CheckpointString)
   ASSERT_SOME_EQ(expected, os::read(file));
 }
 
-
 template <typename T>
-class SlaveRecoveryTest : public IsolatorTest<T>
+class SlaveRecoveryTest : public ContainerizerTest<T>
 {
 public:
   virtual slave::Flags CreateSlaveFlags()
   {
-    slave::Flags flags = IsolatorTest<T>::CreateSlaveFlags();
+    slave::Flags flags = ContainerizerTest<T>::CreateSlaveFlags();
 
     // Setup recovery slave flags.
     flags.checkpoint = true;
@@ -133,15 +123,12 @@ public:
   }
 };
 
+// Note: Although these tests are typed it is Containerizer::create() that
+// decides which Containerizer to create based on the flags - see
+// SlaveRecoveryTest.
+typedef ::testing::Types<slave::MesosContainerizer> ContainerizerTypes;
 
-#ifdef __linux__
-typedef ::testing::Types<ProcessIsolator, CgroupsIsolator> IsolatorTypes;
-#else
-typedef ::testing::Types<ProcessIsolator> IsolatorTypes;
-#endif
-
-TYPED_TEST_CASE(SlaveRecoveryTest, IsolatorTypes);
-
+TYPED_TEST_CASE(SlaveRecoveryTest, ContainerizerTypes);
 
 // Enable checkpointing on the slave and ensure recovery works.
 TYPED_TEST(SlaveRecoveryTest, RecoverSlaveState)
@@ -149,11 +136,12 @@ TYPED_TEST(SlaveRecoveryTest, RecoverSlaveState)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator, flags);
+  Try<Containerizer*> containerizer = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -244,21 +232,21 @@ TYPED_TEST(SlaveRecoveryTest, RecoverSlaveState)
   ASSERT_TRUE(state.frameworks[frameworkId].executors.contains(executorId));
 
   // Check executor id and pids.
-  const Option<UUID>& uuid=
+  const Option<ContainerID>& containerId =
       state.frameworks[frameworkId].executors[executorId].latest;
-  ASSERT_SOME(uuid);
+  ASSERT_SOME(containerId);
 
   ASSERT_TRUE(state
                 .frameworks[frameworkId]
                 .executors[executorId]
-                .runs.contains(uuid.get()));
+                .runs.contains(containerId.get()));
 
   ASSERT_SOME_EQ(
       libprocessPid,
       state
         .frameworks[frameworkId]
         .executors[executorId]
-        .runs[uuid.get()]
+        .runs[containerId.get()]
         .libprocessPid);
 
 
@@ -266,7 +254,7 @@ TYPED_TEST(SlaveRecoveryTest, RecoverSlaveState)
   ASSERT_TRUE(state
                 .frameworks[frameworkId]
                 .executors[executorId]
-                .runs[uuid.get()]
+                .runs[containerId.get()]
                 .tasks.contains(task.task_id()));
 
   const Task& t = mesos::internal::protobuf::createTask(
@@ -277,7 +265,7 @@ TYPED_TEST(SlaveRecoveryTest, RecoverSlaveState)
       state
         .frameworks[frameworkId]
         .executors[executorId]
-        .runs[uuid.get()]
+        .runs[containerId.get()]
         .tasks[task.task_id()]
         .info);
 
@@ -287,7 +275,7 @@ TYPED_TEST(SlaveRecoveryTest, RecoverSlaveState)
       state
         .frameworks[frameworkId]
         .executors[executorId]
-        .runs[uuid.get()]
+        .runs[containerId.get()]
         .tasks[task.task_id()]
         .updates.size());
 
@@ -296,14 +284,14 @@ TYPED_TEST(SlaveRecoveryTest, RecoverSlaveState)
       state
         .frameworks[frameworkId]
         .executors[executorId]
-        .runs[uuid.get()]
+        .runs[containerId.get()]
         .tasks[task.task_id()]
         .updates.front().uuid());
 
   ASSERT_TRUE(state
                 .frameworks[frameworkId]
                 .executors[executorId]
-                .runs[uuid.get()]
+                .runs[containerId.get()]
                 .tasks[task.task_id()]
                 .acks.contains(UUID::fromBytes(ack.get().uuid())));
 
@@ -314,7 +302,9 @@ TYPED_TEST(SlaveRecoveryTest, RecoverSlaveState)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+
+  delete containerizer.get();
 }
 
 
@@ -325,11 +315,12 @@ TYPED_TEST(SlaveRecoveryTest, RecoverStatusUpdateManager)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -376,16 +367,18 @@ TYPED_TEST(SlaveRecoveryTest, RecoverStatusUpdateManager)
   AWAIT_READY(update);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   Future<TaskStatus> status;
   EXPECT_CALL(sched, statusUpdate(_, _))
     .WillOnce(FutureArg<1>(&status))
     .WillRepeatedly(Return());       // Ignore subsequent updates.
 
-  // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  // Restart the slave (use same flags) with a new containerizer.
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(status);
@@ -400,7 +393,8 @@ TYPED_TEST(SlaveRecoveryTest, RecoverStatusUpdateManager)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -412,11 +406,12 @@ TYPED_TEST(SlaveRecoveryTest, ReconnectExecutor)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -455,6 +450,7 @@ TYPED_TEST(SlaveRecoveryTest, ReconnectExecutor)
   AWAIT_READY(statusUpdate);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   Future<Message> reregisterExecutorMessage =
     FUTURE_MESSAGE(Eq(ReregisterExecutorMessage().GetTypeName()), _, _);
@@ -464,10 +460,11 @@ TYPED_TEST(SlaveRecoveryTest, ReconnectExecutor)
     .WillOnce(FutureArg<1>(&status))
     .WillRepeatedly(Return());       // Ignore subsequent updates.
 
-  // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  // Restart the slave (use same flags) with a new containerizer.
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   // Ensure the executor re-registers.
@@ -496,7 +493,8 @@ TYPED_TEST(SlaveRecoveryTest, ReconnectExecutor)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -508,11 +506,12 @@ TYPED_TEST(SlaveRecoveryTest, RecoverUnregisteredExecutor)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -551,6 +550,7 @@ TYPED_TEST(SlaveRecoveryTest, RecoverUnregisteredExecutor)
   UPID executorPid = registerExecutor.get().from;
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   Future<TaskStatus> status;
   EXPECT_CALL(sched, statusUpdate(_, _))
@@ -559,15 +559,16 @@ TYPED_TEST(SlaveRecoveryTest, RecoverUnregisteredExecutor)
 
   Future<Nothing> _recover = FUTURE_DISPATCH(_, &Slave::_recover);
 
-  // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  // Restart the slave (use same flags) with a new containerizer.
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
   Future<vector<Offer> > offers2;
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillOnce(FutureArg<1>(&offers2))
     .WillRepeatedly(Return());        // Ignore subsequent offers.
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   Clock::pause();
@@ -601,7 +602,8 @@ TYPED_TEST(SlaveRecoveryTest, RecoverUnregisteredExecutor)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -614,11 +616,12 @@ TYPED_TEST(SlaveRecoveryTest, RecoverTerminatedExecutor)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -664,6 +667,7 @@ TYPED_TEST(SlaveRecoveryTest, RecoverTerminatedExecutor)
   AWAIT_READY(ack);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   Future<TaskStatus> status;
   EXPECT_CALL(sched, statusUpdate(_, _))
@@ -674,15 +678,16 @@ TYPED_TEST(SlaveRecoveryTest, RecoverTerminatedExecutor)
 
   Future<Nothing> _recover = FUTURE_DISPATCH(_, &Slave::_recover);
 
-  // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  // Restart the slave (use same flags) with a new containerizer.
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
   Future<vector<Offer> > offers2;
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillOnce(FutureArg<1>(&offers2))
     .WillRepeatedly(Return());        // Ignore subsequent offers.
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   Clock::pause();
@@ -716,7 +721,8 @@ TYPED_TEST(SlaveRecoveryTest, RecoverTerminatedExecutor)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -732,14 +738,15 @@ TYPED_TEST(SlaveRecoveryTest, DISABLED_RecoveryTimeout)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   // Set a short recovery timeout, as we can't control the executor
   // driver time when using the process / cgroups isolators.
   slave::Flags flags = this->CreateSlaveFlags();
   flags.recovery_timeout = Milliseconds(1);
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -779,6 +786,7 @@ TYPED_TEST(SlaveRecoveryTest, DISABLED_RecoveryTimeout)
   AWAIT_READY(_statusUpdateAcknowledgement);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   Future<TaskStatus> status;
   EXPECT_CALL(sched, statusUpdate(_, _))
@@ -790,10 +798,11 @@ TYPED_TEST(SlaveRecoveryTest, DISABLED_RecoveryTimeout)
 
   Future<Nothing> _recover = FUTURE_DISPATCH(_, &Slave::_recover);
 
-  // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  // Restart the slave (use same flags) with a new containerizer.
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   Clock::pause();
@@ -812,7 +821,8 @@ TYPED_TEST(SlaveRecoveryTest, DISABLED_RecoveryTimeout)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -825,11 +835,12 @@ TYPED_TEST(SlaveRecoveryTest, RecoverCompletedExecutor)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -873,19 +884,21 @@ TYPED_TEST(SlaveRecoveryTest, RecoverCompletedExecutor)
   AWAIT_READY(schedule);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   Future<Nothing> schedule2 = FUTURE_DISPATCH(
       _, &GarbageCollectorProcess::schedule);
 
-  // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  // Restart the slave (use same flags) with a new containerizer.
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
   Future<vector<Offer> > offers2;
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillOnce(FutureArg<1>(&offers2))
     .WillRepeatedly(Return());        // Ignore subsequent offers.
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   // We use 'gc.schedule' as a proxy for the cleanup of the executor.
@@ -899,7 +912,8 @@ TYPED_TEST(SlaveRecoveryTest, RecoverCompletedExecutor)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -911,11 +925,12 @@ TYPED_TEST(SlaveRecoveryTest, CleanupExecutor)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -955,6 +970,7 @@ TYPED_TEST(SlaveRecoveryTest, CleanupExecutor)
   AWAIT_READY(ack);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   // Slave in cleanup mode shouldn't reregister with slave and hence
   // no offers should be made by the master.
@@ -968,11 +984,12 @@ TYPED_TEST(SlaveRecoveryTest, CleanupExecutor)
   Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
 
   // Restart the slave in 'cleanup' recovery mode with a new isolator.
-  TypeParam isolator2;
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
   flags.recover = "cleanup";
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   Clock::pause();
@@ -996,7 +1013,8 @@ TYPED_TEST(SlaveRecoveryTest, CleanupExecutor)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -1007,9 +1025,12 @@ TYPED_TEST(SlaveRecoveryTest, RemoveNonCheckpointingFramework)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator;
+  slave::Flags flags = this->CreateSlaveFlags();
+
+  Try<Containerizer*> containerizer = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer);
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator);
+  Try<PID<Slave> > slave = this->StartSlave(containerizer.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -1069,6 +1090,7 @@ TYPED_TEST(SlaveRecoveryTest, RemoveNonCheckpointingFramework)
     .WillOnce(FutureArg<1>(&status2));
 
   this->Stop(slave.get());
+  delete containerizer.get();
 
   // Scheduler should receive the TASK_LOST updates.
   AWAIT_READY(status1);
@@ -1080,7 +1102,7 @@ TYPED_TEST(SlaveRecoveryTest, RemoveNonCheckpointingFramework)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
 }
 
 
@@ -1091,11 +1113,12 @@ TYPED_TEST(SlaveRecoveryTest, NonCheckpointingFramework)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator, flags);
+  Try<Containerizer*> containerizer = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -1161,7 +1184,8 @@ TYPED_TEST(SlaveRecoveryTest, NonCheckpointingFramework)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer.get();
 }
 
 
@@ -1172,8 +1196,6 @@ TYPED_TEST(SlaveRecoveryTest, NonCheckpointingSlave)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator;
-
   // Disable checkpointing for the slave.
   slave::Flags flags = this->CreateSlaveFlags();
   flags.checkpoint = false;
@@ -1183,7 +1205,10 @@ TYPED_TEST(SlaveRecoveryTest, NonCheckpointingSlave)
   Future<RegisterSlaveMessage> registerSlaveMessage =
     FUTURE_PROTOBUF(RegisterSlaveMessage(), _, _);
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator, flags);
+  Try<Containerizer*> containerizer = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(registerSlaveMessage);
@@ -1215,10 +1240,10 @@ TYPED_TEST(SlaveRecoveryTest, NonCheckpointingSlave)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer.get();
 }
 
-
 // Scheduler asks a restarted slave to kill a task that has been
 // running before the slave restarted. This test ensures that a
 // restarted slave is able to communicate with all components
@@ -1228,11 +1253,12 @@ TYPED_TEST(SlaveRecoveryTest, KillTask)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -1272,6 +1298,7 @@ TYPED_TEST(SlaveRecoveryTest, KillTask)
   AWAIT_READY(ack);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   Future<Nothing> _recover = FUTURE_DISPATCH(_, &Slave::_recover);
 
@@ -1279,9 +1306,10 @@ TYPED_TEST(SlaveRecoveryTest, KillTask)
     FUTURE_PROTOBUF(ReregisterSlaveMessage(), _, _);
 
   // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   Clock::pause();
@@ -1329,7 +1357,8 @@ TYPED_TEST(SlaveRecoveryTest, KillTask)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -1340,12 +1369,13 @@ TYPED_TEST(SlaveRecoveryTest, Reboot)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
   flags.strict = false;
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -1398,6 +1428,7 @@ TYPED_TEST(SlaveRecoveryTest, Reboot)
   AWAIT_READY(status);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   // Shut down the executor manually so that it doesn't hang around
   // after the test finishes.
@@ -1411,15 +1442,16 @@ TYPED_TEST(SlaveRecoveryTest, Reboot)
   Future<RegisterSlaveMessage> registerSlave =
     FUTURE_PROTOBUF(RegisterSlaveMessage(), _, _);
 
-  // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  // Restart the slave (use same flags) with a new containerizer.
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
   Future<vector<Offer> > offers2;
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillOnce(FutureArg<1>(&offers2))
     .WillRepeatedly(Return());        // Ignore subsequent offers.
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(registerSlave);
@@ -1432,7 +1464,8 @@ TYPED_TEST(SlaveRecoveryTest, Reboot)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -1445,12 +1478,13 @@ TYPED_TEST(SlaveRecoveryTest, GCExecutor)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
   flags.strict = false;
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -1503,6 +1537,7 @@ TYPED_TEST(SlaveRecoveryTest, GCExecutor)
   AWAIT_READY(status);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   // Shut down the executor manually so that it doesn't hang around
   // after the test finishes.
@@ -1522,14 +1557,15 @@ TYPED_TEST(SlaveRecoveryTest, GCExecutor)
     FUTURE_PROTOBUF(ReregisterSlaveMessage(), _, _);
 
   // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
   Future<vector<Offer> > offers2;
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillOnce(FutureArg<1>(&offers2))
     .WillRepeatedly(Return());        // Ignore subsequent offers.
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   Clock::pause();
@@ -1573,7 +1609,8 @@ TYPED_TEST(SlaveRecoveryTest, GCExecutor)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -1584,11 +1621,12 @@ TYPED_TEST(SlaveRecoveryTest, ShutdownSlave)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -1659,16 +1697,18 @@ TYPED_TEST(SlaveRecoveryTest, ShutdownSlave)
   Clock::resume();
 
   this->Stop(slave.get(), true); // Send a "shut down".
+  delete containerizer1.get();
 
   Future<vector<Offer> > offers3;
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillOnce(FutureArg<1>(&offers3))
     .WillRepeatedly(Return());        // Ignore subsequent offers.
 
-  // Now restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  // Now restart the slave (use same flags) with a new containerizer.
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   // Ensure that the slave registered with a new id.
@@ -1686,7 +1726,8 @@ TYPED_TEST(SlaveRecoveryTest, ShutdownSlave)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -1701,11 +1742,12 @@ TYPED_TEST(SlaveRecoveryTest, RegisterDisconnectedSlave)
   Future<RegisterSlaveMessage> registerSlaveMessage =
     FUTURE_PROTOBUF(RegisterSlaveMessage(), _, _);
 
-  TypeParam isolator;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator, flags);
+  Try<Containerizer*> containerizer = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(registerSlaveMessage);
@@ -1784,7 +1826,8 @@ TYPED_TEST(SlaveRecoveryTest, RegisterDisconnectedSlave)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer.get();
 }
 
 
@@ -1799,11 +1842,12 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileKillTask)
   Future<RegisterSlaveMessage> registerSlaveMessage =
       FUTURE_PROTOBUF(RegisterSlaveMessage(), _, _);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(registerSlaveMessage);
@@ -1848,6 +1892,7 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileKillTask)
   AWAIT_READY(_statusUpdateAcknowledgement);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   // Now send a KillTask message to the master. This will not be
   // received by the slave because it is down.
@@ -1857,15 +1902,16 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileKillTask)
   EXPECT_CALL(sched, statusUpdate(_, _))
     .WillOnce(FutureArg<1>(&status));
 
-  // Now restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  // Now restart the slave (use same flags) with a new containerizer.
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
   Future<vector<Offer> > offers2;
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillOnce(FutureArg<1>(&offers2))
     .WillRepeatedly(Return());        // Ignore subsequent offers.
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   // Scheduler should get a TASK_KILLED message.
@@ -1880,7 +1926,8 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileKillTask)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -1895,11 +1942,12 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileShutdownFramework)
   Future<RegisterSlaveMessage> registerSlaveMessage =
     FUTURE_PROTOBUF(RegisterSlaveMessage(), _, _);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(registerSlaveMessage);
@@ -1945,6 +1993,7 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileShutdownFramework)
   AWAIT_READY(_statusUpdateAcknowledgement);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   Future<UnregisterFrameworkMessage> unregisterFrameworkMessage =
     FUTURE_PROTOBUF(UnregisterFrameworkMessage(), _, _);
@@ -1962,10 +2011,11 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileShutdownFramework)
   Future<Nothing> executorTerminated =
     FUTURE_DISPATCH(_, &Slave::executorTerminated);
 
-  // Now restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  // Now restart the slave (use same flags) with a new containerizer.
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   // Slave should get a ShutdownFrameworkMessage.
@@ -1974,7 +2024,8 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileShutdownFramework)
   // Ensure that the executor is terminated.
   AWAIT_READY(executorTerminated);
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -1994,13 +2045,14 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileTasksMissingFromSlave)
   Try<PID<Master> > master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
   EXPECT_CALL(allocator, slaveAdded(_, _, _));
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -2049,6 +2101,7 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileTasksMissingFromSlave)
   EXPECT_CALL(allocator, slaveDisconnected(_));
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   // Construct the framework meta directory that needs wiping.
   string frameworkPath = paths::getFrameworkPath(
@@ -2099,10 +2152,11 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileTasksMissingFromSlave)
     .WillOnce(FutureArg<1>(&offers2))
     .WillRepeatedly(Return());        // Ignore subsequent offers.
 
-  // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  // Restart the slave (use same flags) with a new containerizer.
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   Clock::pause();
@@ -2143,7 +2197,8 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileTasksMissingFromSlave)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -2156,10 +2211,12 @@ TYPED_TEST(SlaveRecoveryTest, SchedulerFailover)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   // Launch the first (i.e., failing) scheduler.
@@ -2202,6 +2259,7 @@ TYPED_TEST(SlaveRecoveryTest, SchedulerFailover)
   AWAIT_READY(_statusUpdateAcknowledgement);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   // Now launch the second (i.e., failover) scheduler using the
   // framework id recorded from the first scheduler.
@@ -2233,10 +2291,11 @@ TYPED_TEST(SlaveRecoveryTest, SchedulerFailover)
   Future<ReregisterSlaveMessage> reregisterSlaveMessage =
       FUTURE_PROTOBUF(ReregisterSlaveMessage(), _, _);
 
-  // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  // Restart the slave (use same flags) with a new containerizer.
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   Clock::pause();
@@ -2287,7 +2346,8 @@ TYPED_TEST(SlaveRecoveryTest, SchedulerFailover)
   driver1.stop();
   driver1.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -2309,10 +2369,12 @@ TYPED_TEST(SlaveRecoveryTest, PartitionedSlave)
   // Drop all the PONGs to simulate slave partition.
   DROP_MESSAGES(Eq("PONG"), _, _);
 
-  TypeParam isolator1;
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   // Enable checkpointing for the framework.
@@ -2404,14 +2466,16 @@ TYPED_TEST(SlaveRecoveryTest, PartitionedSlave)
   Clock::settle();
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   Future<RegisterSlaveMessage> registerSlaveMessage =
     FUTURE_PROTOBUF(RegisterSlaveMessage(), _, _);
 
   // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   AWAIT_READY(registerSlaveMessage);
@@ -2421,7 +2485,8 @@ TYPED_TEST(SlaveRecoveryTest, PartitionedSlave)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -2434,11 +2499,12 @@ TYPED_TEST(SlaveRecoveryTest, MasterFailover)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -2483,6 +2549,7 @@ TYPED_TEST(SlaveRecoveryTest, MasterFailover)
   AWAIT_READY(_statusUpdateAcknowledgement);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   // Step 2. Simulate failed over master by restarting the master.
   this->Stop(master.get());
@@ -2508,9 +2575,10 @@ TYPED_TEST(SlaveRecoveryTest, MasterFailover)
     FUTURE_PROTOBUF(ReregisterSlaveMessage(), _, _);
 
   // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   Clock::pause();
@@ -2558,7 +2626,8 @@ TYPED_TEST(SlaveRecoveryTest, MasterFailover)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
@@ -2571,11 +2640,12 @@ TYPED_TEST(SlaveRecoveryTest, MultipleFrameworks)
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
 
-  TypeParam isolator1;
-
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Try<PID<Slave> > slave = this->StartSlave(&isolator1, flags);
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer1);
+
+  Try<PID<Slave> > slave = this->StartSlave(containerizer1.get(), flags);
   ASSERT_SOME(slave);
 
   // Framework 1.
@@ -2661,16 +2731,18 @@ TYPED_TEST(SlaveRecoveryTest, MultipleFrameworks)
   AWAIT_READY(_statusUpdateAcknowledgement2);
 
   this->Stop(slave.get());
+  delete containerizer1.get();
 
   Future<Nothing> _recover = FUTURE_DISPATCH(_, &Slave::_recover);
 
   Future<ReregisterSlaveMessage> reregisterSlaveMessage =
     FUTURE_PROTOBUF(ReregisterSlaveMessage(), _, _);
 
-  // Restart the slave (use same flags) with a new isolator.
-  TypeParam isolator2;
+  // Restart the slave (use same flags) with a new containerizer.
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags, true);
+  ASSERT_SOME(containerizer2);
 
-  slave = this->StartSlave(&isolator2, flags);
+  slave = this->StartSlave(containerizer2.get(), flags);
   ASSERT_SOME(slave);
 
   Clock::pause();
@@ -2727,39 +2799,14 @@ TYPED_TEST(SlaveRecoveryTest, MultipleFrameworks)
   driver2.stop();
   driver2.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer2.get();
 }
 
 
-// Create a test fixture for those slave recovery tests that only work
-// when ProcessIsolator is used.
-// TODO(jieyu): We use typed test here because it magically allows us
-// to access protected members in Slave (e.g. Slave::reconnect).
-template <typename T>
-class SlaveRecoveryProcessIsolatorTest : public IsolatorTest<T>
-{
-public:
-  virtual slave::Flags CreateSlaveFlags()
-  {
-    slave::Flags flags = IsolatorTest<T>::CreateSlaveFlags();
-
-    // Setup recovery slave flags.
-    flags.checkpoint = true;
-    flags.recover = "reconnect";
-    flags.strict = true;
-
-    return flags;
-  }
-};
-
-
-TYPED_TEST_CASE(SlaveRecoveryProcessIsolatorTest,
-                ::testing::Types<ProcessIsolator>);
-
-
 // This test verifies that slave recovery works properly even if
 // multiple slaves are co-located on the same host.
-TYPED_TEST(SlaveRecoveryProcessIsolatorTest, MultipleSlaves)
+TYPED_TEST(SlaveRecoveryTest, MultipleSlaves)
 {
   Try<PID<Master> > master = this->StartMaster();
   ASSERT_SOME(master);
@@ -2783,9 +2830,10 @@ TYPED_TEST(SlaveRecoveryProcessIsolatorTest, MultipleSlaves)
 
   // Start the first slave.
   slave::Flags flags1 = this->CreateSlaveFlags();
-  slave::ProcessIsolator isolator1;
+  Try<Containerizer*> containerizer1 = Containerizer::create(flags1, true);
+  ASSERT_SOME(containerizer1);
 
-  Try<PID<Slave> > slave1 = this->StartSlave(&isolator1, flags1);
+  Try<PID<Slave> > slave1 = this->StartSlave(containerizer1.get(), flags1);
   ASSERT_SOME(slave1);
 
   AWAIT_READY(offers1);
@@ -2813,9 +2861,10 @@ TYPED_TEST(SlaveRecoveryProcessIsolatorTest, MultipleSlaves)
 
   // Start the second slave.
   slave::Flags flags2 = this->CreateSlaveFlags();
-  slave::ProcessIsolator isolator2;
+  Try<Containerizer*> containerizer2 = Containerizer::create(flags2, true);
+  ASSERT_SOME(containerizer2);
 
-  Try<PID<Slave> > slave2 = this->StartSlave(&isolator2, flags2);
+  Try<PID<Slave> > slave2 = this->StartSlave(containerizer2.get(), flags2);
   ASSERT_SOME(slave2);
 
   AWAIT_READY(offers2);
@@ -2838,7 +2887,9 @@ TYPED_TEST(SlaveRecoveryProcessIsolatorTest, MultipleSlaves)
   AWAIT_READY(_statusUpdateAcknowledgement2);
 
   this->Stop(slave1.get());
+  delete containerizer1.get();
   this->Stop(slave2.get());
+  delete containerizer2.get();
 
   Future<Nothing> _recover1 = FUTURE_DISPATCH(_, &Slave::_recover);
   Future<Nothing> _recover2 = FUTURE_DISPATCH(_, &Slave::_recover);
@@ -2848,15 +2899,17 @@ TYPED_TEST(SlaveRecoveryProcessIsolatorTest, MultipleSlaves)
   Future<ReregisterSlaveMessage> reregisterSlave2 =
     FUTURE_PROTOBUF(ReregisterSlaveMessage(), _, _);
 
-  // Restart both slaves using the same flags with new isolators.
-  slave::ProcessIsolator isolator3;
+  // Restart both slaves using the same flags with new containerizers.
+  Try<Containerizer*> containerizer3 = Containerizer::create(flags1, true);
+  ASSERT_SOME(containerizer3);
 
-  slave1 = this->StartSlave(&isolator3, flags1);
+  slave1 = this->StartSlave(containerizer3.get(), flags1);
   ASSERT_SOME(slave1);
 
-  slave::ProcessIsolator isolator4;
+  Try<Containerizer*> containerizer4 = Containerizer::create(flags2, true);
+  ASSERT_SOME(containerizer4);
 
-  slave2 = this->StartSlave(&isolator4, flags2);
+  slave2 = this->StartSlave(containerizer4.get(), flags2);
   ASSERT_SOME(slave2);
 
   Clock::pause();
@@ -2910,5 +2963,7 @@ TYPED_TEST(SlaveRecoveryProcessIsolatorTest, MultipleSlaves)
   driver.stop();
   driver.join();
 
-  this->Shutdown(); // Shutdown before isolator(s) get deallocated.
+  this->Shutdown();
+  delete containerizer3.get();
+  delete containerizer4.get();
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/0f3f8f35/src/tests/test_framework_test.sh
----------------------------------------------------------------------
diff --git a/src/tests/test_framework_test.sh b/src/tests/test_framework_test.sh
index 277245d..a521b47 100755
--- a/src/tests/test_framework_test.sh
+++ b/src/tests/test_framework_test.sh
@@ -1,5 +1,7 @@
 #!/usr/bin/env bash
 
+set -x
+
 # Expecting MESOS_SOURCE_DIR and MESOS_BUILD_DIR to be in environment.
 
 env | grep MESOS_SOURCE_DIR >/dev/null


[02/11] git commit: Added missing return value for hashmap::containsValue and test.

Posted by vi...@apache.org.
Added missing return value for hashmap::containsValue and test.

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


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

Branch: refs/heads/master
Commit: c578fe5480c4d357db2c3f1019722b7fbe5e33f8
Parents: 413947c
Author: Ian Downes <ia...@gmail.com>
Authored: Tue Feb 11 16:58:18 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Tue Feb 11 16:59:49 2014 -0800

----------------------------------------------------------------------
 .../3rdparty/stout/include/stout/hashmap.hpp           |  1 +
 .../libprocess/3rdparty/stout/tests/hashmap_tests.cpp  | 13 +++++++++++++
 2 files changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c578fe54/3rdparty/libprocess/3rdparty/stout/include/stout/hashmap.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/hashmap.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/hashmap.hpp
index d14b4aa..aa4d9ba 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/hashmap.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/hashmap.hpp
@@ -53,6 +53,7 @@ public:
         return true;
       }
     }
+    return false;
   }
 
   // Inserts a key, value pair into the map replacing an old value

http://git-wip-us.apache.org/repos/asf/mesos/blob/c578fe54/3rdparty/libprocess/3rdparty/stout/tests/hashmap_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/hashmap_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/hashmap_tests.cpp
index ff8bafb..eb3abfc 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/hashmap_tests.cpp
+++ b/3rdparty/libprocess/3rdparty/stout/tests/hashmap_tests.cpp
@@ -23,3 +23,16 @@ TEST(HashMapTest, Insert)
   ASSERT_SOME_EQ(4, map.get("def"));
   ASSERT_EQ(2, map.size());
 }
+
+
+TEST(HashMapTest, Contains)
+{
+  hashmap<string, int> map;
+  map["abc"] = 1;
+
+  ASSERT_TRUE(map.contains("abc"));
+  ASSERT_TRUE(map.containsValue(1));
+
+  ASSERT_FALSE(map.contains("def"));
+  ASSERT_FALSE(map.containsValue(2));
+}


[03/11] git commit: Containerizer - isolators (part 3).

Posted by vi...@apache.org.
Containerizer - isolators (part 3).

Isolators perform isolator for the MesosContainerizer.

Isolator interface and implementations of Posix CPU and Mem isolators
(no isolation, just usage())

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


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

Branch: refs/heads/master
Commit: d5266b8c9c76cccea6b7a70c78ec4c81b4a9a6b4
Parents: 71c6b5c
Author: Ian Downes <ia...@gmail.com>
Authored: Tue Feb 11 16:28:20 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Tue Feb 11 16:59:50 2014 -0800

----------------------------------------------------------------------
 src/Makefile.am                             |   3 +
 src/slave/containerizer/isolator.cpp        | 104 ++++++++++
 src/slave/containerizer/isolator.hpp        | 144 +++++++++++++
 src/slave/containerizer/isolators/posix.hpp | 252 +++++++++++++++++++++++
 4 files changed, 503 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d5266b8c/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index a4b5a52..d9cb9e9 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -177,6 +177,7 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	slave/slave.cpp							\
 	slave/http.cpp							\
 	slave/containerizer/containerizer.cpp				\
+	slave/containerizer/isolator.cpp				\
 	slave/containerizer/launcher.cpp				\
 	slave/containerizer/mesos_containerizer.cpp			\
 	slave/status_update_manager.cpp					\
@@ -233,6 +234,8 @@ libmesos_no_3rdparty_la_SOURCES += common/attributes.hpp		\
 	messages/messages.hpp slave/constants.hpp			\
 	slave/containerizer/cgroups_launcher.hpp			\
 	slave/containerizer/containerizer.hpp				\
+	slave/containerizer/isolator.hpp				\
+	slave/containerizer/isolators/posix.hpp				\
 	slave/containerizer/launcher.hpp				\
 	slave/containerizer/mesos_containerizer.hpp			\
 	slave/flags.hpp slave/gc.hpp slave/monitor.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/d5266b8c/src/slave/containerizer/isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolator.cpp b/src/slave/containerizer/isolator.cpp
new file mode 100644
index 0000000..f7935b3
--- /dev/null
+++ b/src/slave/containerizer/isolator.cpp
@@ -0,0 +1,104 @@
+/**
+ * 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 <process/dispatch.hpp>
+
+#include "slave/containerizer/isolator.hpp"
+
+using namespace process;
+
+using std::string;
+using std::list;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+
+Isolator::Isolator(Owned<IsolatorProcess> _process)
+  : process(_process)
+{
+  process::spawn(CHECK_NOTNULL(process.get()));
+}
+
+
+Isolator::~Isolator()
+{
+  process::terminate(process.get());
+  process::wait(process.get());
+}
+
+
+Future<Nothing> Isolator::recover(const list<state::RunState>& state)
+{
+  return dispatch(process.get(), &IsolatorProcess::recover, state);
+}
+
+
+Future<Nothing> Isolator::prepare(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo)
+{
+  return dispatch(process.get(),
+                  &IsolatorProcess::prepare,
+                  containerId,
+                  executorInfo);
+}
+
+
+Future<Option<CommandInfo> > Isolator::isolate(
+    const ContainerID& containerId,
+    pid_t pid)
+{
+  return dispatch(process.get(), &IsolatorProcess::isolate, containerId, pid);
+}
+
+
+Future<Limitation> Isolator::watch(const ContainerID& containerId)
+{
+  return dispatch(process.get(), &IsolatorProcess::watch, containerId);
+}
+
+
+Future<Nothing> Isolator::update(
+    const ContainerID& containerId,
+    const Resources& resources)
+{
+  return dispatch(
+      process.get(),
+      &IsolatorProcess::update,
+      containerId,
+      resources);
+}
+
+
+Future<ResourceStatistics> Isolator::usage(
+    const ContainerID& containerId) const
+{
+  return dispatch(process.get(), &IsolatorProcess::usage, containerId);
+}
+
+
+Future<Nothing> Isolator::cleanup(const ContainerID& containerId)
+{
+  return dispatch(process.get(), &IsolatorProcess::cleanup, containerId);
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/d5266b8c/src/slave/containerizer/isolator.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolator.hpp b/src/slave/containerizer/isolator.hpp
new file mode 100644
index 0000000..fc6c9ab
--- /dev/null
+++ b/src/slave/containerizer/isolator.hpp
@@ -0,0 +1,144 @@
+/**
+ * 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 __ISOLATOR_HPP__
+#define __ISOLATOR_HPP__
+
+#include <list>
+#include <string>
+
+#include <process/dispatch.hpp>
+#include <process/future.hpp>
+#include <process/owned.hpp>
+#include <process/process.hpp>
+
+#include <stout/try.hpp>
+
+#include "slave/flags.hpp"
+#include "slave/state.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// Forward declaration.
+class IsolatorProcess;
+
+// Information when an executor is impacted by a resource limitation and should
+// be terminated. Intended to support resources like memory where the Linux
+// kernel may invoke the OOM killer, killing some/all of a container's
+// processes.
+struct Limitation
+{
+  Limitation(
+      const Resource& _resource,
+      const std::string& _message)
+    : resource(_resource),
+      message(_message) {}
+
+  // Resource (type and value) that triggered the limitation.
+  const Resource resource;
+  // Description of the limitation.
+  const std::string message;
+};
+
+
+class Isolator
+{
+public:
+  Isolator(process::Owned<IsolatorProcess> process);
+  ~Isolator();
+
+  // Recover containers from the run states.
+  process::Future<Nothing> recover(
+      const std::list<state::RunState>& states);
+
+  // Prepare for isolation of the executor.
+  process::Future<Nothing> prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo);
+
+  // Isolate the executor. Any steps that require execution in the
+  // containerized context (e.g. inside a network namespace) can be returned in
+  // the optional CommandInfo and they will be run by the Launcher.  This could
+  // be a simple command or a URI (including a local file) that will be fetched
+  // and executed.
+  process::Future<Option<CommandInfo> > isolate(
+      const ContainerID& containerId,
+      pid_t pid);
+
+  // Watch the containerized executor and report if any resource constraint
+  // impacts the container, e.g., the kernel killing some processes.
+  process::Future<Limitation> watch(const ContainerID& containerId);
+
+  // Update the resources allocated to the container.
+  process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources);
+
+  // Gather resource usage statistics for the container.
+  process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId) const;
+
+  // Clean up a terminated container. This is called after the executor and all
+  // processes in the container have terminated.
+  process::Future<Nothing> cleanup(const ContainerID& containerId);
+
+private:
+  Isolator(const Isolator&); // Not copyable.
+  Isolator& operator=(const Isolator&); // Not assignable.
+
+  process::Owned<IsolatorProcess> process;
+};
+
+
+class IsolatorProcess : public process::Process<IsolatorProcess>
+{
+public:
+  virtual ~IsolatorProcess() {}
+
+  virtual process::Future<Nothing> recover(
+      const std::list<state::RunState>& state) = 0;
+
+  virtual process::Future<Nothing> prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo) = 0;
+
+  virtual process::Future<Option<CommandInfo> > isolate(
+      const ContainerID& containerId,
+      pid_t pid) = 0;
+
+  virtual process::Future<Limitation> watch(
+      const ContainerID& containerId) = 0;
+
+  virtual process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources) = 0;
+
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId) = 0;
+
+  virtual process::Future<Nothing> cleanup(const ContainerID& containerId) = 0;
+};
+
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/d5266b8c/src/slave/containerizer/isolators/posix.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/posix.hpp b/src/slave/containerizer/isolators/posix.hpp
new file mode 100644
index 0000000..7fbc6dd
--- /dev/null
+++ b/src/slave/containerizer/isolators/posix.hpp
@@ -0,0 +1,252 @@
+/**
+ * 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 __POSIX_ISOLATOR_HPP__
+#define __POSIX_ISOLATOR_HPP__
+
+#include <stout/hashmap.hpp>
+#include <stout/os/pstree.hpp>
+
+#include <process/future.hpp>
+
+#include "slave/containerizer/isolator.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// A basic IsolatorProcess that keeps track of the pid but doesn't do any
+// resource isolation. Subclasses must implement usage() for their appropriate
+// resource(s).
+class PosixIsolatorProcess : public IsolatorProcess
+{
+public:
+  virtual process::Future<Nothing> recover(
+      const std::list<state::RunState>& state)
+  {
+    foreach (const state::RunState& run, state) {
+      if (!run.id.isSome()) {
+        return process::Failure("ContainerID is required to recover");
+      }
+
+      if (!run.forkedPid.isSome()) {
+        return process::Failure("Executor pid is required to recover");
+      }
+
+      // This should (almost) never occur: see comment in
+      // PosixLauncher::recover().
+      if (pids.contains(run.id.get())) {
+        return process::Failure("Container already recovered");
+      }
+
+      pids.put(run.id.get(), run.forkedPid.get());
+
+      process::Owned<process::Promise<Limitation> > promise(
+          new process::Promise<Limitation>());
+      promises.put(run.id.get(), promise);
+    }
+
+    return Nothing();
+  }
+
+  virtual process::Future<Nothing> prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo)
+  {
+    if (promises.contains(containerId)) {
+      return process::Failure("Container " + stringify(containerId) +
+                              " has already been prepared");
+    }
+
+    process::Owned<process::Promise<Limitation> > promise(
+        new process::Promise<Limitation>());
+    promises.put(containerId, promise);
+
+    return Nothing();
+  }
+
+  virtual process::Future<Option<CommandInfo> > isolate(
+      const ContainerID& containerId,
+      pid_t pid)
+  {
+    if (!promises.contains(containerId)) {
+      return process::Failure("Unknown container: " + stringify(containerId));
+    }
+
+    pids.put(containerId, pid);
+
+    return None();
+  }
+
+  virtual process::Future<Limitation> watch(
+      const ContainerID& containerId)
+  {
+    if (!promises.contains(containerId)) {
+      return process::Failure("Unknown container: " + stringify(containerId));
+    }
+
+    return promises[containerId]->future();
+  }
+
+  virtual process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources)
+  {
+    if (!promises.contains(containerId)) {
+      return process::Failure("Unknown container: " + stringify(containerId));
+    }
+
+    // No resources are actually isolated so nothing to do.
+    return Nothing();
+  }
+
+  virtual process::Future<Nothing> cleanup(const ContainerID& containerId)
+  {
+    if (!promises.contains(containerId)) {
+      return process::Failure("Unknown container: " + stringify(containerId));
+    }
+
+    // TODO(idownes): We should discard the container's promise here to signal
+    // to anyone that holds the future from watch().
+    promises.erase(containerId);
+
+    pids.erase(containerId);
+
+    return Nothing();
+  }
+
+protected:
+  hashmap<ContainerID, pid_t> pids;
+  hashmap<ContainerID,
+          process::Owned<process::Promise<Limitation> > > promises;
+};
+
+
+class PosixCpuIsolatorProcess : public PosixIsolatorProcess
+{
+public:
+  static Try<Isolator*> create(const Flags& flags)
+  {
+    process::Owned<IsolatorProcess> process(new PosixCpuIsolatorProcess());
+
+    return new Isolator(process);
+  }
+
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId)
+  {
+    if (!pids.contains(containerId)) {
+      LOG(WARNING) << "No resource usage for unknown container '"
+                   << containerId << "'";
+      return ResourceStatistics();
+    }
+
+    Try<os::ProcessTree> tree = os::pstree(pids.get(containerId).get());
+
+    if (!tree.isSome()) {
+      return ResourceStatistics();
+    }
+
+    ResourceStatistics result;
+
+    std::deque<os::ProcessTree> trees;
+    trees.push_back(tree.get());
+
+    while (!trees.empty()) {
+      os::ProcessTree root = trees.front();
+
+      // We only show utime and stime when both are available, otherwise
+      // we're exposing a partial view of the CPU times.
+      if (root.process.utime.isSome() && root.process.stime.isSome()) {
+        result.set_cpus_user_time_secs(
+            result.cpus_user_time_secs() + root.process.utime.get().secs());
+        result.set_cpus_system_time_secs(
+            result.cpus_system_time_secs() + root.process.stime.get().secs());
+      }
+
+      trees.pop_front();
+      foreach (const os::ProcessTree& child, root.children) {
+        trees.push_back(child);
+      }
+    }
+
+    return result;
+  }
+
+private:
+  PosixCpuIsolatorProcess() {}
+};
+
+
+class PosixMemIsolatorProcess : public PosixIsolatorProcess
+{
+public:
+  static Try<Isolator*> create(const Flags& flags)
+  {
+    process::Owned<IsolatorProcess> process(new PosixMemIsolatorProcess());
+
+    return new Isolator(process);
+  }
+
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId)
+  {
+    if (!pids.contains(containerId)) {
+      LOG(WARNING) << "No resource usage for unknown container '"
+                   << containerId << "'";
+      return ResourceStatistics();
+    }
+
+    Try<os::ProcessTree> tree = os::pstree(pids.get(containerId).get());
+
+    if (!tree.isSome()) {
+      return ResourceStatistics();
+    }
+
+    ResourceStatistics result;
+
+    std::deque<os::ProcessTree> trees;
+    trees.push_back(tree.get());
+
+    while (!trees.empty()) {
+      os::ProcessTree root = trees.front();
+
+      if (root.process.rss.isSome()) {
+        result.set_mem_rss_bytes(
+            result.mem_rss_bytes() + root.process.rss.get().bytes());
+      }
+
+      trees.pop_front();
+      foreach (const os::ProcessTree& child, root.children) {
+        trees.push_back(child);
+      }
+    }
+
+    return result;
+  }
+
+private:
+  PosixMemIsolatorProcess() {}
+};
+
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __POSIX_ISOLATOR_HPP__