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 2016/08/22 20:46:30 UTC

[01/12] mesos git commit: Simplified the mesos containerizer launch logic.

Repository: mesos
Updated Branches:
  refs/heads/master e3143e756 -> c3228f3c3


Simplified the mesos containerizer launch logic.

This patch removes the image volume provisioning code in
MesosContainerizer. The logic will be moved to an isolator.

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


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

Branch: refs/heads/master
Commit: cf775bad0559b4fa212285ebc0d3480ce25e09fe
Parents: e3143e7
Author: Gilbert Song <so...@gmail.com>
Authored: Mon Aug 22 13:00:09 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Aug 22 13:00:09 2016 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 139 +++----------------
 src/slave/containerizer/mesos/containerizer.hpp |  11 --
 2 files changed, 22 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/cf775bad/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index d065464..fa9513f 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -810,30 +810,7 @@ Future<bool> MesosContainerizerProcess::launch(
 
   containers_.put(containerId, Owned<Container>(container));
 
-  // If 'container' is not set in 'executorInfo', one of the following
-  // is true:
-  //  1) This is a custom executor without ContainerInfo.
-  //  2) This is a command task without ContainerInfo (since we copy
-  //     ContainerInfo for command tasks if exists).
-  // In either of the above cases, no provisioning is needed.
-  // Therefore, we can go straight to 'prepare'.
-  if (!executorInfo.has_container()) {
-    return prepare(containerId, taskInfo, executorInfo, directory, user, None())
-      .then(defer(self(),
-                  &Self::__launch,
-                  containerId,
-                  taskInfo,
-                  executorInfo,
-                  directory,
-                  user,
-                  slaveId,
-                  environment,
-                  checkpoint,
-                  None(),
-                  lambda::_1));
-  }
-
-  // We'll first provision the image for the container , and then
+  // We'll first provision the image for the container, and then
   // provision the images specified in Volumes.
   Option<Image> containerImage;
 
@@ -842,21 +819,26 @@ Future<bool> MesosContainerizerProcess::launch(
       taskInfo->container().mesos().has_image()) {
     // Command task.
     containerImage = taskInfo->container().mesos().image();
-  } else if (executorInfo.container().mesos().has_image()) {
+  } else if (executorInfo.has_container() &&
+             executorInfo.container().mesos().has_image()) {
     // Custom executor.
     containerImage = executorInfo.container().mesos().image();
   }
 
   if (containerImage.isNone()) {
-    return _launch(containerId,
-                   taskInfo,
-                   executorInfo,
-                   directory,
-                   user,
-                   slaveId,
-                   environment,
-                   checkpoint,
-                   None());
+    return prepare(containerId, taskInfo, executorInfo, directory, user, None())
+      .then(defer(self(),
+                  &Self::_launch,
+                  containerId,
+                  taskInfo,
+                  executorInfo,
+                  directory,
+                  user,
+                  slaveId,
+                  environment,
+                  checkpoint,
+                  None(),
+                  lambda::_1));
   }
 
   Future<ProvisionInfo> provisioning = provisioner->provision(
@@ -866,96 +848,19 @@ Future<bool> MesosContainerizerProcess::launch(
   container->provisionInfos.push_back(provisioning);
 
   return provisioning
-    .then(defer(PID<MesosContainerizerProcess>(this),
-                &MesosContainerizerProcess::_launch,
-                containerId,
-                taskInfo,
-                executorInfo,
-                directory,
-                user,
-                slaveId,
-                environment,
-                checkpoint,
-                lambda::_1));
-}
-
-
-Future<bool> MesosContainerizerProcess::_launch(
-    const ContainerID& containerId,
-    const Option<TaskInfo>& taskInfo,
-    const ExecutorInfo& executorInfo,
-    const string& directory,
-    const Option<string>& user,
-    const SlaveID& slaveId,
-    const map<string, string>& environment,
-    bool checkpoint,
-    const Option<ProvisionInfo>& provisionInfo)
-{
-  CHECK(executorInfo.has_container());
-  CHECK_EQ(executorInfo.container().type(), ContainerInfo::MESOS);
-
-  // This is because if a 'destroy' happens after 'launch' and before
-  // '_launch', even if the '___destroy' will wait for the 'provision'
-  // in 'launch' to finish, there is still a chance that '___destroy'
-  // and its dependencies finish before '_launch' starts since onAny
-  // is not guaranteed to be executed in order.
-  if (!containers_.contains(containerId)) {
-    return Failure("Container destroyed during provisioning");
-  }
-
-  // Make sure containerizer is not in DESTROYING state, to avoid
-  // a possible race that containerizer is destroying the container
-  // while it is provisioning the image from volumes.
-  if (containers_[containerId]->state == DESTROYING) {
-    return Failure("Container is being destroyed during provisioning");
-  }
-
-  CHECK_EQ(containers_[containerId]->state, PROVISIONING);
-
-  // We will provision the images specified in ContainerInfo::volumes
-  // as well. We will mutate ContainerInfo::volumes to include the
-  // paths to the provisioned root filesystems (by setting the
-  // 'host_path') if the volume specifies an image as the source.
-  //
-  // TODO(gilbert): We need to figure out a way to support passing
-  // runtime configurations specified in the image to the container.
-  Owned<ExecutorInfo> _executorInfo(new ExecutorInfo(executorInfo));
-
-  list<Future<Nothing>> futures;
-
-  for (int i = 0; i < _executorInfo->container().volumes_size(); i++) {
-    Volume* volume = _executorInfo->mutable_container()->mutable_volumes(i);
-
-    if (!volume->has_image()) {
-      continue;
-    }
-
-    const Image& image = volume->image();
-
-    Future<ProvisionInfo> future = provisioner->provision(containerId, image);
-    containers_[containerId]->provisionInfos.push_back(future);
-
-    futures.push_back(future.then([=](const ProvisionInfo& info) {
-        volume->set_host_path(info.rootfs);
-        return Nothing();
-      }));
-  }
-
-  // We put `prepare` inside of a lambda expression, in order to get
-  // _executorInfo object after host path set in volume.
-  return collect(futures)
-    .then(defer([=]() -> Future<bool> {
+    .then(defer(self(),
+                [=](const ProvisionInfo& provisionInfo) -> Future<bool> {
       return prepare(containerId,
                      taskInfo,
-                     *_executorInfo,
+                     executorInfo,
                      directory,
                      user,
                      provisionInfo)
         .then(defer(self(),
-                    &Self::__launch,
+                    &Self::_launch,
                     containerId,
                     taskInfo,
-                    *_executorInfo,
+                    executorInfo,
                     directory,
                     user,
                     slaveId,
@@ -1086,7 +991,7 @@ Future<Nothing> MesosContainerizerProcess::fetch(
 }
 
 
-Future<bool> MesosContainerizerProcess::__launch(
+Future<bool> MesosContainerizerProcess::_launch(
     const ContainerID& containerId,
     const Option<TaskInfo>& taskInfo,
     const ExecutorInfo& executorInfo,

http://git-wip-us.apache.org/repos/asf/mesos/blob/cf775bad/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 4993961..041a62e 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -219,17 +219,6 @@ private:
       const std::string& directory,
       const Option<std::string>& user,
       const SlaveID& slaveId,
-      const std::map<std::string, std::string>& environment,
-      bool checkpoint,
-      const Option<ProvisionInfo>& provisionInfo);
-
-  process::Future<bool> __launch(
-      const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
       const std::map<std::string, std::string>& _environment,
       bool checkpoint,
       const Option<ProvisionInfo>& provisionInfo,


[12/12] mesos git commit: Fixed image provisioning comment in mesos containerizer.

Posted by ji...@apache.org.
Fixed image provisioning comment in mesos containerizer.

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


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

Branch: refs/heads/master
Commit: c3228f3c3d1a1b2c145d1377185cfe22da6079eb
Parents: 2f78a44
Author: Gilbert Song <so...@gmail.com>
Authored: Mon Aug 22 13:01:02 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Aug 22 13:37:20 2016 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c3228f3c/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index e2af95d..1cd3b38 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -883,8 +883,9 @@ Future<bool> MesosContainerizerProcess::launch(
 
   containers_.put(containerId, Owned<Container>(container));
 
-  // We'll first provision the image for the container, and then
-  // provision the images specified in Volumes.
+  // We'll first provision the image for the container, and
+  // then provision the images specified in `volumes` using
+  // the 'volume/image' isolator.
   if (!containerConfig.has_container_info() ||
       !containerConfig.container_info().mesos().has_image()) {
     return prepare(containerId, None())


[03/12] mesos git commit: Kept a single provisioning future in MesosContainerizer.

Posted by ji...@apache.org.
Kept a single provisioning future in MesosContainerizer.

Since we move the image volume provisioning code to an isolator, there
is no reason to keep a list of provisioning futures.

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


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

Branch: refs/heads/master
Commit: e006df1a0ecfcf9d8b8925be0cbf94aa48b434d2
Parents: 80676e6
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Aug 22 13:00:14 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Aug 22 13:00:14 2016 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 8 +++-----
 src/slave/containerizer/mesos/containerizer.hpp | 2 +-
 2 files changed, 4 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e006df1a/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index fa9513f..6c2441a 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -841,13 +841,11 @@ Future<bool> MesosContainerizerProcess::launch(
                   lambda::_1));
   }
 
-  Future<ProvisionInfo> provisioning = provisioner->provision(
+  container->provisioning = provisioner->provision(
       containerId,
       containerImage.get());
 
-  container->provisionInfos.push_back(provisioning);
-
-  return provisioning
+  return container->provisioning
     .then(defer(self(),
                 [=](const ProvisionInfo& provisionInfo) -> Future<bool> {
       return prepare(containerId,
@@ -1557,7 +1555,7 @@ void MesosContainerizerProcess::destroy(
 
     // Wait for the provisioner to finish provisioning before we
     // start destroying the container.
-    await(container->provisionInfos)
+    container->provisioning
       .onAny(defer(
           self(),
           &Self::____destroy,

http://git-wip-us.apache.org/repos/asf/mesos/blob/e006df1a/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 041a62e..bfbae9d 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -305,7 +305,7 @@ private:
     // We keep track of the future that is waiting for the provisioner's
     // `ProvisionInfo`, so that destroy will only start calling
     // provisioner->destroy after provisioner->provision has finished.
-    std::list<process::Future<ProvisionInfo>> provisionInfos;
+    process::Future<ProvisionInfo> provisioning;
 
     // We keep track of the future that is waiting for all the
     // isolators' prepare futures, so that destroy will only start


[04/12] mesos git commit: Refactored MesosContainerizer to prepare for nested container support.

Posted by ji...@apache.org.
Refactored MesosContainerizer to prepare for nested container support.

This patch simplified the MesosContainerizer logic by introducing
`ContainerConfig` in `Container` struct. This avoids some parameter
passing in the launch path. To prepare for the nested container
support, this patch introduces a private `launch` method which takes a
`ContainerConfig` (instead of ExecutorInfo and TaskInfo) since nested
container might not tie to a task.

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


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

Branch: refs/heads/master
Commit: 8e4805179ff3fcc13814faf2b0dc210f3793446c
Parents: e006df1
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Aug 22 13:00:16 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Aug 22 13:00:16 2016 -0700

----------------------------------------------------------------------
 include/mesos/slave/isolator.proto              |  12 +
 src/slave/containerizer/mesos/containerizer.cpp | 319 ++++++++++---------
 src/slave/containerizer/mesos/containerizer.hpp |  33 +-
 .../containerizer/mesos_containerizer_tests.cpp |  27 +-
 4 files changed, 205 insertions(+), 186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8e480517/include/mesos/slave/isolator.proto
----------------------------------------------------------------------
diff --git a/include/mesos/slave/isolator.proto b/include/mesos/slave/isolator.proto
index e945514..83c748d 100644
--- a/include/mesos/slave/isolator.proto
+++ b/include/mesos/slave/isolator.proto
@@ -67,9 +67,21 @@ message ContainerState {
  * during `prepare`.
  */
 message ContainerConfig {
+  // The executor associated with this container.
   required ExecutorInfo executor_info = 8;
+
+  // The task (optional) associated with this container.
   optional TaskInfo task_info = 9;
 
+  // The command used to launch the container.
+  required CommandInfo command_info = 11;
+
+  // The information about the container.
+  optional ContainerInfo container_info = 12;
+
+  // Resources associated with the container during launch.
+  repeated Resource resources = 13;
+
   // The work directory for the container in the host filesystem.
   required string directory = 3;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e480517/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 6c2441a..64dac61 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -798,9 +798,58 @@ Future<bool> MesosContainerizerProcess::launch(
             << "' for executor '" << executorInfo.executor_id()
             << "' of framework " << executorInfo.framework_id();
 
+  ContainerConfig containerConfig;
+  containerConfig.mutable_executor_info()->CopyFrom(executorInfo);
+  containerConfig.mutable_command_info()->CopyFrom(executorInfo.command());
+  containerConfig.mutable_resources()->CopyFrom(executorInfo.resources());
+  containerConfig.set_directory(directory);
+
+  if (user.isSome()) {
+    containerConfig.set_user(user.get());
+  }
+
+  if (taskInfo.isSome()) {
+    // Command task case.
+    containerConfig.mutable_task_info()->CopyFrom(taskInfo.get());
+
+    if (taskInfo->has_container()) {
+      ContainerInfo* containerInfo = containerConfig.mutable_container_info();
+      containerInfo->CopyFrom(taskInfo->container());
+
+      if (taskInfo->container().mesos().has_image()) {
+        // For command tasks, We need to set the command executor user
+        // as root as it needs to perform chroot (even when
+        // switch_user is set to false).
+        containerConfig.mutable_command_info()->set_user("root");
+      }
+    }
+  } else {
+    // Other cases.
+    if (executorInfo.has_container()) {
+      ContainerInfo* containerInfo = containerConfig.mutable_container_info();
+      containerInfo->CopyFrom(executorInfo.container());
+    }
+  }
+
+  return launch(containerId,
+                containerConfig,
+                environment,
+                slaveId,
+                checkpoint);
+}
+
+
+Future<bool> MesosContainerizerProcess::launch(
+    const ContainerID& containerId,
+    const ContainerConfig& containerConfig,
+    const map<string, string>& environment,
+    const SlaveID& slaveId,
+    bool checkpoint)
+{
   Container* container = new Container();
   container->state = PROVISIONING;
-  container->resources = executorInfo.resources();
+  container->config = containerConfig;
+  container->resources = containerConfig.resources();
 
   // We need to set the `launchInfos` to be a ready future initially
   // before we starting calling isolator->prepare() because otherwise,
@@ -812,70 +861,37 @@ Future<bool> MesosContainerizerProcess::launch(
 
   // We'll first provision the image for the container, and then
   // provision the images specified in Volumes.
-  Option<Image> containerImage;
-
-  if (taskInfo.isSome() &&
-      taskInfo->has_container() &&
-      taskInfo->container().mesos().has_image()) {
-    // Command task.
-    containerImage = taskInfo->container().mesos().image();
-  } else if (executorInfo.has_container() &&
-             executorInfo.container().mesos().has_image()) {
-    // Custom executor.
-    containerImage = executorInfo.container().mesos().image();
-  }
-
-  if (containerImage.isNone()) {
-    return prepare(containerId, taskInfo, executorInfo, directory, user, None())
+  if (!containerConfig.has_container_info() ||
+      !containerConfig.container_info().mesos().has_image()) {
+    return prepare(containerId, None())
       .then(defer(self(),
                   &Self::_launch,
                   containerId,
-                  taskInfo,
-                  executorInfo,
-                  directory,
-                  user,
-                  slaveId,
                   environment,
-                  checkpoint,
-                  None(),
-                  lambda::_1));
+                  slaveId,
+                  checkpoint));
   }
 
   container->provisioning = provisioner->provision(
       containerId,
-      containerImage.get());
+      containerConfig.container_info().mesos().image());
 
   return container->provisioning
     .then(defer(self(),
                 [=](const ProvisionInfo& provisionInfo) -> Future<bool> {
-      return prepare(containerId,
-                     taskInfo,
-                     executorInfo,
-                     directory,
-                     user,
-                     provisionInfo)
+      return prepare(containerId, provisionInfo)
         .then(defer(self(),
                     &Self::_launch,
                     containerId,
-                    taskInfo,
-                    executorInfo,
-                    directory,
-                    user,
-                    slaveId,
                     environment,
-                    checkpoint,
-                    provisionInfo,
-                    lambda::_1));
+                    slaveId,
+                    checkpoint));
     }));
 }
 
 
-Future<list<Option<ContainerLaunchInfo>>> MesosContainerizerProcess::prepare(
+Future<Nothing> MesosContainerizerProcess::prepare(
     const ContainerID& containerId,
-    const Option<TaskInfo>& taskInfo,
-    const ExecutorInfo& executorInfo,
-    const string& directory,
-    const Option<string>& user,
     const Option<ProvisionInfo>& provisionInfo)
 {
   // This is because if a 'destroy' happens during the provisoiner is
@@ -888,32 +904,21 @@ Future<list<Option<ContainerLaunchInfo>>> MesosContainerizerProcess::prepare(
     return Failure("Container destroyed during provisioning");
   }
 
+  const Owned<Container>& container = containers_[containerId];
+
   // Make sure containerizer is not in DESTROYING state, to avoid
   // a possible race that containerizer is destroying the container
   // while it is preparing isolators for the container.
-  if (containers_[containerId]->state == DESTROYING) {
+  if (container->state == DESTROYING) {
     return Failure("Container is being destroyed during provisioning");
   }
 
-  CHECK_EQ(containers_[containerId]->state, PROVISIONING);
-
-  containers_[containerId]->state = PREPARING;
+  CHECK_EQ(container->state, PROVISIONING);
 
-  // Construct ContainerConfig.
-  ContainerConfig containerConfig;
-  containerConfig.set_directory(directory);
-  containerConfig.mutable_executor_info()->CopyFrom(executorInfo);
-
-  if (taskInfo.isSome()) {
-    containerConfig.mutable_task_info()->CopyFrom(taskInfo.get());
-  }
-
-  if (user.isSome()) {
-    containerConfig.set_user(user.get());
-  }
+  container->state = PREPARING;
 
   if (provisionInfo.isSome()) {
-    containerConfig.set_rootfs(provisionInfo->rootfs);
+    container->config.set_rootfs(provisionInfo->rootfs);
 
     if (provisionInfo->dockerManifest.isSome() &&
         provisionInfo->appcManifest.isSome()) {
@@ -921,16 +926,19 @@ Future<list<Option<ContainerLaunchInfo>>> MesosContainerizerProcess::prepare(
     }
 
     if (provisionInfo->dockerManifest.isSome()) {
-      ContainerConfig::Docker* docker = containerConfig.mutable_docker();
+      ContainerConfig::Docker* docker = container->config.mutable_docker();
       docker->mutable_manifest()->CopyFrom(provisionInfo->dockerManifest.get());
     }
 
     if (provisionInfo->appcManifest.isSome()) {
-      ContainerConfig::Appc* appc = containerConfig.mutable_appc();
+      ContainerConfig::Appc* appc = container->config.mutable_appc();
       appc->mutable_manifest()->CopyFrom(provisionInfo->appcManifest.get());
     }
   }
 
+  // Captured for lambdas below.
+  ContainerConfig containerConfig = container->config;
+
   // We prepare the isolators sequentially according to their ordering
   // to permit basic dependency specification, e.g., preparing a
   // filesystem isolator before other isolators.
@@ -948,34 +956,40 @@ Future<list<Option<ContainerLaunchInfo>>> MesosContainerizerProcess::prepare(
       });
   }
 
-  containers_[containerId]->launchInfos = f;
+  container->launchInfos = f;
 
-  return f;
+  return f.then([]() { return Nothing(); });
 }
 
 
 Future<Nothing> MesosContainerizerProcess::fetch(
     const ContainerID& containerId,
-    const CommandInfo& commandInfo,
-    const string& directory,
-    const Option<string>& user,
     const SlaveID& slaveId)
 {
   if (!containers_.contains(containerId)) {
     return Failure("Container destroyed during isolating");
   }
 
-  if (containers_[containerId]->state == DESTROYING) {
+  const Owned<Container>& container = containers_[containerId];
+
+  if (container->state == DESTROYING) {
     return Failure("Container is being destroyed during isolating");
   }
 
-  CHECK_EQ(containers_[containerId]->state, ISOLATING);
+  CHECK_EQ(container->state, ISOLATING);
 
-  containers_[containerId]->state = FETCHING;
+  container->state = FETCHING;
+
+  const string directory = container->config.directory();
+
+  Option<string> user;
+  if (container->config.has_user()) {
+    user = container->config.user();
+  }
 
   return fetcher->fetch(
       containerId,
-      commandInfo,
+      container->config.command_info(),
       directory,
       user,
       slaveId,
@@ -991,42 +1005,65 @@ Future<Nothing> MesosContainerizerProcess::fetch(
 
 Future<bool> MesosContainerizerProcess::_launch(
     const ContainerID& containerId,
-    const Option<TaskInfo>& taskInfo,
-    const ExecutorInfo& executorInfo,
-    const string& directory,
-    const Option<string>& user,
+    map<string, string> environment,
     const SlaveID& slaveId,
-    const map<string, string>& _environment,
-    bool checkpoint,
-    const Option<ProvisionInfo>& provisionInfo,
-    const list<Option<ContainerLaunchInfo>>& launchInfos)
+    bool checkpoint)
 {
   if (!containers_.contains(containerId)) {
     return Failure("Container destroyed during preparing");
   }
 
-  if (containers_[containerId]->state == DESTROYING) {
+  const Owned<Container>& container = containers_[containerId];
+
+  if (container->state == DESTROYING) {
     return Failure("Container is being destroyed during preparing");
   }
 
-  CHECK_EQ(containers_[containerId]->state, PREPARING);
+  CHECK_EQ(container->state, PREPARING);
 
-  map<string, string> environment = _environment;
+  // TODO(jieyu): Consider moving this to 'executorEnvironment' and
+  // consolidating with docker containerizer.
+  //
+  // NOTE: For the command executor case, although it uses the host
+  // filesystem for itself, we still set 'MESOS_SANDBOX' according to
+  // the root filesystem of the task (if specified). Command executor
+  // itself does not use this environment variable.
+  environment["MESOS_SANDBOX"] = container->config.has_rootfs()
+    ? flags.sandbox_directory
+    : container->config.directory();
 
-  // Determine the root filesystem for the executor.
-  Option<string> executorRootfs;
-  if (taskInfo.isNone() && provisionInfo.isSome()) {
-    executorRootfs = provisionInfo->rootfs;
+  // Include any enviroment variables from CommandInfo.
+  foreach (const Environment::Variable& variable,
+           container->config.command_info().environment().variables()) {
+    environment[variable.name()] = variable.value();
   }
 
-  // Determine the executor launch command for the container.
-  // At most one command can be returned from docker runtime
-  // isolator if a docker image is specified.
-  Option<CommandInfo> executorLaunchCommand;
+  // NOTE: Command task is a special case. Even if the container
+  // config has a root filesystem, the executor container still uses
+  // the host filesystem.
+  Option<string> rootfs;
+  if (!container->config.has_task_info() &&
+      container->config.has_rootfs()) {
+    rootfs = container->config.rootfs();
+  }
+
+  Option<CommandInfo> launchCommand;
   Option<string> workingDirectory;
+  JSON::Array preExecCommands;
+
+  // TODO(jieyu): We should use Option here. If no namespace is
+  // required, we should pass None() to 'launcher->fork'.
+  int namespaces = 0;
 
-  foreach (const Option<ContainerLaunchInfo>& launchInfo, launchInfos) {
-    if (launchInfo.isSome() && launchInfo->has_environment()) {
+  CHECK_READY(container->launchInfos);
+
+  foreach (const Option<ContainerLaunchInfo>& launchInfo,
+           container->launchInfos.get()) {
+    if (launchInfo.isNone()) {
+      continue;
+    }
+
+    if (launchInfo->has_environment()) {
       foreach (const Environment::Variable& variable,
                launchInfo->environment().variables()) {
         const string& name = variable.name();
@@ -1044,15 +1081,15 @@ Future<bool> MesosContainerizerProcess::_launch(
       }
     }
 
-    if (launchInfo.isSome() && launchInfo->has_command()) {
-      if (executorLaunchCommand.isSome()) {
+    if (launchInfo->has_command()) {
+      if (launchCommand.isSome()) {
         return Failure("At most one command can be returned from isolators");
       } else {
-        executorLaunchCommand = launchInfo->command();
+        launchCommand = launchInfo->command();
       }
     }
 
-    if (launchInfo.isSome() && launchInfo->has_working_directory()) {
+    if (launchInfo->has_working_directory()) {
       if (workingDirectory.isSome()) {
         return Failure(
             "At most one working directory can be returned from isolators");
@@ -1060,70 +1097,39 @@ Future<bool> MesosContainerizerProcess::_launch(
         workingDirectory = launchInfo->working_directory();
       }
     }
-  }
 
-  // TODO(jieyu): Consider moving this to 'executorEnvironment' and
-  // consolidating with docker containerizer.
-  //
-  // NOTE: For the command executor case, although it uses the host
-  // filesystem for itself, we still set 'MESOS_SANDBOX' according to
-  // the root filesystem of the task (if specified). Command executor
-  // itself does not use this environment variable.
-  environment["MESOS_SANDBOX"] = provisionInfo.isSome()
-    ? flags.sandbox_directory
-    : directory;
-
-  // Include any enviroment variables from CommandInfo.
-  foreach (const Environment::Variable& variable,
-           executorInfo.command().environment().variables()) {
-    environment[variable.name()] = variable.value();
-  }
-
-  JSON::Array preExecCommands;
-
-  // TODO(jieyu): We should use Option here. If no namespace is
-  // required, we should pass None() to 'launcher->fork'.
-  int namespaces = 0;
-
-  foreach (const Option<ContainerLaunchInfo>& launchInfo, launchInfos) {
-    if (!launchInfo.isSome()) {
-      continue;
-    }
-
-    // Populate the list of additional commands to be run inside the container
-    // context.
     foreach (const CommandInfo& command, launchInfo->pre_exec_commands()) {
       preExecCommands.values.emplace_back(JSON::protobuf(command));
     }
 
-    // Process additional environment variables returned by isolators.
-    if (launchInfo->has_environment()) {
-      foreach (const Environment::Variable& variable,
-               launchInfo->environment().variables()) {
-        environment[variable.name()] = variable.value();
-      }
-    }
-
     if (launchInfo->has_namespaces()) {
       namespaces |= launchInfo->namespaces();
     }
   }
 
-  if (executorLaunchCommand.isNone()) {
-    executorLaunchCommand = executorInfo.command();
+  if (launchCommand.isNone()) {
+    launchCommand = container->config.command_info();
   }
 
-  // Inform the command executor about the rootfs of the task.
-  if (taskInfo.isSome() && provisionInfo.isSome()) {
-    CHECK_SOME(executorLaunchCommand);
-    executorLaunchCommand->add_arguments("--rootfs=" + provisionInfo->rootfs);
+  // For the command executor case, we should add the rootfs flag to
+  // the launch command of the command executor.
+  if (container->config.has_task_info() &&
+      container->config.has_rootfs()) {
+    CHECK_SOME(launchCommand);
+    launchCommand->add_arguments(
+        "--rootfs=" + container->config.rootfs());
   }
 
-  return logger->prepare(executorInfo, directory)
+  return logger->prepare(
+      container->config.executor_info(),
+      container->config.directory())
     .then(defer(
         self(),
         [=](const ContainerLogger::SubprocessInfo& subprocessInfo)
           -> Future<bool> {
+    CHECK(containers_.contains(containerId));
+    const Owned<Container>& container = containers_[containerId];
+
     // Use a pipe to block the child until it's been isolated.
     // The `pipes` array is captured later in a lambda.
     std::array<int, 2> pipes;
@@ -1135,9 +1141,9 @@ Future<bool> MesosContainerizerProcess::_launch(
     // Prepare the flags to pass to the launch process.
     MesosContainerizerLaunch::Flags launchFlags;
 
-    launchFlags.command = JSON::protobuf(executorLaunchCommand.get());
+    launchFlags.command = JSON::protobuf(launchCommand.get());
 
-    if (executorRootfs.isNone()) {
+    if (rootfs.isNone()) {
       // NOTE: If the executor shares the host filesystem, we should
       // not allow them to 'cd' into an arbitrary directory because
       // that'll create security issues.
@@ -1148,7 +1154,7 @@ Future<bool> MesosContainerizerProcess::_launch(
                      << "host filesystem";
       }
 
-      launchFlags.working_directory = directory;
+      launchFlags.working_directory = container->config.directory();
     } else {
       launchFlags.working_directory = workingDirectory.isSome()
         ? workingDirectory
@@ -1156,20 +1162,23 @@ Future<bool> MesosContainerizerProcess::_launch(
     }
 
 #ifdef __WINDOWS__
-    if (executorRootfs.isSome()) {
+    if (rootfs.isSome()) {
       return Failure(
           "`chroot` is not supported on Windows, but the executor "
           "specifies a root filesystem.");
     }
 
-    if (user.isSome()) {
+    if (container->config.has_user()) {
       return Failure(
           "`su` is not supported on Windows, but the executor "
           "specifies a user.");
     }
 #else
-    launchFlags.rootfs = executorRootfs;
-    launchFlags.user = user;
+    launchFlags.rootfs = rootfs;
+
+    if (container->config.has_user()) {
+      launchFlags.user = container->config.user();
+    }
 #endif // __WINDOWS__
 
 #ifndef __WINDOWS__
@@ -1214,8 +1223,8 @@ Future<bool> MesosContainerizerProcess::_launch(
       const string& path = slave::paths::getForkedPidPath(
           slave::paths::getMetaRootDir(flags.work_dir),
           slaveId,
-          executorInfo.framework_id(),
-          executorInfo.executor_id(),
+          container->config.executor_info().framework_id(),
+          container->config.executor_info().executor_id(),
           containerId);
 
       LOG(INFO) << "Checkpointing executor's forked pid " << pid
@@ -1236,15 +1245,13 @@ Future<bool> MesosContainerizerProcess::_launch(
     // refer to it again during container destroy.
     Future<Option<int>> status = process::reap(pid);
     status.onAny(defer(self(), &Self::reaped, containerId));
-    containers_[containerId]->status = status;
+
+    container->status = status;
 
     return isolate(containerId, pid)
       .then(defer(self(),
                   &Self::fetch,
                   containerId,
-                  executorInfo.command(),
-                  directory,
-                  user,
                   slaveId))
       .then(defer(self(), &Self::exec, containerId, pipes[1]))
       .onAny([pipes]() { os::close(pipes[0]); })

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e480517/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index bfbae9d..6bd16e6 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -197,32 +197,26 @@ private:
       const std::list<mesos::slave::ContainerState>& recovered,
       const hashset<ContainerID>& orphans);
 
-  process::Future<std::list<Option<mesos::slave::ContainerLaunchInfo>>>
-    prepare(const ContainerID& containerId,
-            const Option<TaskInfo>& taskInfo,
-            const ExecutorInfo& executorInfo,
-            const std::string& directory,
-            const Option<std::string>& user,
-            const Option<ProvisionInfo>& provisionInfo);
+  process::Future<Nothing> prepare(
+      const ContainerID& containerId,
+      const Option<ProvisionInfo>& provisionInfo);
 
   process::Future<Nothing> fetch(
       const ContainerID& containerId,
-      const CommandInfo& commandInfo,
-      const std::string& directory,
-      const Option<std::string>& user,
       const SlaveID& slaveId);
 
+  process::Future<bool> launch(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig,
+      const std::map<std::string, std::string>& environment,
+      const SlaveID& slaveId,
+      bool checkpoint);
+
   process::Future<bool> _launch(
       const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const Option<std::string>& user,
+      std::map<std::string, std::string> environment,
       const SlaveID& slaveId,
-      const std::map<std::string, std::string>& _environment,
-      bool checkpoint,
-      const Option<ProvisionInfo>& provisionInfo,
-      const std::list<Option<mesos::slave::ContainerLaunchInfo>>& launchInfos);
+      bool checkpoint);
 
   process::Future<bool> isolate(
       const ContainerID& containerId,
@@ -326,6 +320,9 @@ private:
     // ResourceStatistics limits in usage().
     Resources resources;
 
+    // The configuration for the container to be launched.
+    mesos::slave::ContainerConfig config;
+
     State state;
 
     // Used when `status` needs to be collected from isolators

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e480517/src/tests/containerizer/mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/mesos_containerizer_tests.cpp b/src/tests/containerizer/mesos_containerizer_tests.cpp
index 2a2ef51..e021b68 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -827,10 +827,6 @@ TEST_F(MesosContainerizerProvisionerTest, ProvisionFailed)
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
-  TaskInfo taskInfo;
-  CommandInfo commandInfo;
-  taskInfo.mutable_command()->MergeFrom(commandInfo);
-
   Image image;
   image.set_type(Image::DOCKER);
   Image::Docker dockerImage;
@@ -844,6 +840,11 @@ TEST_F(MesosContainerizerProvisionerTest, ProvisionFailed)
   containerInfo.set_type(ContainerInfo::MESOS);
   containerInfo.mutable_mesos()->CopyFrom(mesosInfo);
 
+  TaskInfo taskInfo;
+  CommandInfo commandInfo;
+  taskInfo.mutable_command()->MergeFrom(commandInfo);
+  taskInfo.mutable_container()->CopyFrom(containerInfo);
+
   ExecutorInfo executorInfo = CREATE_EXECUTOR_INFO("executor", "exit 0");
   executorInfo.mutable_container()->CopyFrom(containerInfo);
 
@@ -922,10 +923,6 @@ TEST_F(MesosContainerizerProvisionerTest, DestroyWhileProvisioning)
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
-  TaskInfo taskInfo;
-  CommandInfo commandInfo;
-  taskInfo.mutable_command()->MergeFrom(commandInfo);
-
   Image image;
   image.set_type(Image::DOCKER);
   Image::Docker dockerImage;
@@ -939,6 +936,11 @@ TEST_F(MesosContainerizerProvisionerTest, DestroyWhileProvisioning)
   containerInfo.set_type(ContainerInfo::MESOS);
   containerInfo.mutable_mesos()->CopyFrom(mesosInfo);
 
+  TaskInfo taskInfo;
+  CommandInfo commandInfo;
+  taskInfo.mutable_command()->MergeFrom(commandInfo);
+  taskInfo.mutable_container()->CopyFrom(containerInfo);
+
   ExecutorInfo executorInfo = CREATE_EXECUTOR_INFO("executor", "exit 0");
   executorInfo.mutable_container()->CopyFrom(containerInfo);
 
@@ -1024,10 +1026,6 @@ TEST_F(MesosContainerizerProvisionerTest, IsolatorCleanupBeforePrepare)
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
-  TaskInfo taskInfo;
-  CommandInfo commandInfo;
-  taskInfo.mutable_command()->MergeFrom(commandInfo);
-
   Image image;
   image.set_type(Image::DOCKER);
   Image::Docker dockerImage;
@@ -1041,6 +1039,11 @@ TEST_F(MesosContainerizerProvisionerTest, IsolatorCleanupBeforePrepare)
   containerInfo.set_type(ContainerInfo::MESOS);
   containerInfo.mutable_mesos()->CopyFrom(mesosInfo);
 
+  TaskInfo taskInfo;
+  CommandInfo commandInfo;
+  taskInfo.mutable_command()->MergeFrom(commandInfo);
+  taskInfo.mutable_container()->CopyFrom(containerInfo);
+
   ExecutorInfo executorInfo = CREATE_EXECUTOR_INFO("executor", "exit 0");
   executorInfo.mutable_container()->CopyFrom(containerInfo);
 


[10/12] mesos git commit: Plugged in the volume/image isolator to mesos containerizer.

Posted by ji...@apache.org.
Plugged in the volume/image isolator to mesos containerizer.

This patch includes:
1 Plugged in the volume/image isolator.
2 Changed MesosContainerizer interface. The parameter for
  provisioner is changed from an owned pointer to a shared
  pointer.
3 Fixed corresponding unit tests.

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


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

Branch: refs/heads/master
Commit: 9642d3c67b1215c7e3e9626d95cd0c4954fa9f7b
Parents: 192283f
Author: Gilbert Song <so...@gmail.com>
Authored: Mon Aug 22 13:00:39 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Aug 22 13:37:08 2016 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 44 +++++++++++++++-----
 src/slave/containerizer/mesos/containerizer.hpp |  7 ++--
 .../mesos/isolators/filesystem/linux.cpp        |  9 +++-
 src/tests/container_logger_tests.cpp            |  2 +-
 .../docker_volume_isolator_tests.cpp            |  2 +-
 .../containerizer/filesystem_isolator_tests.cpp | 26 +++++++++---
 .../containerizer/mesos_containerizer_tests.cpp | 17 ++++----
 7 files changed, 76 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9642d3c6/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 808fc83..e2af95d 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -109,6 +109,10 @@
 #include "slave/containerizer/mesos/isolators/network/port_mapping.hpp"
 #endif
 
+#ifdef __linux__
+#include "slave/containerizer/mesos/isolators/volume/image.hpp"
+#endif
+
 #include "slave/containerizer/mesos/constants.hpp"
 #include "slave/containerizer/mesos/containerizer.hpp"
 #include "slave/containerizer/mesos/launch.hpp"
@@ -196,7 +200,18 @@ Try<MesosContainerizer*> MesosContainerizer::create(
   if (!strings::contains(flags_.isolation, "network/")) {
     flags_.isolation += ",network/cni";
   }
-#endif
+
+  // Always enable 'volume/image' on linux if 'filesystem/linux' is
+  // enabled, to ensure backwards compatibility.
+  //
+  // TODO(gilbert): Make sure the 'gpu/nvidia' isolator to be created
+  // after all volume isolators, so that the nvidia gpu libraries
+  // '/usr/local/nvidia' will be overwritten.
+  if (strings::contains(flags_.isolation, "filesystem/linux") &&
+      !strings::contains(flags_.isolation, "volume/image")) {
+    flags_.isolation += ",volume/image";
+  }
+#endif // __linux__
 
   LOG(INFO) << "Using isolation: " << flags_.isolation;
 
@@ -249,11 +264,13 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     return Error("Failed to create launcher: " + launcher.error());
   }
 
-  Try<Owned<Provisioner>> provisioner = Provisioner::create(flags_);
-  if (provisioner.isError()) {
-    return Error("Failed to create provisioner: " + provisioner.error());
+  Try<Owned<Provisioner>> _provisioner = Provisioner::create(flags_);
+  if (_provisioner.isError()) {
+    return Error("Failed to create provisioner: " + _provisioner.error());
   }
 
+  Shared<Provisioner> provisioner = _provisioner.get().share();
+
   // Create the isolators.
   //
   // Currently, the order of the entries in the --isolation flag
@@ -313,6 +330,11 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     {"docker/runtime", &DockerRuntimeIsolatorProcess::create},
     {"docker/volume", &DockerVolumeIsolatorProcess::create},
 
+    {"volume/image",
+      [&provisioner] (const Flags& flags) -> Try<Isolator*> {
+        return VolumeImageIsolatorProcess::create(flags, provisioner);
+      }},
+
     {"gpu/nvidia",
       [&nvidia] (const Flags& flags) -> Try<Isolator*> {
         if (!nvml::isAvailable()) {
@@ -335,12 +357,12 @@ Try<MesosContainerizer*> MesosContainerizer::create(
 #endif
   };
 
-  const vector<string> isolations = strings::tokenize(flags_.isolation, ",");
+  vector<string> tokens = strings::tokenize(flags_.isolation, ",");
+  set<string> isolations = set<string>(tokens.begin(), tokens.end());
 
-  if (isolations.size() !=
-      set<string>(isolations.begin(), isolations.end()).size()) {
-    return Error("Duplicate entries found in --isolation flag"
-                 " '" + stringify(isolations) + "'");
+  if (tokens.size() != isolations.size()) {
+    return Error("Duplicate entries found in --isolation flag '" +
+                 stringify(tokens) + "'");
   }
 
   vector<Owned<Isolator>> isolators;
@@ -376,7 +398,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
       fetcher,
       Owned<ContainerLogger>(logger.get()),
       Owned<Launcher>(launcher.get()),
-      provisioner.get(),
+      provisioner,
       isolators);
 }
 
@@ -387,7 +409,7 @@ MesosContainerizer::MesosContainerizer(
     Fetcher* fetcher,
     const Owned<ContainerLogger>& logger,
     const Owned<Launcher>& launcher,
-    const Owned<Provisioner>& provisioner,
+    const Shared<Provisioner>& provisioner,
     const vector<Owned<Isolator>>& isolators)
   : process(new MesosContainerizerProcess(
       flags,

http://git-wip-us.apache.org/repos/asf/mesos/blob/9642d3c6/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 0cfe7b0..1f414cf 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -22,6 +22,7 @@
 
 #include <process/id.hpp>
 #include <process/sequence.hpp>
+#include <process/shared.hpp>
 
 #include <process/metrics/counter.hpp>
 
@@ -63,7 +64,7 @@ public:
       Fetcher* fetcher,
       const process::Owned<mesos::slave::ContainerLogger>& logger,
       const process::Owned<Launcher>& launcher,
-      const process::Owned<Provisioner>& provisioner,
+      const process::Shared<Provisioner>& provisioner,
       const std::vector<process::Owned<mesos::slave::Isolator>>& isolators);
 
   // Used for testing.
@@ -122,7 +123,7 @@ public:
       Fetcher* _fetcher,
       const process::Owned<mesos::slave::ContainerLogger>& _logger,
       const process::Owned<Launcher>& _launcher,
-      const process::Owned<Provisioner>& _provisioner,
+      const process::Shared<Provisioner>& _provisioner,
       const std::vector<process::Owned<mesos::slave::Isolator>>& _isolators)
     : ProcessBase(process::ID::generate("mesos-containerizer")),
       flags(_flags),
@@ -271,7 +272,7 @@ private:
   Fetcher* fetcher;
   process::Owned<mesos::slave::ContainerLogger> logger;
   const process::Owned<Launcher> launcher;
-  const process::Owned<Provisioner> provisioner;
+  const process::Shared<Provisioner> provisioner;
   const std::vector<process::Owned<mesos::slave::Isolator>> isolators;
 
   enum State

http://git-wip-us.apache.org/repos/asf/mesos/blob/9642d3c6/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp b/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp
index 6bf9a4a..ea41825 100644
--- a/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp
+++ b/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp
@@ -369,8 +369,13 @@ Try<vector<CommandInfo>> LinuxFilesystemIsolatorProcess::getPreExecCommands(
     // NOTE: Volumes with source will be handled by the corresponding
     // isolators (e.g., docker/volume).
     if (volume.has_source()) {
-      VLOG(1) << "Ignored a volume with source for container '"
-              << containerId << "'";
+      VLOG(1) << "Ignored a volume with source for container "
+              << containerId;
+      continue;
+    }
+
+    if (volume.has_image()) {
+      VLOG(1) << "Ignored an image volume for container " << containerId;
       continue;
     }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/9642d3c6/src/tests/container_logger_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/container_logger_tests.cpp b/src/tests/container_logger_tests.cpp
index efadcea..e8f9341 100644
--- a/src/tests/container_logger_tests.cpp
+++ b/src/tests/container_logger_tests.cpp
@@ -168,7 +168,7 @@ TEST_F(ContainerLoggerTest, MesosContainerizerRecover)
       &fetcher,
       Owned<ContainerLogger>(logger),
       Owned<Launcher>(launcher.get()),
-      provisioner.get(),
+      provisioner->share(),
       vector<Owned<Isolator>>());
 
   // Create the container's sandbox to get past a `CHECK` inside

http://git-wip-us.apache.org/repos/asf/mesos/blob/9642d3c6/src/tests/containerizer/docker_volume_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/docker_volume_isolator_tests.cpp b/src/tests/containerizer/docker_volume_isolator_tests.cpp
index da96f86..236834e 100644
--- a/src/tests/containerizer/docker_volume_isolator_tests.cpp
+++ b/src/tests/containerizer/docker_volume_isolator_tests.cpp
@@ -200,7 +200,7 @@ protected:
             &fetcher,
             Owned<ContainerLogger>(logger.get()),
             Owned<Launcher>(launcher.get()),
-            provisioner.get(),
+            provisioner->share(),
             {Owned<Isolator>(linuxIsolator.get()),
              Owned<Isolator>(runtimeIsolator.get()),
              Owned<Isolator>(volumeIsolator.get())}));

http://git-wip-us.apache.org/repos/asf/mesos/blob/9642d3c6/src/tests/containerizer/filesystem_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/filesystem_isolator_tests.cpp b/src/tests/containerizer/filesystem_isolator_tests.cpp
index 9f6b2ed..a2db8dd 100644
--- a/src/tests/containerizer/filesystem_isolator_tests.cpp
+++ b/src/tests/containerizer/filesystem_isolator_tests.cpp
@@ -41,6 +41,9 @@
 #include "slave/containerizer/mesos/linux_launcher.hpp"
 
 #include "slave/containerizer/mesos/isolators/filesystem/linux.hpp"
+
+#include "slave/containerizer/mesos/isolators/volume/image.hpp"
+
 #include "slave/containerizer/mesos/provisioner/backend.hpp"
 #include "slave/containerizer/mesos/provisioner/paths.hpp"
 #include "slave/containerizer/mesos/provisioner/backends/copy.hpp"
@@ -73,6 +76,7 @@ using mesos::internal::slave::Provisioner;
 using mesos::internal::slave::ProvisionerProcess;
 using mesos::internal::slave::Slave;
 using mesos::internal::slave::Store;
+using mesos::internal::slave::VolumeImageIsolatorProcess;
 
 using mesos::master::detector::MasterDetector;
 
@@ -145,14 +149,25 @@ protected:
         stores,
         backends));
 
-    Owned<Provisioner> provisioner(new Provisioner(provisionerProcess));
+    Owned<Provisioner> _provisioner(new Provisioner(provisionerProcess));
+    Shared<Provisioner> provisioner = _provisioner.share();
 
-    Try<Isolator*> isolator = LinuxFilesystemIsolatorProcess::create(flags);
+    Try<Isolator*> linuxIsolator =
+      LinuxFilesystemIsolatorProcess::create(flags);
 
-    if (isolator.isError()) {
+    if (linuxIsolator.isError()) {
       return Error(
           "Failed to create LinuxFilesystemIsolatorProcess: " +
-          isolator.error());
+          linuxIsolator.error());
+    }
+
+    Try<Isolator*> imageIsolator =
+      VolumeImageIsolatorProcess::create(flags, provisioner);
+
+    if (imageIsolator.isError()) {
+      return Error(
+          "Failed to create VolumeImageIsolatorProcess: " +
+          imageIsolator.error());
     }
 
     Try<Launcher*> launcher = LinuxLauncher::create(flags);
@@ -177,7 +192,8 @@ protected:
             Owned<ContainerLogger>(logger.get()),
             Owned<Launcher>(launcher.get()),
             provisioner,
-            {Owned<Isolator>(isolator.get())}));
+            {Owned<Isolator>(linuxIsolator.get()),
+             Owned<Isolator>(imageIsolator.get())}));
   }
 
   ContainerInfo createContainerInfo(

http://git-wip-us.apache.org/repos/asf/mesos/blob/9642d3c6/src/tests/containerizer/mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/mesos_containerizer_tests.cpp b/src/tests/containerizer/mesos_containerizer_tests.cpp
index 2fd4ec5..c0dda11 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -28,6 +28,7 @@
 
 #include <process/future.hpp>
 #include <process/owned.hpp>
+#include <process/shared.hpp>
 
 #include <stout/net.hpp>
 #include <stout/strings.hpp>
@@ -170,7 +171,7 @@ public:
         fetcher,
         Owned<ContainerLogger>(logger.get()),
         Owned<Launcher>(launcher.get()),
-        provisioner.get(),
+        provisioner->share(),
         isolators));
   }
 
@@ -509,7 +510,7 @@ public:
       Fetcher* fetcher,
       const Owned<ContainerLogger>& logger,
       const Owned<Launcher>& launcher,
-      const Owned<Provisioner>& provisioner,
+      const Shared<Provisioner>& provisioner,
       const vector<Owned<Isolator>>& isolators)
     : MesosContainerizerProcess(
           flags,
@@ -630,7 +631,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhileFetching)
       &fetcher,
       Owned<ContainerLogger>(logger.get()),
       Owned<Launcher>(launcher.get()),
-      provisioner.get(),
+      provisioner->share(),
       vector<Owned<Isolator>>());
 
   Future<Nothing> exec;
@@ -705,7 +706,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
       &fetcher,
       Owned<ContainerLogger>(logger.get()),
       Owned<Launcher>(launcher.get()),
-      provisioner.get(),
+      provisioner->share(),
       {Owned<Isolator>(isolator)});
 
   MesosContainerizer containerizer((Owned<MesosContainerizerProcess>(process)));
@@ -810,7 +811,7 @@ TEST_F(MesosContainerizerProvisionerTest, ProvisionFailed)
       &fetcher,
       Owned<ContainerLogger>(logger.get()),
       Owned<Launcher>(launcher),
-      Owned<Provisioner>(provisioner),
+      Shared<Provisioner>(provisioner),
       vector<Owned<Isolator>>());
 
   MesosContainerizer containerizer((Owned<MesosContainerizerProcess>(process)));
@@ -906,7 +907,7 @@ TEST_F(MesosContainerizerProvisionerTest, DestroyWhileProvisioning)
       &fetcher,
       Owned<ContainerLogger>(logger.get()),
       Owned<Launcher>(launcher),
-      Owned<Provisioner>(provisioner),
+      Shared<Provisioner>(provisioner),
       vector<Owned<Isolator>>());
 
   MesosContainerizer containerizer((Owned<MesosContainerizerProcess>(process)));
@@ -1009,7 +1010,7 @@ TEST_F(MesosContainerizerProvisionerTest, IsolatorCleanupBeforePrepare)
       &fetcher,
       Owned<ContainerLogger>(logger.get()),
       Owned<Launcher>(launcher),
-      Owned<Provisioner>(provisioner),
+      Shared<Provisioner>(provisioner),
       {Owned<Isolator>(isolator)});
 
   MesosContainerizer containerizer((Owned<MesosContainerizerProcess>(process)));
@@ -1107,7 +1108,7 @@ TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
       &fetcher,
       Owned<ContainerLogger>(logger.get()),
       Owned<Launcher>(launcher),
-      provisioner.get(),
+      provisioner->share(),
       vector<Owned<Isolator>>());
 
   MesosContainerizer containerizer((Owned<MesosContainerizerProcess>(process)));


[02/12] mesos git commit: Removed DestroyWhileProvisioningVolumeImage test.

Posted by ji...@apache.org.
Removed DestroyWhileProvisioningVolumeImage test.

Given that we moved the image volume provisioning code into an
isolator, this test no longer applies.

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


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

Branch: refs/heads/master
Commit: 80676e6dbc2a1773a1b2b47e61140064d64e94b5
Parents: cf775ba
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Aug 22 13:00:12 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Aug 22 13:00:12 2016 -0700

----------------------------------------------------------------------
 .../containerizer/mesos_containerizer_tests.cpp | 101 -------------------
 1 file changed, 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/80676e6d/src/tests/containerizer/mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/mesos_containerizer_tests.cpp b/src/tests/containerizer/mesos_containerizer_tests.cpp
index 4a49247..2a2ef51 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -974,107 +974,6 @@ TEST_F(MesosContainerizerProvisionerTest, DestroyWhileProvisioning)
 }
 
 
-// This test verifies that there is no race (or leaked provisioned
-// directories) if the containerizer destroy a container while it
-// is provisioning a volume image in '_launch'.
-TEST_F(MesosContainerizerProvisionerTest, DestroyWhileProvisioningVolumeImage)
-{
-  slave::Flags flags = CreateSlaveFlags();
-
-  Try<Launcher*> launcher_ = PosixLauncher::create(flags);
-  ASSERT_SOME(launcher_);
-
-  TestLauncher* launcher = new TestLauncher(Owned<Launcher>(launcher_.get()));
-
-  MockProvisioner* provisioner = new MockProvisioner();
-
-  Future<Nothing> provision;
-  Promise<ProvisionInfo> promise;
-
-  EXPECT_CALL(*provisioner, provision(_, _))
-    .WillOnce(DoAll(FutureSatisfy(&provision),
-                    Return(promise.future())));
-
-  EXPECT_CALL(*provisioner, destroy(_))
-    .WillOnce(Return(true));
-
-  Fetcher fetcher;
-
-  Try<ContainerLogger*> logger =
-    ContainerLogger::create(flags.container_logger);
-
-  ASSERT_SOME(logger);
-
-  MesosContainerizerProcess* process = new MesosContainerizerProcess(
-      flags,
-      true,
-      &fetcher,
-      Owned<ContainerLogger>(logger.get()),
-      Owned<Launcher>(launcher),
-      Owned<Provisioner>(provisioner),
-      vector<Owned<Isolator>>());
-
-  MesosContainerizer containerizer((Owned<MesosContainerizerProcess>(process)));
-
-  ContainerID containerId;
-  containerId.set_value(UUID::random().toString());
-
-  TaskInfo taskInfo;
-  CommandInfo commandInfo;
-  taskInfo.mutable_command()->MergeFrom(commandInfo);
-
-  ContainerInfo::MesosInfo mesosInfo;
-
-  ContainerInfo containerInfo;
-  containerInfo.set_type(ContainerInfo::MESOS);
-  containerInfo.mutable_mesos()->CopyFrom(mesosInfo);
-
-  Image image;
-  image.set_type(Image::DOCKER);
-  Image::Docker dockerImage;
-  dockerImage.set_name(UUID::random().toString());
-  image.mutable_docker()->CopyFrom(dockerImage);
-
-  Volume* volume = containerInfo.add_volumes();
-  volume->mutable_image()->CopyFrom(image);
-  volume->set_container_path("tmp");
-  volume->set_mode(Volume::RW);
-
-  ExecutorInfo executorInfo = CREATE_EXECUTOR_INFO("executor", "exit 0");
-  executorInfo.mutable_container()->CopyFrom(containerInfo);
-
-  Future<bool> launch = containerizer.launch(
-      containerId,
-      taskInfo,
-      executorInfo,
-      os::getcwd(),
-      None(),
-      SlaveID(),
-      map<string, string>(),
-      false);
-
-  Future<containerizer::Termination> wait = containerizer.wait(containerId);
-
-  AWAIT_READY(provision);
-
-  containerizer.destroy(containerId);
-
-  ASSERT_TRUE(wait.isPending());
-  promise.set(ProvisionInfo{"rootfs", None()});
-
-  AWAIT_FAILED(launch);
-  AWAIT_READY(wait);
-
-  containerizer::Termination termination = wait.get();
-
-  EXPECT_EQ(
-    "Container destroyed while provisioning images",
-    termination.message());
-
-  EXPECT_FALSE(termination.has_status());
-}
-
-
 // This test verifies that isolator cleanup will not be invoked
 // if the containerizer destroy a container while it is provisioning
 // an image, because isolators are not prepared yet.


[07/12] mesos git commit: Renamed containerizer::Termination to ContainerTermination.

Posted by ji...@apache.org.
Renamed containerizer::Termination to ContainerTermination.

`containerizer::Termination` is a legacy protobuf for external
containerizer. Since we already removed the external containerizer, this
patch renamed it to `ContainerTermination` and moved the definition to
`containerizer.proto`. This patch also moved all definitions in
`isolator.proto` to `containerizer.proto` to be more consistent.

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


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

Branch: refs/heads/master
Commit: c28846fbf12e1fcbe947c2ae2b9372015727e084
Parents: c4ce258
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Aug 22 13:00:25 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Aug 22 13:00:25 2016 -0700

----------------------------------------------------------------------
 include/mesos/containerizer/containerizer.hpp   |  23 ---
 include/mesos/containerizer/containerizer.proto |  38 -----
 include/mesos/slave/containerizer.hpp           |  23 +++
 include/mesos/slave/containerizer.proto         | 159 +++++++++++++++++++
 include/mesos/slave/isolator.hpp                |   3 +-
 include/mesos/slave/isolator.proto              | 143 -----------------
 src/CMakeLists.txt                              |   4 +-
 src/Makefile.am                                 |  41 +----
 src/slave/containerizer/composing.cpp           |  10 +-
 src/slave/containerizer/composing.hpp           |   6 +-
 src/slave/containerizer/containerizer.hpp       |  13 +-
 src/slave/containerizer/docker.cpp              |  36 +++--
 src/slave/containerizer/docker.hpp              |   6 +-
 src/slave/containerizer/mesos/containerizer.cpp |  16 +-
 src/slave/containerizer/mesos/containerizer.hpp |   6 +-
 src/slave/slave.cpp                             |  19 +--
 src/slave/slave.hpp                             |   6 +-
 src/tests/cluster.cpp                           |   4 +-
 src/tests/containerizer.cpp                     |  13 +-
 src/tests/containerizer.hpp                     |   8 +-
 .../composing_containerizer_tests.cpp           |   4 +-
 .../docker_containerizer_tests.cpp              |  27 ++--
 .../containerizer/filesystem_isolator_tests.cpp |  54 ++-----
 src/tests/containerizer/isolator_tests.cpp      |   6 +-
 .../containerizer/mesos_containerizer_tests.cpp |  41 ++---
 src/tests/containerizer/port_mapping_tests.cpp  |   4 +-
 src/tests/health_check_tests.cpp                |   5 +-
 src/tests/hook_tests.cpp                        |   8 +-
 src/tests/slave_recovery_tests.cpp              |  16 +-
 29 files changed, 333 insertions(+), 409 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/include/mesos/containerizer/containerizer.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/containerizer/containerizer.hpp b/include/mesos/containerizer/containerizer.hpp
deleted file mode 100644
index 01b1bfa..0000000
--- a/include/mesos/containerizer/containerizer.hpp
+++ /dev/null
@@ -1,23 +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 __CONTAINERIZER_PROTO_HPP__
-#define __CONTAINERIZER_PROTO_HPP__
-
-// ONLY USEFUL AFTER RUNNING PROTOC.
-#include <mesos/containerizer/containerizer.pb.h>
-
-#endif // __CONTAINERIZER_PROTO_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/include/mesos/containerizer/containerizer.proto
----------------------------------------------------------------------
diff --git a/include/mesos/containerizer/containerizer.proto b/include/mesos/containerizer/containerizer.proto
deleted file mode 100644
index 7c35ef3..0000000
--- a/include/mesos/containerizer/containerizer.proto
+++ /dev/null
@@ -1,38 +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.
-
-import "mesos/mesos.proto";
-
-package mesos.containerizer;
-
-option java_package = "org.apache.mesos.containerizer";
-option java_outer_classname = "Protos";
-
-
-/**
- * Information about a container termination, returned by the
- * containerizer to the slave.
- */
-message Termination {
-  // Exit status of the process.
-  optional int32 status = 3;
-
-  // The 'state', 'reasons' and 'message' of a status update for
-  // non-terminal tasks when the executor is terminated.
-  optional TaskState state = 4;
-  repeated TaskStatus.Reason reasons = 5;
-  optional string message = 2;
-}

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/include/mesos/slave/containerizer.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/slave/containerizer.hpp b/include/mesos/slave/containerizer.hpp
new file mode 100644
index 0000000..d0096b9
--- /dev/null
+++ b/include/mesos/slave/containerizer.hpp
@@ -0,0 +1,23 @@
+// 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_CONTAINERIZER_HPP__
+#define __MESOS_SLAVE_CONTAINERIZER_HPP__
+
+// ONLY USEFUL AFTER RUNNING PROTOC.
+#include <mesos/slave/containerizer.pb.h>
+
+#endif // __MESOS_SLAVE_CONTAINERIZER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/include/mesos/slave/containerizer.proto
----------------------------------------------------------------------
diff --git a/include/mesos/slave/containerizer.proto b/include/mesos/slave/containerizer.proto
new file mode 100644
index 0000000..16dd3a1
--- /dev/null
+++ b/include/mesos/slave/containerizer.proto
@@ -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.
+
+package mesos.slave;
+
+import "mesos/mesos.proto";
+
+import "mesos/appc/spec.proto";
+
+import "mesos/docker/v1.proto";
+
+/**
+ * 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.
+ */
+message ContainerLimitation {
+  // Resources that triggered the limitation.
+  // NOTE: 'Resources' is used here because the resource may span
+  // multiple roles (e.g. `"mem(*):1;mem(role):2"`).
+  repeated Resource resources = 1;
+
+  // Description of the limitation.
+  optional string message = 2;
+
+  // The container will be terminated when a resource limitation is
+  // reached. This field specifies the 'reason' that will be sent in
+  // the status update for any remaining non-terminal tasks when the
+  // container is terminated.
+  optional TaskStatus.Reason reason = 3;
+}
+
+
+/**
+ * This message 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.
+ */
+message ContainerState {
+  required ExecutorInfo executor_info = 1;
+
+  // Container id of the last executor run.
+  required ContainerID container_id = 2;
+
+  required uint64 pid = 3;            // Executor pid.
+  required string directory = 4;      // Executor work directory.
+}
+
+
+/**
+ * The container configuration that will be passed to each isolator
+ * during `prepare`.
+ */
+message ContainerConfig {
+  // The executor associated with this container.
+  required ExecutorInfo executor_info = 8;
+
+  // The task (optional) associated with this container.
+  optional TaskInfo task_info = 9;
+
+  // The command used to launch the container.
+  required CommandInfo command_info = 11;
+
+  // The information about the container.
+  optional ContainerInfo container_info = 12;
+
+  // Resources associated with the container during launch.
+  repeated Resource resources = 13;
+
+  // The work directory for the container in the host filesystem.
+  required string directory = 3;
+
+  // The user the task will be run as.
+  optional string user = 4;
+
+  // NOTE: 'rootfs' and 'docker' below are for the executor in custom
+  // executor case, and they are for the task in command task case.
+
+  // The root filesystem for the container.
+  optional string rootfs = 5;
+
+  // Docker v1 image manifest.
+  message Docker {
+    optional docker.spec.v1.ImageManifest manifest = 1;
+  }
+
+  optional Docker docker = 7;
+
+  message Appc {
+    optional appc.spec.ImageManifest manifest = 1;
+  }
+
+  optional Appc appc = 10;
+}
+
+
+/**
+ * Protobuf returned by Isolator::prepare(). The command is executed
+ * by the Launcher in the containerized context.
+ * Note: Currently, any URIs or Environment in the CommandInfo will be
+ * ignored; only the command value is used. Further, we only accept
+ * shell commands for the preparation commands.
+ */
+message ContainerLaunchInfo {
+  // The additional preparation commands to execute before
+  // executing the command.
+  repeated CommandInfo pre_exec_commands = 1;
+  optional Environment environment = 2;
+
+  // The root filesystem for the container.
+  //
+  // This field is not used since 0.28, and it is deprecated
+  // since 1.0.1.
+  optional string rootfs = 3 [deprecated = true]; // Since 1.0.1.
+
+  // (Linux only) The namespaces required for the container.
+  // The namespaces are created while launching the executor.
+  optional uint32 namespaces = 4 [default = 0];
+
+  // If specified, it'll become the launch command for the custom
+  // executor, or the launch command for the user task in the case of
+  // a command task.
+  optional CommandInfo command = 5;
+
+  // The working directory for the container.
+  // NOTE: This is different than Mesos sandbox.
+  optional string working_directory = 6;
+}
+
+
+/**
+ * Information about a container termination, returned by the
+ * containerizer to the slave.
+ */
+message ContainerTermination {
+  // Exit status of the process.
+  optional int32 status = 3;
+
+  // The 'state', 'reasons' and 'message' of a status update for
+  // non-terminal tasks when the executor is terminated.
+  optional TaskState state = 4;
+  repeated TaskStatus.Reason reasons = 5;
+  optional string message = 2;
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/include/mesos/slave/isolator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/slave/isolator.hpp b/include/mesos/slave/isolator.hpp
index ea1fa4f..d5880ee 100644
--- a/include/mesos/slave/isolator.hpp
+++ b/include/mesos/slave/isolator.hpp
@@ -22,8 +22,7 @@
 
 #include <mesos/resources.hpp>
 
-// ONLY USEFUL AFTER RUNNING PROTOC.
-#include <mesos/slave/isolator.pb.h>
+#include <mesos/slave/containerizer.hpp>
 
 #include <process/dispatch.hpp>
 #include <process/future.hpp>

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/include/mesos/slave/isolator.proto
----------------------------------------------------------------------
diff --git a/include/mesos/slave/isolator.proto b/include/mesos/slave/isolator.proto
deleted file mode 100644
index 83c748d..0000000
--- a/include/mesos/slave/isolator.proto
+++ /dev/null
@@ -1,143 +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.
-
-package mesos.slave;
-
-import "mesos/mesos.proto";
-
-import "mesos/appc/spec.proto";
-
-import "mesos/docker/v1.proto";
-
-/**
- * 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.
- */
-message ContainerLimitation {
-  // Resources that triggered the limitation.
-  // NOTE: 'Resources' is used here because the resource may span
-  // multiple roles (e.g. `"mem(*):1;mem(role):2"`).
-  repeated Resource resources = 1;
-
-  // Description of the limitation.
-  optional string message = 2;
-
-  // The container will be terminated when a resource limitation is
-  // reached. This field specifies the 'reason' that will be sent in
-  // the status update for any remaining non-terminal tasks when the
-  // container is terminated.
-  optional TaskStatus.Reason reason = 3;
-}
-
-
-/**
- * This message 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.
- */
-message ContainerState {
-  required ExecutorInfo executor_info = 1;
-
-  // Container id of the last executor run.
-  required ContainerID container_id = 2;
-
-  required uint64 pid = 3;            // Executor pid.
-  required string directory = 4;      // Executor work directory.
-}
-
-
-/**
- * The container configuration that will be passed to each isolator
- * during `prepare`.
- */
-message ContainerConfig {
-  // The executor associated with this container.
-  required ExecutorInfo executor_info = 8;
-
-  // The task (optional) associated with this container.
-  optional TaskInfo task_info = 9;
-
-  // The command used to launch the container.
-  required CommandInfo command_info = 11;
-
-  // The information about the container.
-  optional ContainerInfo container_info = 12;
-
-  // Resources associated with the container during launch.
-  repeated Resource resources = 13;
-
-  // The work directory for the container in the host filesystem.
-  required string directory = 3;
-
-  // The user the task will be run as.
-  optional string user = 4;
-
-  // NOTE: 'rootfs' and 'docker' below are for the executor in custom
-  // executor case, and they are for the task in command task case.
-
-  // The root filesystem for the container.
-  optional string rootfs = 5;
-
-  // Docker v1 image manifest.
-  message Docker {
-    optional docker.spec.v1.ImageManifest manifest = 1;
-  }
-
-  optional Docker docker = 7;
-
-  message Appc {
-    optional appc.spec.ImageManifest manifest = 1;
-  }
-
-  optional Appc appc = 10;
-}
-
-
-/**
- * Protobuf returned by Isolator::prepare(). The command is executed
- * by the Launcher in the containerized context.
- * Note: Currently, any URIs or Environment in the CommandInfo will be
- * ignored; only the command value is used. Further, we only accept
- * shell commands for the preparation commands.
- */
-message ContainerLaunchInfo {
-  // The additional preparation commands to execute before
-  // executing the command.
-  repeated CommandInfo pre_exec_commands = 1;
-  optional Environment environment = 2;
-
-  // The root filesystem for the container.
-  //
-  // This field is not used since 0.28, and it is deprecated
-  // since 1.0.1.
-  optional string rootfs = 3 [deprecated = true]; // Since 1.0.1.
-
-  // (Linux only) The namespaces required for the container.
-  // The namespaces are created while launching the executor.
-  optional uint32 namespaces = 4 [default = 0];
-
-  // If specified, it'll become the launch command for the custom
-  // executor, or the launch command for the user task in the case of
-  // a command task.
-  optional CommandInfo command = 5;
-
-  // The working directory for the container.
-  // NOTE: This is different than Mesos sandbox.
-  optional string working_directory = 6;
-}

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 7ffe377..9668fcf 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -25,13 +25,12 @@ PROTOC_TO_INCLUDE_DIR(ALLOCATOR        mesos/allocator/allocator)
 PROTOC_TO_INCLUDE_DIR(APPC_SPEC        mesos/appc/spec)
 PROTOC_TO_INCLUDE_DIR(AUTHENTICATION   mesos/authentication/authentication)
 PROTOC_TO_INCLUDE_DIR(AUTHORIZATION    mesos/authorizer/authorizer)
-PROTOC_TO_INCLUDE_DIR(CONTAINERIZER    mesos/containerizer/containerizer)
+PROTOC_TO_INCLUDE_DIR(CONTAINERIZER    mesos/slave/containerizer)
 PROTOC_TO_INCLUDE_DIR(DOCKER_SPEC      mesos/docker/spec)
 PROTOC_TO_INCLUDE_DIR(DOCKER_V1        mesos/docker/v1)
 PROTOC_TO_INCLUDE_DIR(DOCKER_V2        mesos/docker/v2)
 PROTOC_TO_INCLUDE_DIR(EXECUTOR         mesos/executor/executor)
 PROTOC_TO_INCLUDE_DIR(FETCHER          mesos/fetcher/fetcher)
-PROTOC_TO_INCLUDE_DIR(ISOLATOR         mesos/slave/isolator)
 PROTOC_TO_INCLUDE_DIR(MAINTENANCE      mesos/maintenance/maintenance)
 PROTOC_TO_INCLUDE_DIR(MASTER           mesos/master/master)
 PROTOC_TO_INCLUDE_DIR(MESOS            mesos/mesos)
@@ -76,7 +75,6 @@ set(MESOS_PROTOBUF_SRC
   ${INTERNAL_LOG_PROTO_CC}
   ${INTERNAL_MESSAGES_PROTO_CC}
   ${INTERNAL_STATE_PROTO_CC}
-  ${ISOLATOR_PROTO_CC}
   ${ISOLATOR_CNI_SPEC_PROTO_CC}
   ${ISOLATOR_DOCKER_VOLUME_STATE_PROTO_CC}
   ${MAINTENANCE_PROTO_CC}

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 9734460..dd24f53 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -216,7 +216,7 @@ ALLOCATOR_PROTO = $(top_srcdir)/include/mesos/allocator/allocator.proto
 APPC_SPEC_PROTO = $(top_srcdir)/include/mesos/appc/spec.proto
 AUTHENTICATION_PROTO = $(top_srcdir)/include/mesos/authentication/authentication.proto
 AUTHORIZATION_PROTO = $(top_srcdir)/include/mesos/authorizer/authorizer.proto
-CONTAINERIZER_PROTO = $(top_srcdir)/include/mesos/containerizer/containerizer.proto
+CONTAINERIZER_PROTO = $(top_srcdir)/include/mesos/slave/containerizer.proto
 DOCKER_SPEC_PROTO = $(top_srcdir)/include/mesos/docker/spec.proto
 DOCKER_V1_PROTO = $(top_srcdir)/include/mesos/docker/v1.proto
 DOCKER_V2_PROTO = $(top_srcdir)/include/mesos/docker/v2.proto
@@ -229,7 +229,6 @@ MODULE_PROTO = $(top_srcdir)/include/mesos/module/module.proto
 QUOTA_PROTO = $(top_srcdir)/include/mesos/quota/quota.proto
 SCHEDULER_PROTO = $(top_srcdir)/include/mesos/scheduler/scheduler.proto
 STATE_PROTO = $(top_srcdir)/include/mesos/state/state.proto
-ISOLATOR_PROTO = $(top_srcdir)/include/mesos/slave/isolator.proto
 OVERSUBSCRIPTION_PROTO = $(top_srcdir)/include/mesos/slave/oversubscription.proto
 URI_PROTO = $(top_srcdir)/include/mesos/uri/uri.proto
 V1_AGENT_PROTO = $(top_srcdir)/include/mesos/v1/agent/agent.proto
@@ -257,8 +256,6 @@ CXX_PROTOS =								\
   ../include/mesos/authorizer/acls.pb.h					\
   ../include/mesos/authorizer/authorizer.pb.cc				\
   ../include/mesos/authorizer/authorizer.pb.h				\
-  ../include/mesos/containerizer/containerizer.pb.cc			\
-  ../include/mesos/containerizer/containerizer.pb.h			\
   ../include/mesos/docker/spec.pb.cc					\
   ../include/mesos/docker/spec.pb.h					\
   ../include/mesos/docker/v1.pb.cc					\
@@ -279,8 +276,8 @@ CXX_PROTOS =								\
   ../include/mesos/quota/quota.pb.h					\
   ../include/mesos/scheduler/scheduler.pb.cc				\
   ../include/mesos/scheduler/scheduler.pb.h				\
-  ../include/mesos/slave/isolator.pb.cc					\
-  ../include/mesos/slave/isolator.pb.h					\
+  ../include/mesos/slave/containerizer.pb.cc				\
+  ../include/mesos/slave/containerizer.pb.h				\
   ../include/mesos/slave/oversubscription.pb.cc				\
   ../include/mesos/slave/oversubscription.pb.h				\
   ../include/mesos/state/state.pb.cc					\
@@ -328,13 +325,11 @@ CXX_STATE_PROTOS =							\
 
 JAVA_PROTOS =								\
   java/generated/org/apache/mesos/Protos.java				\
-  java/generated/org/apache/mesos/containerizer/Protos.java             \
   java/generated/org/apache/mesos/executor/Protos.java			\
   java/generated/org/apache/mesos/scheduler/Protos.java
 
 PYTHON_PROTOS =								\
-  python/interface/src/mesos/interface/mesos_pb2.py			\
-  python/interface/src/mesos/interface/containerizer_pb2.py
+  python/interface/src/mesos/interface/mesos_pb2.py
 
 V1_JAVA_PROTOS =							\
   java/generated/org/apache/mesos/v1/Protos.java			\
@@ -374,10 +369,6 @@ CLEANFILES +=								\
 
 
 # Targets for generating Java protocol buffer code.
-java/generated/org/apache/mesos/containerizer/Protos.java: $(CONTAINERIZER_PROTO)
-	$(MKDIR_P) $(@D)
-	$(PROTOC) $(PROTOCFLAGS) --java_out=java/generated $^
-
 java/generated/org/apache/mesos/executor/Protos.java: $(EXECUTOR_PROTO)
 	$(MKDIR_P) $(@D)
 	$(PROTOC) $(PROTOCFLAGS) --java_out=java/generated $^
@@ -412,12 +403,6 @@ java/generated/org/apache/mesos/v1/scheduler/Protos.java: $(V1_SCHEDULER_PROTO)
 # has to get patched using sed, removing the leading 'mesos.'
 # namespace qualifier due to the flat hierachy in the resulting
 # mesos.interface egg. See MESOS-1750 for more.
-python/interface/src/mesos/interface/containerizer_pb2.py: $(CONTAINERIZER_PROTO)
-	$(MKDIR_P) $(@D)
-	$(PROTOC) -I$(top_srcdir)/include/mesos/containerizer $(PROTOCFLAGS)	\
-		--python_out=python/interface/src/mesos/interface $^
-	$(SED) -i.bak 's/mesos\.mesos_pb2/mesos_pb2/' $@ && rm $@.bak
-
 python/interface/src/mesos/interface/executor_pb2.py: $(EXECUTOR_PROTO)
 	$(MKDIR_P) $(@D)
 	$(PROTOC) -I$(top_srcdir)/include/mesos/executor $(PROTOCFLAGS)		\
@@ -513,15 +498,6 @@ nodist_authorizer_HEADERS =						\
   ../include/mesos/authorizer/acls.pb.h					\
   ../include/mesos/authorizer/authorizer.pb.h
 
-containerizerdir = $(pkgincludedir)/containerizer
-
-containerizer_HEADERS =							\
-  $(top_srcdir)/include/mesos/containerizer/containerizer.hpp		\
-  $(top_srcdir)/include/mesos/containerizer/containerizer.proto
-
-nodist_containerizer_HEADERS =						\
-  ../include/mesos/containerizer/containerizer.pb.h
-
 dockerdir = $(pkgincludedir)/docker
 
 docker_HEADERS =							\
@@ -624,10 +600,9 @@ agentdir = $(pkgincludedir)/agent
 
 agent_HEADERS =								\
   $(top_srcdir)/include/mesos/agent/agent.hpp				\
-  $(top_srcdir)/include/mesos/agent/agent.proto			\
+  $(top_srcdir)/include/mesos/agent/agent.proto				\
   $(top_srcdir)/include/mesos/slave/container_logger.hpp		\
   $(top_srcdir)/include/mesos/slave/isolator.hpp			\
-  $(top_srcdir)/include/mesos/slave/isolator.proto			\
   $(top_srcdir)/include/mesos/slave/oversubscription.hpp		\
   $(top_srcdir)/include/mesos/slave/oversubscription.proto		\
   $(top_srcdir)/include/mesos/slave/qos_controller.hpp			\
@@ -635,7 +610,7 @@ agent_HEADERS =								\
 
 nodist_agent_HEADERS =							\
   ../include/mesos/agent/agent.pb.h					\
-  ../include/mesos/slave/isolator.pb.h					\
+  ../include/mesos/slave/containerizer.pb.h				\
   ../include/mesos/slave/oversubscription.pb.h
 
 statedir = $(pkgincludedir)/state
@@ -1274,7 +1249,7 @@ lib_LTLIBRARIES += libmesos.la
 # Include as part of the distribution.
 libmesos_la_SOURCES =							\
   $(ACLS_PROTO)								\
-  $(AGENT_PROTO)								\
+  $(AGENT_PROTO)							\
   $(ALLOCATOR_PROTO)							\
   $(APPC_SPEC_PROTO)							\
   $(AUTHENTICATION_PROTO)						\
@@ -1285,7 +1260,6 @@ libmesos_la_SOURCES =							\
   $(DOCKER_V1_PROTO)							\
   $(DOCKER_V2_PROTO)							\
   $(FETCHER_PROTO)							\
-  $(ISOLATOR_PROTO)							\
   $(MAINTENANCE_PROTO)							\
   $(MASTER_PROTO)							\
   $(MESOS_PROTO)							\
@@ -1590,7 +1564,6 @@ libjava_la_CPPFLAGS += $(JAVA_CPPFLAGS)
 libjava_la_CPPFLAGS += -I$(srcdir)/java/jni -Ijava/jni
 
 libjava_la_DEPENDENCIES =						\
-  $(CONTAINERIZER_PROTO)						\
   $(MESOS_PROTO)							\
   $(V1_EXECUTOR_PROTO)							\
   $(V1_MESOS_PROTO)							\

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/slave/containerizer/composing.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.cpp b/src/slave/containerizer/composing.cpp
index c51cb4b..5ff3e65 100644
--- a/src/slave/containerizer/composing.cpp
+++ b/src/slave/containerizer/composing.cpp
@@ -32,12 +32,14 @@
 #include "slave/containerizer/containerizer.hpp"
 #include "slave/containerizer/composing.hpp"
 
+using namespace process;
+
 using std::list;
 using std::map;
 using std::string;
 using std::vector;
 
-using namespace process;
+using mesos::slave::ContainerTermination;
 
 namespace mesos {
 namespace internal {
@@ -78,7 +80,7 @@ public:
   Future<ContainerStatus> status(
       const ContainerID& containerId);
 
-  Future<containerizer::Termination> wait(
+  Future<ContainerTermination> wait(
       const ContainerID& containerId);
 
   void destroy(const ContainerID& containerId);
@@ -204,7 +206,7 @@ Future<ContainerStatus> ComposingContainerizer::status(
 }
 
 
-Future<containerizer::Termination> ComposingContainerizer::wait(
+Future<ContainerTermination> ComposingContainerizer::wait(
     const ContainerID& containerId)
 {
   return dispatch(process, &ComposingContainerizerProcess::wait, containerId);
@@ -434,7 +436,7 @@ Future<ContainerStatus> ComposingContainerizerProcess::status(
 }
 
 
-Future<containerizer::Termination> ComposingContainerizerProcess::wait(
+Future<ContainerTermination> ComposingContainerizerProcess::wait(
     const ContainerID& containerId)
 {
   if (!containers_.contains(containerId)) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/slave/containerizer/composing.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.hpp b/src/slave/containerizer/composing.hpp
index 406015a..ef3c2ee 100644
--- a/src/slave/containerizer/composing.hpp
+++ b/src/slave/containerizer/composing.hpp
@@ -22,8 +22,6 @@
 #include <mesos/mesos.hpp>
 #include <mesos/resources.hpp>
 
-#include <mesos/containerizer/containerizer.hpp>
-
 #include <process/future.hpp>
 #include <process/process.hpp>
 
@@ -32,7 +30,6 @@
 #include <stout/option.hpp>
 #include <stout/try.hpp>
 
-
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -40,6 +37,7 @@ namespace slave {
 // Forward declaration.
 class ComposingContainerizerProcess;
 
+
 class ComposingContainerizer : public Containerizer
 {
 public:
@@ -74,7 +72,7 @@ public:
   virtual process::Future<ContainerStatus> status(
       const ContainerID& containerId);
 
-  virtual process::Future<containerizer::Termination> wait(
+  virtual process::Future<mesos::slave::ContainerTermination> wait(
       const ContainerID& containerId);
 
   virtual void destroy(const ContainerID& containerId);

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/slave/containerizer/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
index bf39aaa..f13669d 100644
--- a/src/slave/containerizer/containerizer.hpp
+++ b/src/slave/containerizer/containerizer.hpp
@@ -22,7 +22,7 @@
 #include <mesos/mesos.hpp>
 #include <mesos/resources.hpp>
 
-#include <mesos/containerizer/containerizer.hpp>
+#include <mesos/slave/containerizer.hpp>
 
 #include <process/future.hpp>
 #include <process/owned.hpp>
@@ -120,12 +120,11 @@ public:
     return ContainerStatus();
   }
 
-  // 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<containerizer::Termination> wait(
+  // Wait on the 'ContainerTermination'. 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<mesos::slave::ContainerTermination> wait(
       const ContainerID& containerId) = 0;
 
   // Destroy a running container, killing all processes and releasing

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index e447c58..8ecd773 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -60,35 +60,37 @@
 
 #include "usage/usage.hpp"
 
+using namespace process;
 
 using std::list;
 using std::map;
 using std::string;
 using std::vector;
 
-using namespace process;
-
 using mesos::slave::ContainerLogger;
+using mesos::slave::ContainerTermination;
+
+using mesos::internal::slave::state::SlaveState;
+using mesos::internal::slave::state::FrameworkState;
+using mesos::internal::slave::state::ExecutorState;
+using mesos::internal::slave::state::RunState;
 
 namespace mesos {
 namespace internal {
 namespace slave {
 
-using state::SlaveState;
-using state::FrameworkState;
-using state::ExecutorState;
-using state::RunState;
-
-
 // Declared in header, see explanation there.
 const string DOCKER_NAME_PREFIX = "mesos-";
 
+
 // Declared in header, see explanation there.
 const string DOCKER_NAME_SEPERATOR = ".";
 
+
 // Declared in header, see explanation there.
 const string DOCKER_SYMLINK_DIRECTORY = "docker/links";
 
+
 // Parse the ContainerID from a Docker container and return None if
 // the container was not launched from Mesos.
 Option<ContainerID> parse(const Docker::Container& container)
@@ -712,7 +714,7 @@ Future<ResourceStatistics> DockerContainerizer::usage(
 }
 
 
-Future<containerizer::Termination> DockerContainerizer::wait(
+Future<ContainerTermination> DockerContainerizer::wait(
     const ContainerID& containerId)
 {
   return dispatch(
@@ -809,8 +811,8 @@ Future<Nothing> DockerContainerizerProcess::_recover(
         // 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.
+        // which will return a failed 'ContainerTermination' and
+        // everything will get cleaned up.
         if (!run.get().forkedPid.isSome()) {
           continue;
         }
@@ -1779,7 +1781,7 @@ Try<ResourceStatistics> DockerContainerizerProcess::cgroupsStatistics(
 }
 
 
-Future<containerizer::Termination> DockerContainerizerProcess::wait(
+Future<ContainerTermination> DockerContainerizerProcess::wait(
     const ContainerID& containerId)
 {
   CHECK(!containerId.has_parent());
@@ -1814,7 +1816,7 @@ void DockerContainerizerProcess::destroy(
 
     // NOTE: The launch error message will be retrieved by the slave
     // and properly set in the corresponding status update.
-    container->termination.set(containerizer::Termination());
+    container->termination.set(ContainerTermination());
 
     containers_.erase(containerId);
     delete container;
@@ -1856,7 +1858,7 @@ void DockerContainerizerProcess::destroy(
 
     fetcher->kill(containerId);
 
-    containerizer::Termination termination;
+    ContainerTermination termination;
     termination.set_message("Container destroyed while fetching");
     container->termination.set(termination);
 
@@ -1875,7 +1877,7 @@ void DockerContainerizerProcess::destroy(
 
     container->pull.discard();
 
-    containerizer::Termination termination;
+    ContainerTermination termination;
     termination.set_message("Container destroyed while pulling image");
     container->termination.set(termination);
 
@@ -1898,7 +1900,7 @@ void DockerContainerizerProcess::destroy(
                    << unmount.error();
     }
 
-    containerizer::Termination termination;
+    ContainerTermination termination;
     termination.set_message("Container destroyed while mounting volumes");
     container->termination.set(termination);
 
@@ -2037,7 +2039,7 @@ void DockerContainerizerProcess::___destroy(
 
   Container* container = containers_[containerId];
 
-  containerizer::Termination termination;
+  ContainerTermination termination;
 
   if (status.isReady() && status.get().isSome()) {
     termination.set_status(status.get().get());

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/slave/containerizer/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.hpp b/src/slave/containerizer/docker.hpp
index 6597d86..b172a8e 100644
--- a/src/slave/containerizer/docker.hpp
+++ b/src/slave/containerizer/docker.hpp
@@ -96,7 +96,7 @@ public:
   virtual process::Future<ResourceStatistics> usage(
       const ContainerID& containerId);
 
-  virtual process::Future<containerizer::Termination> wait(
+  virtual process::Future<mesos::slave::ContainerTermination> wait(
       const ContainerID& containerId);
 
   virtual void destroy(const ContainerID& containerId);
@@ -146,7 +146,7 @@ public:
   virtual process::Future<ResourceStatistics> usage(
       const ContainerID& containerId);
 
-  virtual process::Future<containerizer::Termination> wait(
+  virtual process::Future<mesos::slave::ContainerTermination> wait(
       const ContainerID& containerId);
 
   virtual void destroy(
@@ -438,7 +438,7 @@ private:
     const Flags flags;
 
     // Promise for future returned from wait().
-    process::Promise<containerizer::Termination> termination;
+    process::Promise<mesos::slave::ContainerTermination> termination;
 
     // Exit status of executor or container (depending on whether or
     // not we used the command executor). Represented as a promise so

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 687d9bb..808fc83 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -133,6 +133,7 @@ using mesos::slave::ContainerLaunchInfo;
 using mesos::slave::ContainerLimitation;
 using mesos::slave::ContainerLogger;
 using mesos::slave::ContainerState;
+using mesos::slave::ContainerTermination;
 using mesos::slave::Isolator;
 
 using state::SlaveState;
@@ -510,7 +511,7 @@ Future<ContainerStatus> MesosContainerizer::status(
 }
 
 
-Future<containerizer::Termination> MesosContainerizer::wait(
+Future<ContainerTermination> MesosContainerizer::wait(
     const ContainerID& containerId)
 {
   return dispatch(process.get(), &MesosContainerizerProcess::wait, containerId);
@@ -562,10 +563,11 @@ Future<Nothing> MesosContainerizerProcess::recover(
         CHECK_SOME(run);
         CHECK_SOME(run.get().id);
 
-        // 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.
+        // 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 ContainerTermination and
+        // everything will get cleaned up.
         if (!run.get().forkedPid.isSome()) {
           continue;
         }
@@ -1357,7 +1359,7 @@ Future<Nothing> MesosContainerizerProcess::launch(
 }
 
 
-Future<containerizer::Termination> MesosContainerizerProcess::wait(
+Future<ContainerTermination> MesosContainerizerProcess::wait(
     const ContainerID& containerId)
 {
   CHECK(!containerId.has_parent());
@@ -1759,7 +1761,7 @@ void MesosContainerizerProcess::_____destroy(
     return;
   }
 
-  containerizer::Termination termination;
+  ContainerTermination termination;
 
   if (status.isReady() && status->isSome()) {
     termination.set_status(status->get());

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 6bd16e6..0cfe7b0 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -100,7 +100,7 @@ public:
   virtual process::Future<ContainerStatus> status(
       const ContainerID& containerId);
 
-  virtual process::Future<containerizer::Termination> wait(
+  virtual process::Future<mesos::slave::ContainerTermination> wait(
       const ContainerID& containerId);
 
   virtual void destroy(const ContainerID& containerId);
@@ -164,7 +164,7 @@ public:
   virtual process::Future<ContainerStatus> status(
       const ContainerID& containerId);
 
-  virtual process::Future<containerizer::Termination> wait(
+  virtual process::Future<mesos::slave::ContainerTermination> wait(
       const ContainerID& containerId);
 
   virtual process::Future<bool> exec(
@@ -289,7 +289,7 @@ private:
     Container() : sequence("mesos-container-status-updates") {}
 
     // Promise for futures returned from wait().
-    process::Promise<containerizer::Termination> promise;
+    process::Promise<mesos::slave::ContainerTermination> promise;
 
     // We need to keep track of the future exit status for each
     // executor because we'll only get a single notification when

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 3688f42..c686a97 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -104,6 +104,7 @@ using mesos::executor::Call;
 
 using mesos::master::detector::MasterDetector;
 
+using mesos::slave::ContainerTermination;
 using mesos::slave::QoSController;
 using mesos::slave::QoSCorrection;
 using mesos::slave::ResourceEstimator;
@@ -1914,7 +1915,7 @@ void Slave::runTasks(
 
     Executor* executor = getExecutor(frameworkId, executorId);
     if (executor != nullptr) {
-      containerizer::Termination termination;
+      ContainerTermination termination;
       termination.set_state(TASK_LOST);
       termination.add_reasons(TaskStatus::REASON_CONTAINER_UPDATE_FAILED);
       termination.set_message(
@@ -3182,7 +3183,7 @@ void Slave::_reregisterExecutor(
 
     Executor* executor = getExecutor(frameworkId, executorId);
     if (executor != nullptr) {
-      containerizer::Termination termination;
+      ContainerTermination termination;
       termination.set_state(TASK_LOST);
       termination.add_reasons(TaskStatus::REASON_CONTAINER_UPDATE_FAILED);
       termination.set_message(
@@ -3225,7 +3226,7 @@ void Slave::reregisterExecutorTimeout()
 
           executor->state = Executor::TERMINATING;
 
-          containerizer::Termination termination;
+          ContainerTermination termination;
           termination.set_state(TASK_LOST);
           termination.add_reasons(
               TaskStatus::REASON_EXECUTOR_REREGISTRATION_TIMEOUT);
@@ -3516,7 +3517,7 @@ void Slave::__statusUpdate(
 
     Executor* executor = getExecutor(update.framework_id(), executorId);
     if (executor != nullptr) {
-      containerizer::Termination termination;
+      ContainerTermination termination;
       termination.set_state(TASK_LOST);
       termination.add_reasons(TaskStatus::REASON_CONTAINER_UPDATE_FAILED);
       termination.set_message(
@@ -4038,7 +4039,7 @@ void Slave::executorLaunched(
 
     Executor* executor = getExecutor(frameworkId, executorId);
     if (executor != nullptr) {
-      containerizer::Termination termination;
+      ContainerTermination termination;
       termination.set_state(TASK_FAILED);
       termination.add_reasons(TaskStatus::REASON_CONTAINER_LAUNCH_FAILED);
       termination.set_message(
@@ -4115,7 +4116,7 @@ void Slave::executorLaunched(
 void Slave::executorTerminated(
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
-    const Future<containerizer::Termination>& termination)
+    const Future<ContainerTermination>& termination)
 {
   int status;
   // A termination failure indicates the containerizer could not destroy a
@@ -4598,7 +4599,7 @@ void Slave::registerExecutorTimeout(
 
       executor->state = Executor::TERMINATING;
 
-      containerizer::Termination termination;
+      ContainerTermination termination;
       termination.set_state(TASK_FAILED);
       termination.add_reasons(TaskStatus::REASON_EXECUTOR_REGISTRATION_TIMEOUT);
       termination.set_message(
@@ -5215,7 +5216,7 @@ void Slave::_qosCorrections(const Future<list<QoSCorrection>>& future)
           // (MESOS-2875).
           executor->state = Executor::TERMINATING;
 
-          containerizer::Termination termination;
+          ContainerTermination termination;
           termination.set_state(TASK_LOST);
           termination.add_reasons(TaskStatus::REASON_CONTAINER_PREEMPTED);
           termination.set_message("Container preempted by QoS correction");
@@ -5509,7 +5510,7 @@ Future<bool> Slave::authorizeSandboxAccess(
 
 void Slave::sendExecutorTerminatedStatusUpdate(
     const TaskID& taskId,
-    const Future<containerizer::Termination>& termination,
+    const Future<ContainerTermination>& termination,
     const FrameworkID& frameworkId,
     const Executor* executor)
 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 9f29d80..7ca9923 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -274,7 +274,7 @@ public:
   void executorTerminated(
       const FrameworkID& frameworkId,
       const ExecutorID& executorId,
-      const process::Future<containerizer::Termination>& termination);
+      const process::Future<mesos::slave::ContainerTermination>& termination);
 
   // NOTE: Pulled these to public to make it visible for testing.
   // TODO(vinod): Make tests friends to this class instead.
@@ -635,7 +635,7 @@ private:
 
   void sendExecutorTerminatedStatusUpdate(
       const TaskID& taskId,
-      const Future<containerizer::Termination>& termination,
+      const Future<mesos::slave::ContainerTermination>& termination,
       const FrameworkID& frameworkId,
       const Executor* executor);
 
@@ -903,7 +903,7 @@ struct Executor
   // slave initiated the destruction and will influence the
   // information sent in the status updates for any remaining
   // non-terminal tasks.
-  Option<containerizer::Termination> pendingTermination;
+  Option<mesos::slave::ContainerTermination> pendingTermination;
 
 private:
   Executor(const Executor&);              // No copying.

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/tests/cluster.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cluster.cpp b/src/tests/cluster.cpp
index dd41544..b04653a 100644
--- a/src/tests/cluster.cpp
+++ b/src/tests/cluster.cpp
@@ -96,6 +96,8 @@ using mesos::master::detector::MasterDetector;
 using mesos::master::detector::StandaloneMasterDetector;
 using mesos::master::detector::ZooKeeperMasterDetector;
 
+using mesos::slave::ContainerTermination;
+
 namespace mesos {
 namespace internal {
 namespace tests {
@@ -550,7 +552,7 @@ Slave::~Slave()
     AWAIT_READY(containers);
 
     foreach (const ContainerID& containerId, containers.get()) {
-      process::Future<containerizer::Termination> wait =
+      process::Future<ContainerTermination> wait =
         containerizer->wait(containerId);
 
       containerizer->destroy(containerId);

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/tests/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.cpp b/src/tests/containerizer.cpp
index 761b1ed..bda3e6f 100644
--- a/src/tests/containerizer.cpp
+++ b/src/tests/containerizer.cpp
@@ -22,6 +22,8 @@
 
 #include "tests/mesos.hpp"
 
+using namespace process;
+
 using std::map;
 using std::shared_ptr;
 using std::string;
@@ -30,7 +32,7 @@ using testing::_;
 using testing::Invoke;
 using testing::Return;
 
-using namespace process;
+using mesos::slave::ContainerTermination;
 
 using mesos::v1::executor::Mesos;
 
@@ -38,7 +40,6 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
-
 TestContainerizer::TestContainerizer(
     const ExecutorID& executorId,
     const shared_ptr<MockV1HTTPExecutor>& executor)
@@ -186,14 +187,14 @@ Future<bool> TestContainerizer::_launch(
   }
 
   promises[containerId] =
-    Owned<Promise<containerizer::Termination>>(
-      new Promise<containerizer::Termination>());
+    Owned<Promise<ContainerTermination>>(
+      new Promise<ContainerTermination>());
 
   return true;
 }
 
 
-Future<containerizer::Termination> TestContainerizer::_wait(
+Future<ContainerTermination> TestContainerizer::_wait(
     const ContainerID& containerId)
 {
   // An unknown container is possible for tests where we "drop" the
@@ -234,7 +235,7 @@ void TestContainerizer::_destroy(const ContainerID& containerId)
   }
 
   if (promises.contains(containerId)) {
-    containerizer::Termination termination;
+    ContainerTermination termination;
     termination.set_message("Killed executor");
     termination.set_status(0);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/tests/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.hpp b/src/tests/containerizer.hpp
index cd7eeb0..f1fd579 100644
--- a/src/tests/containerizer.hpp
+++ b/src/tests/containerizer.hpp
@@ -103,7 +103,7 @@ public:
 
   MOCK_METHOD1(
       wait,
-      process::Future<containerizer::Termination>(const ContainerID&));
+      process::Future<mesos::slave::ContainerTermination>(const ContainerID&));
 
   MOCK_METHOD1(
       destroy,
@@ -127,7 +127,7 @@ private:
       const std::map<std::string, std::string>& environment,
       bool checkpoint);
 
-  process::Future<containerizer::Termination> _wait(
+  process::Future<mesos::slave::ContainerTermination> _wait(
       const ContainerID& containerId);
 
   void _destroy(const ContainerID& containerID);
@@ -138,8 +138,8 @@ private:
   hashmap<std::pair<FrameworkID, ExecutorID>, ContainerID> containers_;
   hashmap<ContainerID, process::Owned<MesosExecutorDriver>> drivers;
   hashmap<ContainerID, process::Owned<executor::TestV1Mesos>> v1Libraries;
-  hashmap<ContainerID,
-      process::Owned<process::Promise<containerizer::Termination>>> promises;
+  hashmap<ContainerID, process::Owned<
+      process::Promise<mesos::slave::ContainerTermination>>> promises;
 };
 
 } // namespace tests {

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/tests/containerizer/composing_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/composing_containerizer_tests.cpp b/src/tests/containerizer/composing_containerizer_tests.cpp
index 5a9d5e4..51aab33 100644
--- a/src/tests/containerizer/composing_containerizer_tests.cpp
+++ b/src/tests/containerizer/composing_containerizer_tests.cpp
@@ -41,6 +41,8 @@ using std::vector;
 using testing::_;
 using testing::Return;
 
+using mesos::slave::ContainerTermination;
+
 namespace mesos {
 namespace internal {
 namespace tests {
@@ -81,7 +83,7 @@ public:
 
   MOCK_METHOD1(
       wait,
-      process::Future<containerizer::Termination>(
+      process::Future<ContainerTermination>(
           const ContainerID&));
 
   MOCK_METHOD1(

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/tests/containerizer/docker_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/docker_containerizer_tests.cpp b/src/tests/containerizer/docker_containerizer_tests.cpp
index 42d4364..28cd3fa 100644
--- a/src/tests/containerizer/docker_containerizer_tests.cpp
+++ b/src/tests/containerizer/docker_containerizer_tests.cpp
@@ -59,6 +59,7 @@ using mesos::internal::slave::Slave;
 using mesos::master::detector::MasterDetector;
 
 using mesos::slave::ContainerLogger;
+using mesos::slave::ContainerTermination;
 
 using std::list;
 using std::string;
@@ -272,7 +273,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor)
 
   ASSERT_TRUE(exists(docker, slaveId, containerId.get()));
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     dockerContainerizer.wait(containerId.get());
 
   driver.stop();
@@ -400,7 +401,7 @@ TEST_F(DockerContainerizerTest, DISABLED_ROOT_DOCKER_Launch_Executor_Bridged)
 
   ASSERT_TRUE(exists(docker, slaveId, containerId.get()));
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     dockerContainerizer.wait(containerId.get());
 
   driver.stop();
@@ -551,7 +552,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch)
 
   ASSERT_TRUE(exists(docker, slaveId, containerId.get()));
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     dockerContainerizer.wait(containerId.get());
 
   driver.stop();
@@ -662,7 +663,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Kill)
   EXPECT_CALL(sched, statusUpdate(&driver, _))
     .WillOnce(FutureArg<1>(&statusKilled));
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     dockerContainerizer.wait(containerId.get());
 
   driver.killTask(task.task_id());
@@ -788,7 +789,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_TaskKillingCapability)
     .WillOnce(FutureArg<1>(&statusKilling))
     .WillOnce(FutureArg<1>(&statusKilled));
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     dockerContainerizer.wait(containerId.get());
 
   driver.killTask(task.task_id());
@@ -937,7 +938,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Usage)
   EXPECT_LT(0, statistics.cpus_system_time_secs());
   EXPECT_GT(statistics.mem_rss_bytes(), 0u);
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     dockerContainerizer.wait(containerId.get());
 
   dockerContainerizer.destroy(containerId.get());
@@ -1229,7 +1230,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Recover)
 
   AWAIT_READY(recover);
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     dockerContainerizer.wait(containerId);
 
   ASSERT_FALSE(termination.isFailed());
@@ -1360,7 +1361,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_KillOrphanContainers)
 
   AWAIT_READY(recover);
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     dockerContainerizer.wait(containerId);
 
   ASSERT_FALSE(termination.isFailed());
@@ -1563,7 +1564,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchWithPersistentVolumes)
   AWAIT_READY(statusFinished);
   EXPECT_EQ(TASK_FINISHED, statusFinished.get().state());
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     dockerContainerizer.wait(containerId.get());
 
   driver.stop();
@@ -1733,7 +1734,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_RecoverPersistentVolumes)
   // Wait until containerizer recover is complete.
   AWAIT_READY(_recover);
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     dockerContainerizer->wait(containerId.get());
 
   dockerContainerizer->destroy(containerId.get());
@@ -2610,7 +2611,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_SlaveRecoveryTaskContainer)
 
   ASSERT_TRUE(exists(docker, slaveId, containerId.get()));
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     dockerContainerizer->wait(containerId.get());
 
   driver.stop();
@@ -2947,7 +2948,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_NC_PortMapping)
   // to stdout by the docker container running nc -l.
   EXPECT_TRUE(containsLine(lines, uuid));
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     dockerContainerizer.wait(containerId.get());
 
   driver.stop();
@@ -3055,7 +3056,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchSandboxWithColon)
 
   ASSERT_TRUE(exists(docker, slaveId, containerId.get()));
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     dockerContainerizer.wait(containerId.get());
 
   driver.stop();

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/tests/containerizer/filesystem_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/filesystem_isolator_tests.cpp b/src/tests/containerizer/filesystem_isolator_tests.cpp
index 2b216c3..9f6b2ed 100644
--- a/src/tests/containerizer/filesystem_isolator_tests.cpp
+++ b/src/tests/containerizer/filesystem_isolator_tests.cpp
@@ -77,6 +77,7 @@ using mesos::internal::slave::Store;
 using mesos::master::detector::MasterDetector;
 
 using mesos::slave::ContainerLogger;
+using mesos::slave::ContainerTermination;
 using mesos::slave::Isolator;
 
 namespace mesos {
@@ -271,9 +272,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ChangeRootFilesystem)
   AWAIT_READY_FOR(launch, Seconds(60));
 
   // Wait on the container.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.
@@ -792,9 +791,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_Metrics)
   containerizer.get()->destroy(containerId);
 
   // Wait on the container.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Executor was killed.
@@ -845,9 +842,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromSandbox)
   AWAIT_READY_FOR(launch, Seconds(60));
 
   // Wait on the container.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.
@@ -900,9 +895,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromHost)
   AWAIT_READY_FOR(launch, Seconds(60));
 
   // Wait on the container.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.
@@ -953,9 +946,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_FileVolumeFromHost)
 
   AWAIT_READY_FOR(launch, Seconds(60));
 
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.
@@ -1006,9 +997,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromHostSandboxMountPoint)
   AWAIT_READY_FOR(launch, Seconds(60));
 
   // Wait on the container.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.
@@ -1059,9 +1048,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_FileVolumeFromHostSandboxMountPoint)
 
   AWAIT_READY_FOR(launch, Seconds(60));
 
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.
@@ -1127,9 +1114,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithRootFilesystem)
   AWAIT_READY_FOR(launch, Seconds(60));
 
   // Wait on the container.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.
@@ -1201,9 +1186,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithoutRootFilesystem)
   AWAIT_READY_FOR(launch, Seconds(60));
 
   // Wait on the container.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.
@@ -1256,9 +1239,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ImageInVolumeWithoutRootFilesystem)
   AWAIT_READY_FOR(launch, Seconds(60));
 
   // Wait on the container.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.
@@ -1311,8 +1292,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ImageInVolumeWithRootFilesystem)
   AWAIT_READY_FOR(launch, Seconds(240));
 
   // Wait on the container.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
 
   // Because destroy rootfs spents a lot of time, we use 30s as timeout here.
   AWAIT_READY_FOR(wait, Seconds(30));
@@ -1426,10 +1406,8 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_MultipleContainers)
   containerizer.get()->destroy(containerId1);
 
   // Wait on the containers.
-  Future<containerizer::Termination> wait1 =
-    containerizer.get()->wait(containerId1);
-  Future<containerizer::Termination> wait2 =
-    containerizer.get()->wait(containerId2);
+  Future<ContainerTermination> wait1 = containerizer.get()->wait(containerId1);
+  Future<ContainerTermination> wait2 = containerizer.get()->wait(containerId2);
 
   AWAIT_READY_FOR(wait1, Seconds(60));
   AWAIT_READY_FOR(wait2, Seconds(60));
@@ -1499,9 +1477,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_SandboxEnvironmentVariable)
   AWAIT_READY_FOR(launch, Seconds(60));
 
   // Wait on the container.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/tests/containerizer/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index 05620d2..2725eb0 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -102,6 +102,7 @@ using mesos::master::detector::MasterDetector;
 
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::ContainerTermination;
 using mesos::slave::Isolator;
 
 using process::http::OK;
@@ -157,7 +158,7 @@ Try<Nothing> isolatePid(
   }
 
   // Isolate process.
-  process::Future<Nothing> isolate = isolator->isolate(containerId, child);
+  Future<Nothing> isolate = isolator->isolate(containerId, child);
 
   // Note this is following the implementation of AWAIT_READY.
   if (!process::internal::await(isolate, Seconds(15))) {
@@ -1534,8 +1535,7 @@ TEST_F(NamespacesPidIsolatorTest, ROOT_PidNamespace)
   ASSERT_TRUE(launch.get());
 
   // Wait on the container.
-  process::Future<containerizer::Termination> wait =
-    containerizer->wait(containerId);
+  Future<ContainerTermination> wait = containerizer->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/tests/containerizer/mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/mesos_containerizer_tests.cpp b/src/tests/containerizer/mesos_containerizer_tests.cpp
index e021b68..49f4552 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -73,6 +73,7 @@ using mesos::slave::ContainerLaunchInfo;
 using mesos::slave::ContainerLimitation;
 using mesos::slave::ContainerLogger;
 using mesos::slave::ContainerState;
+using mesos::slave::ContainerTermination;
 using mesos::slave::Isolator;
 
 using std::list;
@@ -218,9 +219,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptSucceeds)
   AWAIT_READY(launch);
 
   // Wait for the child (preparation script + executor) to complete.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the child exited correctly.
@@ -270,9 +269,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptFails)
   AWAIT_READY(launch);
 
   // Wait for the child (preparation script + executor) to complete.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the child failed to exit correctly.
@@ -333,8 +330,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
   AWAIT_READY(launch);
 
   // Wait for the child (preparation script(s) + executor) to complete.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the child failed to exit correctly.
@@ -415,9 +411,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ExecutorEnvironmentVariable)
   AWAIT_READY(launch);
 
   // Wait for the child (preparation script + executor) to complete.
-  Future<containerizer::Termination> wait =
-    containerizer.get()->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the child exited correctly.
@@ -485,9 +479,7 @@ TEST_F(MesosContainerizerExecuteTest, IoRedirection)
   AWAIT_READY(launch);
 
   // Wait on the container.
-  Future<containerizer::Termination> wait =
-    containerizer->wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.
@@ -668,8 +660,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhileFetching)
       map<string, string>(),
       false);
 
-  Future<containerizer::Termination> wait = containerizer.wait(containerId);
-
+  Future<ContainerTermination> wait = containerizer.wait(containerId);
   AWAIT_READY(exec);
 
   containerizer.destroy(containerId);
@@ -736,7 +727,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
       map<string, string>(),
       false);
 
-  Future<containerizer::Termination> wait = containerizer.wait(containerId);
+  Future<ContainerTermination> wait = containerizer.wait(containerId);
 
   AWAIT_READY(prepare);
 
@@ -753,7 +744,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
 
   AWAIT_READY(wait);
 
-  containerizer::Termination termination = wait.get();
+  ContainerTermination termination = wait.get();
 
   EXPECT_EQ(
       "Container destroyed while preparing isolators",
@@ -862,13 +853,13 @@ TEST_F(MesosContainerizerProvisionerTest, ProvisionFailed)
 
   AWAIT_FAILED(launch);
 
-  Future<containerizer::Termination> wait = containerizer.wait(containerId);
+  Future<ContainerTermination> wait = containerizer.wait(containerId);
 
   containerizer.destroy(containerId);
 
   AWAIT_READY(wait);
 
-  containerizer::Termination termination = wait.get();
+  ContainerTermination termination = wait.get();
 
   EXPECT_EQ(
     "Container destroyed while provisioning images",
@@ -954,7 +945,7 @@ TEST_F(MesosContainerizerProvisionerTest, DestroyWhileProvisioning)
       map<string, string>(),
       false);
 
-  Future<containerizer::Termination> wait = containerizer.wait(containerId);
+  Future<ContainerTermination> wait = containerizer.wait(containerId);
 
   AWAIT_READY(provision);
 
@@ -966,7 +957,7 @@ TEST_F(MesosContainerizerProvisionerTest, DestroyWhileProvisioning)
   AWAIT_FAILED(launch);
   AWAIT_READY(wait);
 
-  containerizer::Termination termination = wait.get();
+  ContainerTermination termination = wait.get();
 
   EXPECT_EQ(
     "Container destroyed while provisioning images",
@@ -1057,7 +1048,7 @@ TEST_F(MesosContainerizerProvisionerTest, IsolatorCleanupBeforePrepare)
       map<string, string>(),
       false);
 
-  Future<containerizer::Termination> wait = containerizer.wait(containerId);
+  Future<ContainerTermination> wait = containerizer.wait(containerId);
 
   AWAIT_READY(provision);
 
@@ -1069,7 +1060,7 @@ TEST_F(MesosContainerizerProvisionerTest, IsolatorCleanupBeforePrepare)
   AWAIT_FAILED(launch);
   AWAIT_READY(wait);
 
-  containerizer::Termination termination = wait.get();
+  ContainerTermination termination = wait.get();
 
   EXPECT_EQ(
     "Container destroyed while provisioning images",
@@ -1146,7 +1137,7 @@ TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
 
   AWAIT_READY(launch);
 
-  Future<containerizer::Termination> wait = containerizer.wait(containerId);
+  Future<ContainerTermination> wait = containerizer.wait(containerId);
 
   containerizer.destroy(containerId);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/tests/containerizer/port_mapping_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/port_mapping_tests.cpp b/src/tests/containerizer/port_mapping_tests.cpp
index fd181ca..db619ea 100644
--- a/src/tests/containerizer/port_mapping_tests.cpp
+++ b/src/tests/containerizer/port_mapping_tests.cpp
@@ -83,6 +83,7 @@ using mesos::master::detector::MasterDetector;
 
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::ContainerTermination;
 using mesos::slave::Isolator;
 
 using std::list;
@@ -2197,8 +2198,7 @@ TEST_F(PortMappingMesosTest, ROOT_NetworkNamespaceHandleSymlink)
   EXPECT_TRUE(os::exists(symlink));
   EXPECT_TRUE(os::stat::islink(symlink));
 
-  Future<containerizer::Termination> termination =
-    containerizer->wait(containerId);
+  Future<ContainerTermination> termination = containerizer->wait(containerId);
 
   driver.killTask(task.task_id());
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/tests/health_check_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/health_check_tests.cpp b/src/tests/health_check_tests.cpp
index e69c4f5..50a252b 100644
--- a/src/tests/health_check_tests.cpp
+++ b/src/tests/health_check_tests.cpp
@@ -52,6 +52,7 @@ using mesos::internal::slave::Slave;
 using mesos::master::detector::MasterDetector;
 
 using mesos::slave::ContainerLogger;
+using mesos::slave::ContainerTermination;
 
 using process::Clock;
 using process::Future;
@@ -578,7 +579,7 @@ TEST_F(HealthCheckTest, ROOT_DOCKER_DockerHealthyTask)
   EXPECT_TRUE(statusHealth.get().has_healthy());
   EXPECT_TRUE(statusHealth.get().healthy());
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     containerizer.wait(containerId.get());
 
   driver.stop();
@@ -1001,7 +1002,7 @@ TEST_F(HealthCheckTest, ROOT_DOCKER_DockerHealthStatusChange)
   ASSERT_SOME(os::read(tmpPath));
   EXPECT_EQ("bar", os::read(tmpPath).get());
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     containerizer.wait(containerId.get());
 
   driver.stop();

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/tests/hook_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hook_tests.cpp b/src/tests/hook_tests.cpp
index ac1188e..d864ef3 100644
--- a/src/tests/hook_tests.cpp
+++ b/src/tests/hook_tests.cpp
@@ -65,6 +65,7 @@ using mesos::internal::slave::Slave;
 using mesos::master::detector::MasterDetector;
 
 using mesos::slave::ContainerLogger;
+using mesos::slave::ContainerTermination;
 
 using process::Clock;
 using process::Future;
@@ -322,8 +323,7 @@ TEST_F(HookTest, VerifySlaveExecutorEnvironmentDecorator)
   ASSERT_TRUE(launch.get());
 
   // Wait on the container.
-  process::Future<containerizer::Termination> wait =
-    containerizer->wait(containerId);
+  process::Future<ContainerTermination> wait = containerizer->wait(containerId);
   AWAIT_READY(wait);
 
   // Check the executor exited correctly.
@@ -711,7 +711,7 @@ TEST_F(HookTest, ROOT_DOCKER_VerifySlavePreLaunchDockerEnvironmentDecorator)
   AWAIT_READY_FOR(statusFinished, Seconds(60));
   EXPECT_EQ(TASK_FINISHED, statusFinished.get().state());
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     containerizer.wait(containerId.get());
 
   driver.stop();
@@ -927,7 +927,7 @@ TEST_F(HookTest, ROOT_DOCKER_VerifySlavePreLaunchDockerHook)
   AWAIT_READY_FOR(statusFinished, Seconds(60));
   EXPECT_EQ(TASK_FINISHED, statusFinished.get().state());
 
-  Future<containerizer::Termination> termination =
+  Future<ContainerTermination> termination =
     containerizer.wait(containerId.get());
 
   driver.stop();

http://git-wip-us.apache.org/repos/asf/mesos/blob/c28846fb/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index b6d4ae5..9ff19f4 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -77,6 +77,8 @@ using mesos::internal::master::Master;
 using mesos::master::detector::MasterDetector;
 using mesos::master::detector::StandaloneMasterDetector;
 
+using mesos::slave::ContainerTermination;
+
 using mesos::v1::executor::Call;
 
 using std::map;
@@ -1717,8 +1719,7 @@ TYPED_TEST(SlaveRecoveryTest, RemoveNonCheckpointingFramework)
   AWAIT_READY(containers);
 
   foreach (const ContainerID& containerId, containers.get()) {
-    Future<containerizer::Termination> wait =
-      containerizer.get()->wait(containerId);
+    Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
 
     containerizer.get()->destroy(containerId);
     AWAIT_READY(wait);
@@ -2651,8 +2652,7 @@ TYPED_TEST(SlaveRecoveryTest, RegisterDisconnectedSlave)
   AWAIT_READY(containers);
 
   foreach (const ContainerID& containerId, containers.get()) {
-    Future<containerizer::Termination> wait =
-      containerizer.get()->wait(containerId);
+    Future<ContainerTermination> wait = containerizer.get()->wait(containerId);
 
     containerizer.get()->destroy(containerId);
     AWAIT_READY(wait);
@@ -4005,7 +4005,7 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, ResourceStatistics)
   EXPECT_TRUE(usage.get().has_cpus_limit());
   EXPECT_TRUE(usage.get().has_mem_limit_bytes());
 
-  Future<containerizer::Termination> wait = containerizer->wait(containerId);
+  Future<ContainerTermination> wait = containerizer->wait(containerId);
 
   containerizer->destroy(containerId);
 
@@ -4258,8 +4258,7 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, CGROUPS_ROOT_PidNamespaceForward)
   EXPECT_NE(0u, offers2.get().size());
 
   // Set up to wait on the container's termination.
-  Future<containerizer::Termination> termination =
-    containerizer->wait(containerId);
+  Future<ContainerTermination> termination = containerizer->wait(containerId);
 
   // Destroy the container.
   containerizer->destroy(containerId);
@@ -4364,8 +4363,7 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, CGROUPS_ROOT_PidNamespaceBackward)
   EXPECT_NE(0u, offers2.get().size());
 
   // Set up to wait on the container's termination.
-  Future<containerizer::Termination> termination =
-    containerizer->wait(containerId);
+  Future<ContainerTermination> termination = containerizer->wait(containerId);
 
   // Destroy the container.
   containerizer->destroy(containerId);


[08/12] mesos git commit: Changed the provisioner recover/provision/destroy to be const methods.

Posted by ji...@apache.org.
Changed the provisioner recover/provision/destroy to be const methods.

This is necessary for updating the provisioner as a shared pointer
in both mesos containerizer and the volume/image isolator.

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


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

Branch: refs/heads/master
Commit: 836ad5e00fc02cec39920ede1b08c94b2eddfb65
Parents: c28846f
Author: Gilbert Song <so...@gmail.com>
Authored: Mon Aug 22 13:00:34 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Aug 22 13:00:34 2016 -0700

----------------------------------------------------------------------
 .../containerizer/mesos/provisioner/provisioner.cpp |  6 +++---
 .../containerizer/mesos/provisioner/provisioner.hpp |  6 +++---
 .../containerizer/mesos_containerizer_tests.cpp     | 16 ++++++++--------
 3 files changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/836ad5e0/src/slave/containerizer/mesos/provisioner/provisioner.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/provisioner/provisioner.cpp b/src/slave/containerizer/mesos/provisioner/provisioner.cpp
index ad19fec..8e35ff4 100644
--- a/src/slave/containerizer/mesos/provisioner/provisioner.cpp
+++ b/src/slave/containerizer/mesos/provisioner/provisioner.cpp
@@ -118,7 +118,7 @@ Provisioner::~Provisioner()
 
 Future<Nothing> Provisioner::recover(
     const list<ContainerState>& states,
-    const hashset<ContainerID>& orphans)
+    const hashset<ContainerID>& orphans) const
 {
   return dispatch(
       CHECK_NOTNULL(process.get()),
@@ -130,7 +130,7 @@ Future<Nothing> Provisioner::recover(
 
 Future<ProvisionInfo> Provisioner::provision(
     const ContainerID& containerId,
-    const Image& image)
+    const Image& image) const
 {
   return dispatch(
       CHECK_NOTNULL(process.get()),
@@ -140,7 +140,7 @@ Future<ProvisionInfo> Provisioner::provision(
 }
 
 
-Future<bool> Provisioner::destroy(const ContainerID& containerId)
+Future<bool> Provisioner::destroy(const ContainerID& containerId) const
 {
   return dispatch(
       CHECK_NOTNULL(process.get()),

http://git-wip-us.apache.org/repos/asf/mesos/blob/836ad5e0/src/slave/containerizer/mesos/provisioner/provisioner.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/provisioner/provisioner.hpp b/src/slave/containerizer/mesos/provisioner/provisioner.hpp
index 8f378f6..64cfa50 100644
--- a/src/slave/containerizer/mesos/provisioner/provisioner.hpp
+++ b/src/slave/containerizer/mesos/provisioner/provisioner.hpp
@@ -85,19 +85,19 @@ public:
   // directories) to not leak anything.
   virtual process::Future<Nothing> recover(
       const std::list<mesos::slave::ContainerState>& states,
-      const hashset<ContainerID>& orphans);
+      const hashset<ContainerID>& orphans) const;
 
   // Provision a root filesystem for the container using the specified
   // image and return the absolute path to the root filesystem.
   virtual process::Future<ProvisionInfo> provision(
       const ContainerID& containerId,
-      const Image& image);
+      const Image& image) const;
 
   // Destroy a previously provisioned root filesystem. Assumes that
   // all references (e.g., mounts, open files) to the provisioned
   // filesystem have been removed. Return false if there is no
   // provisioned root filesystem for the given container.
-  virtual process::Future<bool> destroy(const ContainerID& containerId);
+  virtual process::Future<bool> destroy(const ContainerID& containerId) const;
 
 protected:
   Provisioner() {} // For creating mock object.

http://git-wip-us.apache.org/repos/asf/mesos/blob/836ad5e0/src/tests/containerizer/mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/mesos_containerizer_tests.cpp b/src/tests/containerizer/mesos_containerizer_tests.cpp
index 49f4552..2fd4ec5 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -761,16 +761,16 @@ class MockProvisioner : public mesos::internal::slave::Provisioner
 {
 public:
   MockProvisioner() {}
-  MOCK_METHOD2(recover,
-               Future<Nothing>(const list<ContainerState>&,
-                               const hashset<ContainerID>&));
+  MOCK_CONST_METHOD2(recover,
+                     Future<Nothing>(const list<ContainerState>&,
+                                     const hashset<ContainerID>&));
 
-  MOCK_METHOD2(provision,
-               Future<mesos::internal::slave::ProvisionInfo>(
-                   const ContainerID&,
-                   const Image&));
+  MOCK_CONST_METHOD2(provision,
+                     Future<mesos::internal::slave::ProvisionInfo>(
+                         const ContainerID&,
+                         const Image&));
 
-  MOCK_METHOD1(destroy, Future<bool>(const ContainerID&));
+  MOCK_CONST_METHOD1(destroy, Future<bool>(const ContainerID&));
 };
 
 


[05/12] mesos git commit: Fixed container ID related logging in MesosContainerizer.

Posted by ji...@apache.org.
Fixed container ID related logging in MesosContainerizer.

ContainerID is generated by the agent. Thus, quote is not needed.
Also, no need for `:` before the ID.

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


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

Branch: refs/heads/master
Commit: 10dda3a2d6db52ba3567122f1d75aa6ab663d357
Parents: 8e48051
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Aug 22 13:00:19 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Aug 22 13:00:19 2016 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 28 ++++++++++----------
 1 file changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/10dda3a2/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 64dac61..4025bfa 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -590,8 +590,8 @@ Future<Nothing> MesosContainerizerProcess::recover(
           continue;
         }
 
-        LOG(INFO) << "Recovering container '" << containerId
-                  << "' for executor '" << executor.id
+        LOG(INFO) << "Recovering container " << containerId
+                  << " for executor '" << executor.id
                   << "' of framework " << framework.id;
 
         // NOTE: We create the executor directory before checkpointing
@@ -794,8 +794,8 @@ Future<bool> MesosContainerizerProcess::launch(
         flags.default_container_info.get());
   }
 
-  LOG(INFO) << "Starting container '" << containerId
-            << "' for executor '" << executorInfo.executor_id()
+  LOG(INFO) << "Starting container " << containerId
+            << " for executor '" << executorInfo.executor_id()
             << "' of framework " << executorInfo.framework_id();
 
   ContainerConfig containerConfig;
@@ -1373,15 +1373,15 @@ Future<Nothing> MesosContainerizerProcess::update(
     // because the slave will attempt to update the container's
     // resources on a task's terminal state change but the executor
     // may have already exited and the container cleaned up.
-    LOG(WARNING) << "Ignoring update for unknown container: " << containerId;
+    LOG(WARNING) << "Ignoring update for unknown container " << containerId;
     return Nothing();
   }
 
   const Owned<Container>& container = containers_[containerId];
 
   if (container->state == DESTROYING) {
-    LOG(WARNING) << "Ignoring update for currently being destroyed container: "
-                 << containerId;
+    LOG(WARNING) << "Ignoring update for currently being destroyed "
+                 << "container " << containerId;
     return Nothing();
   }
 
@@ -1450,7 +1450,7 @@ Future<ResourceStatistics> MesosContainerizerProcess::usage(
   CHECK(!containerId.has_parent());
 
   if (!containers_.contains(containerId)) {
-    return Failure("Unknown container: " + stringify(containerId));
+    return Failure("Unknown container " + stringify(containerId));
   }
 
   list<Future<ResourceStatistics>> futures;
@@ -1487,7 +1487,7 @@ Future<ContainerStatus> _status(
     }
   }
 
-  VLOG(2) << "Aggregating status for container: " << containerId;
+  VLOG(2) << "Aggregating status for container " << containerId;
 
   return result;
 }
@@ -1513,7 +1513,7 @@ Future<ContainerStatus> MesosContainerizerProcess::status(
   // serialize the invocation to `await` in order to maintain the
   // order of requests for `ContainerStatus` by the agent.  See
   // MESOS-4671 for more details.
-  VLOG(2) << "Serializing status request for container: " << containerId;
+  VLOG(2) << "Serializing status request for container " << containerId;
 
   return containers_[containerId]->sequence.add<ContainerStatus>(
       [=]() -> Future<ContainerStatus> {
@@ -1541,18 +1541,18 @@ void MesosContainerizerProcess::destroy(
     //
     // The guard here and `if (container->state == DESTROYING)` below
     // make sure redundant destroys short-circuit.
-    VLOG(1) << "Ignoring destroy of unknown container: " << containerId;
+    VLOG(1) << "Ignoring destroy of unknown container " << containerId;
     return;
   }
 
   Container* container = containers_[containerId].get();
 
   if (container->state == DESTROYING) {
-    VLOG(1) << "Destroy has already been initiated for '" << containerId << "'";
+    VLOG(1) << "Destroy has already been initiated for " << containerId;
     return;
   }
 
-  LOG(INFO) << "Destroying container '" << containerId << "'";
+  LOG(INFO) << "Destroying container " << containerId;
 
   if (container->state == PROVISIONING) {
     VLOG(1) << "Waiting for the provisioner to complete for container '"
@@ -1793,7 +1793,7 @@ void MesosContainerizerProcess::reaped(const ContainerID& containerId)
     return;
   }
 
-  LOG(INFO) << "Executor for container '" << containerId << "' has exited";
+  LOG(INFO) << "Executor for container " << containerId << " has exited";
 
   // The executor has exited so destroy the container.
   destroy(containerId);


[06/12] mesos git commit: Changed the order so user specified env variables take precedence.

Posted by ji...@apache.org.
Changed the order so user specified env variables take precedence.

This addressed MESOS-6057. Isolator generated environment variables
should be overwritten by user specified env variables.

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


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

Branch: refs/heads/master
Commit: c4ce258d30072236fab8c954654f2f00000f6361
Parents: 10dda3a
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Aug 22 13:00:21 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Aug 22 13:00:21 2016 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 23 +++++++++++++++-----
 1 file changed, 17 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c4ce258d/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 4025bfa..687d9bb 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1032,12 +1032,6 @@ Future<bool> MesosContainerizerProcess::_launch(
     ? flags.sandbox_directory
     : container->config.directory();
 
-  // Include any enviroment variables from CommandInfo.
-  foreach (const Environment::Variable& variable,
-           container->config.command_info().environment().variables()) {
-    environment[variable.name()] = variable.value();
-  }
-
   // NOTE: Command task is a special case. Even if the container
   // config has a root filesystem, the executor container still uses
   // the host filesystem.
@@ -1120,6 +1114,23 @@ Future<bool> MesosContainerizerProcess::_launch(
         "--rootfs=" + container->config.rootfs());
   }
 
+  // Include any enviroment variables from CommandInfo.
+  foreach (const Environment::Variable& variable,
+           container->config.command_info().environment().variables()) {
+    const string& name = variable.name();
+    const string& value = variable.value();
+
+    if (environment.count(name)) {
+      VLOG(1) << "Overwriting environment variable '"
+              << name << "', original: '"
+              << environment[name] << "', new: '"
+              << value << "', for container "
+              << containerId;
+    }
+
+    environment[name] = value;
+  }
+
   return logger->prepare(
       container->config.executor_info(),
       container->config.directory())


[09/12] mesos git commit: Implemented the volume/image isolator.

Posted by ji...@apache.org.
Implemented the volume/image isolator.

This is a followup patch for r/51141, to move the image volumes
support from the mesos containerizer to the 'volume/image' isolator.

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


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

Branch: refs/heads/master
Commit: 192283f62a12b17108541be17197407470f44567
Parents: 836ad5e
Author: Gilbert Song <so...@gmail.com>
Authored: Mon Aug 22 13:00:37 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Aug 22 13:00:37 2016 -0700

----------------------------------------------------------------------
 src/CMakeLists.txt                              |   1 +
 src/Makefile.am                                 |   2 +
 .../mesos/isolators/volume/image.cpp            | 226 +++++++++++++++++++
 .../mesos/isolators/volume/image.hpp            |  69 ++++++
 4 files changed, 298 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/192283f6/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 9668fcf..ccd9892 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -180,6 +180,7 @@ set(LINUX_SRC
   slave/containerizer/mesos/isolators/gpu/volume.cpp
   slave/containerizer/mesos/isolators/namespaces/pid.cpp
   slave/containerizer/mesos/isolators/network/cni/cni.cpp
+  slave/containerizer/mesos/isolators/volume/image.cpp
   slave/containerizer/mesos/provisioner/backends/aufs.cpp
   slave/containerizer/mesos/provisioner/backends/bind.cpp
   slave/containerizer/mesos/provisioner/backends/overlay.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/192283f6/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index dd24f53..8dc4175 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1042,6 +1042,7 @@ MESOS_LINUX_FILES =							\
   slave/containerizer/mesos/isolators/gpu/volume.cpp			\
   slave/containerizer/mesos/isolators/namespaces/pid.cpp		\
   slave/containerizer/mesos/isolators/network/cni/cni.cpp		\
+  slave/containerizer/mesos/isolators/volume/image.cpp			\
   slave/containerizer/mesos/provisioner/backends/aufs.cpp		\
   slave/containerizer/mesos/provisioner/backends/bind.cpp		\
   slave/containerizer/mesos/provisioner/backends/overlay.cpp
@@ -1081,6 +1082,7 @@ MESOS_LINUX_FILES +=							\
   slave/containerizer/mesos/isolators/gpu/volume.hpp			\
   slave/containerizer/mesos/isolators/namespaces/pid.hpp		\
   slave/containerizer/mesos/isolators/network/cni/cni.hpp		\
+  slave/containerizer/mesos/isolators/volume/image.hpp			\
   slave/containerizer/mesos/provisioner/backends/aufs.hpp		\
   slave/containerizer/mesos/provisioner/backends/bind.hpp		\
   slave/containerizer/mesos/provisioner/backends/overlay.hpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/192283f6/src/slave/containerizer/mesos/isolators/volume/image.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/volume/image.cpp b/src/slave/containerizer/mesos/isolators/volume/image.cpp
new file mode 100644
index 0000000..c25205b
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/volume/image.cpp
@@ -0,0 +1,226 @@
+// 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 <list>
+#include <string>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include <process/collect.hpp>
+#include <process/defer.hpp>
+#include <process/id.hpp>
+
+#include <stout/error.hpp>
+#include <stout/foreach.hpp>
+#include <stout/strings.hpp>
+
+#include "slave/containerizer/mesos/isolators/volume/image.hpp"
+
+#include "slave/containerizer/mesos/provisioner/provisioner.hpp"
+
+using std::list;
+using std::string;
+using std::vector;
+
+using process::defer;
+using process::Failure;
+using process::Future;
+using process::Owned;
+using process::PID;
+using process::Shared;
+
+using mesos::slave::ContainerConfig;
+using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::Isolator;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+VolumeImageIsolatorProcess::VolumeImageIsolatorProcess(
+    const Flags& _flags,
+    const Shared<Provisioner>& _provisioner)
+  : ProcessBase(process::ID::generate("volume-image-isolator")),
+    flags(_flags),
+    provisioner(_provisioner) {}
+
+
+VolumeImageIsolatorProcess::~VolumeImageIsolatorProcess() {}
+
+
+Try<Isolator*> VolumeImageIsolatorProcess::create(
+    const Flags& flags,
+    const Shared<Provisioner>& provisioner)
+{
+  process::Owned<MesosIsolatorProcess> process(
+      new VolumeImageIsolatorProcess(flags, provisioner));
+
+  return new MesosIsolator(process);
+}
+
+
+Future<Option<ContainerLaunchInfo>> VolumeImageIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ContainerConfig& containerConfig)
+{
+  const ExecutorInfo& executorInfo = containerConfig.executor_info();
+
+  if (!executorInfo.has_container()) {
+    return None();
+  }
+
+  if (executorInfo.container().type() != ContainerInfo::MESOS) {
+    return Failure("Can only prepare image volumes for a MESOS container");
+  }
+
+  vector<string> targets;
+  list<Future<ProvisionInfo>> futures;
+
+  for (int i = 0; i < executorInfo.container().volumes_size(); i++) {
+    const Volume& volume = executorInfo.container().volumes(i);
+
+    if (!volume.has_image()) {
+      continue;
+    }
+
+    // Determine the target of the mount. The mount target
+    // is determined by 'container_path'.
+    string target;
+
+    // The logic to determine a volume mount target is identical to
+    // linux filesystem isolator, because volume image isolator has
+    // a dependency on that isolator, and it assumes that if the
+    // container specifies a rootfs the sandbox is already bind
+    // mounted into the container.
+    if (path::absolute(volume.container_path())) {
+      // To specify an image volume for a container, operators should
+      // be allowed to define the 'container_path' either as an absolute
+      // path or a relative path. Please see linux filesystem isolator
+      // for detail.
+      if (containerConfig.has_rootfs()) {
+        target = path::join(
+            containerConfig.rootfs(),
+            volume.container_path());
+
+        Try<Nothing> mkdir = os::mkdir(target);
+        if (mkdir.isError()) {
+          return Failure(
+              "Failed to create the target of the mount at '" +
+              target + "': " + mkdir.error());
+        }
+      } else {
+        target = volume.container_path();
+
+        if (!os::exists(target)) {
+          return Failure("Absolute container path '" + target + "' "
+                         "does not exist");
+        }
+      }
+    } else {
+      if (containerConfig.has_rootfs()) {
+        target = path::join(containerConfig.rootfs(),
+                            flags.sandbox_directory,
+                            volume.container_path());
+      } else {
+        target = path::join(containerConfig.directory(),
+                            volume.container_path());
+      }
+
+      // NOTE: We cannot create the mount point at 'target' if
+      // container has rootfs defined. The bind mount of the sandbox
+      // will hide what's inside 'target'. So we should always create
+      // the mount point in 'directory'.
+      const string mountPoint = path::join(
+          containerConfig.directory(),
+          volume.container_path());
+
+      Try<Nothing> mkdir = os::mkdir(mountPoint);
+      if (mkdir.isError()) {
+        return Failure(
+            "Failed to create the target of the mount at '" +
+            mountPoint + "': " + mkdir.error());
+      }
+    }
+
+    targets.push_back(target);
+    futures.push_back(provisioner->provision(containerId, volume.image()));
+  }
+
+  return await(futures)
+    .then(defer(
+        PID<VolumeImageIsolatorProcess>(this),
+        &VolumeImageIsolatorProcess::_prepare,
+        containerId,
+        targets,
+        lambda::_1));
+}
+
+
+Future<Option<ContainerLaunchInfo>> VolumeImageIsolatorProcess::_prepare(
+    const ContainerID& containerId,
+    const vector<string>& targets,
+    const list<Future<ProvisionInfo>>& futures)
+{
+  ContainerLaunchInfo launchInfo;
+  launchInfo.set_namespaces(CLONE_NEWNS);
+
+  vector<string> messages;
+  vector<string> sources;
+
+  foreach (const Future<ProvisionInfo>& future, futures) {
+    if (!future.isReady()) {
+      messages.push_back(future.isFailed() ? future.failure() : "discarded");
+      continue;
+    }
+
+    sources.push_back(future.get().rootfs);
+  }
+
+  if (!messages.empty()) {
+    return Failure(strings::join("\n", messages));
+  }
+
+  CHECK_EQ(sources.size(), targets.size());
+
+  for (size_t i = 0; i < sources.size(); i++) {
+    const string& source = sources[i];
+    const string& target = targets[i];
+
+    LOG(INFO) << "Mounting image volume rootfs '" << source
+              << "' to '" << target << "' for container " << containerId;
+
+    if (!os::exists(source)) {
+      return Failure(
+          "Provisioned rootfs '" + source + "' does not exist");
+    }
+
+    CommandInfo* command = launchInfo.add_pre_exec_commands();
+    command->set_shell(false);
+    command->set_value("mount");
+    command->add_arguments("mount");
+    command->add_arguments("-n");
+    command->add_arguments("--rbind");
+    command->add_arguments(source);
+    command->add_arguments(target);
+  }
+
+  return launchInfo;
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/192283f6/src/slave/containerizer/mesos/isolators/volume/image.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/volume/image.hpp b/src/slave/containerizer/mesos/isolators/volume/image.hpp
new file mode 100644
index 0000000..6333e9c
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/volume/image.hpp
@@ -0,0 +1,69 @@
+// 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 __VOLUME_IMAGE_ISOLATOR_HPP__
+#define __VOLUME_IMAGE_ISOLATOR_HPP__
+
+#include <list>
+#include <string>
+#include <vector>
+
+#include <process/shared.hpp>
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolator.hpp"
+
+#include "slave/containerizer/mesos/provisioner/provisioner.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// The volume image isolator is responsible for preparing image
+// volumes for mesos container.
+class VolumeImageIsolatorProcess : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(
+      const Flags& flags,
+      const process::Shared<Provisioner>& provisioner);
+
+  virtual ~VolumeImageIsolatorProcess();
+
+  virtual process::Future<Option<mesos::slave::ContainerLaunchInfo>> prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig);
+
+private:
+  VolumeImageIsolatorProcess(
+      const Flags& flags,
+      const process::Shared<Provisioner>& provisioner);
+
+  process::Future<Option<mesos::slave::ContainerLaunchInfo>> _prepare(
+      const ContainerID& containerId,
+      const std::vector<std::string>& targets,
+      const std::list<process::Future<ProvisionInfo>>& futures);
+
+  const Flags flags;
+  const process::Shared<Provisioner> provisioner;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __VOLUME_IMAGE_ISOLATOR_HPP__


[11/12] mesos git commit: Fixed potential flakiness in ROOT_RecoverOrphanedPersistentVolume.

Posted by ji...@apache.org.
Fixed potential flakiness in ROOT_RecoverOrphanedPersistentVolume.

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


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

Branch: refs/heads/master
Commit: 2f78a440ef4201c5b11fb92c225694e84a60369c
Parents: 9642d3c
Author: Gilbert Song <so...@gmail.com>
Authored: Mon Aug 22 13:00:58 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Aug 22 13:37:20 2016 -0700

----------------------------------------------------------------------
 src/tests/containerizer/filesystem_isolator_tests.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2f78a440/src/tests/containerizer/filesystem_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/filesystem_isolator_tests.cpp b/src/tests/containerizer/filesystem_isolator_tests.cpp
index a2db8dd..df4642d 100644
--- a/src/tests/containerizer/filesystem_isolator_tests.cpp
+++ b/src/tests/containerizer/filesystem_isolator_tests.cpp
@@ -734,7 +734,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_RecoverOrphanedPersistentVolume)
 
   // Wait until slave recovery is complete.
   Future<Nothing> _recover = FUTURE_DISPATCH(_, &Slave::_recover);
-  AWAIT_READY(_recover);
+  AWAIT_READY_FOR(_recover, Seconds(60));
 
   // Wait until the containerizer's recovery is done too.
   // This is called once orphans are cleaned up.  But this future is not