You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by nn...@apache.org on 2015/02/10 19:22:39 UTC

[5/9] mesos git commit: Out of tree build 5: Exposed slave/containerizer/isolator.hpp as mesos/slave/isolator.hpp

Out of tree build 5: Exposed slave/containerizer/isolator.hpp as mesos/slave/isolator.hpp

Expose "slave/state.hpp" and "slave/containerizer/isolator.hpp" to
allow modules to include them directly from the Mesos install
location.

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


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

Branch: refs/heads/master
Commit: 0082eb6d1a4cb87b2fb3672524a02d04e02854f9
Parents: 79a18e6
Author: Kapil Arya <ka...@mesosphere.io>
Authored: Tue Feb 10 09:54:37 2015 -0800
Committer: Niklas Q. Nielsen <ni...@mesosphere.io>
Committed: Tue Feb 10 09:54:37 2015 -0800

----------------------------------------------------------------------
 include/mesos/slave/isolator.hpp                | 163 +++++++++++++++++++
 src/Makefile.am                                 |   6 +-
 src/examples/test_isolator_module.cpp           |   3 +-
 src/module/isolator.hpp                         |   2 +-
 src/slave/containerizer/containerizer.cpp       |   1 -
 .../containerizer/external_containerizer.hpp    |   1 -
 src/slave/containerizer/isolator.cpp            |   4 +-
 src/slave/containerizer/isolator.hpp            | 163 -------------------
 .../isolators/cgroups/cpushare.hpp              |   4 +-
 .../containerizer/isolators/cgroups/mem.hpp     |   4 +-
 .../isolators/cgroups/perf_event.hpp            |   4 +-
 .../isolators/filesystem/shared.hpp             |   4 +-
 .../containerizer/isolators/namespaces/pid.hpp  |   4 +-
 .../isolators/network/port_mapping.hpp          |   4 +-
 src/slave/containerizer/isolators/posix.hpp     |   4 +-
 .../containerizer/isolators/posix/disk.hpp      |   4 +-
 src/slave/containerizer/launcher.hpp            |   4 +-
 src/slave/containerizer/mesos/containerizer.cpp |   3 +-
 src/slave/containerizer/mesos/containerizer.hpp |   3 +-
 src/tests/containerizer_tests.cpp               |   7 +-
 src/tests/isolator.hpp                          |   2 +-
 src/tests/isolator_tests.cpp                    |   3 +-
 src/tests/module_tests.cpp                      |   5 +-
 src/tests/port_mapping_tests.cpp                |   1 +
 24 files changed, 206 insertions(+), 197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/include/mesos/slave/isolator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/slave/isolator.hpp b/include/mesos/slave/isolator.hpp
