You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2015/07/24 20:01:47 UTC

mesos git commit: Made the Isolator interface not dependent on IsolatorProcess.

Repository: mesos
Updated Branches:
  refs/heads/master 71b57cfed -> 0942b055c


Made the Isolator interface not dependent on IsolatorProcess.

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


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

Branch: refs/heads/master
Commit: 0942b055ccd9a9b6a1c188e208591e172b78fbac
Parents: 71b57cf
Author: Jie Yu <yu...@gmail.com>
Authored: Thu Jul 23 15:36:54 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Jul 24 11:01:27 2015 -0700

----------------------------------------------------------------------
 include/mesos/slave/isolator.hpp                |  70 +++--------
 src/Makefile.am                                 |   9 +-
 src/slave/containerizer/isolator.cpp            |  71 ++++++-----
 src/slave/containerizer/isolator.hpp            | 118 +++++++++++++++++++
 .../isolators/cgroups/cpushare.cpp              |  11 +-
 .../isolators/cgroups/cpushare.hpp              |  10 +-
 .../containerizer/isolators/cgroups/mem.cpp     |  11 +-
 .../containerizer/isolators/cgroups/mem.hpp     |  11 +-
 .../isolators/cgroups/perf_event.cpp            |  12 +-
 .../isolators/cgroups/perf_event.hpp            |  11 +-
 .../isolators/filesystem/posix.cpp              |   8 +-
 .../isolators/filesystem/posix.hpp              |   6 +-
 .../isolators/filesystem/shared.cpp             |  13 +-
 .../isolators/filesystem/shared.hpp             |   6 +-
 .../containerizer/isolators/namespaces/pid.cpp  |   3 +-
 .../containerizer/isolators/namespaces/pid.hpp  |  10 +-
 .../isolators/network/port_mapping.cpp          |  11 +-
 .../isolators/network/port_mapping.hpp          |   6 +-
 src/slave/containerizer/isolators/posix.hpp     |  23 ++--
 .../containerizer/isolators/posix/disk.cpp      |  13 +-
 .../containerizer/isolators/posix/disk.hpp      |   6 +-
 src/slave/containerizer/mesos/containerizer.cpp |   1 -
 src/tests/containerizer_tests.cpp               |  16 +--
 src/tests/isolator.hpp                          |   9 +-
 24 files changed, 270 insertions(+), 195 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/include/mesos/slave/isolator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/slave/isolator.hpp b/include/mesos/slave/isolator.hpp
index 8387efd..879e03e 100644
--- a/include/mesos/slave/isolator.hpp
+++ b/include/mesos/slave/isolator.hpp
@@ -42,11 +42,11 @@ namespace slave {
 // Forward declaration.
 class IsolatorProcess;
 
+
 class Isolator
 {
 public:
-  explicit Isolator(process::Owned<IsolatorProcess> process);
-  ~Isolator();
+  virtual ~Isolator() {}
 
   // Returns the namespaces required by the isolator. The namespaces
   // are created while launching the executor. Isolators may return
@@ -55,14 +55,14 @@ public:
   // TODO(karya): Since namespaces are Linux-only, create a separate
   // LinuxIsolator (and corresponding LinuxIsolatorProcess) class
   // for Linux-specific isolators.
-  process::Future<Option<int>> namespaces();
+  virtual process::Future<Option<int>> namespaces() { return None(); }
 
   // Recover containers from the run states and the orphan containers
   // (known to the launcher but not known to the slave) detected by
   // the launcher.
-  process::Future<Nothing> recover(
+  virtual process::Future<Nothing> recover(
       const std::list<ExecutorRunState>& states,
-      const hashset<ContainerID>& orphans);
+      const hashset<ContainerID>& orphans) = 0;
 
   // Prepare for isolation of the executor. Any steps that require
   // execution in the containerized context (e.g. inside a network
@@ -70,55 +70,6 @@ public:
   // will be run by the Launcher.
   // TODO(idownes): Any URIs or Environment in the CommandInfo will be
   // ignored; only the command value is used.
-  process::Future<Option<CommandInfo>> prepare(
-      const ContainerID& containerId,
-      const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const Option<std::string>& rootfs,
-      const Option<std::string>& user);
-
-  // Isolate the executor.
-  process::Future<Nothing> 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<ExecutorLimitation> 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<Option<int>> namespaces() { return None(); }
-
-  virtual process::Future<Nothing> recover(
-      const std::list<ExecutorRunState>& state,
-      const hashset<ContainerID>& orphans) = 0;
-
   virtual process::Future<Option<CommandInfo>> prepare(
       const ContainerID& containerId,
       const ExecutorInfo& executorInfo,
@@ -126,21 +77,30 @@ public:
       const Option<std::string>& rootfs,
       const Option<std::string>& user) = 0;
 
+  // Isolate the executor.
   virtual process::Future<Nothing> isolate(
       const ContainerID& containerId,
       pid_t pid) = 0;
 
+  // Watch the containerized executor and report if any resource
+  // constraint impacts the container, e.g., the kernel killing some
+  // processes.
   virtual process::Future<ExecutorLimitation> watch(
       const ContainerID& containerId) = 0;
 
+  // Update the resources allocated to the container.
   virtual process::Future<Nothing> update(
       const ContainerID& containerId,
       const Resources& resources) = 0;
 
+  // Gather resource usage statistics for the container.
   virtual process::Future<ResourceStatistics> usage(
       const ContainerID& containerId) = 0;
 
-  virtual process::Future<Nothing> cleanup(const ContainerID& containerId) = 0;
+  // Clean up a terminated container. This is called after the
+  // executor and all processes in the container have terminated.
+  virtual process::Future<Nothing> cleanup(
+      const ContainerID& containerId) = 0;
 };
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 9f2d7e3..93a6a7a 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -636,12 +636,12 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/state.hpp							\
 	slave/status_update_manager.hpp					\
 	slave/containerizer/containerizer.hpp				\
-	slave/containerizer/fetcher.hpp					\
 	slave/containerizer/external_containerizer.hpp			\
+	slave/containerizer/fetcher.hpp					\
+	slave/containerizer/isolator.hpp				\
 	slave/containerizer/launcher.hpp				\
 	slave/containerizer/linux_launcher.hpp				\
-	slave/containerizer/mesos/containerizer.hpp			\
-	slave/containerizer/mesos/launch.hpp				\
+	slave/containerizer/provisioner.hpp				\
 	slave/containerizer/isolators/posix.hpp				\
 	slave/containerizer/isolators/posix/disk.hpp			\
 	slave/containerizer/isolators/cgroups/constants.hpp		\
@@ -651,7 +651,8 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/containerizer/isolators/namespaces/pid.hpp		\
 	slave/containerizer/isolators/filesystem/posix.hpp		\
 	slave/containerizer/isolators/filesystem/shared.hpp		\
-	slave/containerizer/provisioner.hpp				\
+	slave/containerizer/mesos/containerizer.hpp			\
+	slave/containerizer/mesos/launch.hpp				\
 	slave/resource_estimators/noop.hpp				\
 	tests/cluster.hpp						\
 	tests/containerizer.hpp						\

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolator.cpp b/src/slave/containerizer/isolator.cpp
index ec14b20..9d2af8b 100644
--- a/src/slave/containerizer/isolator.cpp
+++ b/src/slave/containerizer/isolator.cpp
@@ -16,50 +16,54 @@
  * limitations under the License.
  */
 
-#include <mesos/slave/isolator.hpp>
-
 #include <process/dispatch.hpp>
 
+#include "slave/containerizer/isolator.hpp"
+
 using namespace process;
 
 using std::string;
 using std::list;
 
+using mesos::slave::ExecutorLimitation;
+using mesos::slave::ExecutorRunState;
+
 namespace mesos {
+namespace internal {
 namespace slave {
 
-Isolator::Isolator(Owned<IsolatorProcess> _process)
+MesosIsolator::MesosIsolator(Owned<MesosIsolatorProcess> _process)
   : process(_process)
 {
-  process::spawn(CHECK_NOTNULL(process.get()));
+  spawn(CHECK_NOTNULL(process.get()));
 }
 
 
-Isolator::~Isolator()
+MesosIsolator::~MesosIsolator()
 {
-  process::terminate(process.get());
-  process::wait(process.get());
+  terminate(process.get());
+  wait(process.get());
 }
 
 
-Future<Option<int>> Isolator::namespaces()
+Future<Option<int>> MesosIsolator::namespaces()
 {
-  return dispatch(process.get(), &IsolatorProcess::namespaces);
+  return dispatch(process.get(), &MesosIsolatorProcess::namespaces);
 }
 
 
-Future<Nothing> Isolator::recover(
+Future<Nothing> MesosIsolator::recover(
     const list<ExecutorRunState>& state,
     const hashset<ContainerID>& orphans)
 {
   return dispatch(process.get(),
-                  &IsolatorProcess::recover,
+                  &MesosIsolatorProcess::recover,
                   state,
                   orphans);
 }
 
 
-Future<Option<CommandInfo>> Isolator::prepare(
+Future<Option<CommandInfo>> MesosIsolator::prepare(
     const ContainerID& containerId,
     const ExecutorInfo& executorInfo,
     const string& directory,
@@ -67,7 +71,7 @@ Future<Option<CommandInfo>> Isolator::prepare(
     const Option<string>& user)
 {
   return dispatch(process.get(),
-                  &IsolatorProcess::prepare,
+                  &MesosIsolatorProcess::prepare,
                   containerId,
                   executorInfo,
                   directory,
@@ -76,43 +80,54 @@ Future<Option<CommandInfo>> Isolator::prepare(
 }
 
 
-Future<Nothing> Isolator::isolate(
+Future<Nothing> MesosIsolator::isolate(
     const ContainerID& containerId,
     pid_t pid)
 {
-  return dispatch(process.get(), &IsolatorProcess::isolate, containerId, pid);
+  return dispatch(process.get(),
+                  &MesosIsolatorProcess::isolate,
+                  containerId,
+                  pid);
 }
 
 
-Future<ExecutorLimitation> Isolator::watch(const ContainerID& containerId)
+Future<ExecutorLimitation> MesosIsolator::watch(
+    const ContainerID& containerId)
 {
-  return dispatch(process.get(), &IsolatorProcess::watch, containerId);
+  return dispatch(process.get(),
+                  &MesosIsolatorProcess::watch,
+                  containerId);
 }
 
 
-Future<Nothing> Isolator::update(
+Future<Nothing> MesosIsolator::update(
     const ContainerID& containerId,
     const Resources& resources)
 {
-  return dispatch(
-      process.get(),
-      &IsolatorProcess::update,
-      containerId,
-      resources);
+  return dispatch(process.get(),
+                  &MesosIsolatorProcess::update,
+                  containerId,
+                  resources);
 }
 
 
-Future<ResourceStatistics> Isolator::usage(
-    const ContainerID& containerId) const
+Future<ResourceStatistics> MesosIsolator::usage(
+    const ContainerID& containerId)
 {
-  return dispatch(process.get(), &IsolatorProcess::usage, containerId);
+  return dispatch(process.get(),
+                  &MesosIsolatorProcess::usage,
+                  containerId);
 }
 
 
-Future<Nothing> Isolator::cleanup(const ContainerID& containerId)
+Future<Nothing> MesosIsolator::cleanup(
+    const ContainerID& containerId)
 {
-  return dispatch(process.get(), &IsolatorProcess::cleanup, containerId);
+  return dispatch(process.get(),
+                  &MesosIsolatorProcess::cleanup,
+                  containerId);
 }
 
 } // namespace slave {
+} // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/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..aa6f1e6
--- /dev/null
+++ b/src/slave/containerizer/isolator.hpp
@@ -0,0 +1,118 @@
+/**
+ * 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 <mesos/slave/isolator.hpp>
+
+#include <process/owned.hpp>
+#include <process/process.hpp>
+
+#include <stout/none.hpp>
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// Forward declaration.
+class MesosIsolatorProcess;
+
+
+class MesosIsolator : public mesos::slave::Isolator
+{
+public:
+  explicit MesosIsolator(process::Owned<MesosIsolatorProcess> process);
+  virtual ~MesosIsolator();
+
+  virtual process::Future<Option<int>> namespaces();
+
+  virtual process::Future<Nothing> recover(
+      const std::list<mesos::slave::ExecutorRunState>& states,
+      const hashset<ContainerID>& orphans);
+
+  virtual process::Future<Option<CommandInfo>> prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo,
+      const std::string& directory,
+      const Option<std::string>& rootfs,
+      const Option<std::string>& user);
+
+  virtual process::Future<Nothing> isolate(
+      const ContainerID& containerId,
+      pid_t pid);
+
+  virtual process::Future<mesos::slave::ExecutorLimitation> 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:
+  process::Owned<MesosIsolatorProcess> process;
+};
+
+
+class MesosIsolatorProcess : public process::Process<MesosIsolatorProcess>
+{
+public:
+  virtual ~MesosIsolatorProcess() {}
+
+  virtual process::Future<Option<int>> namespaces() { return None(); }
+
+  virtual process::Future<Nothing> recover(
+      const std::list<mesos::slave::ExecutorRunState>& states,
+      const hashset<ContainerID>& orphans) = 0;
+
+  virtual process::Future<Option<CommandInfo>> prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo,
+      const std::string& directory,
+      const Option<std::string>& rootfs,
+      const Option<std::string>& user) = 0;
+
+  virtual process::Future<Nothing> isolate(
+      const ContainerID& containerId,
+      pid_t pid) = 0;
+
+  virtual process::Future<mesos::slave::ExecutorLimitation> 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/0942b055/src/slave/containerizer/isolators/cgroups/cpushare.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/cpushare.cpp b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
index b1ebdad..ccaf1e3 100644
--- a/src/slave/containerizer/isolators/cgroups/cpushare.cpp
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
@@ -18,9 +18,6 @@
 
 #include <stdint.h>
 
-#include <vector>
-
-#include <mesos/resources.hpp>
 #include <mesos/type_utils.hpp>
 #include <mesos/values.hpp>
 
@@ -32,7 +29,6 @@
 #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/os.hpp>
@@ -42,8 +38,6 @@
 
 #include "linux/cgroups.hpp"
 
-#include "slave/flags.hpp"
-
 #include "slave/containerizer/isolators/cgroups/cpushare.hpp"
 
 using namespace process;
@@ -56,7 +50,6 @@ using std::vector;
 using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
-using mesos::slave::IsolatorProcess;
 
 namespace mesos {
 namespace internal {
@@ -163,10 +156,10 @@ Try<Isolator*> CgroupsCpushareIsolatorProcess::create(const Flags& flags)
     }
   }
 
-  process::Owned<IsolatorProcess> process(
+  process::Owned<MesosIsolatorProcess> process(
       new CgroupsCpushareIsolatorProcess(flags, hierarchies, subsystems));
 
-  return new Isolator(process);
+  return new MesosIsolator(process);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/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
index 4fa9015..409a3a9 100644
--- a/src/slave/containerizer/isolators/cgroups/cpushare.hpp
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.hpp
@@ -19,14 +19,20 @@
 #ifndef __CPUSHARE_ISOLATOR_HPP__
 #define __CPUSHARE_ISOLATOR_HPP__
 
+#include <sys/types.h>
+
 #include <string>
+#include <vector>
 
-#include <mesos/slave/isolator.hpp>
+#include <process/future.hpp>
 
 #include <stout/hashmap.hpp>
+#include <stout/option.hpp>
 
 #include "slave/flags.hpp"
 
+#include "slave/containerizer/isolator.hpp"
+
 #include "slave/containerizer/isolators/cgroups/constants.hpp"
 
 namespace mesos {
@@ -37,7 +43,7 @@ namespace slave {
 // Completely Fair Scheduler (CFS).
 // - cpushare implements proportionally weighted scheduling.
 // - cfs implements hard quota based scheduling.
-class CgroupsCpushareIsolatorProcess : public mesos::slave::IsolatorProcess
+class CgroupsCpushareIsolatorProcess : public MesosIsolatorProcess
 {
 public:
   static Try<mesos::slave::Isolator*> create(const Flags& flags);

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/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
index 919e0f7..70c9868 100644
--- a/src/slave/containerizer/isolators/cgroups/mem.cpp
+++ b/src/slave/containerizer/isolators/cgroups/mem.cpp
@@ -21,7 +21,6 @@
 #include <list>
 #include <vector>
 
-#include <mesos/resources.hpp>
 #include <mesos/type_utils.hpp>
 #include <mesos/values.hpp>
 
@@ -33,18 +32,15 @@
 #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/path.hpp>
 #include <stout/stringify.hpp>
 #include <stout/try.hpp>
 
 #include "common/protobuf_utils.hpp"
 
-#include "linux/cgroups.hpp"
-
+#include "slave/containerizer/isolators/cgroups/constants.hpp"
 #include "slave/containerizer/isolators/cgroups/mem.hpp"
 
 using namespace process;
@@ -61,7 +57,6 @@ using std::vector;
 using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
-using mesos::slave::IsolatorProcess;
 
 namespace mesos {
 namespace internal {
@@ -153,10 +148,10 @@ Try<Isolator*> CgroupsMemIsolatorProcess::create(const Flags& flags)
     limitSwap = true;
   }
 
-  process::Owned<IsolatorProcess> process(
+  process::Owned<MesosIsolatorProcess> process(
       new CgroupsMemIsolatorProcess(flags, hierarchy.get(), limitSwap));
 
-  return new Isolator(process);
+  return new MesosIsolator(process);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/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
index c198c83..d43ba6d 100644
--- a/src/slave/containerizer/isolators/cgroups/mem.hpp
+++ b/src/slave/containerizer/isolators/cgroups/mem.hpp
@@ -19,25 +19,26 @@
 #ifndef __MEM_ISOLATOR_HPP__
 #define __MEM_ISOLATOR_HPP__
 
-#include <list>
-
-#include <mesos/slave/isolator.hpp>
+#include <sys/types.h>
 
+#include <process/future.hpp>
 #include <process/owned.hpp>
 
 #include <stout/hashmap.hpp>
+#include <stout/nothing.hpp>
+#include <stout/option.hpp>
 
 #include "linux/cgroups.hpp"
 
 #include "slave/flags.hpp"
 
-#include "slave/containerizer/isolators/cgroups/constants.hpp"
+#include "slave/containerizer/isolator.hpp"
 
 namespace mesos {
 namespace internal {
 namespace slave {
 
-class CgroupsMemIsolatorProcess : public mesos::slave::IsolatorProcess
+class CgroupsMemIsolatorProcess : public MesosIsolatorProcess
 {
 public:
   static Try<mesos::slave::Isolator*> create(const Flags& flags);

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolators/cgroups/perf_event.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/perf_event.cpp b/src/slave/containerizer/isolators/cgroups/perf_event.cpp
index 367cb43..a62a4e6 100644
--- a/src/slave/containerizer/isolators/cgroups/perf_event.cpp
+++ b/src/slave/containerizer/isolators/cgroups/perf_event.cpp
@@ -19,14 +19,10 @@
 #include <stdint.h>
 
 #include <vector>
-#include <set>
 
 #include <google/protobuf/descriptor.h>
 #include <google/protobuf/message.h>
 
-#include <mesos/resources.hpp>
-#include <mesos/values.hpp>
-
 #include <process/collect.hpp>
 #include <process/defer.hpp>
 #include <process/delay.hpp>
@@ -38,16 +34,15 @@
 #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/os.hpp>
 #include <stout/path.hpp>
 #include <stout/stringify.hpp>
 #include <stout/try.hpp>
 
 #include "linux/cgroups.hpp"
+#include "linux/perf.hpp"
 
 #include "slave/containerizer/isolators/cgroups/perf_event.hpp"
 
@@ -61,7 +56,6 @@ using std::vector;
 using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
-using mesos::slave::IsolatorProcess;
 
 namespace mesos {
 namespace internal {
@@ -111,10 +105,10 @@ Try<Isolator*> CgroupsPerfEventIsolatorProcess::create(const Flags& flags)
             << " every " << flags.perf_interval
             << " for events: " << stringify(events);
 
-  process::Owned<IsolatorProcess> process(
+  process::Owned<MesosIsolatorProcess> process(
       new CgroupsPerfEventIsolatorProcess(flags, hierarchy.get()));
 
-  return new Isolator(process);
+  return new MesosIsolator(process);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolators/cgroups/perf_event.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/perf_event.hpp b/src/slave/containerizer/isolators/cgroups/perf_event.hpp
index 243cf5a..3796b30 100644
--- a/src/slave/containerizer/isolators/cgroups/perf_event.hpp
+++ b/src/slave/containerizer/isolators/cgroups/perf_event.hpp
@@ -21,21 +21,20 @@
 
 #include <set>
 
-#include <mesos/slave/isolator.hpp>
-
 #include <process/time.hpp>
 
 #include <stout/hashmap.hpp>
-
-#include "linux/perf.hpp"
+#include <stout/nothing.hpp>
 
 #include "slave/flags.hpp"
 
+#include "slave/containerizer/isolator.hpp"
+
 namespace mesos {
 namespace internal {
 namespace slave {
 
-class CgroupsPerfEventIsolatorProcess : public mesos::slave::IsolatorProcess
+class CgroupsPerfEventIsolatorProcess : public MesosIsolatorProcess
 {
 public:
   static Try<mesos::slave::Isolator*> create(const Flags& flags);
@@ -110,9 +109,11 @@ private:
 
   // The path to the cgroups subsystem hierarchy root.
   const std::string hierarchy;
+
   // Set of events to sample.
   std::set<std::string> events;
 
+  // TODO(jieyu): Use Owned<Info>.
   hashmap<ContainerID, Info*> infos;
 };
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolators/filesystem/posix.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/posix.cpp b/src/slave/containerizer/isolators/filesystem/posix.cpp
index 2aa8406..5e5a896 100644
--- a/src/slave/containerizer/isolators/filesystem/posix.cpp
+++ b/src/slave/containerizer/isolators/filesystem/posix.cpp
@@ -23,9 +23,6 @@
 #include <stout/os.hpp>
 #include <stout/path.hpp>
 
-#include <mesos/mesos.hpp>
-#include <mesos/resources.hpp>
-
 #include "slave/paths.hpp"
 
 #include "slave/containerizer/isolators/filesystem/posix.hpp"
@@ -38,7 +35,6 @@ using std::string;
 using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
-using mesos::slave::IsolatorProcess;
 
 namespace mesos {
 namespace internal {
@@ -54,10 +50,10 @@ PosixFilesystemIsolatorProcess::~PosixFilesystemIsolatorProcess() {}
 
 Try<Isolator*> PosixFilesystemIsolatorProcess::create(const Flags& flags)
 {
-  process::Owned<IsolatorProcess> process(
+  process::Owned<MesosIsolatorProcess> process(
       new PosixFilesystemIsolatorProcess(flags));
 
-  return new Isolator(process);
+  return new MesosIsolator(process);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolators/filesystem/posix.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/posix.hpp b/src/slave/containerizer/isolators/filesystem/posix.hpp
index d44023e..2afc572 100644
--- a/src/slave/containerizer/isolators/filesystem/posix.hpp
+++ b/src/slave/containerizer/isolators/filesystem/posix.hpp
@@ -19,15 +19,17 @@
 #ifndef __POSIX_FILESYSTEM_ISOLATOR_HPP__
 #define __POSIX_FILESYSTEM_ISOLATOR_HPP__
 
-#include <mesos/slave/isolator.hpp>
+#include <mesos/resources.hpp>
 
 #include "slave/flags.hpp"
 
+#include "slave/containerizer/isolator.hpp"
+
 namespace mesos {
 namespace internal {
 namespace slave {
 
-class PosixFilesystemIsolatorProcess : public mesos::slave::IsolatorProcess
+class PosixFilesystemIsolatorProcess : public MesosIsolatorProcess
 {
 public:
   static Try<mesos::slave::Isolator*> create(const Flags& flags);

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolators/filesystem/shared.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/shared.cpp b/src/slave/containerizer/isolators/filesystem/shared.cpp
index f90045e..4694025 100644
--- a/src/slave/containerizer/isolators/filesystem/shared.cpp
+++ b/src/slave/containerizer/isolators/filesystem/shared.cpp
@@ -28,14 +28,13 @@ using std::list;
 using std::set;
 using std::string;
 
-namespace mesos {
-namespace internal {
-namespace slave {
-
 using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
-using mesos::slave::IsolatorProcess;
+
+namespace mesos {
+namespace internal {
+namespace slave {
 
 SharedFilesystemIsolatorProcess::SharedFilesystemIsolatorProcess(
     const Flags& _flags)
@@ -57,10 +56,10 @@ Try<Isolator*> SharedFilesystemIsolatorProcess::create(const Flags& flags)
     return Error("SharedFilesystemIsolator requires root privileges");
   }
 
-  process::Owned<IsolatorProcess> process(
+  process::Owned<MesosIsolatorProcess> process(
       new SharedFilesystemIsolatorProcess(flags));
 
-  return new Isolator(process);
+  return new MesosIsolator(process);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolators/filesystem/shared.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/shared.hpp b/src/slave/containerizer/isolators/filesystem/shared.hpp
index 4a5dcc3..f6ba43a 100644
--- a/src/slave/containerizer/isolators/filesystem/shared.hpp
+++ b/src/slave/containerizer/isolators/filesystem/shared.hpp
@@ -19,10 +19,10 @@
 #ifndef __SHARED_FILESYSTEM_ISOLATOR_HPP__
 #define __SHARED_FILESYSTEM_ISOLATOR_HPP__
 
-#include <mesos/slave/isolator.hpp>
-
 #include "slave/flags.hpp"
 
+#include "slave/containerizer/isolator.hpp"
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -32,7 +32,7 @@ namespace slave {
 // into each container's mount namespace. In particular, this can be
 // used to give each container a "private" system directory, such as
 // /tmp and /var/tmp.
-class SharedFilesystemIsolatorProcess : public mesos::slave::IsolatorProcess
+class SharedFilesystemIsolatorProcess : public MesosIsolatorProcess
 {
 public:
   static Try<mesos::slave::Isolator*> create(const Flags& flags);

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolators/namespaces/pid.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/namespaces/pid.cpp b/src/slave/containerizer/isolators/namespaces/pid.cpp
index 4241fa7..29be94b 100644
--- a/src/slave/containerizer/isolators/namespaces/pid.cpp
+++ b/src/slave/containerizer/isolators/namespaces/pid.cpp
@@ -42,7 +42,6 @@ using std::string;
 using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
-using mesos::slave::IsolatorProcess;
 
 namespace mesos {
 namespace internal {
@@ -103,7 +102,7 @@ Try<Isolator*> NamespacesPidIsolatorProcess::create(const Flags& flags)
         string(PID_NS_BIND_MOUNT_MASK_DIR) + ": " + mkdir.error());
   }
 
-  return new Isolator(Owned<IsolatorProcess>(
+  return new MesosIsolator(Owned<MesosIsolatorProcess>(
       new NamespacesPidIsolatorProcess()));
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolators/namespaces/pid.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/namespaces/pid.hpp b/src/slave/containerizer/isolators/namespaces/pid.hpp
index 702f331..61db386 100644
--- a/src/slave/containerizer/isolators/namespaces/pid.hpp
+++ b/src/slave/containerizer/isolators/namespaces/pid.hpp
@@ -19,16 +19,16 @@
 #ifndef __NAMESPACES_PID_ISOLATOR_HPP__
 #define __NAMESPACES_PID_ISOLATOR_HPP__
 
-#include <mesos/slave/isolator.hpp>
-
-#include "slave/flags.hpp"
-
 #include <sys/types.h>
 
 #include <string>
 
 #include <stout/result.hpp>
 
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/isolator.hpp"
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -37,7 +37,7 @@ namespace slave {
 // (see the LinuxLauncher for that) but it is used to keep track of a
 // container's pid namespace through a bind mount and exposed by
 // getNamespace().
-class NamespacesPidIsolatorProcess : public mesos::slave::IsolatorProcess
+class NamespacesPidIsolatorProcess : public MesosIsolatorProcess
 {
 public:
   static Try<mesos::slave::Isolator*> create(const Flags& flags);

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolators/network/port_mapping.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.cpp b/src/slave/containerizer/isolators/network/port_mapping.cpp
index 39d1813..3f35782 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.cpp
@@ -106,6 +106,10 @@ using std::vector;
 
 using filter::ip::PortRange;
 
+using mesos::slave::ExecutorLimitation;
+using mesos::slave::ExecutorRunState;
+using mesos::slave::Isolator;
+
 
 // An old glibc might not have this symbol.
 #ifndef MNT_DETACH
@@ -117,11 +121,6 @@ namespace mesos {
 namespace internal {
 namespace slave {
 
-using mesos::slave::ExecutorLimitation;
-using mesos::slave::ExecutorRunState;
-using mesos::slave::Isolator;
-using mesos::slave::IsolatorProcess;
-
 // The minimum number of ephemeral ports a container should have.
 static const uint16_t MIN_EPHEMERAL_PORTS_SIZE = 16;
 
@@ -1611,7 +1610,7 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
         PORT_MAPPING_BIND_MOUNT_SYMLINK_ROOT() + ": " + mkdir.error());
   }
 
-  return new Isolator(Owned<IsolatorProcess>(
+  return new MesosIsolator(Owned<MesosIsolatorProcess>(
       new PortMappingIsolatorProcess(
           flags,
           eth0.get(),

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolators/network/port_mapping.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.hpp b/src/slave/containerizer/isolators/network/port_mapping.hpp
index 6ffd729..fb9ce9c 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.hpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.hpp
@@ -27,8 +27,6 @@
 #include <string>
 #include <vector>
 
-#include <mesos/slave/isolator.hpp>
-
 #include <process/owned.hpp>
 #include <process/subprocess.hpp>
 
@@ -49,6 +47,8 @@
 
 #include "slave/flags.hpp"
 
+#include "slave/containerizer/isolator.hpp"
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -145,7 +145,7 @@ std::vector<routing::filter::ip::PortRange> getPortRanges(
 // isolator is useful when the operator wants to reuse the host IP for
 // all containers running on the host (e.g., there are insufficient
 // IPs).
-class PortMappingIsolatorProcess : public mesos::slave::IsolatorProcess
+class PortMappingIsolatorProcess : public MesosIsolatorProcess
 {
 public:
   static Try<mesos::slave::Isolator*> create(const Flags& flags);

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolators/posix.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/posix.hpp b/src/slave/containerizer/isolators/posix.hpp
index 6ddab7d..58d9911 100644
--- a/src/slave/containerizer/isolators/posix.hpp
+++ b/src/slave/containerizer/isolators/posix.hpp
@@ -19,26 +19,26 @@
 #ifndef __POSIX_ISOLATOR_HPP__
 #define __POSIX_ISOLATOR_HPP__
 
-#include <mesos/slave/isolator.hpp>
+#include <process/future.hpp>
 
 #include <stout/hashmap.hpp>
 
 #include <stout/os/pstree.hpp>
 
-#include <process/future.hpp>
-
 #include "slave/flags.hpp"
 
+#include "slave/containerizer/isolator.hpp"
+
 #include "usage/usage.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 mesos::slave::IsolatorProcess
+// A basic MesosIsolatorProcess 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 MesosIsolatorProcess
 {
 public:
   virtual process::Future<Nothing> recover(
@@ -144,10 +144,10 @@ class PosixCpuIsolatorProcess : public PosixIsolatorProcess
 public:
   static Try<mesos::slave::Isolator*> create(const Flags& flags)
   {
-    process::Owned<mesos::slave::IsolatorProcess> process(
+    process::Owned<MesosIsolatorProcess> process(
         new PosixCpuIsolatorProcess());
 
-    return new mesos::slave::Isolator(process);
+    return new MesosIsolator(process);
   }
 
   virtual process::Future<ResourceStatistics> usage(
@@ -178,10 +178,10 @@ class PosixMemIsolatorProcess : public PosixIsolatorProcess
 public:
   static Try<mesos::slave::Isolator*> create(const Flags& flags)
   {
-    process::Owned<mesos::slave::IsolatorProcess> process(
+    process::Owned<MesosIsolatorProcess> process(
         new PosixMemIsolatorProcess());
 
-    return new mesos::slave::Isolator(process);
+    return new MesosIsolator(process);
   }
 
   virtual process::Future<ResourceStatistics> usage(
@@ -206,7 +206,6 @@ private:
   PosixMemIsolatorProcess() {}
 };
 
-
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolators/posix/disk.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/posix/disk.cpp b/src/slave/containerizer/isolators/posix/disk.cpp
index 238f179..845496c 100644
--- a/src/slave/containerizer/isolators/posix/disk.cpp
+++ b/src/slave/containerizer/isolators/posix/disk.cpp
@@ -55,21 +55,20 @@ using std::list;
 using std::string;
 using std::vector;
 
-namespace mesos {
-namespace internal {
-namespace slave {
-
 using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
-using mesos::slave::IsolatorProcess;
+
+namespace mesos {
+namespace internal {
+namespace slave {
 
 Try<Isolator*> PosixDiskIsolatorProcess::create(const Flags& flags)
 {
   // TODO(jieyu): Check the availability of command 'du'.
 
-  return new Isolator(
-      process::Owned<IsolatorProcess>(new PosixDiskIsolatorProcess(flags)));
+  return new MesosIsolator(process::Owned<MesosIsolatorProcess>(
+        new PosixDiskIsolatorProcess(flags)));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/isolators/posix/disk.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/posix/disk.hpp b/src/slave/containerizer/isolators/posix/disk.hpp
index fdf24a1..86aea37 100644
--- a/src/slave/containerizer/isolators/posix/disk.hpp
+++ b/src/slave/containerizer/isolators/posix/disk.hpp
@@ -21,8 +21,6 @@
 
 #include <string>
 
-#include <mesos/slave/isolator.hpp>
-
 #include <process/owned.hpp>
 
 #include <stout/bytes.hpp>
@@ -32,6 +30,8 @@
 #include "slave/flags.hpp"
 #include "slave/state.hpp"
 
+#include "slave/containerizer/isolator.hpp"
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -71,7 +71,7 @@ private:
 // TODO(jieyu): Consider handling each container independently, or
 // triggering an initial collection when the container starts, to
 // ensure that we have usage statistics without a large delay.
-class PosixDiskIsolatorProcess : public mesos::slave::IsolatorProcess
+class PosixDiskIsolatorProcess : public MesosIsolatorProcess
 {
 public:
   static Try<mesos::slave::Isolator*> create(const Flags& flags);

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index c21e925..11eedf3 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -87,7 +87,6 @@ using mesos::modules::ModuleManager;
 using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
-using mesos::slave::IsolatorProcess;
 
 using state::SlaveState;
 using state::FrameworkState;

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/tests/containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer_tests.cpp b/src/tests/containerizer_tests.cpp
index 0b13381..9508613 100644
--- a/src/tests/containerizer_tests.cpp
+++ b/src/tests/containerizer_tests.cpp
@@ -94,7 +94,7 @@ public:
     vector<Owned<Isolator>> isolators;
 
     foreach (const Option<CommandInfo>& prepare, prepares) {
-      Try<Isolator*> isolator = tests::TestIsolatorProcess::create(prepare);
+      Try<Isolator*> isolator = TestIsolatorProcess::create(prepare);
       if (isolator.isError()) {
         return Error(isolator.error());
       }
@@ -288,7 +288,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
 }
 
 
-class MesosContainerizerExecuteTest : public tests::TemporaryDirectoryTest {};
+class MesosContainerizerExecuteTest : public TemporaryDirectoryTest {};
 
 
 TEST_F(MesosContainerizerExecuteTest, IoRedirection)
@@ -395,10 +395,10 @@ public:
 };
 
 
-class MockIsolatorProcess : public IsolatorProcess
+class MockIsolator : public mesos::slave::Isolator
 {
 public:
-  MockIsolatorProcess()
+  MockIsolator()
   {
     EXPECT_CALL(*this, watch(_))
       .WillRepeatedly(Return(watchPromise.future()));
@@ -410,7 +410,7 @@ public:
       .WillRepeatedly(Return(Nothing()));
 
     EXPECT_CALL(*this, prepare(_, _, _, _, _))
-      .WillRepeatedly(Invoke(this, &MockIsolatorProcess::_prepare));
+      .WillRepeatedly(Invoke(this, &MockIsolator::_prepare));
   }
 
   MOCK_METHOD2(
@@ -528,13 +528,13 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
   Try<Launcher*> launcher = PosixLauncher::create(flags);
   ASSERT_SOME(launcher);
 
-  MockIsolatorProcess* isolatorProcess = new MockIsolatorProcess();
+  MockIsolator* isolator = new MockIsolator();
 
   Future<Nothing> prepare;
   Promise<Option<CommandInfo>> promise;
 
   // Simulate a long prepare from the isolator.
-  EXPECT_CALL(*isolatorProcess, prepare(_, _, _, _, _))
+  EXPECT_CALL(*isolator, prepare(_, _, _, _, _))
     .WillOnce(DoAll(FutureSatisfy(&prepare),
                     Return(promise.future())));
 
@@ -545,7 +545,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
       true,
       &fetcher,
       Owned<Launcher>(launcher.get()),
-      {Owned<Isolator>(new Isolator(Owned<IsolatorProcess>(isolatorProcess)))},
+      {Owned<Isolator>(isolator)},
       hashmap<ContainerInfo::Image::Type, Owned<Provisioner>>());
 
   MesosContainerizer containerizer((Owned<MesosContainerizerProcess>(process)));

http://git-wip-us.apache.org/repos/asf/mesos/blob/0942b055/src/tests/isolator.hpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator.hpp b/src/tests/isolator.hpp
index 671b021..8aaf88c 100644
--- a/src/tests/isolator.hpp
+++ b/src/tests/isolator.hpp
@@ -21,22 +21,22 @@
 
 #include <gmock/gmock.h>
 
-#include <mesos/slave/isolator.hpp>
+#include "slave/containerizer/isolator.hpp"
 
 namespace mesos {
 namespace internal {
 namespace tests {
 
-class TestIsolatorProcess : public mesos::slave::IsolatorProcess
+class TestIsolatorProcess : public slave::MesosIsolatorProcess
 {
 public:
   static Try<mesos::slave::Isolator*> create(
       const Option<CommandInfo>& commandInfo)
   {
-    process::Owned<mesos::slave::IsolatorProcess> process(
+    process::Owned<MesosIsolatorProcess> process(
         new TestIsolatorProcess(commandInfo));
 
-    return new mesos::slave::Isolator(process);
+    return new slave::MesosIsolator(process);
   }
 
   MOCK_METHOD2(
@@ -89,7 +89,6 @@ private:
       .WillRepeatedly(testing::Return(Nothing()));
   }
 
-
   const Option<CommandInfo> commandInfo;
 
   process::Promise<mesos::slave::ExecutorLimitation> promise;