new file mode 100644
index 0000000..907be5b
--- /dev/null
+++ b/include/mesos/slave/isolator.hpp
@@ -0,0 +1,163 @@
+/**
+ * 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_SLAVE_ISOLATOR_HPP__
+#define __MESOS_SLAVE_ISOLATOR_HPP__
+
+#include <list>
+#include <string>
+
+#include <mesos/resources.hpp>
+
+#include <process/dispatch.hpp>
+#include <process/future.hpp>
+#include <process/owned.hpp>
+#include <process/process.hpp>
+
+#include <stout/try.hpp>
+
+namespace mesos {
+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 Resources& _resources,
+      const std::string& _message)
+    : resources(_resources),
+      message(_message) {}
+
+  // Resources that triggered the limitation.
+  // NOTE: 'Resources' is used here because the resource may span
+  // multiple roles (e.g. `"mem(*):1;mem(role):2"`).
+  Resources resources;
+
+  // Description of the limitation.
+  std::string message;
+};
+
+
+// This struct is derived from slave::state::RunState. It contains
+// only those fields that are needed by Isolators for recovering the
+// containers. The reason for not using RunState instead is to avoid
+// any dependency on RunState and in turn on internal protobufs.
+struct ExecutorRunState
+{
+  ExecutorRunState(ContainerID id_, pid_t pid_, std::string directory_)
+    : id(id_), pid(pid_), directory(directory_) {}
+
+  ContainerID id;        // Container id of the last executor run.
+  pid_t pid;             // Executor pid.
+  std::string directory; // Executor work directory.
+};
+
+
+class Isolator
+{
+public:
+  explicit Isolator(process::Owned<IsolatorProcess> process);
+  ~Isolator();
+
+  // Recover containers from the run states.
+  process::Future<Nothing> recover(
+      const std::list<ExecutorRunState>& states);
+
+  // Prepare for isolation of 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.
+  // 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>& 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<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<ExecutorRunState>& state) = 0;
+
+  virtual process::Future<Option<CommandInfo> > prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo,
+      const std::string& directory,
+      const Option<std::string>& user) = 0;
+
+  virtual process::Future<Nothing> 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 mesos {
+
+#endif // __MESOS_SLAVE_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 7e26f23..00fb2a6 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -389,6 +389,11 @@ scheduler_HEADERS =							\
 
 nodist_scheduler_HEADERS = ../include/mesos/scheduler/scheduler.pb.h
 
+slavedir = $(pkgincludedir)/slave
+
+slave_HEADERS =								\
+  $(top_srcdir)/include/mesos/slave/isolator.hpp
+
 if OS_LINUX
   libmesos_no_3rdparty_la_SOURCES += linux/cgroups.cpp
   libmesos_no_3rdparty_la_SOURCES += linux/fs.cpp
@@ -506,7 +511,6 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/containerizer/containerizer.hpp				\
 	slave/containerizer/fetcher.hpp					\
 	slave/containerizer/external_containerizer.hpp			\
-	slave/containerizer/isolator.hpp				\
 	slave/containerizer/launcher.hpp				\
 	slave/containerizer/linux_launcher.hpp				\
 	slave/containerizer/mesos/containerizer.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/examples/test_isolator_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_isolator_module.cpp b/src/examples/test_isolator_module.cpp
index 3b92b04..dc804fe 100644
--- a/src/examples/test_isolator_module.cpp
+++ b/src/examples/test_isolator_module.cpp
@@ -19,11 +19,12 @@
 #include <mesos/mesos.hpp>
 #include <mesos/module.hpp>
 
+#include <mesos/slave/isolator.hpp>
+
 #include <stout/try.hpp>
 
 #include "module/isolator.hpp"
 
-#include "slave/containerizer/isolator.hpp"
 #include "slave/containerizer/isolators/posix.hpp"
 #include "slave/flags.hpp"
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/module/isolator.hpp
----------------------------------------------------------------------
diff --git a/src/module/isolator.hpp b/src/module/isolator.hpp
index b5c86e3..4457186 100644
--- a/src/module/isolator.hpp
+++ b/src/module/isolator.hpp
@@ -22,7 +22,7 @@
 #include <mesos/mesos.hpp>
 #include <mesos/module.hpp>
 
-#include "slave/containerizer/isolator.hpp"
+#include <mesos/slave/isolator.hpp>
 
 namespace mesos {
 namespace modules {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/slave/containerizer/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.cpp b/src/slave/containerizer/containerizer.cpp
index cf2ece8..4e0dc3b 100644
--- a/src/slave/containerizer/containerizer.cpp
+++ b/src/slave/containerizer/containerizer.cpp
@@ -38,7 +38,6 @@
 #include "slave/containerizer/composing.hpp"
 #include "slave/containerizer/containerizer.hpp"
 #include "slave/containerizer/docker.hpp"
-#include "slave/containerizer/isolator.hpp"
 #include "slave/containerizer/launcher.hpp"
 #include "slave/containerizer/external_containerizer.hpp"
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/slave/containerizer/external_containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/external_containerizer.hpp b/src/slave/containerizer/external_containerizer.hpp
index d7ae6d6..5035454 100644
--- a/src/slave/containerizer/external_containerizer.hpp
+++ b/src/slave/containerizer/external_containerizer.hpp
@@ -34,7 +34,6 @@
 #include "slave/state.hpp"
 
 #include "slave/containerizer/containerizer.hpp"
-#include "slave/containerizer/isolator.hpp"
 #include "slave/containerizer/launcher.hpp"
 
 namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/slave/containerizer/isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolator.cpp b/src/slave/containerizer/isolator.cpp
index 9f5ba99..ee294a2 100644
--- a/src/slave/containerizer/isolator.cpp
+++ b/src/slave/containerizer/isolator.cpp
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-#include <process/dispatch.hpp>
+#include <mesos/slave/isolator.hpp>
 
-#include "slave/containerizer/isolator.hpp"
+#include <process/dispatch.hpp>
 
 using namespace process;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/slave/containerizer/isolator.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolator.hpp b/src/slave/containerizer/isolator.hpp
deleted file mode 100644
index 0823f4a..0000000
--- a/src/slave/containerizer/isolator.hpp
+++ /dev/null
@@ -1,163 +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 <list>
-#include <string>
-
-#include <mesos/resources.hpp>
-
-#include <process/dispatch.hpp>
-#include <process/future.hpp>
-#include <process/owned.hpp>
-#include <process/process.hpp>
-
-#include <stout/try.hpp>
-
-namespace mesos {
-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 Resources& _resources,
-      const std::string& _message)
-    : resources(_resources),
-      message(_message) {}
-
-  // Resources that triggered the limitation.
-  // NOTE: 'Resources' is used here because the resource may span
-  // multiple roles (e.g. `"mem(*):1;mem(role):2"`).
-  Resources resources;
-
-  // Description of the limitation.
-  std::string message;
-};
-
-
-// This struct is derived from slave::state::RunState. It contains
-// only those fields that are needed by Isolators for recovering the
-// containers. The reason for not using RunState instead is to avoid
-// any dependency on RunState and in turn on internal protobufs.
-struct ExecutorRunState
-{
-  ExecutorRunState(ContainerID id_, pid_t pid_, std::string directory_)
-    : id(id_), pid(pid_), directory(directory_) {}
-
-  ContainerID id;        // Container id of the last executor run.
-  pid_t pid;             // Executor pid.
-  std::string directory; // Executor work directory.
-};
-
-
-class Isolator
-{
-public:
-  explicit Isolator(process::Owned<IsolatorProcess> process);
-  ~Isolator();
-
-  // Recover containers from the run states.
-  process::Future<Nothing> recover(
-      const std::list<ExecutorRunState>& states);
-
-  // Prepare for isolation of 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.
-  // 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>& 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<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<ExecutorRunState>& state) = 0;
-
-  virtual process::Future<Option<CommandInfo> > prepare(
-      const ContainerID& containerId,
-      const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const Option<std::string>& user) = 0;
-
-  virtual process::Future<Nothing> 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 mesos {
-
-#endif // __ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/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 6c6c57d..f4ae149 100644
--- a/src/slave/containerizer/isolators/cgroups/cpushare.hpp
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.hpp
@@ -21,12 +21,12 @@
 
 #include <string>
 
+#include <mesos/slave/isolator.hpp>
+
 #include <stout/hashmap.hpp>
 
 #include "slave/flags.hpp"
 
-#include "slave/containerizer/isolator.hpp"
-
 #include "slave/containerizer/isolators/cgroups/constants.hpp"
 
 namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/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 c3793c4..92f197b 100644
--- a/src/slave/containerizer/isolators/cgroups/mem.hpp
+++ b/src/slave/containerizer/isolators/cgroups/mem.hpp
@@ -19,12 +19,12 @@
 #ifndef __MEM_ISOLATOR_HPP__
 #define __MEM_ISOLATOR_HPP__
 
+#include <mesos/slave/isolator.hpp>
+
 #include <stout/hashmap.hpp>
 
 #include "slave/flags.hpp"
 
-#include "slave/containerizer/isolator.hpp"
-
 #include "slave/containerizer/isolators/cgroups/constants.hpp"
 
 namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/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 a546697..4ad6cb9 100644
--- a/src/slave/containerizer/isolators/cgroups/perf_event.hpp
+++ b/src/slave/containerizer/isolators/cgroups/perf_event.hpp
@@ -21,6 +21,8 @@
 
 #include <set>
 
+#include <mesos/slave/isolator.hpp>
+
 #include <process/time.hpp>
 
 #include <stout/hashmap.hpp>
@@ -29,8 +31,6 @@
 
 #include "slave/flags.hpp"
 
-#include "slave/containerizer/isolator.hpp"
-
 namespace mesos {
 namespace slave {
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/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 27c7a65..779caca 100644
--- a/src/slave/containerizer/isolators/filesystem/shared.hpp
+++ b/src/slave/containerizer/isolators/filesystem/shared.hpp
@@ -19,9 +19,9 @@
 #ifndef __SHARED_FILESYSTEM_ISOLATOR_HPP__
 #define __SHARED_FILESYSTEM_ISOLATOR_HPP__
 
-#include "slave/flags.hpp"
+#include <mesos/slave/isolator.hpp>
 
-#include "slave/containerizer/isolator.hpp"
+#include "slave/flags.hpp"
 
 namespace mesos {
 namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/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 21674ce..88227a3 100644
--- a/src/slave/containerizer/isolators/namespaces/pid.hpp
+++ b/src/slave/containerizer/isolators/namespaces/pid.hpp
@@ -19,9 +19,9 @@
 #ifndef __NAMESPACES_PID_ISOLATOR_HPP__
 #define __NAMESPACES_PID_ISOLATOR_HPP__
 
-#include "slave/flags.hpp"
+#include <mesos/slave/isolator.hpp>
 
-#include "slave/containerizer/isolator.hpp"
+#include "slave/flags.hpp"
 
 #include <sys/types.h>
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/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 a74bd65..d0d5619 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.hpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.hpp
@@ -26,6 +26,8 @@
 #include <string>
 #include <vector>
 
+#include <mesos/slave/isolator.hpp>
+
 #include <process/owned.hpp>
 #include <process/subprocess.hpp>
 
@@ -45,8 +47,6 @@
 
 #include "slave/flags.hpp"
 
-#include "slave/containerizer/isolator.hpp"
-
 namespace mesos {
 namespace slave {
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/slave/containerizer/isolators/posix.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/posix.hpp b/src/slave/containerizer/isolators/posix.hpp
index 0d59384..f871606 100644
--- a/src/slave/containerizer/isolators/posix.hpp
+++ b/src/slave/containerizer/isolators/posix.hpp
@@ -19,6 +19,8 @@
 #ifndef __POSIX_ISOLATOR_HPP__
 #define __POSIX_ISOLATOR_HPP__
 
+#include <mesos/slave/isolator.hpp>
+
 #include <stout/hashmap.hpp>
 
 #include <stout/os/pstree.hpp>
@@ -27,8 +29,6 @@
 
 #include "slave/flags.hpp"
 
-#include "slave/containerizer/isolator.hpp"
-
 #include "usage/usage.hpp"
 
 namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/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 4760d06..f5b294e 100644
--- a/src/slave/containerizer/isolators/posix/disk.hpp
+++ b/src/slave/containerizer/isolators/posix/disk.hpp
@@ -21,6 +21,8 @@
 
 #include <string>
 
+#include <mesos/slave/isolator.hpp>
+
 #include <process/owned.hpp>
 
 #include <stout/bytes.hpp>
@@ -29,8 +31,6 @@
 
 #include "slave/flags.hpp"
 
-#include "slave/containerizer/isolator.hpp"
-
 namespace mesos {
 namespace slave {
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/slave/containerizer/launcher.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/launcher.hpp b/src/slave/containerizer/launcher.hpp
index 75ade8b..f292e2d 100644
--- a/src/slave/containerizer/launcher.hpp
+++ b/src/slave/containerizer/launcher.hpp
@@ -23,6 +23,8 @@
 #include <map>
 #include <string>
 
+#include <mesos/slave/isolator.hpp>
+
 #include <process/future.hpp>
 #include <process/subprocess.hpp>
 
@@ -33,8 +35,6 @@
 
 #include "slave/flags.hpp"
 
-#include "slave/containerizer/isolator.hpp"
-
 namespace mesos {
 namespace slave {
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index e1b85b6..9910a17 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 
+#include <mesos/slave/isolator.hpp>
+
 #include <process/collect.hpp>
 #include <process/defer.hpp>
 #include <process/io.hpp>
@@ -32,7 +34,6 @@
 
 #include "slave/containerizer/containerizer.hpp"
 #include "slave/containerizer/fetcher.hpp"
-#include "slave/containerizer/isolator.hpp"
 #include "slave/containerizer/launcher.hpp"
 #ifdef __linux__
 #include "slave/containerizer/linux_launcher.hpp"

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 293c2d8..b7015be 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -22,13 +22,14 @@
 #include <list>
 #include <vector>
 
+#include <mesos/slave/isolator.hpp>
+
 #include <stout/hashmap.hpp>
 #include <stout/multihashmap.hpp>
 
 #include "slave/state.hpp"
 
 #include "slave/containerizer/containerizer.hpp"
-#include "slave/containerizer/isolator.hpp"
 #include "slave/containerizer/launcher.hpp"
 
 namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/tests/containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer_tests.cpp b/src/tests/containerizer_tests.cpp
index 7300333..0678849 100644
--- a/src/tests/containerizer_tests.cpp
+++ b/src/tests/containerizer_tests.cpp
@@ -22,17 +22,18 @@
 
 #include <gmock/gmock.h>
 
+#include <mesos/mesos.hpp>
+
+#include <mesos/slave/isolator.hpp>
+
 #include <process/future.hpp>
 #include <process/owned.hpp>
 
 #include <stout/strings.hpp>
 
-#include <mesos/mesos.hpp>
-
 #include "slave/flags.hpp"
 
 #include "slave/containerizer/fetcher.hpp"
-#include "slave/containerizer/isolator.hpp"
 #include "slave/containerizer/launcher.hpp"
 
 #include "slave/containerizer/mesos/containerizer.hpp"

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/tests/isolator.hpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator.hpp b/src/tests/isolator.hpp
index 547b0bb..5ad338b 100644
--- a/src/tests/isolator.hpp
+++ b/src/tests/isolator.hpp
@@ -21,7 +21,7 @@
 
 #include <gmock/gmock.h>
 
-#include "slave/containerizer/isolator.hpp"
+#include <mesos/slave/isolator.hpp>
 
 namespace mesos {
 namespace tests {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/tests/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator_tests.cpp b/src/tests/isolator_tests.cpp
index 093b9fd..7123a40 100644
--- a/src/tests/isolator_tests.cpp
+++ b/src/tests/isolator_tests.cpp
@@ -26,6 +26,8 @@
 
 #include <mesos/resources.hpp>
 
+#include <mesos/slave/isolator.hpp>
+
 #include <process/future.hpp>
 #include <process/owned.hpp>
 #include <process/reap.hpp>
@@ -47,7 +49,6 @@
 #include "slave/flags.hpp"
 #include "slave/slave.hpp"
 
-#include "slave/containerizer/isolator.hpp"
 #ifdef __linux__
 #include "slave/containerizer/isolators/cgroups/cpushare.hpp"
 #include "slave/containerizer/isolators/cgroups/mem.hpp"

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/tests/module_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/module_tests.cpp b/src/tests/module_tests.cpp
index c9d7381..5131309 100644
--- a/src/tests/module_tests.cpp
+++ b/src/tests/module_tests.cpp
@@ -20,6 +20,8 @@
 
 #include <mesos/module/module.hpp>
 
+#include <mesos/slave/isolator.hpp>
+
 #include <stout/dynamiclibrary.hpp>
 #include <stout/os.hpp>
 
@@ -27,7 +29,6 @@
 #include "examples/test_module.hpp"
 #include "module/isolator.hpp"
 #include "module/manager.hpp"
-#include "slave/containerizer/isolator.hpp"
 
 #include "tests/flags.hpp"
 #include "tests/mesos.hpp"
@@ -35,9 +36,9 @@
 using std::string;
 
 using namespace mesos;
+using namespace mesos::modules;
 using namespace mesos::slave;
 using namespace mesos::tests;
-using namespace mesos::modules;
 
 const char* DEFAULT_MODULE_LIBRARY_NAME = "examplemodule";
 const char* DEFAULT_MODULE_NAME = "org_apache_mesos_TestModule";

http://git-wip-us.apache.org/repos/asf/mesos/blob/0082eb6d/src/tests/port_mapping_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/port_mapping_tests.cpp b/src/tests/port_mapping_tests.cpp
index 462e49a..308f629 100644
--- a/src/tests/port_mapping_tests.cpp
+++ b/src/tests/port_mapping_tests.cpp
@@ -72,6 +72,7 @@ using namespace routing::queueing;
 
 using mesos::master::Master;
 
+using mesos::slave::Isolator;
 using mesos::slave::Launcher;
 using mesos::slave::LinuxLauncher;
 using mesos::slave::MesosContainerizer;