You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by jo...@apache.org on 2017/11/15 08:13:25 UTC

[01/15] mesos git commit: Moved GPU resource validation into common file.

Repository: mesos
Updated Branches:
  refs/heads/master 69a75519a -> 57334da8f


Moved GPU resource validation into common file.

Both master and agent expect GPUs to be specified in whole numbers
only.  This is some of the common resource validation which will
be used by the master and by the Standalone Container API.

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


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

Branch: refs/heads/master
Commit: 96d52a0931590fa859c77bcd9bedaf7af60194c9
Parents: e05dc9e
Author: Joseph Wu <jo...@apache.org>
Authored: Wed Sep 6 13:48:17 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 16:58:40 2017 -0800

----------------------------------------------------------------------
 src/common/validation.cpp | 17 +++++++++++++++++
 src/common/validation.hpp |  3 +++
 src/master/validation.cpp | 15 +--------------
 3 files changed, 21 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/96d52a09/src/common/validation.cpp
----------------------------------------------------------------------
diff --git a/src/common/validation.cpp b/src/common/validation.cpp
index 75c73e3..bad9ea8 100644
--- a/src/common/validation.cpp
+++ b/src/common/validation.cpp
@@ -21,6 +21,8 @@
 #include <algorithm>
 #include <cctype>
 
+#include <mesos/resources.hpp>
+
 #include <stout/foreach.hpp>
 #include <stout/stringify.hpp>
 #include <stout/unreachable.hpp>
@@ -29,6 +31,8 @@
 
 using std::string;
 
+using google::protobuf::RepeatedPtrField;
+
 namespace mesos {
 namespace internal {
 namespace common {
@@ -267,6 +271,19 @@ Option<Error> validateContainerInfo(const ContainerInfo& containerInfo)
   return None();
 }
 
+
+// Validates that the `gpus` resource is not fractional.
+// We rely on scalar resources only having 3 digits of precision.
+Option<Error> validateGpus(const RepeatedPtrField<Resource>& resources)
+{
+  double gpus = Resources(resources).gpus().getOrElse(0.0);
+  if (static_cast<long long>(gpus * 1000.0) % 1000 != 0) {
+    return Error("The 'gpus' resource must be an unsigned integer");
+  }
+
+  return None();
+}
+
 } // namespace validation {
 } // namespace common {
 } // namespace internal {

http://git-wip-us.apache.org/repos/asf/mesos/blob/96d52a09/src/common/validation.hpp
----------------------------------------------------------------------
diff --git a/src/common/validation.hpp b/src/common/validation.hpp
index c17c07d..3f60d7a 100644
--- a/src/common/validation.hpp
+++ b/src/common/validation.hpp
@@ -51,6 +51,9 @@ Option<Error> validateVolume(const Volume& volume);
 
 Option<Error> validateContainerInfo(const ContainerInfo& containerInfo);
 
+Option<Error> validateGpus(
+    const google::protobuf::RepeatedPtrField<Resource>& resources);
+
 } // namespace validation {
 } // namespace common {
 } // namespace internal {

http://git-wip-us.apache.org/repos/asf/mesos/blob/96d52a09/src/master/validation.cpp
----------------------------------------------------------------------
diff --git a/src/master/validation.cpp b/src/master/validation.cpp
index 55aac2f..8b5848b 100644
--- a/src/master/validation.cpp
+++ b/src/master/validation.cpp
@@ -646,19 +646,6 @@ Option<Error> validate(
 
 namespace resource {
 
-// Validates that the `gpus` resource is not fractional.
-// We rely on scalar resources only having 3 digits of precision.
-Option<Error> validateGpus(const RepeatedPtrField<Resource>& resources)
-{
-  double gpus = Resources(resources).gpus().getOrElse(0.0);
-  if (static_cast<long long>(gpus * 1000.0) % 1000 != 0) {
-    return Error("The 'gpus' resource must be an unsigned integer");
-  }
-
-  return None();
-}
-
-
 // Validates the ReservationInfos specified in the given resources (if
 // exist). Returns error if any ReservationInfo is found invalid or
 // unsupported.
@@ -867,7 +854,7 @@ Option<Error> validate(const RepeatedPtrField<Resource>& resources)
     return Error("Invalid resources: " + error.get().message);
   }
 
-  error = validateGpus(resources);
+  error = common::validation::validateGpus(resources);
   if (error.isSome()) {
     return Error("Invalid 'gpus' resource: " + error.get().message);
   }


[15/15] mesos git commit: Updated some isolators' usage of ExecutorInfo.

Posted by jo...@apache.org.
Updated some isolators' usage of ExecutorInfo.

These four isolators refered to the ExecutorInfo field of the given
ContainerConfig to determine the total resources to isolate.  This
changes these isolators to refer instead to the Resources field of the
given ContainerConfig because the resource amounts should be identical
(enforced by the caller to Containerizer::launch, such as the agent)
and in order to support standalone containers, which specify resources
but no ExecutorInfo.

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


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

Branch: refs/heads/master
Commit: 57334da8f8a9842344a80f2161c3838b1b46d01f
Parents: 38951cf
Author: Joseph Wu <jo...@apache.org>
Authored: Thu Nov 2 04:23:11 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 17:16:56 2017 -0800

----------------------------------------------------------------------
 src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp     | 5 +----
 src/slave/containerizer/mesos/isolators/gpu/isolator.cpp        | 2 +-
 .../containerizer/mesos/isolators/network/port_mapping.cpp      | 3 +--
 src/slave/containerizer/mesos/isolators/xfs/disk.cpp            | 2 +-
 4 files changed, 4 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/57334da8/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
index 549a455..c0ebc4e 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
@@ -517,10 +517,7 @@ Future<Option<ContainerLaunchInfo>> CgroupsIsolatorProcess::_prepare(
         strings::join(";", errors));
   }
 
-  // TODO(haosdent): Here we assume the command executor's resources
-  // include the task's resources. Revisit here if this semantics
-  // changes.
-  return update(containerId, containerConfig.executor_info().resources())
+  return update(containerId, containerConfig.resources())
     .then([]() { return Option<ContainerLaunchInfo>::none(); });
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/57334da8/src/slave/containerizer/mesos/isolators/gpu/isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/gpu/isolator.cpp b/src/slave/containerizer/mesos/isolators/gpu/isolator.cpp
index 25636b5..4d6f640 100644
--- a/src/slave/containerizer/mesos/isolators/gpu/isolator.cpp
+++ b/src/slave/containerizer/mesos/isolators/gpu/isolator.cpp
@@ -351,7 +351,7 @@ Future<Option<ContainerLaunchInfo>> NvidiaGpuIsolatorProcess::prepare(
     }
   }
 
-  return update(containerId, containerConfig.executor_info().resources())
+  return update(containerId, containerConfig.resources())
     .then(defer(PID<NvidiaGpuIsolatorProcess>(this),
                 &NvidiaGpuIsolatorProcess::_prepare,
                 containerConfig));

http://git-wip-us.apache.org/repos/asf/mesos/blob/57334da8/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp b/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
index 55672b1..d60052e 100644
--- a/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
@@ -2494,8 +2494,7 @@ Future<Option<ContainerLaunchInfo>> PortMappingIsolatorProcess::prepare(
   }
 
   const ExecutorInfo& executorInfo = containerConfig.executor_info();
-
-  Resources resources(executorInfo.resources());
+  const Resources resources(containerConfig.resources());
 
   IntervalSet<uint16_t> nonEphemeralPorts;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/57334da8/src/slave/containerizer/mesos/isolators/xfs/disk.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/xfs/disk.cpp b/src/slave/containerizer/mesos/isolators/xfs/disk.cpp
index 2113f86..8d9f8f8 100644
--- a/src/slave/containerizer/mesos/isolators/xfs/disk.cpp
+++ b/src/slave/containerizer/mesos/isolators/xfs/disk.cpp
@@ -291,7 +291,7 @@ Future<Option<ContainerLaunchInfo>> XfsDiskIsolatorProcess::prepare(
   LOG(INFO) << "Assigned project " << stringify(projectId.get()) << " to '"
             << containerConfig.directory() << "'";
 
-  return update(containerId, containerConfig.executor_info().resources())
+  return update(containerId, containerConfig.resources())
     .then([]() -> Future<Option<ContainerLaunchInfo>> {
       return None();
     });


[06/15] mesos git commit: Added validation for Standalone Container APIs.

Posted by jo...@apache.org.
Added validation for Standalone Container APIs.

The Standalone Container APIs act much like the nested container APIs,
except that ContainerIDs do not necessarily need to have a parent.

Additionally, the 'resources' field in the `LAUNCH_CONTAINER` API
has some restrictions due to how these resources are not reported
to the master.

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


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

Branch: refs/heads/master
Commit: 2d7ec26835ca3551ce952ea57b3e369cdcb2cb99
Parents: 96d52a0
Author: Joseph Wu <jo...@apache.org>
Authored: Wed Sep 6 13:50:47 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 16:58:41 2017 -0800

----------------------------------------------------------------------
 src/slave/validation.cpp | 137 ++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 137 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2d7ec268/src/slave/validation.cpp
----------------------------------------------------------------------
diff --git a/src/slave/validation.cpp b/src/slave/validation.cpp
index a575d88..32781fd 100644
--- a/src/slave/validation.cpp
+++ b/src/slave/validation.cpp
@@ -18,6 +18,8 @@
 
 #include <string>
 
+#include <mesos/resources.hpp>
+
 #include <mesos/agent/agent.hpp>
 
 #include <stout/stringify.hpp>
@@ -26,6 +28,7 @@
 
 #include "checks/checker.hpp"
 
+#include "common/resources_utils.hpp"
 #include "common/validation.hpp"
 
 using std::string;
@@ -348,6 +351,140 @@ Option<Error> validate(
 
       return None();
     }
+
+    case mesos::agent::Call::LAUNCH_CONTAINER: {
+      if (!call.has_launch_container()) {
+        return Error("Expecting 'launch_container' to be present");
+      }
+
+      Option<Error> error = validation::container::validateContainerId(
+          call.launch_container().container_id());
+
+      if (error.isSome()) {
+        return Error(
+            "'launch_container.container_id' is invalid: " + error->message);
+      }
+
+      // Nested containers share resources with their parent so are
+      // not allowed to specify resources in this call.
+      if (call.launch_container().container_id().has_parent() &&
+          call.launch_container().resources().size() != 0) {
+        return Error(
+            "Resources may not be specified when using "
+            "'launch_container' to launch nested containers");
+      }
+
+      // General resource validation first.
+      error = Resources::validate(call.launch_container().resources());
+      if (error.isSome()) {
+        return Error("Invalid resources: " + error.get().message);
+      }
+
+      error = common::validation::validateGpus(
+          call.launch_container().resources());
+
+      if (error.isSome()) {
+        return Error("Invalid GPU resources: " + error.get().message);
+      }
+
+      // Because standalone containers are launched outside of the master's
+      // offer cycle, some resource types or fields may not be specified.
+      foreach (Resource resource, call.launch_container().resources()) {
+        // Normalize the resources (in place) to simplify validation.
+        convertResourceFormat(&resource, POST_RESERVATION_REFINEMENT);
+
+        // Standalone containers may only use unreserved resources.
+        // There is no accounting in the master for resources consumed
+        // by standalone containers, so allowing reserved resources would
+        // only increase code complexity with no change in behavior.
+        if (Resources::isReserved(resource)) {
+          return Error("'launch_container.resources' must be unreserved");
+        }
+
+        // NOTE: The master normally requires all volumes be persistent,
+        // and that all persistent volumes belong to a role. Standalone
+        // containers therefore cannot use persistent volumes.
+        if (Resources::isPersistentVolume(resource)) {
+          return Error(
+              "'launch_container.resources' may not use persistent volumes");
+        }
+
+        // Standalone containers are expected to occupy resources *not*
+        // advertised by the agent and hence do not need to worry about
+        // being preempted or throttled.
+        if (Resources::isRevocable(resource)) {
+          return Error("'launch_container.resources' must be non-revocable");
+        }
+      }
+
+      if (call.launch_container().has_command()) {
+        error = common::validation::validateCommandInfo(
+            call.launch_container().command());
+        if (error.isSome()) {
+          return Error(
+              "'launch_container.command' is invalid: " + error->message);
+        }
+      }
+
+      if (call.launch_container().has_container()) {
+        error = common::validation::validateContainerInfo(
+            call.launch_container().container());
+        if (error.isSome()) {
+          return Error(
+              "'launch_container.container' is invalid: " + error->message);
+        }
+      }
+
+      return None();
+    }
+
+    case mesos::agent::Call::WAIT_CONTAINER: {
+      if (!call.has_wait_container()) {
+        return Error("Expecting 'wait_container' to be present");
+      }
+
+      Option<Error> error = validation::container::validateContainerId(
+          call.wait_container().container_id());
+
+      if (error.isSome()) {
+        return Error("'wait_container.container_id' is invalid"
+                     ": " + error->message);
+      }
+
+      return None();
+    }
+
+    case mesos::agent::Call::KILL_CONTAINER: {
+      if (!call.has_kill_container()) {
+        return Error("Expecting 'kill_container' to be present");
+      }
+
+      Option<Error> error = validation::container::validateContainerId(
+          call.kill_container().container_id());
+
+      if (error.isSome()) {
+        return Error("'kill_container.container_id' is invalid"
+                     ": " + error->message);
+      }
+
+      return None();
+    }
+
+    case mesos::agent::Call::REMOVE_CONTAINER: {
+      if (!call.has_remove_container()) {
+        return Error("Expecting 'remove_container' to be present");
+      }
+
+      Option<Error> error = validation::container::validateContainerId(
+          call.remove_container().container_id());
+
+      if (error.isSome()) {
+        return Error("'remove_container.container_id' is invalid"
+                     ": " + error->message);
+      }
+
+      return None();
+    }
   }
 
   UNREACHABLE();


[05/15] mesos git commit: Defined API for launching standalone containers.

Posted by jo...@apache.org.
Defined API for launching standalone containers.

Launching a standalone container is very similar to launching a
nested container, except that the caller must specify some Resources.
As such, this patch deprecates some nested container APIs
and replaces them with more generically named APIs.

This applies to the Launch, Wait, and Kill (nested) container APIs.

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


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

Branch: refs/heads/master
Commit: 3f25be13e5c1b52877dc0722f3aa7506102e072e
Parents: f370a32
Author: Joseph Wu <jo...@apache.org>
Authored: Fri Jul 14 11:12:42 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 16:58:40 2017 -0800

----------------------------------------------------------------------
 include/mesos/agent/agent.proto    | 161 +++++++++++++++++++++++++++----
 include/mesos/v1/agent/agent.proto | 165 ++++++++++++++++++++++++++++----
 2 files changed, 294 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3f25be13/include/mesos/agent/agent.proto
----------------------------------------------------------------------
diff --git a/include/mesos/agent/agent.proto b/include/mesos/agent/agent.proto
index 4df3dce..0f92f73 100644
--- a/include/mesos/agent/agent.proto
+++ b/include/mesos/agent/agent.proto
@@ -59,16 +59,25 @@ message Call {
     GET_AGENT = 20;         // Retrieves the agent information.
 
     // Calls for managing nested containers underneath an executor's container.
-    LAUNCH_NESTED_CONTAINER = 14;  // See 'LaunchNestedContainer' below.
-    WAIT_NESTED_CONTAINER = 15;    // See 'WaitNestedContainer' below.
-    KILL_NESTED_CONTAINER = 16;    // See 'KillNestedContainer' below.
-    REMOVE_NESTED_CONTAINER = 21;  // See 'RemoveNestedContainer' below.
+    // Some of these calls are deprecated in favor of the calls
+    // for both standalone or nested containers further below.
+    LAUNCH_NESTED_CONTAINER = 14 [deprecated = true];
+    WAIT_NESTED_CONTAINER = 15 [deprecated = true];
+    KILL_NESTED_CONTAINER = 16 [deprecated = true];
+    REMOVE_NESTED_CONTAINER = 21 [deprecated = true];
 
     // See 'LaunchNestedContainerSession' below.
     LAUNCH_NESTED_CONTAINER_SESSION = 17;
 
     ATTACH_CONTAINER_INPUT = 18; // See 'AttachContainerInput' below.
     ATTACH_CONTAINER_OUTPUT = 19; // see 'AttachContainerOutput' below.
+
+    // Calls for managing standalone containers
+    // or containers nested underneath another container.
+    LAUNCH_CONTAINER = 22; // See 'LaunchContainer' below.
+    WAIT_CONTAINER = 23;   // See 'WaitContainer' below.
+    KILL_CONTAINER = 24;   // See 'KillContainer' below.
+    REMOVE_CONTAINER = 25; // See 'RemoveContainer' below.
   }
 
   // Provides a snapshot of the current metrics tracked by the agent.
@@ -109,28 +118,25 @@ message Call {
     optional uint64 length = 3;
   }
 
- // Launches a nested container within an executor's tree of containers.
+  // Deprecated in favor of `LaunchContainer`.
   message LaunchNestedContainer {
     required ContainerID container_id = 1;
     optional CommandInfo command = 2;
     optional ContainerInfo container = 3;
   }
 
-  // Waits for the nested container to terminate and receives the exit status.
+  // Deprecated in favor of `WaitContainer`.
   message WaitNestedContainer {
     required ContainerID container_id = 1;
   }
 
-  // Kills the nested container. The signal (e.g., SIGTERM, SIGKILL, etc.)
-  // to be sent to the container can be specified in the 'signal' field.
-  // If 'signal' is not set, SIGKILL is used by default.
+  // Deprecated in favor of `KillContainer`.
   message KillNestedContainer {
     required ContainerID container_id = 1;
     optional int32 signal = 2;
   }
 
-  // Removes a nested container and its artifacts (runtime and sandbox
-  // directories).
+  // Deprecated in favor of `RemoveContainer`.
   message RemoveNestedContainer {
     required ContainerID container_id = 1;
   }
@@ -175,19 +181,118 @@ message Call {
     required ContainerID container_id = 1;
   }
 
+  // Launches a either a "standalone" container on this agent
+  // or a nested container within another tree of containers.
+  //
+  // A standalone container is launched by specifying a ContainerID
+  // with no parent. Standalone containers bypass the normal offer cycle
+  // between the master and agent. Unlike other containers, a standalone
+  // container does not have an executor or any tasks. This means the
+  // standalone container does not report back to Mesos or any framework
+  // and must be supervised separately.
+  //
+  // A nested container is launched by specifying a ContainerID with
+  // another existing container (including standalone containers)
+  // as the parent.
+  //
+  // Returns 200 OK if the new container launch succeeds.
+  // Returns 202 Accepted if the requested ContainerID is already in use
+  //   by a standalone or nested container.
+  // Returns 400 Bad Request if the container launch fails.
+  message LaunchContainer {
+    // NOTE: Some characters cannot be used in the ID. All characters
+    // must be valid filesystem path characters.  In addition, '/' and '.'
+    // are reserved.
+    required ContainerID container_id = 1;
+
+    optional CommandInfo command = 2;
+
+    // NOTE: Nested containers may not specify resources and instead
+    // share resources with its parent container.
+    //
+    // TODO(josephw): These resources are purely used for isolation
+    // and are not accounted for by the Mesos master (if connected).
+    // It is the caller's responsibility to ensure that resources are
+    // not overcommitted (e.g. CPU and memory) or conflicting (e.g. ports
+    // and volumes). Once there is support for preempting tasks and a
+    // way to update the resources advertised by the agent, these standalone
+    // container resources should be accounted for by the master.
+    repeated Resource resources = 3;
+
+    optional ContainerInfo container = 4;
+  }
+
+  // Waits for the standalone or nested container to terminate
+  // and returns the exit status.
+  //
+  // Returns 200 OK if and when the container exits.
+  // Returns 404 Not Found if the container does not exist.
+  message WaitContainer {
+    required ContainerID container_id = 1;
+  }
+
+  // Kills the standalone or nested container. The signal to be sent
+  // to the container can be specified in the 'signal' field.
+  //
+  // Returns 200 OK if the signal is sent successfully.
+  // Returns 404 Not Found if the container does not exist.
+  message KillContainer {
+    required ContainerID container_id = 1;
+
+    // Defaults to SIGKILL.
+    optional int32 signal = 2;
+  }
+
+  // Removes a container's artifacts (runtime and sandbox directories).
+  //
+  // For nested containers, it is important to use this call if multiple
+  // nested containers are launched under the same parent container, because
+  // garbage collection only takes place at the parent container. Artifacts
+  // belonging to nested containers will not be garbage collected while
+  // the parent container is running.
+  //
+  // TODO(josephw): A standalone container's runtime directory is currently
+  // garbage collected as soon as the container exits. To allow the user to
+  // retrieve the exit status reliably, the runtime directory cannot be
+  // garbage collected immediately. Instead, the user will eventually be
+  // required to make this call after the standalone container has exited.
+  // Also, a standalone container's sandbox directory is currently not
+  // garbage collected and is only deleted via this call.
+  //
+  // Returns 200 OK if the removal is successful or if the parent container
+  //   (for nested containers) does not exist.
+  // Returns 500 Internal Server Error if anything goes wrong, including
+  //   if the container is still running or does not exist.
+  //
+  // TODO(josephw): Consider returning a 400 Bad Request instead of 500
+  // Internal Server Error when the user tries to remove a running or
+  // nonexistent nested container.
+  message RemoveContainer {
+    required ContainerID container_id = 1;
+  }
+
   optional Type type = 1;
 
   optional GetMetrics get_metrics = 2;
   optional SetLoggingLevel set_logging_level = 3;
   optional ListFiles list_files = 4;
   optional ReadFile read_file = 5;
-  optional LaunchNestedContainer launch_nested_container = 6;
-  optional WaitNestedContainer wait_nested_container = 7;
-  optional KillNestedContainer kill_nested_container = 8;
-  optional RemoveNestedContainer remove_nested_container = 12;
+
+  optional LaunchNestedContainer launch_nested_container = 6
+    [deprecated = true];
+
+  optional WaitNestedContainer wait_nested_container = 7 [deprecated = true];
+  optional KillNestedContainer kill_nested_container = 8 [deprecated = true];
+  optional RemoveNestedContainer remove_nested_container = 12
+    [deprecated = true];
+
   optional LaunchNestedContainerSession launch_nested_container_session = 9;
   optional AttachContainerInput attach_container_input = 10;
   optional AttachContainerOutput attach_container_output = 11;
+  optional LaunchContainer launch_container = 13;
+  optional WaitContainer wait_container = 14;
+  optional KillContainer kill_container = 15;
+  optional RemoveContainer remove_container = 16;
 }
 
 
@@ -217,7 +322,8 @@ message Response {
     GET_TASKS = 12;                // See 'GetTasks' below.
     GET_AGENT = 14;                // See 'GetAgent' below.
 
-    WAIT_NESTED_CONTAINER = 13;    // See 'WaitNestedContainer' below.
+    WAIT_NESTED_CONTAINER = 13 [deprecated = true];
+    WAIT_CONTAINER = 15;           // See 'WaitContainer' below.
   }
 
   // `healthy` would be true if the agent is healthy. Delayed responses are also
@@ -353,6 +459,28 @@ message Response {
     optional string message = 5;
   }
 
+  // Returns termination information about the standalone or nested container.
+  message WaitContainer {
+    // Wait status of the lead process in the container. Note that this
+    // is the return value of `wait(2)`, so callers must use the `wait(2)`
+    // family of macros to extract whether the process exited cleanly and
+    // what the exit code was.
+    optional int32 exit_status = 1;
+
+    // The `state` and `reason` fields may be populated if the Mesos agent
+    // terminates the container. In the absence of any special knowledge,
+    // executors should propagate this information via the `status` field
+    // of an `Update` call for the corresponding TaskID.
+    optional TaskState state = 2;
+    optional TaskStatus.Reason reason = 3;
+
+    // This field will be populated if the task was terminated due to
+    // a resource limitation.
+    optional TaskResourceLimitation limitation = 4;
+
+    optional string message = 5;
+  }
+
   optional Type type = 1;
 
   optional GetHealth get_health = 2;
@@ -369,6 +497,7 @@ message Response {
   optional GetTasks get_tasks = 13;
   optional GetAgent get_agent = 15;
   optional WaitNestedContainer wait_nested_container = 14;
+  optional WaitContainer wait_container = 16;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/3f25be13/include/mesos/v1/agent/agent.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/agent/agent.proto b/include/mesos/v1/agent/agent.proto
index e99d23d..012ffef 100644
--- a/include/mesos/v1/agent/agent.proto
+++ b/include/mesos/v1/agent/agent.proto
@@ -59,16 +59,25 @@ message Call {
     GET_AGENT = 20;         // Retrieves the agent information.
 
     // Calls for managing nested containers underneath an executor's container.
-    LAUNCH_NESTED_CONTAINER = 14;  // See 'LaunchNestedContainer' below.
-    WAIT_NESTED_CONTAINER = 15;    // See 'WaitNestedContainer' below.
-    KILL_NESTED_CONTAINER = 16;    // See 'KillNestedContainer' below.
-    REMOVE_NESTED_CONTAINER = 21;  // See 'RemoveNestedContainer' below.
+    // Some of these calls are deprecated in favor of the calls
+    // for both standalone or nested containers further below.
+    LAUNCH_NESTED_CONTAINER = 14 [deprecated = true];
+    WAIT_NESTED_CONTAINER = 15 [deprecated = true];
+    KILL_NESTED_CONTAINER = 16 [deprecated = true];
+    REMOVE_NESTED_CONTAINER = 21 [deprecated = true];
 
     // See 'LaunchNestedContainerSession' below.
     LAUNCH_NESTED_CONTAINER_SESSION = 17;
 
     ATTACH_CONTAINER_INPUT = 18; // See 'AttachContainerInput' below.
     ATTACH_CONTAINER_OUTPUT = 19; // see 'AttachContainerOutput' below.
+
+    // Calls for managing standalone containers
+    // or containers nested underneath another container.
+    LAUNCH_CONTAINER = 22; // See 'LaunchContainer' below.
+    WAIT_CONTAINER = 23;   // See 'WaitContainer' below.
+    KILL_CONTAINER = 24;   // See 'KillContainer' below.
+    REMOVE_CONTAINER = 25; // See 'RemoveContainer' below.
   }
 
   // Provides a snapshot of the current metrics tracked by the agent.
@@ -109,28 +118,25 @@ message Call {
     optional uint64 length = 3;
   }
 
-  // Launches a nested container within an executor's tree of containers.
+  // Deprecated in favor of `LaunchContainer`.
   message LaunchNestedContainer {
     required ContainerID container_id = 1;
     optional CommandInfo command = 2;
     optional ContainerInfo container = 3;
   }
 
-  // Waits for the nested container to terminate and receives the exit status.
+  // Deprecated in favor of `WaitContainer`.
   message WaitNestedContainer {
     required ContainerID container_id = 1;
   }
 
-  // Kills the nested container. The signal (e.g., SIGTERM, SIGKILL, etc.)
-  // to be sent to the container can be specified in the 'signal' field.
-  // If 'signal' is not set, SIGKILL is used by default.
+  // Deprecated in favor of `KillContainer`.
   message KillNestedContainer {
     required ContainerID container_id = 1;
     optional int32 signal = 2;
   }
 
-  // Removes a nested container and its artifacts (runtime and sandbox
-  // directories).
+  // Deprecated in favor of `RemoveContainer`.
   message RemoveNestedContainer {
     required ContainerID container_id = 1;
   }
@@ -175,19 +181,118 @@ message Call {
     required ContainerID container_id = 1;
   }
 
+  // Launches a either a "standalone" container on this agent
+  // or a nested container within another tree of containers.
+  //
+  // A standalone container is launched by specifying a ContainerID
+  // with no parent. Standalone containers bypass the normal offer cycle
+  // between the master and agent. Unlike other containers, a standalone
+  // container does not have an executor or any tasks. This means the
+  // standalone container does not report back to Mesos or any framework
+  // and must be supervised separately.
+  //
+  // A nested container is launched by specifying a ContainerID with
+  // another existing container (including standalone containers)
+  // as the parent.
+  //
+  // Returns 200 OK if the new container launch succeeds.
+  // Returns 202 Accepted if the requested ContainerID is already in use
+  //   by a standalone or nested container.
+  // Returns 400 Bad Request if the container launch fails.
+  message LaunchContainer {
+    // NOTE: Some characters cannot be used in the ID. All characters
+    // must be valid filesystem path characters.  In addition, '/' and '.'
+    // are reserved.
+    required ContainerID container_id = 1;
+
+    optional CommandInfo command = 2;
+
+    // NOTE: Nested containers may not specify resources and instead
+    // share resources with its parent container.
+    //
+    // TODO(josephw): These resources are purely used for isolation
+    // and are not accounted for by the Mesos master (if connected).
+    // It is the caller's responsibility to ensure that resources are
+    // not overcommitted (e.g. CPU and memory) or conflicting (e.g. ports
+    // and volumes). Once there is support for preempting tasks and a
+    // way to update the resources advertised by the agent, these standalone
+    // container resources should be accounted for by the master.
+    repeated Resource resources = 3;
+
+    optional ContainerInfo container = 4;
+  }
+
+  // Waits for the standalone or nested container to terminate
+  // and returns the exit status.
+  //
+  // Returns 200 OK if and when the container exits.
+  // Returns 404 Not Found if the container does not exist.
+  message WaitContainer {
+    required ContainerID container_id = 1;
+  }
+
+  // Kills the standalone or nested container. The signal to be sent
+  // to the container can be specified in the 'signal' field.
+  //
+  // Returns 200 OK if the signal is sent successfully.
+  // Returns 404 Not Found if the container does not exist.
+  message KillContainer {
+    required ContainerID container_id = 1;
+
+    // Defaults to SIGKILL.
+    optional int32 signal = 2;
+  }
+
+  // Removes a container's artifacts (runtime and sandbox directories).
+  //
+  // For nested containers, it is important to use this call if multiple
+  // nested containers are launched under the same parent container, because
+  // garbage collection only takes place at the parent container. Artifacts
+  // belonging to nested containers will not be garbage collected while
+  // the parent container is running.
+  //
+  // TODO(josephw): A standalone container's runtime directory is currently
+  // garbage collected as soon as the container exits. To allow the user to
+  // retrieve the exit status reliably, the runtime directory cannot be
+  // garbage collected immediately. Instead, the user will eventually be
+  // required to make this call after the standalone container has exited.
+  // Also, a standalone container's sandbox directory is currently not
+  // garbage collected and is only deleted via this call.
+  //
+  // Returns 200 OK if the removal is successful or if the parent container
+  //   (for nested containers) does not exist.
+  // Returns 500 Internal Server Error if anything goes wrong, including
+  //   if the container is still running or does not exist.
+  //
+  // TODO(josephw): Consider returning a 400 Bad Request instead of 500
+  // Internal Server Error when the user tries to remove a running or
+  // nonexistent nested container.
+  message RemoveContainer {
+    required ContainerID container_id = 1;
+  }
+
   optional Type type = 1;
 
   optional GetMetrics get_metrics = 2;
   optional SetLoggingLevel set_logging_level = 3;
   optional ListFiles list_files = 4;
   optional ReadFile read_file = 5;
-  optional LaunchNestedContainer launch_nested_container = 6;
-  optional WaitNestedContainer wait_nested_container = 7;
-  optional KillNestedContainer kill_nested_container = 8;
-  optional RemoveNestedContainer remove_nested_container = 12;
+
+  optional LaunchNestedContainer launch_nested_container = 6
+    [deprecated = true];
+
+  optional WaitNestedContainer wait_nested_container = 7 [deprecated = true];
+  optional KillNestedContainer kill_nested_container = 8 [deprecated = true];
+  optional RemoveNestedContainer remove_nested_container = 12
+    [deprecated = true];
+
   optional LaunchNestedContainerSession launch_nested_container_session = 9;
   optional AttachContainerInput attach_container_input = 10;
   optional AttachContainerOutput attach_container_output = 11;
+  optional LaunchContainer launch_container = 13;
+  optional WaitContainer wait_container = 14;
+  optional KillContainer kill_container = 15;
+  optional RemoveContainer remove_container = 16;
 }
 
 
@@ -217,7 +322,8 @@ message Response {
     GET_TASKS = 12;                // See 'GetTasks' below.
     GET_AGENT = 14;                // See 'GetAgent' below.
 
-    WAIT_NESTED_CONTAINER = 13;    // See 'WaitNestedContainer' below.
+    WAIT_NESTED_CONTAINER = 13 [deprecated = true];
+    WAIT_CONTAINER = 15;           // See 'WaitContainer' below.
   }
 
   // `healthy` would be true if the agent is healthy. Delayed responses are also
@@ -339,6 +445,32 @@ message Response {
     // what the exit code was.
     optional int32 exit_status = 1;
 
+    // The `state` and `reason` fields may be populated if the Mesos agent
+    // terminates the container. In the absence of any special knowledge,
+    // executors should propagate this information via the `status` field
+    // of an `Update` call for the corresponding TaskID.
+    optional TaskState state = 2;
+    optional TaskStatus.Reason reason = 3;
+
+    // This field will be populated if the task was terminated due to
+    // a resource limitation.
+    optional TaskResourceLimitation limitation = 4;
+
+    optional string message = 5;
+  }
+
+  // Returns termination information about the standalone or nested container.
+  message WaitContainer {
+    // Wait status of the lead process in the container. Note that this
+    // is the return value of `wait(2)`, so callers must use the `wait(2)`
+    // family of macros to extract whether the process exited cleanly and
+    // what the exit code was.
+    optional int32 exit_status = 1;
+
+    // The `state` and `reason` fields may be populated if the Mesos agent
+    // terminates the container. In the absence of any special knowledge,
+    // executors should propagate this information via the `status` field
+    // of an `Update` call for the corresponding TaskID.
     optional TaskState state = 2;
     optional TaskStatus.Reason reason = 3;
 
@@ -365,6 +497,7 @@ message Response {
   optional GetTasks get_tasks = 13;
   optional GetAgent get_agent = 15;
   optional WaitNestedContainer wait_nested_container = 14;
+  optional WaitContainer wait_container = 16;
 }
 
 


[11/15] mesos git commit: Modified Containerizer::launch interface to allow repeated launch.

Posted by jo...@apache.org.
Modified Containerizer::launch interface to allow repeated launch.

There is some existing tech debt around requiring the caller of
`Containerizer::launch` to call `Containerizer::destroy` if the launch
fails (see MESOS-6214).  For nested and standalone containers, the
side effect of this results in accidentally destroying running
containers if you make the same call an even number of times.

For example, suppose the user launches a valid nested container
with an ID of 'parent.child'. If the user issues the same call to
launch 'parent.child' again, this second call will fail *and* will
also destroy the first container.

This commit prevents repeated launch calls from destroying containers
by changing the return value of `Containerizer::launch`.  There are
now four possible return values:
  * The launch succeeded.
  * The standalone/nested container already exists.
  * The given ContainerConfig is not supported.
  * The launch failed.

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


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

Branch: refs/heads/master
Commit: feefdba2a7376d45ff9113cd4b157fcc71ca3459
Parents: 89258c0
Author: Joseph Wu <jo...@apache.org>
Authored: Mon Oct 16 17:13:30 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 17:16:21 2017 -0800

----------------------------------------------------------------------
 src/slave/containerizer/composing.cpp           | 44 +++++++-------
 src/slave/containerizer/composing.hpp           |  2 +-
 src/slave/containerizer/containerizer.hpp       | 11 ++--
 src/slave/containerizer/mesos/containerizer.cpp | 49 ++++++++--------
 src/slave/containerizer/mesos/containerizer.hpp | 10 ++--
 src/slave/http.cpp                              | 58 ++++++++++--------
 src/slave/slave.cpp                             | 13 +++-
 src/slave/slave.hpp                             |  2 +-
 src/tests/agent_container_api_tests.cpp         | 62 ++++++++++++++++++++
 9 files changed, 168 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/containerizer/composing.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.cpp b/src/slave/containerizer/composing.cpp
index 587f009..64919ef 100644
--- a/src/slave/containerizer/composing.cpp
+++ b/src/slave/containerizer/composing.cpp
@@ -65,7 +65,7 @@ public:
   Future<Nothing> recover(
       const Option<state::SlaveState>& state);
 
-  Future<bool> launch(
+  Future<Containerizer::LaunchResult> launch(
       const ContainerID& containerId,
       const ContainerConfig& config,
       const map<string, string>& environment,
@@ -103,18 +103,18 @@ private:
       const hashset<ContainerID>& containers);
   static Future<Nothing> ___recover();
 
-  Future<bool> _launch(
+  Future<Containerizer::LaunchResult> _launch(
       const ContainerID& containerId,
       const ContainerConfig& config,
       const map<string, string>& environment,
       const Option<std::string>& pidCheckpointPath,
       vector<Containerizer*>::iterator containerizer,
-      bool launched);
+      Containerizer::LaunchResult launchResult);
 
   // Continuation for nested containers.
-  Future<bool> _launch(
+  Future<Containerizer::LaunchResult> _launch(
       const ContainerID& containerId,
-      bool launched);
+      Containerizer::LaunchResult launchResult);
 
   vector<Containerizer*> containerizers_;
 
@@ -170,7 +170,7 @@ Future<Nothing> ComposingContainerizer::recover(
 }
 
 
-Future<bool> ComposingContainerizer::launch(
+Future<Containerizer::LaunchResult> ComposingContainerizer::launch(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig,
     const map<string, string>& environment,
@@ -321,22 +321,22 @@ Future<Nothing> ComposingContainerizerProcess::___recover()
 }
 
 
-Future<bool> ComposingContainerizerProcess::_launch(
+Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig,
     const map<string, string>& environment,
     const Option<std::string>& pidCheckpointPath,
     vector<Containerizer*>::iterator containerizer,
-    bool launched)
+    Containerizer::LaunchResult launchResult)
 {
   if (!containers_.contains(containerId)) {
     // If we are here a destroy started and finished in the interim.
-    return launched;
+    return launchResult;
   }
 
   Container* container = containers_.at(containerId);
 
-  if (launched) {
+  if (launchResult == Containerizer::LaunchResult::SUCCESS) {
     // Note that we don't update the state if a destroy is in progress.
     if (container->state == LAUNCHING) {
       container->state = LAUNCHED;
@@ -349,7 +349,7 @@ Future<bool> ComposingContainerizerProcess::_launch(
 
     // Note that the return value is not impacted
     // by whether a destroy is currently in progress.
-    return true;
+    return Containerizer::LaunchResult::SUCCESS;
   }
 
   // If we are here, the launch is not supported by `containerizer`.
@@ -370,9 +370,8 @@ Future<bool> ComposingContainerizerProcess::_launch(
     containers_.erase(containerId);
     delete container;
 
-    // We return false here because none of the
-    // containerizers support the launch.
-    return false;
+    // None of the containerizers support the launch.
+    return Containerizer::LaunchResult::NOT_SUPPORTED;
   }
 
   if (container->state == DESTROYING) {
@@ -412,14 +411,14 @@ Future<bool> ComposingContainerizerProcess::_launch(
 }
 
 
-Future<bool> ComposingContainerizerProcess::launch(
+Future<Containerizer::LaunchResult> ComposingContainerizerProcess::launch(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig,
     const map<string, string>& environment,
     const Option<std::string>& pidCheckpointPath)
 {
   if (containers_.contains(containerId)) {
-    return Failure("Duplicate container found");
+    return Containerizer::LaunchResult::ALREADY_LAUNCHED;
   }
 
   Container* container = new Container();
@@ -473,18 +472,18 @@ Future<bool> ComposingContainerizerProcess::launch(
 }
 
 
-Future<bool> ComposingContainerizerProcess::_launch(
+Future<Containerizer::LaunchResult> ComposingContainerizerProcess::_launch(
     const ContainerID& containerId,
-    bool launched)
+    Containerizer::LaunchResult launchResult)
 {
   if (!containers_.contains(containerId)) {
     // If we are here a destroy started and finished in the interim.
-    return launched;
+    return launchResult;
   }
 
   Container* container = containers_.at(containerId);
 
-  if (launched) {
+  if (launchResult == Containerizer::LaunchResult::SUCCESS) {
     // Note that we don't update the state if a destroy is in progress.
     if (container->state == LAUNCHING) {
       container->state = LAUNCHED;
@@ -497,7 +496,7 @@ Future<bool> ComposingContainerizerProcess::_launch(
 
     // Note that the return value is not impacted
     // by whether a destroy is currently in progress.
-    return true;
+    return Containerizer::LaunchResult::SUCCESS;
   }
 
   // If we are here, the launch is not supported by the containerizer.
@@ -512,8 +511,7 @@ Future<bool> ComposingContainerizerProcess::_launch(
   containers_.erase(containerId);
   delete container;
 
-  // We return false here because the launch is not supported.
-  return false;
+  return Containerizer::LaunchResult::NOT_SUPPORTED;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/containerizer/composing.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.hpp b/src/slave/containerizer/composing.hpp
index 06d68ee..c2689cf 100644
--- a/src/slave/containerizer/composing.hpp
+++ b/src/slave/containerizer/composing.hpp
@@ -54,7 +54,7 @@ public:
   virtual process::Future<Nothing> recover(
       const Option<state::SlaveState>& state);
 
-  virtual process::Future<bool> launch(
+  virtual process::Future<Containerizer::LaunchResult> launch(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/containerizer/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
index 449bb5d..2027bd9 100644
--- a/src/slave/containerizer/containerizer.hpp
+++ b/src/slave/containerizer/containerizer.hpp
@@ -57,6 +57,12 @@ struct SlaveState;
 class Containerizer
 {
 public:
+  enum class LaunchResult {
+    SUCCESS,
+    ALREADY_LAUNCHED,
+    NOT_SUPPORTED,
+  };
+
   // Attempts to create a containerizer as specified by 'isolation' in
   // flags.
   static Try<Containerizer*> create(
@@ -87,10 +93,7 @@ public:
   // a nested container.
   // NOTE: For nested containers, the required `directory` field of
   // the ContainerConfig will be determined by the containerizer.
-  //
-  // Returns true if launching this container is supported and it has
-  // been launched, otherwise false or a failure if something went wrong.
-  virtual process::Future<bool> launch(
+  virtual process::Future<LaunchResult> launch(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index c5ca76e..db5f044 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -559,7 +559,7 @@ Future<Nothing> MesosContainerizer::recover(
 }
 
 
-Future<bool> MesosContainerizer::launch(
+Future<Containerizer::LaunchResult> MesosContainerizer::launch(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig,
     const map<string, string>& environment,
@@ -1047,21 +1047,19 @@ Future<Nothing> MesosContainerizerProcess::__recover(
 // 4. Fetch any URIs.
 // 5. Signal the helper process to continue. It will first execute any
 //    preparation commands from isolators and then exec the starting command.
-Future<bool> MesosContainerizerProcess::launch(
+Future<Containerizer::LaunchResult> MesosContainerizerProcess::launch(
     const ContainerID& containerId,
     const ContainerConfig& _containerConfig,
     const map<string, string>& environment,
     const Option<std::string>& pidCheckpointPath)
 {
   if (containers_.contains(containerId)) {
-    return Failure(
-        (containerId.has_parent() ? "Nested container" : "Container") +
-        stringify(containerId) + " already started");
+    return Containerizer::LaunchResult::ALREADY_LAUNCHED;
   }
 
   if (_containerConfig.has_container_info() &&
       _containerConfig.container_info().type() != ContainerInfo::MESOS) {
-    return false;
+    return Containerizer::LaunchResult::NOT_SUPPORTED;
   }
 
   // NOTE: We make a copy of the ContainerConfig because we may need
@@ -1238,19 +1236,22 @@ Future<bool> MesosContainerizerProcess::launch(
       containerConfig.container_info().mesos().image());
 
   return container->provisioning
-    .then(defer(self(),
-                [=](const ProvisionInfo& provisionInfo) -> Future<bool> {
-      return prepare(containerId, provisionInfo)
-        .then(defer(self(), [this, containerId] () {
-          return ioSwitchboard->extractContainerIO(containerId);
-        }))
-        .then(defer(self(),
-                    &Self::_launch,
-                    containerId,
-                    lambda::_1,
-                    environment,
-                    pidCheckpointPath));
-    }));
+    .then(defer(
+        self(),
+        [=](const ProvisionInfo& provisionInfo)
+            -> Future<Containerizer::LaunchResult> {
+          return prepare(containerId, provisionInfo)
+            .then(defer(self(), [this, containerId] () {
+              return ioSwitchboard->extractContainerIO(containerId);
+            }))
+            .then(defer(
+                self(),
+                &Self::_launch,
+                containerId,
+                lambda::_1,
+                environment,
+                pidCheckpointPath));
+        }));
 }
 
 
@@ -1367,7 +1368,7 @@ Future<Nothing> MesosContainerizerProcess::fetch(
 }
 
 
-Future<bool> MesosContainerizerProcess::_launch(
+Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
     const ContainerID& containerId,
     const Option<ContainerIO>& containerIO,
     const map<string, string>& environment,
@@ -1908,7 +1909,7 @@ Future<bool> MesosContainerizerProcess::_launch(
 }
 
 
-Future<bool> MesosContainerizerProcess::isolate(
+Future<Nothing> MesosContainerizerProcess::isolate(
     const ContainerID& containerId,
     pid_t _pid)
 {
@@ -1958,11 +1959,11 @@ Future<bool> MesosContainerizerProcess::isolate(
 
   container->isolation = future;
 
-  return future.then([]() { return true; });
+  return future.then([]() { return Nothing(); });
 }
 
 
-Future<bool> MesosContainerizerProcess::exec(
+Future<Containerizer::LaunchResult> MesosContainerizerProcess::exec(
     const ContainerID& containerId,
     int_fd pipeWrite)
 {
@@ -1994,7 +1995,7 @@ Future<bool> MesosContainerizerProcess::exec(
 
   transition(containerId, RUNNING);
 
-  return true;
+  return Containerizer::LaunchResult::SUCCESS;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 6d356cc..f5d5146 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -78,7 +78,7 @@ public:
   virtual process::Future<Nothing> recover(
       const Option<state::SlaveState>& state);
 
-  virtual process::Future<bool> launch(
+  virtual process::Future<Containerizer::LaunchResult> launch(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,
@@ -143,7 +143,7 @@ public:
   virtual process::Future<Nothing> recover(
       const Option<state::SlaveState>& state);
 
-  virtual process::Future<bool> launch(
+  virtual process::Future<Containerizer::LaunchResult> launch(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,
@@ -165,7 +165,7 @@ public:
   virtual process::Future<Option<mesos::slave::ContainerTermination>> wait(
       const ContainerID& containerId);
 
-  virtual process::Future<bool> exec(
+  virtual process::Future<Containerizer::LaunchResult> exec(
       const ContainerID& containerId,
       int_fd pipeWrite);
 
@@ -217,13 +217,13 @@ private:
   process::Future<Nothing> fetch(
       const ContainerID& containerId);
 
-  process::Future<bool> _launch(
+  process::Future<Containerizer::LaunchResult> _launch(
       const ContainerID& containerId,
       const Option<mesos::slave::ContainerIO>& containerIO,
       const std::map<std::string, std::string>& environment,
       const Option<std::string>& pidCheckpointPath);
 
-  process::Future<bool> isolate(
+  process::Future<Nothing> isolate(
       const ContainerID& containerId,
       pid_t _pid);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index ff6d21d..c4d8098 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -2512,7 +2512,7 @@ Future<Response> Http::_launchContainer(
     containerConfig.set_directory(directory);
   }
 
-  Future<bool> launched = slave->containerizer->launch(
+  Future<Containerizer::LaunchResult> launched = slave->containerizer->launch(
       containerId,
       containerConfig,
       map<string, string>(),
@@ -2526,34 +2526,46 @@ Future<Response> Http::_launchContainer(
   //
   // TODO(bmahler): The containerizers currently require that
   // the caller calls destroy if the launch fails. See MESOS-6214.
-  launched
-    .onAny(defer(slave->self(), [=](const Future<bool>& launch) {
-      if (launch.isReady()) {
-        return;
-      }
+  launched.onAny(defer(
+      slave->self(),
+      [=](const Future<Containerizer::LaunchResult>& launchResult) {
+        if (launchResult.isReady()) {
+          return;
+        }
 
-      LOG(WARNING) << "Failed to launch container "
-                   << containerId << ": "
-                   << (launch.isFailed() ? launch.failure() : "discarded");
+        LOG(WARNING)
+          << "Failed to launch container " << containerId << ": "
+          << (launchResult.isFailed() ? launchResult.failure() : "discarded");
 
-      slave->containerizer->destroy(containerId)
-        .onAny([=](const Future<bool>& destroy) {
-          if (destroy.isReady()) {
-            return;
-          }
+        slave->containerizer->destroy(containerId)
+          .onAny([=](const Future<bool>& destroy) {
+            if (destroy.isReady()) {
+              return;
+            }
 
-          LOG(ERROR) << "Failed to destroy container "
-                     << containerId << " after launch failure: "
-                     << (destroy.isFailed() ? destroy.failure() : "discarded");
-        });
-    }));
+            LOG(ERROR)
+              << "Failed to destroy container " << containerId
+              << " after launch failure: "
+              << (destroy.isFailed() ? destroy.failure() : "discarded");
+          });
+      }));
 
   return launched
-    .then([](bool launched) -> Response {
-      if (!launched) {
-        return BadRequest("The provided ContainerInfo is not supported");
+    .then([](const Containerizer::LaunchResult launchResult) -> Response {
+      switch (launchResult) {
+        case Containerizer::LaunchResult::SUCCESS:
+          return OK();
+        case Containerizer::LaunchResult::ALREADY_LAUNCHED:
+          return Accepted();
+        case Containerizer::LaunchResult::NOT_SUPPORTED:
+          return BadRequest("The provided ContainerInfo is not supported");
+
+        // NOTE: By not setting a default we leverage the compiler
+        // errors when the enumeration is augmented to find all
+        // the cases we need to provide.
       }
-      return OK();
+
+      UNREACHABLE();
     })
     .repair([](const Future<Response>& launch) {
       // NOTE: Failures are automatically translated into 500 Internal Server

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index d8baceb..d8edc5e 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -5389,7 +5389,7 @@ void Slave::executorLaunched(
     const FrameworkID& frameworkId,
     const ExecutorID& executorId,
     const ContainerID& containerId,
-    const Future<bool>& future)
+    const Future<Containerizer::LaunchResult>& future)
 {
   // Set up callback for executor termination. Note that we do this
   // regardless of whether or not we have successfully launched the
@@ -5432,7 +5432,7 @@ void Slave::executorLaunched(
     }
 
     return;
-  } else if (!future.get()) {
+  } else if (future.get() == Containerizer::LaunchResult::NOT_SUPPORTED) {
     LOG(ERROR) << "Container '" << containerId
                << "' for executor '" << executorId
                << "' of framework " << frameworkId
@@ -5442,6 +5442,15 @@ void Slave::executorLaunched(
 
     ++metrics.container_launch_errors;
     return;
+  } else if (future.get() == Containerizer::LaunchResult::ALREADY_LAUNCHED) {
+    // This should be extremely rare, as the user would need to launch a
+    // standalone container with a user-specified UUID that happens to
+    // collide with the Agent-generated ContainerID for this launch.
+    LOG(ERROR) << "Container '" << containerId
+               << "' for executor '" << executorId
+               << "' of framework " << frameworkId
+               << " has already been launched.";
+    return;
   }
 
   Framework* framework = getFramework(frameworkId);

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index c0acaa6..40442f2 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -298,7 +298,7 @@ public:
       const FrameworkID& frameworkId,
       const ExecutorID& executorId,
       const ContainerID& containerId,
-      const process::Future<bool>& future);
+      const process::Future<Containerizer::LaunchResult>& future);
 
   // Made 'virtual' for Slave mocking.
   virtual void executorTerminated(

http://git-wip-us.apache.org/repos/asf/mesos/blob/feefdba2/src/tests/agent_container_api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/agent_container_api_tests.cpp b/src/tests/agent_container_api_tests.cpp
index 531ec73..595ce51 100644
--- a/src/tests/agent_container_api_tests.cpp
+++ b/src/tests/agent_container_api_tests.cpp
@@ -774,6 +774,68 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   EXPECT_TRUE(checkWaitContainerResponse(waitChild, SIGKILL));
 }
 
+
+// This test runs tries to send multiple calls to launch the same container
+// The first call is expected to succeed with 200 OK, and subsequent calls
+// should return 202 Accepted.
+TEST_P_TEMP_DISABLED_ON_WINDOWS(
+    AgentContainerAPITest, NestedContainerIdempotentLaunch)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.launcher = std::get<1>(std::get<3>(GetParam()));
+  slaveFlags.isolation = std::get<0>(std::get<3>(GetParam()));
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), slaveFlags);
+  ASSERT_SOME(slave);
+
+  Try<v1::ContainerID> parentContainerId =
+    launchParentContainer(master.get()->pid, slave.get()->pid);
+
+  ASSERT_SOME(parentContainerId);
+
+  // Launch a nested container and wait for it to finish.
+  v1::ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+  containerId.mutable_parent()->CopyFrom(parentContainerId.get());
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::OK().status,
+      launchNestedContainer(slave.get()->pid, containerId));
+
+  // NOTE: There should be an even number of launch requests to guard
+  // against regression related to MESOS-6214. e.g. If a launch request
+  // detects the container is already running, the containerizer should
+  // not accidentally destroy the container.
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::Accepted().status,
+      launchNestedContainer(slave.get()->pid, containerId));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::Accepted().status,
+      launchNestedContainer(slave.get()->pid, containerId));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::Accepted().status,
+      launchNestedContainer(slave.get()->pid, containerId));
+
+  Future<v1::agent::Response> wait =
+    deserialize(waitNestedContainer(slave.get()->pid, containerId));
+
+  EXPECT_TRUE(wait.isPending());
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::OK().status,
+      killNestedContainer(slave.get()->pid, containerId));
+
+  AWAIT_READY(wait);
+  EXPECT_TRUE(checkWaitContainerResponse(wait, SIGKILL));
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[10/15] mesos git commit: Updated test filters to include value-parameterized tests.

Posted by jo...@apache.org.
Updated test filters to include value-parameterized tests.

Test filters currently check the name of tests and type parameters
if they exist.  This adds value-parameterized tests to the set of
strings checked against when applying the default test filters.

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


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

Branch: refs/heads/master
Commit: 3c929e057bb20c72c23be22863b3239b6bb7c203
Parents: 9086ece
Author: Joseph Wu <jo...@apache.org>
Authored: Wed Oct 4 12:12:24 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 17:16:21 2017 -0800

----------------------------------------------------------------------
 3rdparty/stout/include/stout/tests/environment.hpp | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3c929e05/3rdparty/stout/include/stout/tests/environment.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/tests/environment.hpp b/3rdparty/stout/include/stout/tests/environment.hpp
index 0b30c18..8f8a420 100644
--- a/3rdparty/stout/include/stout/tests/environment.hpp
+++ b/3rdparty/stout/include/stout/tests/environment.hpp
@@ -53,6 +53,11 @@ public:
       return true;
     }
 
+    if (test->value_param() != nullptr &&
+        strings::contains(test->value_param(), pattern)) {
+      return true;
+    }
+
     return false;
   }
 };


[03/15] mesos git commit: Added basic tests for launching standalone containers.

Posted by jo...@apache.org.
Added basic tests for launching standalone containers.

Within the containerizer, standalone containers are treated like
any other container (such as those with executors/tasks), so the
MesosContainerizerTests focus on the difference in arguments passed
for standalone containers.

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


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

Branch: refs/heads/master
Commit: f370a32d2c05a7d8d9606aaeb1f6b01dd884abdc
Parents: 1414578
Author: Joseph Wu <jo...@apache.org>
Authored: Wed Jul 12 12:24:59 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 16:58:40 2017 -0800

----------------------------------------------------------------------
 .../containerizer/mesos_containerizer_tests.cpp | 45 ++++++++++++++++++++
 src/tests/mesos.hpp                             | 27 ++++++++++++
 2 files changed, 72 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f370a32d/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 ef42415..f73e0bf 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -149,6 +149,51 @@ TEST_F(MesosContainerizerTest, Launch)
 }
 
 
+TEST_F(MesosContainerizerTest, StandaloneLaunch)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.launcher = "posix";
+  flags.isolation = "posix/cpu,posix/mem";
+
+  Fetcher fetcher(flags);
+
+  Try<MesosContainerizer*> create = MesosContainerizer::create(
+      flags,
+      true,
+      &fetcher);
+
+  ASSERT_SOME(create);
+
+  Owned<MesosContainerizer> containerizer(create.get());
+
+  SlaveState state;
+  state.id = SlaveID();
+
+  AWAIT_READY(containerizer->recover(state));
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  Future<bool> launch = containerizer->launch(
+      containerId,
+      createContainerConfig(
+          createCommandInfo("exit 42"),
+          "cpus:1;mem:64",
+          sandbox.get()),
+      map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_TRUE(launch);
+
+  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(42, wait.get()->status());
+}
+
+
 TEST_F(MesosContainerizerTest, Destroy)
 {
   slave::Flags flags = CreateSlaveFlags();

http://git-wip-us.apache.org/repos/asf/mesos/blob/f370a32d/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index f251732..345b883 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -1420,6 +1420,33 @@ inline mesos::slave::ContainerConfig createContainerConfig(
 }
 
 
+// Helper for creating standalone container configs.
+inline mesos::slave::ContainerConfig createContainerConfig(
+    const CommandInfo& commandInfo,
+    const std::string& resources,
+    const std::string& sandboxDirectory,
+    const Option<ContainerInfo>& containerInfo = None(),
+    const Option<std::string>& user = None())
+{
+  mesos::slave::ContainerConfig containerConfig;
+  containerConfig.mutable_command_info()->CopyFrom(commandInfo);
+  containerConfig.mutable_resources()->CopyFrom(
+      Resources::parse(resources).get());
+
+  containerConfig.set_directory(sandboxDirectory);
+
+  if (user.isSome()) {
+    containerConfig.set_user(user.get());
+  }
+
+  if (containerInfo.isSome()) {
+    containerConfig.mutable_container_info()->CopyFrom(containerInfo.get());
+  }
+
+  return containerConfig;
+}
+
+
 template <typename... Args>
 inline Image createDockerImage(Args&&... args)
 {


[14/15] mesos git commit: Updated tests with Containerizer::launch interface change.

Posted by jo...@apache.org.
Updated tests with Containerizer::launch interface change.

This commit contains the test changes required due to the interface
change in https://reviews.apache.org/r/63063 .

Instead of `AWAIT_ASSERT_TRUE(...)`, the affected test lines now use
`AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, ...)`.

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


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

Branch: refs/heads/master
Commit: 38951cf6046b7fa2d4610999cc1bedd6916ffded
Parents: 6f3a576
Author: Joseph Wu <jo...@apache.org>
Authored: Mon Oct 16 18:40:09 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 17:16:22 2017 -0800

----------------------------------------------------------------------
 src/tests/containerizer.cpp                     |   8 +-
 src/tests/containerizer.hpp                     |   4 +-
 .../composing_containerizer_tests.cpp           |  22 ++--
 .../containerizer/io_switchboard_tests.cpp      |  15 ++-
 src/tests/containerizer/isolator_tests.cpp      |  68 +++++-----
 .../linux_filesystem_isolator_tests.cpp         |  32 ++---
 .../containerizer/mesos_containerizer_tests.cpp |  51 ++++----
 src/tests/containerizer/mock_containerizer.hpp  |   2 +-
 .../nested_mesos_containerizer_tests.cpp        | 131 ++++++++++---------
 .../volume_host_path_isolator_tests.cpp         |  18 +--
 .../volume_image_isolator_tests.cpp             |  12 +-
 .../volume_sandbox_path_isolator_tests.cpp      |  23 ++--
 .../volume_secret_isolator_tests.cpp            |   7 +-
 src/tests/hook_tests.cpp                        |   6 +-
 src/tests/mock_docker.hpp                       |   4 +-
 src/tests/slave_recovery_tests.cpp              |   2 +-
 src/tests/slave_tests.cpp                       |   7 +-
 17 files changed, 210 insertions(+), 202 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.cpp b/src/tests/containerizer.cpp
index cd140f4..c6f1ec0 100644
--- a/src/tests/containerizer.cpp
+++ b/src/tests/containerizer.cpp
@@ -86,7 +86,7 @@ public:
     return Nothing();
   }
 
-  Future<bool> launch(
+  Future<slave::Containerizer::LaunchResult> launch(
       const ContainerID& containerId,
       const ContainerConfig& containerConfig,
       const map<string, string>& environment,
@@ -110,7 +110,7 @@ public:
     if (containerId.has_parent()) {
       // Launching a nested container via the test containerizer is a
       // no-op for now.
-      return true;
+      return slave::Containerizer::LaunchResult::SUCCESS;
     }
 
     CHECK(executors.contains(containerConfig.executor_info().executor_id()))
@@ -196,7 +196,7 @@ public:
       }
     }
 
-    return true;
+    return slave::Containerizer::LaunchResult::SUCCESS;
   }
 
   Future<Nothing> update(
@@ -450,7 +450,7 @@ Future<Nothing> TestContainerizer::_recover(
 }
 
 
-Future<bool> TestContainerizer::_launch(
+Future<slave::Containerizer::LaunchResult> TestContainerizer::_launch(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig,
     const map<string, string>& environment,

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.hpp b/src/tests/containerizer.hpp
index a778b85..c98913f 100644
--- a/src/tests/containerizer.hpp
+++ b/src/tests/containerizer.hpp
@@ -86,7 +86,7 @@ public:
 
   MOCK_METHOD4(
       launch,
-      process::Future<bool>(
+      process::Future<slave::Containerizer::LaunchResult>(
           const ContainerID&,
           const mesos::slave::ContainerConfig&,
           const std::map<std::string, std::string>&,
@@ -138,7 +138,7 @@ private:
   process::Future<Nothing> _recover(
       const Option<slave::state::SlaveState>& state);
 
-  process::Future<bool> _launch(
+  process::Future<slave::Containerizer::LaunchResult> _launch(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/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 61e47e9..b759ba2 100644
--- a/src/tests/containerizer/composing_containerizer_tests.cpp
+++ b/src/tests/containerizer/composing_containerizer_tests.cpp
@@ -83,7 +83,7 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
   SlaveID slaveId;
   map<string, string> environment;
 
-  Promise<bool> launchPromise;
+  Promise<Containerizer::LaunchResult> launchPromise;
 
   EXPECT_CALL(*mockContainerizer1, launch(_, _, _, _))
     .WillOnce(Return(launchPromise.future()));
@@ -94,7 +94,7 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
     .WillOnce(DoAll(FutureSatisfy(&destroy),
                     Return(destroyPromise.future())));
 
-  Future<bool> launched = containerizer.launch(
+  Future<Containerizer::LaunchResult> launched = containerizer.launch(
       containerId,
       createContainerConfig(taskInfo, executorInfo, "dir", "user"),
       environment,
@@ -114,7 +114,7 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
   // container is already destroyed.
   AWAIT_READY(destroy);
 
-  launchPromise.set(false);
+  launchPromise.set(Containerizer::LaunchResult::NOT_SUPPORTED);
   destroyPromise.set(false);
 
   // `launched` should be a failure and `destroyed` should be true
@@ -149,7 +149,7 @@ TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
   SlaveID slaveId;
   map<string, string> environment;
 
-  Promise<bool> launchPromise;
+  Promise<Containerizer::LaunchResult> launchPromise;
 
   EXPECT_CALL(*mockContainerizer1, launch(_, _, _, _))
     .WillOnce(Return(launchPromise.future()));
@@ -160,7 +160,7 @@ TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
     .WillOnce(DoAll(FutureSatisfy(&destroy),
                     Return(destroyPromise.future())));
 
-  Future<bool> launched = containerizer.launch(
+  Future<Containerizer::LaunchResult> launched = containerizer.launch(
       containerId,
       createContainerConfig(taskInfo, executorInfo, "dir", "user"),
       environment,
@@ -180,12 +180,12 @@ TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
   // container is already destroyed.
   AWAIT_READY(destroy);
 
-  launchPromise.set(true);
+  launchPromise.set(Containerizer::LaunchResult::SUCCESS);
   destroyPromise.set(false);
 
   // `launched` should return true and `destroyed` should return false
   // because the launch succeeded and `destroyPromise` was set to false.
-  AWAIT_EXPECT_EQ(true, launched);
+  AWAIT_EXPECT_EQ(Containerizer::LaunchResult::SUCCESS, launched);
   AWAIT_EXPECT_EQ(false, destroyed);
 }
 
@@ -211,7 +211,7 @@ TEST_F(ComposingContainerizerTest, DestroyAfterLaunchLoop)
   SlaveID slaveId;
   map<string, string> environment;
 
-  Promise<bool> launchPromise;
+  Promise<Containerizer::LaunchResult> launchPromise;
 
   EXPECT_CALL(*mockContainerizer1, launch(_, _, _, _))
     .WillOnce(Return(launchPromise.future()));
@@ -222,7 +222,7 @@ TEST_F(ComposingContainerizerTest, DestroyAfterLaunchLoop)
     .WillOnce(DoAll(FutureSatisfy(&destroy),
                     Return(destroyPromise.future())));
 
-  Future<bool> launched = containerizer.launch(
+  Future<Containerizer::LaunchResult> launched = containerizer.launch(
       containerId,
       createContainerConfig(taskInfo, executorInfo, "dir", "user"),
       environment,
@@ -237,12 +237,12 @@ TEST_F(ComposingContainerizerTest, DestroyAfterLaunchLoop)
   // We make sure the destroy is being called on the containerizer.
   AWAIT_READY(destroy);
 
-  launchPromise.set(false);
+  launchPromise.set(Containerizer::LaunchResult::NOT_SUPPORTED);
   destroyPromise.set(false);
 
   // `launch` should return false and `destroyed` should return false
   // because none of the containerizers support the launch.
-  AWAIT_EXPECT_EQ(false, launched);
+  AWAIT_EXPECT_EQ(Containerizer::LaunchResult::NOT_SUPPORTED, launched);
   AWAIT_EXPECT_EQ(false, destroyed);
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/containerizer/io_switchboard_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/io_switchboard_tests.cpp b/src/tests/containerizer/io_switchboard_tests.cpp
index be1078e..fe9643a 100644
--- a/src/tests/containerizer/io_switchboard_tests.cpp
+++ b/src/tests/containerizer/io_switchboard_tests.cpp
@@ -63,6 +63,7 @@ namespace paths = mesos::internal::slave::containerizer::paths;
 using mesos::agent::Call;
 using mesos::agent::ProcessIO;
 
+using mesos::internal::slave::Containerizer;
 using mesos::internal::slave::Fetcher;
 using mesos::internal::slave::IOSwitchboardServer;
 using mesos::internal::slave::MesosContainerizer;
@@ -742,13 +743,13 @@ TEST_F(IOSwitchboardTest, ContainerAttach)
   executorInfo.mutable_container()->set_type(ContainerInfo::MESOS);
   executorInfo.mutable_container()->mutable_tty_info();
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executorInfo, directory.get()),
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<http::Connection> connection = containerizer->attach(containerId);
   AWAIT_READY(connection);
@@ -807,13 +808,13 @@ TEST_F(IOSwitchboardTest, OutputRedirectionWithTTY)
   executorInfo.mutable_container()->set_type(ContainerInfo::MESOS);
   executorInfo.mutable_container()->mutable_tty_info();
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executorInfo, directory.get()),
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -861,13 +862,13 @@ TEST_F(IOSwitchboardTest, KillSwitchboardContainerDestroyed)
       "sleep 1000",
       "cpus:1");
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executorInfo, directory.get()),
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   ContainerID childContainerId;
   childContainerId.mutable_parent()->CopyFrom(containerId);
@@ -882,7 +883,7 @@ TEST_F(IOSwitchboardTest, KillSwitchboardContainerDestroyed)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Result<pid_t> pid = paths::getContainerIOSwitchboardPid(
         flags.runtime_dir, childContainerId);

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/containerizer/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index 4ad42bc..b341a86 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -47,6 +47,7 @@ using std::string;
 using process::Future;
 using process::Owned;
 
+using mesos::internal::slave::Containerizer;
 using mesos::internal::slave::Fetcher;
 using mesos::internal::slave::MesosContainerizer;
 
@@ -121,17 +122,17 @@ TEST_F(NamespacesIsolatorTest, ROOT_PidNamespace)
   const string command =
     "stat -Lc %i /proc/self/ns/pid > ns && (cat /proc/1/comm > init)";
 
-  process::Future<bool> launch = containerizer.get()->launch(
-      containerId,
-      createContainerConfig(
-          None(),
-          createExecutorInfo("executor", command),
-          directory),
-      std::map<string, string>(),
-      None());
+  process::Future<Containerizer::LaunchResult> launch =
+    containerizer.get()->launch(
+        containerId,
+        createContainerConfig(
+            None(),
+            createExecutorInfo("executor", command),
+            directory),
+        std::map<string, string>(),
+        None());
 
-  AWAIT_READY(launch);
-  ASSERT_TRUE(launch.get());
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Wait on the container.
   Future<Option<ContainerTermination>> wait =
@@ -189,14 +190,14 @@ TEST_F(NamespacesIsolatorTest, ROOT_SharePidNamespace)
   container->set_type(ContainerInfo::MESOS);
   container->mutable_linux_info()->set_share_pid_namespace(true);
 
-  process::Future<bool> launch = containerizer.get()->launch(
-      containerId,
-      containerConfig,
-      std::map<string, string>(),
-      None());
+  process::Future<Containerizer::LaunchResult> launch =
+    containerizer.get()->launch(
+        containerId,
+        containerConfig,
+        std::map<string, string>(),
+        None());
 
-  AWAIT_READY(launch);
-  ASSERT_TRUE(launch.get());
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Wait on the container.
   Future<Option<ContainerTermination>> wait =
@@ -242,11 +243,12 @@ TEST_F(NamespacesIsolatorTest, ROOT_SharePidNamespaceWhenDisallow)
   container->set_type(ContainerInfo::MESOS);
   container->mutable_linux_info()->set_share_pid_namespace(true);
 
-  process::Future<bool> launch = containerizer.get()->launch(
-      containerId,
-      containerConfig,
-      std::map<string, string>(),
-      None());
+  process::Future<Containerizer::LaunchResult> launch =
+    containerizer.get()->launch(
+        containerId,
+        containerConfig,
+        std::map<string, string>(),
+        None());
 
   AWAIT_FAILED(launch);
 }
@@ -276,17 +278,17 @@ TEST_F(NamespacesIsolatorTest, ROOT_IPCNamespace)
     "echo " + stringify(shmmaxValue) + " > /proc/sys/kernel/shmmax;"
     "cp /proc/sys/kernel/shmmax shmmax";
 
-  process::Future<bool> launch = containerizer.get()->launch(
-      containerId,
-      createContainerConfig(
-          None(),
-          createExecutorInfo("executor", command),
-          directory),
-      std::map<string, string>(),
-      None());
-
-  AWAIT_READY(launch);
-  ASSERT_TRUE(launch.get());
+  process::Future<Containerizer::LaunchResult> launch =
+    containerizer.get()->launch(
+        containerId,
+        createContainerConfig(
+            None(),
+            createExecutorInfo("executor", command),
+            directory),
+        std::map<string, string>(),
+        None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Wait on the container.
   Future<Option<ContainerTermination>> wait =

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/linux_filesystem_isolator_tests.cpp b/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
index 4dfd90b..655f9f8 100644
--- a/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
+++ b/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
@@ -106,13 +106,13 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ChangeRootFilesystem)
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
       None());
 
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -158,13 +158,13 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_Metrics)
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
       None());
 
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   JSON::Object stats = Metrics();
   EXPECT_EQ(1u, stats.values.count(
@@ -231,13 +231,13 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithRootFilesystem)
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
       None());
 
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -307,13 +307,13 @@ TEST_F(LinuxFilesystemIsolatorTest,
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
       None());
 
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -372,13 +372,13 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithoutRootFilesystem)
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
       None());
 
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -444,13 +444,13 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_MultipleContainers)
   string directory1 = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory1));
 
-  Future<bool> launch1 = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch1 = containerizer->launch(
       containerId1,
       createContainerConfig(None(), executor1, directory1),
       map<string, string>(),
       None());
 
-  AWAIT_READY(launch1);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch1);
 
   ExecutorInfo executor2 = createExecutorInfo(
       "test_executor2",
@@ -461,13 +461,13 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_MultipleContainers)
   string directory2 = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory2));
 
-  Future<bool> launch2 = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch2 = containerizer->launch(
       containerId2,
       createContainerConfig(None(), executor2, directory2),
       map<string, string>(),
       None());
 
-  AWAIT_READY(launch1);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch1);
 
   // Wait on the containers.
   Future<Option<ContainerTermination>> wait1 =
@@ -616,13 +616,13 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeMountPointCleanup)
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
       None());
 
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   ASSERT_SOME(os::touch(path::join(directory, "volume", "abc")));
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/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 f73e0bf..ce67def 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -58,6 +58,7 @@ using namespace process;
 
 using mesos::internal::master::Master;
 
+using mesos::internal::slave::Containerizer;
 using mesos::internal::slave::executorEnvironment;
 using mesos::internal::slave::Fetcher;
 using mesos::internal::slave::FetcherProcess;
@@ -129,7 +130,7 @@ TEST_F(MesosContainerizerTest, Launch)
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -138,7 +139,7 @@ TEST_F(MesosContainerizerTest, Launch)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -174,7 +175,7 @@ TEST_F(MesosContainerizerTest, StandaloneLaunch)
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           createCommandInfo("exit 42"),
@@ -183,7 +184,7 @@ TEST_F(MesosContainerizerTest, StandaloneLaunch)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -222,7 +223,7 @@ TEST_F(MesosContainerizerTest, Destroy)
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -231,7 +232,7 @@ TEST_F(MesosContainerizerTest, Destroy)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -274,7 +275,7 @@ TEST_F(MesosContainerizerTest, StatusWithContainerID)
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -283,7 +284,7 @@ TEST_F(MesosContainerizerTest, StatusWithContainerID)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<ContainerStatus> status = containerizer->status(containerId);
   AWAIT_READY(status);
@@ -374,7 +375,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptSucceeds)
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -384,7 +385,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptSucceeds)
       None());
 
   // Wait until the launch completes.
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Wait for the child (preparation script + executor) to complete.
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
@@ -421,7 +422,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptFails)
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -431,7 +432,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptFails)
       None());
 
   // Wait until the launch completes.
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Wait for the child (preparation script + executor) to complete.
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
@@ -480,7 +481,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -490,7 +491,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
       None());
 
   // Wait until the launch completes.
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Wait for the child (preparation script(s) + executor) to complete.
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
@@ -559,14 +560,14 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ExecutorEnvironmentVariable)
       None(),
       false);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executorInfo, sandbox.get()),
       environment,
       None());
 
   // Wait until the launch completes.
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Wait for the child (preparation script + executor) to complete.
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
@@ -620,7 +621,7 @@ TEST_F(MesosContainerizerExecuteTest, IoRedirection)
   string command =
     "(echo '" + errMsg + "' 1>&2) && echo '" + outMsg + "'";
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -630,7 +631,7 @@ TEST_F(MesosContainerizerExecuteTest, IoRedirection)
       None());
 
   // Wait for the launch to complete.
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Wait on the container.
   Future<Option<ContainerTermination>> wait =
@@ -677,14 +678,14 @@ TEST_F(MesosContainerizerExecuteTest, ROOT_SandboxFileOwnership)
   ExecutorInfo executor = createExecutorInfo("executor", "exit 0");
   executor.mutable_command()->set_user(user);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, sandbox.get()),
       map<string, string>(),
       None());
 
   // Wait for the launch to complete.
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Result<uid_t> uid = os::getuid(user);
   ASSERT_SOME(uid);
@@ -965,7 +966,7 @@ TEST_F(MesosContainerizerProvisionerTest, ProvisionFailed)
   ExecutorInfo executorInfo = createExecutorInfo("executor", "exit 0");
   executorInfo.mutable_container()->CopyFrom(containerInfo);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(taskInfo, executorInfo, sandbox.get()),
       map<string, string>(),
@@ -1049,7 +1050,7 @@ TEST_F(MesosContainerizerProvisionerTest, DestroyWhileProvisioning)
   ExecutorInfo executorInfo = createExecutorInfo("executor", "exit 0");
   executorInfo.mutable_container()->CopyFrom(containerInfo);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(taskInfo, executorInfo, sandbox.get()),
       map<string, string>(),
@@ -1140,7 +1141,7 @@ TEST_F(MesosContainerizerProvisionerTest, IsolatorCleanupBeforePrepare)
   ExecutorInfo executorInfo = createExecutorInfo("executor", "exit 0");
   executorInfo.mutable_container()->CopyFrom(containerInfo);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(taskInfo, executorInfo, sandbox.get()),
       map<string, string>(),
@@ -1218,7 +1219,7 @@ TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
     .WillOnce(DoAll(InvokeDestroyAndWait(testLauncher),
                     Return(Failure("Destroy failure"))));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           taskInfo,
@@ -1227,7 +1228,7 @@ TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
       map<string, string>(),
       None());
 
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/containerizer/mock_containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/mock_containerizer.hpp b/src/tests/containerizer/mock_containerizer.hpp
index 0adcb01..5befccc 100644
--- a/src/tests/containerizer/mock_containerizer.hpp
+++ b/src/tests/containerizer/mock_containerizer.hpp
@@ -48,7 +48,7 @@ public:
 
   MOCK_METHOD4(
       launch,
-      process::Future<bool>(
+      process::Future<slave::Containerizer::LaunchResult>(
           const ContainerID&,
           const mesos::slave::ContainerConfig&,
           const std::map<std::string, std::string>&,

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
index 983ecd1..d9b1173 100644
--- a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
@@ -47,6 +47,7 @@
 #include "tests/environment.hpp"
 #include "tests/mesos.hpp"
 
+using mesos::internal::slave::Containerizer;
 using mesos::internal::slave::Fetcher;
 using mesos::internal::slave::MesosContainerizer;
 
@@ -195,7 +196,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNested)
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -204,7 +205,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNested)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Now launch nested container.
   ContainerID nestedContainerId;
@@ -217,7 +218,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNested)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(
       nestedContainerId);
@@ -278,7 +279,7 @@ TEST_F(NestedMesosContainerizerTest,
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
@@ -289,7 +290,7 @@ TEST_F(NestedMesosContainerizerTest,
           executor.executor_id(),
           containerId));
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<ContainerStatus> status = containerizer->status(containerId);
   AWAIT_READY(status);
@@ -304,7 +305,7 @@ TEST_F(NestedMesosContainerizerTest,
     nestedContainerId.mutable_parent()->CopyFrom(containerId);
     nestedContainerId.set_value(UUID::random().toString());
 
-    Future<bool> launchNested = containerizer->launch(
+    Future<Containerizer::LaunchResult> launchNested = containerizer->launch(
         nestedContainerId,
         createContainerConfig(
             createCommandInfo("exit $" + envKey),
@@ -313,7 +314,7 @@ TEST_F(NestedMesosContainerizerTest,
         map<string, string>(),
         None());
 
-    AWAIT_ASSERT_TRUE(launchNested);
+    AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launchNested);
 
     Future<Option<ContainerTermination>> waitNested = containerizer->wait(
         nestedContainerId);
@@ -338,7 +339,7 @@ TEST_F(NestedMesosContainerizerTest,
     nestedContainerId.mutable_parent()->CopyFrom(containerId);
     nestedContainerId.set_value(UUID::random().toString());
 
-    Future<bool> launchNested = containerizer->launch(
+    Future<Containerizer::LaunchResult> launchNested = containerizer->launch(
         nestedContainerId,
         createContainerConfig(
             nestedCommand,
@@ -347,7 +348,7 @@ TEST_F(NestedMesosContainerizerTest,
         map<string, string>(),
         None());
 
-    AWAIT_ASSERT_TRUE(launchNested);
+    AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launchNested);
 
     Future<Option<ContainerTermination>> waitNested = containerizer->wait(
         nestedContainerId);
@@ -389,7 +390,7 @@ TEST_F(NestedMesosContainerizerTest,
     nestedContainerId.mutable_parent()->CopyFrom(containerId);
     nestedContainerId.set_value(UUID::random().toString());
 
-    Future<bool> launchNested = containerizer->launch(
+    Future<Containerizer::LaunchResult> launchNested = containerizer->launch(
         nestedContainerId,
         createContainerConfig(
             createCommandInfo("exit $" + envKey),
@@ -398,7 +399,7 @@ TEST_F(NestedMesosContainerizerTest,
         map<string, string>(),
         None());
 
-    AWAIT_ASSERT_TRUE(launchNested);
+    AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launchNested);
 
     Future<Option<ContainerTermination>> waitNested = containerizer->wait(
         nestedContainerId);
@@ -472,7 +473,7 @@ TEST_F(NestedMesosContainerizerTest,
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -481,7 +482,7 @@ TEST_F(NestedMesosContainerizerTest,
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Wait for the parent container to start running its task
   // before launching a debug container inside it.
@@ -508,7 +509,7 @@ TEST_F(NestedMesosContainerizerTest,
         "read PARENT_SANDBOX <&" + stringify(pipes[0]) + ";"
         "[ ${PARENT_SANDBOX} == ${MESOS_SANDBOX} ] && exit 0 || exit 1;");
 
-    Future<bool> launchNested = containerizer->launch(
+    Future<Containerizer::LaunchResult> launchNested = containerizer->launch(
         nestedContainerId,
         createContainerConfig(
             nestedCommand,
@@ -517,7 +518,7 @@ TEST_F(NestedMesosContainerizerTest,
         map<string, string>(),
         None());
 
-    AWAIT_ASSERT_TRUE(launchNested);
+    AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launchNested);
 
     Future<Option<ContainerTermination>> waitNested = containerizer->wait(
         nestedContainerId);
@@ -596,7 +597,7 @@ TEST_F(NestedMesosContainerizerTest,
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -610,7 +611,7 @@ TEST_F(NestedMesosContainerizerTest,
           executor.executor_id(),
           containerId));
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Wait for the parent container to start running its task
   // before launching a debug container inside it.
@@ -630,7 +631,7 @@ TEST_F(NestedMesosContainerizerTest,
     nestedContainerId.mutable_parent()->CopyFrom(containerId);
     nestedContainerId.set_value(UUID::random().toString());
 
-    Future<bool> launchNested = containerizer->launch(
+    Future<Containerizer::LaunchResult> launchNested = containerizer->launch(
         nestedContainerId,
         createContainerConfig(
             createCommandInfo("ls " + filename),
@@ -639,7 +640,7 @@ TEST_F(NestedMesosContainerizerTest,
         map<string, string>(),
         None());
 
-    AWAIT_ASSERT_TRUE(launchNested);
+    AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launchNested);
 
     Future<Option<ContainerTermination>> waitNested = containerizer->wait(
         nestedContainerId);
@@ -680,7 +681,7 @@ TEST_F(NestedMesosContainerizerTest,
     nestedContainerId.mutable_parent()->CopyFrom(containerId);
     nestedContainerId.set_value(UUID::random().toString());
 
-    Future<bool> launchNested = containerizer->launch(
+    Future<Containerizer::LaunchResult> launchNested = containerizer->launch(
         nestedContainerId,
         createContainerConfig(
             createCommandInfo("ls " + filename),
@@ -689,7 +690,7 @@ TEST_F(NestedMesosContainerizerTest,
         map<string, string>(),
         None());
 
-    AWAIT_ASSERT_TRUE(launchNested);
+    AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launchNested);
 
     Future<Option<ContainerTermination>> waitNested = containerizer->wait(
         nestedContainerId);
@@ -741,7 +742,7 @@ TEST_F(NestedMesosContainerizerTest,
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -750,7 +751,7 @@ TEST_F(NestedMesosContainerizerTest,
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Now launch nested container.
   ContainerID nestedContainerId;
@@ -779,7 +780,7 @@ TEST_F(NestedMesosContainerizerTest,
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(
       nestedContainerId);
@@ -807,7 +808,7 @@ TEST_F(NestedMesosContainerizerTest,
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   wait = containerizer->wait(nestedContainerId);
 
@@ -860,7 +861,7 @@ TEST_F(NestedMesosContainerizerTest,
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -872,7 +873,7 @@ TEST_F(NestedMesosContainerizerTest,
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Launch the first nested container which will share pid namespace
   // with the parent container.
@@ -892,7 +893,7 @@ TEST_F(NestedMesosContainerizerTest,
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(
       nestedContainerId1);
@@ -917,7 +918,7 @@ TEST_F(NestedMesosContainerizerTest,
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   wait = containerizer->wait(nestedContainerId2);
 
@@ -1066,7 +1067,7 @@ TEST_F(NestedMesosContainerizerTest,
   // Launch a debug container inside the command task and check for
   // the existence of a file we know to be inside the `alpine` docker
   // image (but not on the host filesystem).
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       nestedContainerId,
       createContainerConfig(
           createCommandInfo(
@@ -1079,7 +1080,7 @@ TEST_F(NestedMesosContainerizerTest,
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait =
     containerizer->wait(nestedContainerId);
@@ -1128,7 +1129,7 @@ TEST_F(NestedMesosContainerizerTest,
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
@@ -1139,7 +1140,7 @@ TEST_F(NestedMesosContainerizerTest,
           executor.executor_id(),
           containerId));
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<ContainerStatus> status = containerizer->status(containerId);
   AWAIT_READY(status);
@@ -1161,7 +1162,7 @@ TEST_F(NestedMesosContainerizerTest,
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   status = containerizer->status(nestedContainerId);
   AWAIT_READY(status);
@@ -1241,7 +1242,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyNested)
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -1250,7 +1251,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyNested)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Now launch nested container.
   ContainerID nestedContainerId;
@@ -1263,7 +1264,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyNested)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> nestedWait = containerizer->wait(
       nestedContainerId);
@@ -1318,7 +1319,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyParent)
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -1327,7 +1328,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyParent)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Now launch nested container.
   ContainerID nestedContainerId;
@@ -1340,7 +1341,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyParent)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -1411,7 +1412,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentExit)
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
@@ -1419,7 +1420,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentExit)
 
   close(pipes[0]); // We're never going to read.
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Now launch nested container.
   ContainerID nestedContainerId;
@@ -1432,7 +1433,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentExit)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -1507,13 +1508,13 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentSigterm)
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   close(pipes[1]);
 
@@ -1528,7 +1529,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentSigterm)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -1597,7 +1598,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNested)
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
@@ -1608,7 +1609,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNested)
           executor.executor_id(),
           containerId));
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<ContainerStatus> status = containerizer->status(containerId);
   AWAIT_READY(status);
@@ -1627,7 +1628,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNested)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   status = containerizer->status(nestedContainerId);
   AWAIT_READY(status);
@@ -1778,7 +1779,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNestedLauncherOrphans)
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
@@ -1789,7 +1790,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNestedLauncherOrphans)
           executor.executor_id(),
           containerId));
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<ContainerStatus> status = containerizer->status(containerId);
   AWAIT_READY(status);
@@ -1973,7 +1974,7 @@ TEST_F(NestedMesosContainerizerTest,
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
@@ -1984,7 +1985,7 @@ TEST_F(NestedMesosContainerizerTest,
           executor.executor_id(),
           containerId));
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<ContainerStatus> status = containerizer->status(containerId);
   AWAIT_READY(status);
@@ -2113,7 +2114,7 @@ TEST_F(NestedMesosContainerizerTest,
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
@@ -2124,7 +2125,7 @@ TEST_F(NestedMesosContainerizerTest,
           executor.executor_id(),
           containerId));
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<ContainerStatus> status = containerizer->status(containerId);
   AWAIT_READY(status);
@@ -2143,7 +2144,7 @@ TEST_F(NestedMesosContainerizerTest,
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   status = containerizer->status(nestedContainerId1);
   AWAIT_READY(status);
@@ -2356,7 +2357,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_AgentEnvironmentNotLeaked)
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -2365,7 +2366,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_AgentEnvironmentNotLeaked)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Now launch nested container.
   ContainerID nestedContainerId;
@@ -2395,7 +2396,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_AgentEnvironmentNotLeaked)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(
       nestedContainerId);
@@ -2444,7 +2445,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_Remove)
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -2453,7 +2454,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_Remove)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Now launch nested container.
   ContainerID nestedContainerId;
@@ -2466,7 +2467,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_Remove)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait =
     containerizer->wait(nestedContainerId);
@@ -2533,7 +2534,7 @@ TEST_F(NestedMesosContainerizerTest,
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(
           None(),
@@ -2542,7 +2543,7 @@ TEST_F(NestedMesosContainerizerTest,
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Now launch nested container.
   ContainerID nestedContainerId;
@@ -2555,7 +2556,7 @@ TEST_F(NestedMesosContainerizerTest,
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait =
     containerizer->wait(nestedContainerId);

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/containerizer/volume_host_path_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/volume_host_path_isolator_tests.cpp b/src/tests/containerizer/volume_host_path_isolator_tests.cpp
index 1d00672..89710f7 100644
--- a/src/tests/containerizer/volume_host_path_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_host_path_isolator_tests.cpp
@@ -98,13 +98,13 @@ TEST_F(VolumeHostPathIsolatorTest, ROOT_VolumeFromHost)
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
       None());
 
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -155,13 +155,14 @@ TEST_F(VolumeHostPathIsolatorTest, ROOT_FileVolumeFromHost)
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
       None());
 
-  AWAIT_READY_FOR(launch, Seconds(60));
+  AWAIT_ASSERT_EQ_FOR(
+      Containerizer::LaunchResult::SUCCESS, launch, Seconds(60));
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -212,13 +213,13 @@ TEST_F(VolumeHostPathIsolatorTest, ROOT_VolumeFromHostSandboxMountPoint)
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
       None());
 
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -269,13 +270,14 @@ TEST_F(VolumeHostPathIsolatorTest, ROOT_FileVolumeFromHostSandboxMountPoint)
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
       None());
 
-  AWAIT_READY_FOR(launch, Seconds(60));
+  AWAIT_ASSERT_EQ_FOR(
+      Containerizer::LaunchResult::SUCCESS, launch, Seconds(60));
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/containerizer/volume_image_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/volume_image_isolator_tests.cpp b/src/tests/containerizer/volume_image_isolator_tests.cpp
index 2f91730..da36422 100644
--- a/src/tests/containerizer/volume_image_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_image_isolator_tests.cpp
@@ -113,13 +113,13 @@ TEST_P(VolumeImageIsolatorTest, ROOT_ImageInVolumeWithoutRootFilesystem)
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -134,7 +134,7 @@ TEST_P(VolumeImageIsolatorTest, ROOT_ImageInVolumeWithoutRootFilesystem)
         map<string, string>(),
         None());
 
-    AWAIT_ASSERT_TRUE(launch);
+    AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
     wait = containerizer->wait(nestedContainerId);
   }
@@ -203,13 +203,13 @@ TEST_P(VolumeImageIsolatorTest, ROOT_ImageInVolumeWithRootFilesystem)
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -224,7 +224,7 @@ TEST_P(VolumeImageIsolatorTest, ROOT_ImageInVolumeWithRootFilesystem)
         map<string, string>(),
         None());
 
-    AWAIT_ASSERT_TRUE(launch);
+    AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
     wait = containerizer->wait(nestedContainerId);
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp b/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
index b36c8df..5b75852 100644
--- a/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
@@ -38,6 +38,7 @@ using std::string;
 using process::Future;
 using process::Owned;
 
+using mesos::internal::slave::Containerizer;
 using mesos::internal::slave::Fetcher;
 using mesos::internal::slave::MesosContainerizer;
 
@@ -90,13 +91,13 @@ TEST_F(VolumeSandboxPathIsolatorTest, ROOT_SelfType)
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
       None());
 
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -143,13 +144,13 @@ TEST_F(VolumeSandboxPathIsolatorTest, SharedParentTypeVolume)
   Try<string> directory = environment->mkdtemp();
   ASSERT_SOME(directory);
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   ContainerID nestedContainerId1;
   nestedContainerId1.mutable_parent()->CopyFrom(containerId);
@@ -177,7 +178,7 @@ TEST_F(VolumeSandboxPathIsolatorTest, SharedParentTypeVolume)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   ContainerID nestedContainerId2;
   nestedContainerId2.mutable_parent()->CopyFrom(containerId);
@@ -192,7 +193,7 @@ TEST_F(VolumeSandboxPathIsolatorTest, SharedParentTypeVolume)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait =
     containerizer->wait(nestedContainerId2);
@@ -256,13 +257,13 @@ TEST_F(VolumeSandboxPathIsolatorTest, ROOT_SelfTypeOwnership)
   // from FrameworkInfo.
   ASSERT_SOME(os::chown("nobody", directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory, "nobody"),
       map<string, string>(),
       None());
 
-  AWAIT_READY(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -313,13 +314,13 @@ TEST_F(VolumeSandboxPathIsolatorTest, ROOT_ParentTypeOwnership)
   // from FrameworkInfo.
   ASSERT_SOME(os::chown("nobody", directory.get()));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory.get(), "nobody"),
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   ContainerID nestedContainerId;
   nestedContainerId.mutable_parent()->CopyFrom(containerId);
@@ -349,7 +350,7 @@ TEST_F(VolumeSandboxPathIsolatorTest, ROOT_ParentTypeOwnership)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   Future<Option<ContainerTermination>> wait =
     containerizer->wait(nestedContainerId);

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/containerizer/volume_secret_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/volume_secret_isolator_tests.cpp b/src/tests/containerizer/volume_secret_isolator_tests.cpp
index a55af95..4d03a02 100644
--- a/src/tests/containerizer/volume_secret_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_secret_isolator_tests.cpp
@@ -32,6 +32,7 @@
 using process::Future;
 using process::Owned;
 
+using mesos::internal::slave::Containerizer;
 using mesos::internal::slave::Fetcher;
 using mesos::internal::slave::MesosContainerizer;
 
@@ -204,7 +205,7 @@ TEST_P(VolumeSecretIsolatorTest, ROOT_SecretInVolumeWithRootFilesystem)
   string directory = path::join(flags.work_dir, "sandbox");
   ASSERT_SOME(os::mkdir(directory));
 
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executor, directory),
       map<string, string>(),
@@ -215,7 +216,7 @@ TEST_P(VolumeSecretIsolatorTest, ROOT_SecretInVolumeWithRootFilesystem)
     return;
   }
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Now launch nested container.
   ContainerID nestedContainerId;
@@ -232,7 +233,7 @@ TEST_P(VolumeSecretIsolatorTest, ROOT_SecretInVolumeWithRootFilesystem)
       map<string, string>(),
       None());
 
-  AWAIT_ASSERT_TRUE(launch);
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Wait for nested container.
   Future<Option<ContainerTermination>> wait = containerizer->wait(

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/hook_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hook_tests.cpp b/src/tests/hook_tests.cpp
index dc8d87f..2e58d11 100644
--- a/src/tests/hook_tests.cpp
+++ b/src/tests/hook_tests.cpp
@@ -59,6 +59,7 @@ using mesos::internal::master::Master;
 
 using mesos::internal::protobuf::createLabel;
 
+using mesos::internal::slave::Containerizer;
 using mesos::internal::slave::DockerContainerizer;
 using mesos::internal::slave::executorEnvironment;
 using mesos::internal::slave::Fetcher;
@@ -318,14 +319,13 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   // Test hook adds a new environment variable "FOO" to the executor
   // with a value "bar". A '0' (success) exit status for the following
   // command validates the hook.
-  process::Future<bool> launch = containerizer->launch(
+  process::Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(None(), executorInfo, directory),
       environment,
       None());
 
-  AWAIT_READY(launch);
-  ASSERT_TRUE(launch.get());
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Wait on the container.
   Future<Option<ContainerTermination>> wait =

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/mock_docker.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mock_docker.hpp b/src/tests/mock_docker.hpp
index 5987364..65bc004 100644
--- a/src/tests/mock_docker.hpp
+++ b/src/tests/mock_docker.hpp
@@ -162,7 +162,7 @@ public:
 
   MOCK_METHOD4(
       launch,
-      process::Future<bool>(
+      process::Future<slave::Containerizer::LaunchResult>(
           const ContainerID&,
           const mesos::slave::ContainerConfig&,
           const std::map<std::string, std::string>&,
@@ -176,7 +176,7 @@ public:
 
   // Default 'launch' implementation (necessary because we can't just
   // use &slave::DockerContainerizer::launch with 'Invoke').
-  process::Future<bool> _launch(
+  process::Future<slave::Containerizer::LaunchResult> _launch(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index d85634c..c864aa9 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -4587,7 +4587,7 @@ TYPED_TEST(SlaveRecoveryTest, RestartBeforeContainerizerLaunch)
   Future<Nothing> launch;
   EXPECT_CALL(containerizer1, launch(_, _, _, _))
     .WillOnce(DoAll(FutureSatisfy(&launch),
-                    Return(Future<bool>())));
+                    Return(Future<Containerizer::LaunchResult>())));
 
   // No status update should be sent for now.
   EXPECT_CALL(sched, statusUpdate(_, _))

http://git-wip-us.apache.org/repos/asf/mesos/blob/38951cf6/src/tests/slave_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_tests.cpp b/src/tests/slave_tests.cpp
index 61dd940..a75bb26 100644
--- a/src/tests/slave_tests.cpp
+++ b/src/tests/slave_tests.cpp
@@ -968,18 +968,17 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(SlaveTest, ROOT_LaunchTaskInfoWithContainerInfo)
 
   SlaveID slaveID;
   slaveID.set_value(UUID::random().toString());
-  Future<bool> launch = containerizer->launch(
+  Future<Containerizer::LaunchResult> launch = containerizer->launch(
       containerId,
       createContainerConfig(task, executor, sandbox.get(), "nobody"),
       map<string, string>(),
       None());
-  AWAIT_READY(launch);
 
   // TODO(spikecurtis): With agent capabilities (MESOS-3362), the
   // Containerizer should fail this request since none of the listed
   // isolators can handle NetworkInfo, which implies
   // IP-per-container.
-  EXPECT_TRUE(launch.get());
+  AWAIT_EXPECT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
   // Wait for the container to terminate before shutting down.
   AWAIT_READY(containerizer->wait(containerId));
@@ -7228,7 +7227,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(SlaveTest, DefaultExecutorCommandInfo)
   Future<ContainerConfig> containerConfig;
   EXPECT_CALL(containerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<1>(&containerConfig),
-                    Return(Future<bool>())));
+                    Return(Future<Containerizer::LaunchResult>())));
 
   const v1::Offer& offer = offers->offers(0);
   const SlaveID slaveId = devolve(offer.agent_id());


[13/15] mesos git commit: Updated Docker path for Containerizer::launch interface change.

Posted by jo...@apache.org.
Updated Docker path for Containerizer::launch interface change.

The change in the return type for Containerizer::launch has less of an
impact on the Docker containerizer, as it (currently) will not return
the "new" enum 'ALREADY_LAUNCHED'.

Note that in changing Containerizer::launch, a private helper of the
Docker containerizer `reapExecutor`s return value was changed.
The return value was originally `Future<bool>` because `reapExecutor`
is the final continuation in the launch path so it needed to match
the return value of Containerizer::launch.  However, `reapExecutor`
never returns `false` (only `true` or a Failure).

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


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

Branch: refs/heads/master
Commit: 6f3a57612391e7c81a6decd1258a10dc7ec08d11
Parents: feefdba
Author: Joseph Wu <jo...@apache.org>
Authored: Mon Nov 13 15:18:23 2017 -0800
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 17:16:22 2017 -0800

----------------------------------------------------------------------
 src/slave/containerizer/docker.cpp | 24 +++++++++++++++---------
 src/slave/containerizer/docker.hpp | 10 +++++-----
 2 files changed, 20 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/6f3a5761/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index 292eecb..63432a9 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -802,7 +802,7 @@ Future<Nothing> DockerContainerizer::recover(
 }
 
 
-Future<bool> DockerContainerizer::launch(
+Future<Containerizer::LaunchResult> DockerContainerizer::launch(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig,
     const map<string, string>& environment,
@@ -1096,7 +1096,7 @@ Future<Nothing> DockerContainerizerProcess::__recover(
 }
 
 
-Future<bool> DockerContainerizerProcess::launch(
+Future<Containerizer::LaunchResult> DockerContainerizerProcess::launch(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig,
     const map<string, string>& environment,
@@ -1112,12 +1112,12 @@ Future<bool> DockerContainerizerProcess::launch(
 
   if (!containerConfig.has_container_info()) {
     LOG(INFO) << "No container info found, skipping launch";
-    return false;
+    return Containerizer::LaunchResult::NOT_SUPPORTED;
   }
 
   if (containerConfig.container_info().type() != ContainerInfo::DOCKER) {
     LOG(INFO) << "Skipping non-docker container";
-    return false;
+    return Containerizer::LaunchResult::NOT_SUPPORTED;
   }
 
   Try<Container*> container = Container::create(
@@ -1223,7 +1223,7 @@ Future<bool> DockerContainerizerProcess::launch(
 }
 
 
-Future<bool> DockerContainerizerProcess::_launch(
+Future<Containerizer::LaunchResult> DockerContainerizerProcess::_launch(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig)
 {
@@ -1257,7 +1257,10 @@ Future<bool> DockerContainerizerProcess::_launch(
       }))
       .then(defer(self(), [=](pid_t pid) {
         return reapExecutor(containerId, pid);
-      }));
+      }))
+      .then([]() {
+        return Containerizer::LaunchResult::SUCCESS;
+      });
   }
 
   string containerName = container->containerName;
@@ -1305,7 +1308,10 @@ Future<bool> DockerContainerizerProcess::_launch(
     }))
     .then(defer(self(), [=](pid_t pid) {
       return reapExecutor(containerId, pid);
-    }));
+    }))
+    .then([]() {
+      return Containerizer::LaunchResult::SUCCESS;
+    });
 }
 
 
@@ -1583,7 +1589,7 @@ Future<pid_t> DockerContainerizerProcess::checkpointExecutor(
 }
 
 
-Future<bool> DockerContainerizerProcess::reapExecutor(
+Future<Nothing> DockerContainerizerProcess::reapExecutor(
     const ContainerID& containerId,
     pid_t pid)
 {
@@ -1599,7 +1605,7 @@ Future<bool> DockerContainerizerProcess::reapExecutor(
   container->status.future().get()
     .onAny(defer(self(), &Self::reaped, containerId));
 
-  return true;
+  return Nothing();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/6f3a5761/src/slave/containerizer/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.hpp b/src/slave/containerizer/docker.hpp
index b602a56..105c068 100644
--- a/src/slave/containerizer/docker.hpp
+++ b/src/slave/containerizer/docker.hpp
@@ -86,7 +86,7 @@ public:
   virtual process::Future<Nothing> recover(
       const Option<state::SlaveState>& state);
 
-  virtual process::Future<bool> launch(
+  virtual process::Future<Containerizer::LaunchResult> launch(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,
@@ -134,7 +134,7 @@ public:
   virtual process::Future<Nothing> recover(
       const Option<state::SlaveState>& state);
 
-  virtual process::Future<bool> launch(
+  virtual process::Future<Containerizer::LaunchResult> launch(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,
@@ -176,7 +176,7 @@ private:
       const ContainerID& containerId,
       pid_t pid);
 
-  process::Future<bool> _launch(
+  process::Future<Containerizer::LaunchResult> _launch(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig);
 
@@ -201,7 +201,7 @@ private:
       const Docker::Container& dockerContainer);
 
   // Reaps on the executor pid.
-  process::Future<bool> reapExecutor(
+  process::Future<Nothing> reapExecutor(
       const ContainerID& containerId,
       pid_t pid);
 
@@ -471,7 +471,7 @@ private:
 
     // Future that tells us the return value of last launch stage (fetch, pull,
     // run, etc).
-    process::Future<bool> launch;
+    process::Future<Containerizer::LaunchResult> launch;
 
     // We keep track of the resources for each container so we can set
     // the ResourceStatistics limits in usage(). Note that this is


[02/15] mesos git commit: Added ACLs and AuthZ for standalone containers.

Posted by jo...@apache.org.
Added ACLs and AuthZ for standalone containers.

This defines some coarse-grained AuthZ for launching and managing
standalone containers.  Each HTTP principal can be given the right
to Launch, Wait upon, Kill, or Remove  standalone containers under
a given (posix) user.

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


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

Branch: refs/heads/master
Commit: e05dc9e719bb1e71fda6a42771135425765a6b2d
Parents: 3f25be1
Author: Joseph Wu <jo...@apache.org>
Authored: Fri Jul 14 17:09:41 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 16:58:40 2017 -0800

----------------------------------------------------------------------
 include/mesos/authorizer/acls.proto       | 57 ++++++++++++++++
 include/mesos/authorizer/authorizer.proto | 24 +++++++
 src/authorizer/local/authorizer.cpp       | 92 ++++++++++++++++++++++++++
 3 files changed, 173 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e05dc9e7/include/mesos/authorizer/acls.proto
----------------------------------------------------------------------
diff --git a/include/mesos/authorizer/acls.proto b/include/mesos/authorizer/acls.proto
index 587b714..6f36470 100644
--- a/include/mesos/authorizer/acls.proto
+++ b/include/mesos/authorizer/acls.proto
@@ -414,6 +414,59 @@ message ACL {
     // access.
     required Entity agents = 2;
   }
+
+  // Which principals are authorized to launch standalone containers.
+  message LaunchStandaloneContainer {
+    // Subjects: HTTP Username.
+    required Entity principals = 1;
+
+    // Objects: Given implicitly.
+    // Use Entity type ANY or NONE to allow or deny access.
+    //
+    // TODO(josephw): Consider allowing granular permission to launch as
+    // SOME particular operating system users (e.g., linux users).
+    required Entity users = 2;
+  }
+
+  // Which principals are authorized to kill a standalone container.
+  message KillStandaloneContainer {
+    // Subjects: HTTP Username.
+    required Entity principals = 1;
+
+    // Objects: Given implicitly.
+    // Use Entity type ANY or NONE to allow or deny access.
+    //
+    // TODO(josephw): Consider allowing granular permission to act upon
+    // SOME particular operating system users (e.g., linux users).
+    required Entity users = 2;
+  }
+
+  // Which principals are authorized to wait on a standalone container.
+  message WaitStandaloneContainer {
+    // Subjects: HTTP Username.
+    required Entity principals = 1;
+
+    // Objects: Given implicitly.
+    // Use Entity type ANY or NONE to allow or deny access.
+    //
+    // TODO(josephw): Consider allowing granular permission to act upon
+    // SOME particular operating system users (e.g., linux users).
+    required Entity users = 2;
+  }
+
+  // Which principals are authorized to remove the artifacts (sandbox
+  // and runtime directories) of a standalone container.
+  message RemoveStandaloneContainer {
+    // Subjects: HTTP Username.
+    required Entity principals = 1;
+
+    // Objects: Given implicitly.
+    // Use Entity type ANY or NONE to allow or deny access.
+    //
+    // TODO(josephw): Consider allowing granular permission to act upon
+    // SOME particular operating system users (e.g., linux users).
+    required Entity users = 2;
+  }
 }
 
 
@@ -485,4 +538,8 @@ message ACLs {
   repeated ACL.StopMaintenance stop_maintenances = 38;
   repeated ACL.GetMaintenanceStatus get_maintenance_statuses = 39;
   repeated ACL.MarkAgentGone mark_agents_gone = 40;
+  repeated ACL.LaunchStandaloneContainer launch_standalone_container = 41;
+  repeated ACL.KillStandaloneContainer kill_standalone_container = 42;
+  repeated ACL.WaitStandaloneContainer wait_standalone_container = 43;
+  repeated ACL.RemoveStandaloneContainer remove_standalone_container = 44;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/e05dc9e7/include/mesos/authorizer/authorizer.proto
----------------------------------------------------------------------
diff --git a/include/mesos/authorizer/authorizer.proto b/include/mesos/authorizer/authorizer.proto
index 87a8057..d84295f 100644
--- a/include/mesos/authorizer/authorizer.proto
+++ b/include/mesos/authorizer/authorizer.proto
@@ -217,6 +217,30 @@ enum Action {
   // This action will not fill in any object fields, since a principal is
   // either allowed to mark an agent as gone or is unauthorized.
   MARK_AGENT_GONE = 34;
+
+  // This action will not fill in any object fields. A principal is either
+  // allowed to launch standalone containers or is unauthorized.
+  //
+  // TODO(josephw): This should set the operating system user in the object.
+  LAUNCH_STANDALONE_CONTAINER = 35;
+
+  // This action will not fill in any object fields. A principal is either
+  // allowed to kill standalone containers or is unauthorized.
+  //
+  // TODO(josephw): This should set the operating system user in the object.
+  KILL_STANDALONE_CONTAINER = 36;
+
+  // This action will not fill in any object fields. A principal is either
+  // allowed to wait upon standalone containers or is unauthorized.
+  //
+  // TODO(josephw): This should set the operating system user in the object.
+  WAIT_STANDALONE_CONTAINER = 37;
+
+  // This action will not fill in any object fields. A principal is either
+  // allowed to remove standalone containers or is unauthorized.
+  //
+  // TODO(josephw): This should set the operating system user in the object.
+  REMOVE_STANDALONE_CONTAINER = 38;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/e05dc9e7/src/authorizer/local/authorizer.cpp
----------------------------------------------------------------------
diff --git a/src/authorizer/local/authorizer.cpp b/src/authorizer/local/authorizer.cpp
index 2fe7b87..35bf03c 100644
--- a/src/authorizer/local/authorizer.cpp
+++ b/src/authorizer/local/authorizer.cpp
@@ -396,6 +396,10 @@ public:
           }
 
           break;
+        case authorization::LAUNCH_STANDALONE_CONTAINER:
+        case authorization::KILL_STANDALONE_CONTAINER:
+        case authorization::WAIT_STANDALONE_CONTAINER:
+        case authorization::REMOVE_STANDALONE_CONTAINER:
         case authorization::GET_MAINTENANCE_SCHEDULE:
         case authorization::GET_MAINTENANCE_STATUS:
         case authorization::MARK_AGENT_GONE:
@@ -665,11 +669,14 @@ public:
         case authorization::GET_MAINTENANCE_SCHEDULE:
         case authorization::GET_MAINTENANCE_STATUS:
         case authorization::KILL_NESTED_CONTAINER:
+        case authorization::KILL_STANDALONE_CONTAINER:
         case authorization::LAUNCH_NESTED_CONTAINER:
         case authorization::LAUNCH_NESTED_CONTAINER_SESSION:
+        case authorization::LAUNCH_STANDALONE_CONTAINER:
         case authorization::MARK_AGENT_GONE:
         case authorization::REGISTER_AGENT:
         case authorization::REMOVE_NESTED_CONTAINER:
+        case authorization::REMOVE_STANDALONE_CONTAINER:
         case authorization::RUN_TASK:
         case authorization::SET_LOG_LEVEL:
         case authorization::START_MAINTENANCE:
@@ -683,6 +690,7 @@ public:
         case authorization::VIEW_FRAMEWORK:
         case authorization::VIEW_TASK:
         case authorization::WAIT_NESTED_CONTAINER:
+        case authorization::WAIT_STANDALONE_CONTAINER:
         case authorization::UNKNOWN:
           UNREACHABLE();
       }
@@ -876,11 +884,14 @@ public:
       case authorization::GET_MAINTENANCE_SCHEDULE:
       case authorization::GET_MAINTENANCE_STATUS:
       case authorization::KILL_NESTED_CONTAINER:
+      case authorization::KILL_STANDALONE_CONTAINER:
       case authorization::LAUNCH_NESTED_CONTAINER:
       case authorization::LAUNCH_NESTED_CONTAINER_SESSION:
+      case authorization::LAUNCH_STANDALONE_CONTAINER:
       case authorization::MARK_AGENT_GONE:
       case authorization::REGISTER_AGENT:
       case authorization::REMOVE_NESTED_CONTAINER:
+      case authorization::REMOVE_STANDALONE_CONTAINER:
       case authorization::RUN_TASK:
       case authorization::SET_LOG_LEVEL:
       case authorization::START_MAINTENANCE:
@@ -895,6 +906,7 @@ public:
       case authorization::VIEW_FRAMEWORK:
       case authorization::VIEW_TASK:
       case authorization::WAIT_NESTED_CONTAINER:
+      case authorization::WAIT_STANDALONE_CONTAINER:
         UNREACHABLE();
     }
 
@@ -1043,9 +1055,12 @@ public:
       case authorization::GET_MAINTENANCE_SCHEDULE:
       case authorization::GET_MAINTENANCE_STATUS:
       case authorization::KILL_NESTED_CONTAINER:
+      case authorization::KILL_STANDALONE_CONTAINER:
+      case authorization::LAUNCH_STANDALONE_CONTAINER:
       case authorization::MARK_AGENT_GONE:
       case authorization::REGISTER_AGENT:
       case authorization::REMOVE_NESTED_CONTAINER:
+      case authorization::REMOVE_STANDALONE_CONTAINER:
       case authorization::RUN_TASK:
       case authorization::SET_LOG_LEVEL:
       case authorization::START_MAINTENANCE:
@@ -1059,6 +1074,7 @@ public:
       case authorization::VIEW_FRAMEWORK:
       case authorization::VIEW_TASK:
       case authorization::WAIT_NESTED_CONTAINER:
+      case authorization::WAIT_STANDALONE_CONTAINER:
       case authorization::UNKNOWN: {
         Result<vector<GenericACL>> genericACLs =
           createGenericACLs(action, acls);
@@ -1348,6 +1364,50 @@ private:
         }
 
         return acls_;
+      case authorization::LAUNCH_STANDALONE_CONTAINER:
+        foreach (const ACL::LaunchStandaloneContainer& acl,
+                 acls.launch_standalone_container()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.users();
+
+          acls_.push_back(acl_);
+        }
+
+        return acls_;
+      case authorization::KILL_STANDALONE_CONTAINER:
+        foreach (const ACL::KillStandaloneContainer& acl,
+            acls.kill_standalone_container()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.users();
+
+          acls_.push_back(acl_);
+        }
+
+        return acls_;
+      case authorization::WAIT_STANDALONE_CONTAINER:
+        foreach (const ACL::WaitStandaloneContainer& acl,
+            acls.wait_standalone_container()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.users();
+
+          acls_.push_back(acl_);
+        }
+
+        return acls_;
+      case authorization::REMOVE_STANDALONE_CONTAINER:
+        foreach (const ACL::RemoveStandaloneContainer& acl,
+            acls.remove_standalone_container()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.users();
+
+          acls_.push_back(acl_);
+        }
+
+        return acls_;
       case authorization::REGISTER_FRAMEWORK:
       case authorization::CREATE_VOLUME:
       case authorization::RESERVE_RESOURCES:
@@ -1479,6 +1539,38 @@ Option<Error> LocalAuthorizer::validate(const ACLs& acls)
     }
   }
 
+  foreach (const ACL::LaunchStandaloneContainer& acl,
+           acls.launch_standalone_container()) {
+    if (acl.users().type() == ACL::Entity::SOME) {
+      return Error(
+          "acls.launch_standalone_container type must be either NONE or ANY");
+    }
+  }
+
+  foreach (const ACL::KillStandaloneContainer& acl,
+           acls.kill_standalone_container()) {
+    if (acl.users().type() == ACL::Entity::SOME) {
+      return Error(
+          "acls.kill_standalone_container type must be either NONE or ANY");
+    }
+  }
+
+  foreach (const ACL::WaitStandaloneContainer& acl,
+           acls.wait_standalone_container()) {
+    if (acl.users().type() == ACL::Entity::SOME) {
+      return Error(
+          "acls.wait_standalone_container type must be either NONE or ANY");
+    }
+  }
+
+  foreach (const ACL::RemoveStandaloneContainer& acl,
+           acls.remove_standalone_container()) {
+    if (acl.users().type() == ACL::Entity::SOME) {
+      return Error(
+          "acls.remove_standalone_container type must be either NONE or ANY");
+    }
+  }
+
   // TODO(alexr): Consider validating not only protobuf, but also the original
   // JSON in order to spot misspelled names. A misspelled action may affect
   // authorization result and hence lead to a security issue (e.g. when there


[09/15] mesos git commit: Moved and refactored some nested container tests.

Posted by jo...@apache.org.
Moved and refactored some nested container tests.

This moves tests for the Nested container API found in `api_tests.cpp`
into `agent_container_api_tests.cpp` and refactors the moved tests to:
  * Exercise the MesosContainerizer, rather than mocks.
  * Parameterize tests to use the Standalone container API or Nested
    container API.
  * Parameterize tests to exercise different isolator combinations.

Additionally, the  `NestedContainerWaitNotFound` and
`NestedContainerKillNotFound` tests are now combined into
`NestedContainerNotFound`.

>From the `nested_mesos_containerizer_tests.cpp`, the `WaitAfterDestroy`
and `LaunchNestedThreeLevels` were deleted as the same codepaths are now
covered by `NestedContainerLaunch` and `TwoLevelNestedContainerLaunch`
respectively.

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


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

Branch: refs/heads/master
Commit: fd4b9af1473c8773e08970c8e6aa2eaf27733c42
Parents: e5323f0
Author: Joseph Wu <jo...@apache.org>
Authored: Fri Oct 13 14:18:43 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 17:16:21 2017 -0800

----------------------------------------------------------------------
 src/tests/agent_container_api_tests.cpp         | 207 ++++++++++
 src/tests/api_tests.cpp                         | 376 -------------------
 .../nested_mesos_containerizer_tests.cpp        | 165 --------
 3 files changed, 207 insertions(+), 541 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fd4b9af1/src/tests/agent_container_api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/agent_container_api_tests.cpp b/src/tests/agent_container_api_tests.cpp
index 88c33ae..6b9bace 100644
--- a/src/tests/agent_container_api_tests.cpp
+++ b/src/tests/agent_container_api_tests.cpp
@@ -530,6 +530,213 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentContainerAPITest, NestedContainerLaunch)
   EXPECT_TRUE(checkWaitContainerResponse(wait, SIGKILL));
 }
 
+
+// This test launches a parent and nested container, simulates an agent
+// failover, and then waits/kills the containers.
+TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentContainerAPITest, RecoverNestedContainer)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.launcher = std::get<1>(std::get<3>(GetParam()));
+  slaveFlags.isolation = std::get<0>(std::get<3>(GetParam()));
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), slaveFlags);
+  ASSERT_SOME(slave);
+
+  Try<v1::ContainerID> parentContainerId =
+    launchParentContainer(master.get()->pid, slave.get()->pid);
+
+  ASSERT_SOME(parentContainerId);
+
+  // Launch a nested container.
+  v1::ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+  containerId.mutable_parent()->CopyFrom(parentContainerId.get());
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::OK().status,
+      launchNestedContainer(slave.get()->pid, containerId));
+
+  // Simulate an agent failover.
+  Future<SlaveReregisteredMessage> slaveReregisteredMessage =
+    FUTURE_PROTOBUF(SlaveReregisteredMessage(), _, _);
+
+  slave.get()->terminate();
+  slave = StartSlave(detector.get(), slaveFlags);
+  ASSERT_SOME(slave);
+
+  AWAIT_READY(slaveReregisteredMessage);
+
+  // Now kill and wait for the (recovered) container to exit.
+  Future<v1::agent::Response> wait =
+    deserialize(waitNestedContainer(slave.get()->pid, containerId));
+
+  EXPECT_TRUE(wait.isPending());
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::OK().status,
+      killNestedContainer(slave.get()->pid, containerId));
+
+  AWAIT_READY(wait);
+  EXPECT_TRUE(checkWaitContainerResponse(wait, SIGKILL));
+}
+
+
+// This test checks that using the KILL or WAIT container calls on
+// a non-existent ContainerID returns a 404 Not Found.
+TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentContainerAPITest, NestedContainerNotFound)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.launcher = std::get<1>(std::get<3>(GetParam()));
+  slaveFlags.isolation = std::get<0>(std::get<3>(GetParam()));
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), slaveFlags);
+  ASSERT_SOME(slave);
+
+  // Wait for the agent to finish registering.
+  Future<SlaveRegisteredMessage> slaveRegisteredMessage =
+    FUTURE_PROTOBUF(SlaveRegisteredMessage(), _, _);
+  AWAIT_READY(slaveRegisteredMessage);
+
+  v1::ContainerID unknownContainerId;
+  unknownContainerId.set_value(UUID::random().toString());
+  unknownContainerId.mutable_parent()->set_value(UUID::random().toString());
+
+  // Expect a 404 for waiting on unknown containers.
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::NotFound().status,
+      waitNestedContainer(slave.get()->pid, unknownContainerId));
+
+  // Expect a 404 for waiting on unknown containers.
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::NotFound().status,
+      killNestedContainer(slave.get()->pid, unknownContainerId));
+}
+
+
+// This test attempts to give invalid ContainerInfo when launching a
+// nested container. The invalid nested container LAUNCH call is expected
+// to give a 400 Bad Request, but the parent container should be otherwise
+// unaffected.
+TEST_P_TEMP_DISABLED_ON_WINDOWS(
+    AgentContainerAPITest, NestedContainerLaunchFalse)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.launcher = std::get<1>(std::get<3>(GetParam()));
+  slaveFlags.isolation = std::get<0>(std::get<3>(GetParam()));
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), slaveFlags);
+  ASSERT_SOME(slave);
+
+  Try<v1::ContainerID> parentContainerId =
+    launchParentContainer(master.get()->pid, slave.get()->pid);
+
+  ASSERT_SOME(parentContainerId);
+
+  // Try to launch an "unsupported" container.
+  // In this case, we try to specify a nested container that expects
+  // the Docker containerizer, even though the parent was made with
+  // the Mesos containerizer.
+  v1::ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+  containerId.mutable_parent()->CopyFrom(parentContainerId.get());
+
+  mesos::v1::ContainerInfo containerInfo;
+  containerInfo.set_type(mesos::v1::ContainerInfo::DOCKER);
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::BadRequest().status,
+      launchNestedContainer(
+          slave.get()->pid, containerId, None(), containerInfo));
+}
+
+
+// This test launches three total layers of nested containers,
+// one parent, nested, and double-nested container. Each nested container
+// is killed and reaped like any other nested container.
+TEST_P_TEMP_DISABLED_ON_WINDOWS(
+    AgentContainerAPITest, TwoLevelNestedContainerLaunch)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.launcher = std::get<1>(std::get<3>(GetParam()));
+  slaveFlags.isolation = std::get<0>(std::get<3>(GetParam()));
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), slaveFlags);
+  ASSERT_SOME(slave);
+
+  Try<v1::ContainerID> parentContainerId =
+    launchParentContainer(master.get()->pid, slave.get()->pid);
+
+  ASSERT_SOME(parentContainerId);
+
+  // Launch the first nested container and wait for it to finish.
+  v1::ContainerID childContainerId;
+  childContainerId.set_value(UUID::random().toString());
+  childContainerId.mutable_parent()->CopyFrom(parentContainerId.get());
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::OK().status,
+      launchNestedContainer(slave.get()->pid, childContainerId));
+
+  // Launch the second nested container underneath the first nested contaienr
+  // and wait for it to finish.
+  v1::ContainerID grandchildContainerId;
+  grandchildContainerId.set_value(UUID::random().toString());
+  grandchildContainerId.mutable_parent()->CopyFrom(childContainerId);
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::OK().status,
+      launchNestedContainer(slave.get()->pid, grandchildContainerId));
+
+  // Start waiting for each nested container to exit.
+  Future<v1::agent::Response> waitChild =
+    deserialize(waitNestedContainer(slave.get()->pid, childContainerId));
+
+  Future<v1::agent::Response> waitgrandChild =
+    deserialize(waitNestedContainer(slave.get()->pid, grandchildContainerId));
+
+  EXPECT_TRUE(waitChild.isPending());
+  EXPECT_TRUE(waitgrandChild.isPending());
+
+  // Kill the grandchild container.
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::OK().status,
+      killNestedContainer(slave.get()->pid, grandchildContainerId));
+
+  AWAIT_READY(waitgrandChild);
+  EXPECT_TRUE(checkWaitContainerResponse(waitgrandChild, SIGKILL));
+
+  // The child container should still be running.
+  EXPECT_TRUE(waitChild.isPending());
+
+  // Kill the child container.
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::OK().status,
+      killNestedContainer(slave.get()->pid, childContainerId));
+
+  AWAIT_READY(waitChild);
+  EXPECT_TRUE(checkWaitContainerResponse(waitChild, SIGKILL));
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/fd4b9af1/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 5fc89be..c9c50b9 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -4930,382 +4930,6 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentAPITest, GetState)
 }
 
 
-TEST_P(AgentAPITest, NestedContainerWaitNotFound)
-{
-  ContentType contentType = GetParam();
-
-  Clock::pause();
-
-  StandaloneMasterDetector detector;
-  MockContainerizer mockContainerizer;
-
-  EXPECT_CALL(mockContainerizer, recover(_))
-    .WillOnce(Return(Future<Nothing>(Nothing())));
-
-  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
-
-  Try<Owned<cluster::Slave>> slave =
-    StartSlave(&detector, &mockContainerizer);
-
-  ASSERT_SOME(slave);
-
-  // Wait for the agent to finish recovery.
-  AWAIT_READY(__recover);
-  Clock::settle();
-
-  // Expect a 404 for waiting on unknown containers.
-  {
-    v1::agent::Call call;
-    call.set_type(v1::agent::Call::WAIT_NESTED_CONTAINER);
-
-    v1::ContainerID unknownContainerId;
-    unknownContainerId.set_value(UUID::random().toString());
-    unknownContainerId.mutable_parent()->set_value(UUID::random().toString());
-
-    call.mutable_wait_nested_container()->mutable_container_id()
-      ->CopyFrom(unknownContainerId);
-
-    Future<http::Response> response = http::post(
-      slave.get()->pid,
-      "api/v1",
-      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
-      serialize(contentType, call),
-      stringify(contentType));
-
-    AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::NotFound().status, response);
-  }
-
-  // The destructor of `cluster::Slave` will try to clean up any
-  // remaining containers by inspecting the result of `containers()`.
-  EXPECT_CALL(mockContainerizer, containers())
-    .WillRepeatedly(Return(hashset<ContainerID>()));
-}
-
-
-TEST_P(AgentAPITest, NestedContainerKillNotFound)
-{
-  ContentType contentType = GetParam();
-
-  Clock::pause();
-
-  StandaloneMasterDetector detector;
-  MockContainerizer mockContainerizer;
-
-  EXPECT_CALL(mockContainerizer, recover(_))
-    .WillOnce(Return(Future<Nothing>(Nothing())));
-
-  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
-
-  Try<Owned<cluster::Slave>> slave =
-    StartSlave(&detector, &mockContainerizer);
-
-  ASSERT_SOME(slave);
-
-  // Wait for the agent to finish recovery.
-  AWAIT_READY(__recover);
-  Clock::settle();
-
-  // Expect a 404 for killing unknown containers.
-  {
-    v1::agent::Call call;
-    call.set_type(v1::agent::Call::KILL_NESTED_CONTAINER);
-
-    v1::ContainerID unknownContainerId;
-    unknownContainerId.set_value(UUID::random().toString());
-    unknownContainerId.mutable_parent()->set_value(UUID::random().toString());
-
-    call.mutable_kill_nested_container()->mutable_container_id()
-      ->CopyFrom(unknownContainerId);
-
-    Future<http::Response> response = http::post(
-      slave.get()->pid,
-      "api/v1",
-      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
-      serialize(contentType, call),
-      stringify(contentType));
-
-    AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::NotFound().status, response);
-  }
-
-  // The destructor of `cluster::Slave` will try to clean up any
-  // remaining containers by inspecting the result of `containers()`.
-  EXPECT_CALL(mockContainerizer, containers())
-    .WillRepeatedly(Return(hashset<ContainerID>()));
-}
-
-
-// When containerizer returns false from launching a nested
-// container, it is considered a bad request (e.g. image
-// type is not supported).
-TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentAPITest, NestedContainerLaunchFalse)
-{
-  ContentType contentType = GetParam();
-
-  Clock::pause();
-
-  Try<Owned<cluster::Master>> master = StartMaster();
-  ASSERT_SOME(master);
-
-  Owned<MasterDetector> detector = master.get()->createDetector();
-
-  MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestContainerizer containerizer(&exec);
-
-  slave::Flags agentFlags = CreateSlaveFlags();
-  Try<Owned<cluster::Slave>> slave =
-    StartSlave(detector.get(), &containerizer, agentFlags);
-
-  ASSERT_SOME(slave);
-
-  MockScheduler sched;
-  MesosSchedulerDriver driver(
-      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
-
-  EXPECT_CALL(sched, registered(&driver, _, _));
-
-  EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 0.1, 32, "*"))
-    .WillRepeatedly(Return()); // Ignore subsequent offers.
-
-  Future<Nothing> executorRegistered;
-  EXPECT_CALL(exec, registered(_, _, _, _))
-    .WillOnce(FutureSatisfy(&executorRegistered));
-
-  EXPECT_CALL(exec, launchTask(_, _));
-
-  driver.start();
-
-  // Trigger authentication and registration for the agent.
-  Clock::advance(agentFlags.authentication_backoff_factor);
-  Clock::advance(agentFlags.registration_backoff_factor);
-
-  AWAIT_READY(executorRegistered);
-
-  Future<hashset<ContainerID>> containerIds = containerizer.containers();
-  AWAIT_READY(containerIds);
-  ASSERT_EQ(1u, containerIds->size());
-
-  // Try to launch an "unsupported" container.
-  v1::ContainerID containerId;
-  containerId.set_value(UUID::random().toString());
-  containerId.mutable_parent()->set_value(containerIds->begin()->value());
-
-  {
-    // Return false here to indicate "unsupported".
-    EXPECT_CALL(containerizer, launch(_, _, _, _))
-      .WillOnce(Return(Future<bool>(false)));
-
-    v1::agent::Call call;
-    call.set_type(v1::agent::Call::LAUNCH_NESTED_CONTAINER);
-
-    call.mutable_launch_nested_container()->mutable_container_id()
-      ->CopyFrom(containerId);
-
-    Future<http::Response> response = http::post(
-      slave.get()->pid,
-      "api/v1",
-      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
-      serialize(contentType, call),
-      stringify(contentType));
-
-    AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::BadRequest().status, response);
-  }
-
-  EXPECT_CALL(exec, shutdown(_))
-    .Times(AtMost(1));
-
-  driver.stop();
-  driver.join();
-}
-
-
-TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentAPITest, TwoLevelNestedContainerLaunch)
-{
-  ContentType contentType = GetParam();
-
-  Clock::pause();
-
-  Try<Owned<cluster::Master>> master = StartMaster();
-  ASSERT_SOME(master);
-
-  Owned<MasterDetector> detector = master.get()->createDetector();
-
-  MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestContainerizer containerizer(&exec);
-
-  slave::Flags agentFlags = CreateSlaveFlags();
-  Try<Owned<cluster::Slave>> slave =
-    StartSlave(detector.get(), &containerizer, agentFlags);
-
-  ASSERT_SOME(slave);
-
-  MockScheduler sched;
-  MesosSchedulerDriver driver(
-      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
-
-  EXPECT_CALL(sched, registered(&driver, _, _));
-
-  EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 0.1, 32, "*"))
-    .WillRepeatedly(Return()); // Ignore subsequent offers.
-
-  Future<Nothing> executorRegistered;
-  EXPECT_CALL(exec, registered(_, _, _, _))
-    .WillOnce(FutureSatisfy(&executorRegistered));
-
-  EXPECT_CALL(exec, launchTask(_, _));
-
-  driver.start();
-
-  // Trigger authentication and registration for the agent.
-  Clock::advance(agentFlags.authentication_backoff_factor);
-  Clock::advance(agentFlags.registration_backoff_factor);
-
-  AWAIT_READY(executorRegistered);
-
-  Future<hashset<ContainerID>> containerIds = containerizer.containers();
-  AWAIT_READY(containerIds);
-  ASSERT_EQ(1u, containerIds->size());
-
-  // Launch a two level nested parent/child container and then wait for them to
-  // finish.
-  v1::ContainerID parentContainerId;
-  parentContainerId.set_value(UUID::random().toString());
-  parentContainerId.mutable_parent()->set_value(containerIds->begin()->value());
-
-  // Launch the parent container.
-  {
-    v1::agent::Call call;
-    call.set_type(v1::agent::Call::LAUNCH_NESTED_CONTAINER);
-
-    call.mutable_launch_nested_container()->mutable_container_id()
-      ->CopyFrom(parentContainerId);
-
-    Future<http::Response> response = http::post(
-      slave.get()->pid,
-      "api/v1",
-      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
-      serialize(contentType, call),
-      stringify(contentType));
-
-    AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::OK().status, response);
-  }
-
-  // Launch the child container.
-  v1::ContainerID childContainerId;
-  childContainerId.set_value(UUID::random().toString());
-  childContainerId.mutable_parent()->CopyFrom(parentContainerId);
-
-  {
-    v1::agent::Call call;
-    call.set_type(v1::agent::Call::LAUNCH_NESTED_CONTAINER);
-
-    call.mutable_launch_nested_container()->mutable_container_id()
-      ->CopyFrom(childContainerId);
-
-    Future<http::Response> response = http::post(
-      slave.get()->pid,
-      "api/v1",
-      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
-      serialize(contentType, call),
-      stringify(contentType));
-
-    AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::OK().status, response);
-  }
-
-  // Wait for the parent container.
-  Future<v1::agent::Response> waitParent;
-
-  {
-    v1::agent::Call call;
-    call.set_type(v1::agent::Call::WAIT_NESTED_CONTAINER);
-
-    call.mutable_wait_nested_container()->mutable_container_id()
-      ->CopyFrom(parentContainerId);
-
-    waitParent = post(slave.get()->pid, call, contentType);
-
-    Clock::settle();
-
-    EXPECT_TRUE(waitParent.isPending());
-  }
-
-  // Wait for the child container.
-  Future<v1::agent::Response> waitChild;
-
-  {
-    v1::agent::Call call;
-    call.set_type(v1::agent::Call::WAIT_NESTED_CONTAINER);
-
-    call.mutable_wait_nested_container()->mutable_container_id()
-      ->CopyFrom(childContainerId);
-
-    waitChild = post(slave.get()->pid, call, contentType);
-
-    Clock::settle();
-
-    EXPECT_TRUE(waitChild.isPending());
-  }
-
-  // Kill the child container.
-  {
-    v1::agent::Call call;
-    call.set_type(v1::agent::Call::KILL_NESTED_CONTAINER);
-
-    call.mutable_kill_nested_container()->mutable_container_id()
-      ->CopyFrom(childContainerId);
-
-    Future<http::Response> response = http::post(
-      slave.get()->pid,
-      "api/v1",
-      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
-      serialize(contentType, call),
-      stringify(contentType));
-
-    AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::OK().status, response);
-  }
-
-  AWAIT_READY(waitChild);
-  ASSERT_EQ(v1::agent::Response::WAIT_NESTED_CONTAINER, waitChild->type());
-
-  // The test containerizer sets exit status to 0 when destroyed.
-  EXPECT_EQ(0, waitChild->wait_nested_container().exit_status());
-
-  // The parent container should still be running.
-  EXPECT_TRUE(waitParent.isPending());
-
-  // Kill the parent container.
-  {
-    v1::agent::Call call;
-    call.set_type(v1::agent::Call::KILL_NESTED_CONTAINER);
-
-    call.mutable_kill_nested_container()->mutable_container_id()
-      ->CopyFrom(parentContainerId);
-
-    Future<http::Response> response = http::post(
-      slave.get()->pid,
-      "api/v1",
-      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
-      serialize(contentType, call),
-      stringify(contentType));
-
-    AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::OK().status, response);
-  }
-
-  AWAIT_READY(waitParent);
-  ASSERT_EQ(v1::agent::Response::WAIT_NESTED_CONTAINER, waitParent->type());
-
-  // The test containerizer sets exit status to 0 when destroyed.
-  EXPECT_EQ(0, waitParent->wait_nested_container().exit_status());
-
-  EXPECT_CALL(exec, shutdown(_))
-    .Times(AtMost(1));
-
-  driver.stop();
-  driver.join();
-}
-
-
 // This test verifies that launch nested container session fails when
 // attaching to the output of the container fails. Consequently, the
 // launched container should be destroyed.

http://git-wip-us.apache.org/repos/asf/mesos/blob/fd4b9af1/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
index 3e2a2d1..983ecd1 100644
--- a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
@@ -2325,91 +2325,6 @@ TEST_F(NestedMesosContainerizerTest,
   ASSERT_FALSE(containers->contains(containerId));
 }
 
-
-TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_WaitAfterDestroy)
-{
-  slave::Flags flags = CreateSlaveFlags();
-  flags.launcher = "linux";
-  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
-
-  Fetcher fetcher(flags);
-
-  Try<MesosContainerizer*> create = MesosContainerizer::create(
-      flags,
-      true,
-      &fetcher);
-
-  ASSERT_SOME(create);
-
-  Owned<MesosContainerizer> containerizer(create.get());
-
-  // Launch a top-level container.
-  ContainerID containerId;
-  containerId.set_value(UUID::random().toString());
-
-  Try<string> directory = environment->mkdtemp();
-  ASSERT_SOME(directory);
-
-  Future<bool> launch = containerizer->launch(
-      containerId,
-      createContainerConfig(
-          None(),
-          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
-          directory.get()),
-      map<string, string>(),
-      None());
-
-  AWAIT_ASSERT_TRUE(launch);
-
-  // Launch a nested container.
-  ContainerID nestedContainerId;
-  nestedContainerId.mutable_parent()->CopyFrom(containerId);
-  nestedContainerId.set_value(UUID::random().toString());
-
-  launch = containerizer->launch(
-      nestedContainerId,
-      createContainerConfig(createCommandInfo("exit 42")),
-      map<string, string>(),
-      None());
-
-  AWAIT_ASSERT_TRUE(launch);
-
-  // Wait once (which does a destroy),
-  // then wait again on the nested container.
-  Future<Option<ContainerTermination>> nestedWait = containerizer->wait(
-      nestedContainerId);
-
-  AWAIT_READY(nestedWait);
-  ASSERT_SOME(nestedWait.get());
-  ASSERT_TRUE(nestedWait.get()->has_status());
-  EXPECT_WEXITSTATUS_EQ(42, nestedWait.get()->status());
-
-  nestedWait = containerizer->wait(nestedContainerId);
-
-  AWAIT_READY(nestedWait);
-  ASSERT_SOME(nestedWait.get());
-  ASSERT_TRUE(nestedWait.get()->has_status());
-  EXPECT_WEXITSTATUS_EQ(42, nestedWait.get()->status());
-
-  // Destroy the top-level container.
-  Future<Option<ContainerTermination>> wait = containerizer->wait(
-      containerId);
-
-  AWAIT_READY(containerizer->destroy(containerId));
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
-
-  // Wait on nested container again.
-  nestedWait = containerizer->wait(nestedContainerId);
-
-  AWAIT_READY(nestedWait);
-  ASSERT_NONE(nestedWait.get());
-}
-
-
 // This test verifies that agent environment variables are not leaked
 // to the nested container, and the environment variables specified in
 // the command for the nested container will be honored.
@@ -2501,86 +2416,6 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_AgentEnvironmentNotLeaked)
 }
 
 
-TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNestedThreeLevels)
-{
-  slave::Flags flags = CreateSlaveFlags();
-  flags.launcher = "linux";
-  flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
-
-  Fetcher fetcher(flags);
-
-  Try<MesosContainerizer*> create = MesosContainerizer::create(
-      flags,
-      false,
-      &fetcher);
-
-  ASSERT_SOME(create);
-
-  Owned<MesosContainerizer> containerizer(create.get());
-
-  SlaveState state;
-  state.id = SlaveID();
-
-  AWAIT_READY(containerizer->recover(state));
-
-  ContainerID level1ContainerId;
-  level1ContainerId.set_value(UUID::random().toString());
-
-  Try<string> directory = environment->mkdtemp();
-  ASSERT_SOME(directory);
-
-  Future<bool> launch = containerizer->launch(
-      level1ContainerId,
-      createContainerConfig(
-          None(),
-          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
-          directory.get()),
-      map<string, string>(),
-      None());
-
-  AWAIT_ASSERT_TRUE(launch);
-
-  ContainerID level2ContainerId;
-  level2ContainerId.mutable_parent()->CopyFrom(level1ContainerId);
-  level2ContainerId.set_value(UUID::random().toString());
-
-  launch = containerizer->launch(
-      level2ContainerId,
-      createContainerConfig(createCommandInfo("sleep 1000")),
-      map<string, string>(),
-      None());
-
-  AWAIT_ASSERT_TRUE(launch);
-
-  ContainerID level3ContainerId;
-  level3ContainerId.mutable_parent()->CopyFrom(level2ContainerId);
-  level3ContainerId.set_value(UUID::random().toString());
-
-  launch = containerizer->launch(
-      level3ContainerId,
-      createContainerConfig(createCommandInfo("exit 42")),
-      map<string, string>(),
-      None());
-
-  Future<Option<ContainerTermination>> wait =
-    containerizer->wait(level3ContainerId);
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WEXITSTATUS_EQ(42, wait.get()->status());
-
-  wait = containerizer->wait(level1ContainerId);
-
-  containerizer->destroy(level1ContainerId);
-
-  AWAIT_READY(wait);
-  ASSERT_SOME(wait.get());
-  ASSERT_TRUE(wait.get()->has_status());
-  EXPECT_WTERMSIG_EQ(SIGKILL, wait.get()->status());
-}
-
-
 TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_Remove)
 {
   slave::Flags flags = CreateSlaveFlags();


[08/15] mesos git commit: Parameterized test for nested container launch.

Posted by jo...@apache.org.
Parameterized test for nested container launch.

This introduces a new test class that is parameterized based
on the type of API used to launch (1) the parent container,
(2) the nested container, and (3) the content type (like all
other API tests).  The test is also parameterized based
on the enabled set of isolators.

Because this change overlaps with existing nested container
API tests, those test(s) will be removed.

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


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

Branch: refs/heads/master
Commit: e5323f010f8c24715fad885ce3e4c7fad3181cf8
Parents: 3c929e0
Author: Joseph Wu <jo...@apache.org>
Authored: Thu Sep 21 14:44:02 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 17:16:21 2017 -0800

----------------------------------------------------------------------
 src/Makefile.am                         |   1 +
 src/tests/CMakeLists.txt                |   1 +
 src/tests/agent_container_api_tests.cpp | 535 +++++++++++++++++++++++++++
 src/tests/api_tests.cpp                 | 118 ------
 4 files changed, 537 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e5323f01/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 955f01a..49dec55 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -2385,6 +2385,7 @@ endif
 mesos_tests_SOURCES =						\
   slave/qos_controllers/load.cpp				\
   tests/active_user_test_helper.cpp				\
+  tests/agent_container_api_tests.cpp				\
   tests/anonymous_tests.cpp					\
   tests/api_tests.cpp						\
   tests/attributes_tests.cpp					\

http://git-wip-us.apache.org/repos/asf/mesos/blob/e5323f01/src/tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/tests/CMakeLists.txt b/src/tests/CMakeLists.txt
index 81c85d9..db5e531 100644
--- a/src/tests/CMakeLists.txt
+++ b/src/tests/CMakeLists.txt
@@ -75,6 +75,7 @@ endif ()
 #######################
 set(MESOS_TESTS_SRC
   ${MESOS_TESTS_UTILS_SRC}
+  agent_container_api_tests.cpp
   anonymous_tests.cpp
   api_tests.cpp
   attributes_tests.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/e5323f01/src/tests/agent_container_api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/agent_container_api_tests.cpp b/src/tests/agent_container_api_tests.cpp
new file mode 100644
index 0000000..88c33ae
--- /dev/null
+++ b/src/tests/agent_container_api_tests.cpp
@@ -0,0 +1,535 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include <string>
+#include <tuple>
+#include <vector>
+
+#include <mesos/http.hpp>
+
+#include <mesos/v1/resources.hpp>
+
+#include <mesos/v1/master/master.hpp>
+
+#include <mesos/v1/scheduler/scheduler.hpp>
+
+#include <process/clock.hpp>
+#include <process/future.hpp>
+#include <process/gmock.hpp>
+#include <process/gtest.hpp>
+#include <process/http.hpp>
+#include <process/owned.hpp>
+#include <process/reap.hpp>
+
+#include <stout/gtest.hpp>
+#include <stout/jsonify.hpp>
+#include <stout/nothing.hpp>
+#include <stout/stringify.hpp>
+#include <stout/try.hpp>
+
+#include "common/http.hpp"
+
+#include "master/detector/standalone.hpp"
+
+#include "slave/slave.hpp"
+
+#include "slave/containerizer/fetcher.hpp"
+
+#include "slave/containerizer/mesos/containerizer.hpp"
+
+#include "tests/mesos.hpp"
+
+namespace http = process::http;
+
+using std::string;
+using std::tuple;
+using std::vector;
+
+using mesos::master::detector::MasterDetector;
+using mesos::master::detector::StandaloneMasterDetector;
+
+using mesos::internal::slave::Fetcher;
+using mesos::internal::slave::MesosContainerizer;
+using mesos::internal::slave::Slave;
+
+using process::Clock;
+using process::Failure;
+using process::Future;
+using process::Owned;
+
+using testing::_;
+using testing::Return;
+using testing::WithParamInterface;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+enum ContainerLaunchType
+{
+  NORMAL,
+  STANDALONE
+};
+
+
+// These tests are parameterized by:
+// 1) The type of any parent container.
+//    A NORMAL parent is launched with a framework and executor.
+//    A STANDALONE parent is launched via `LAUNCH_CONTAINER`.
+// 2) The type of any nested container.
+//    A NORMAL nested container is launched via `LAUNCH_NESTED_CONTAINER`.
+//    A STANDALONE nested contaienr is launched via `LAUNCH_CONTAINER`.
+// 3) The content type of the HTTP request(s).
+// 4) A tuple containing:
+//    1) The isolators to enable during the test.
+//    2) The launcher to use.
+//    3) And, in case the isolator/launcher requires specific permissions
+//       or capabilities, the last parameter is a free-form string used
+//       to supply additional test filters.
+class AgentContainerAPITest
+  : public MesosTest,
+    public WithParamInterface<
+      tuple<
+        ContainerLaunchType,
+        ContainerLaunchType,
+        ContentType,
+        tuple<string, string, string>>>
+{
+public:
+  // Helper function to post a request to the `/api/v1` agent endpoint
+  // and return the response.
+  Future<http::Response> post(
+      const process::PID<slave::Slave>& pid,
+      const v1::agent::Call& call)
+  {
+    ContentType contentType = std::get<2>(GetParam());
+
+    http::Headers headers = createBasicAuthHeaders(DEFAULT_CREDENTIAL);
+    headers["Accept"] = stringify(contentType);
+
+    return http::post(
+        pid,
+        "api/v1",
+        headers,
+        serialize(contentType, call),
+        stringify(contentType));
+  }
+
+  // Helper function to deserialize the response from the `/api/v1` agent
+  // endpoint as a V1 response protobuf.
+  Future<v1::agent::Response> deserialize(
+      const Future<http::Response>& response)
+  {
+    ContentType contentType = std::get<2>(GetParam());
+
+    return response
+      .then([contentType](const http::Response& response)
+            -> Future<v1::agent::Response> {
+        if (response.status != http::OK().status) {
+          return Failure(
+              "Unexpected response status: " + response.status +
+              ": " + response.body);
+        }
+
+        return mesos::internal::deserialize<v1::agent::Response>(
+            contentType, response.body);
+      });
+  }
+
+  // Helper function to launch a top level container based on the first
+  // test parameter:
+  //   * NORMAL mode will launch a scheduler, executor, and task in order
+  //     to create a top level container and uses the `GET_CONTAINERS`
+  //     agent API call to retrieve the ContainerID (this method assumes
+  //     there is only one container present on the agent).
+  //   * STANDALONE mode uses the `LAUNCH_CONTAINER` agent API to create
+  //     a top level container. The ContainerID is generated by the callee.
+  //
+  // Returns the ContainerID of the created container.
+  Try<v1::ContainerID> launchParentContainer(
+      const process::PID<master::Master>& master,
+      const process::PID<slave::Slave>& slave)
+  {
+    switch (std::get<0>(GetParam())) {
+      case ContainerLaunchType::NORMAL: {
+        normal = NormalParentContainerDependencies(master);
+
+        // Launch a normal executor and sleep task.
+        EXPECT_CALL(
+            *(normal->scheduler), registered(normal->driver.get(), _, _));
+
+        Future<vector<Offer>> offers;
+        EXPECT_CALL(
+            *(normal->scheduler), resourceOffers(normal->driver.get(), _))
+          .WillOnce(FutureArg<1>(&offers))
+          .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+        normal->driver->start();
+
+        offers.await(Seconds(15));
+        if (!offers.isReady() || offers->empty()) {
+          return Error("Failed to get offer(s)");
+        }
+
+        TaskInfo task = createTask(offers->front(), SLEEP_COMMAND(1000));
+
+        Future<TaskStatus> statusRunning;
+        EXPECT_CALL(*(normal->scheduler), statusUpdate(normal->driver.get(), _))
+          .WillOnce(FutureArg<1>(&statusRunning))
+          .WillRepeatedly(Return()); // Ignore subsequent status updates.
+
+        normal->driver->launchTasks(offers->front().id(), {task});
+
+        statusRunning.await(Seconds(15));
+        if (!statusRunning.isReady() &&
+            statusRunning->state() != TASK_RUNNING) {
+          return Error("Failed to launch parent container");
+        }
+
+        // Use the GET_CONTAINERS call to retrieve the ContainerID.
+        v1::agent::Call call;
+        call.set_type(v1::agent::Call::GET_CONTAINERS);
+
+        Future<v1::agent::Response> containers = deserialize(post(slave, call));
+
+        containers.await(Seconds(15));
+        if (!containers.isReady() ||
+            containers->get_containers().containers_size() != 1u) {
+          return Error("Failed to get parent ContainerID");
+        }
+
+        return containers->get_containers().containers(0).container_id();
+      }
+
+      case ContainerLaunchType::STANDALONE: {
+        // TODO(josephw): The agent should not need to register with
+        // the master to launch standalone containers.
+        Future<SlaveRegisteredMessage> slaveRegisteredMessage =
+          FUTURE_PROTOBUF(SlaveRegisteredMessage(), _, _);
+
+        slaveRegisteredMessage.await(Seconds(15));
+        if (!slaveRegisteredMessage.isReady()) {
+          return Error("Failed to register agent");
+        }
+
+        // Launch a standalone parent container.
+        v1::ContainerID containerId;
+        containerId.set_value(UUID::random().toString());
+
+        v1::agent::Call call;
+        call.set_type(v1::agent::Call::LAUNCH_CONTAINER);
+
+        call.mutable_launch_container()->mutable_container_id()
+          ->CopyFrom(containerId);
+
+        call.mutable_launch_container()->mutable_command()
+          ->set_value(SLEEP_COMMAND(1000));
+
+        v1::Resources resources = v1::Resources::parse("cpus:0.1;mem:32").get();
+        call.mutable_launch_container()->mutable_resources()
+          ->CopyFrom(resources);
+
+        Future<http::Response> response = post(slave, call);
+
+        AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::OK().status, response);
+
+        return containerId;
+      }
+    }
+
+    UNREACHABLE();
+  }
+
+  // Helper function to launch a nested container under the given ContainerID
+  // based on the second test parameter:
+  //   * NORMAL mode uses the (deprecated) `LAUNCH_NESTED_CONTAINER` API.
+  //   * STANDALONE mode uses the `LAUNCH_CONTAINER` API.
+  Future<http::Response> launchNestedContainer(
+      const process::PID<slave::Slave>& slave,
+      const v1::ContainerID& containerId,
+      const Option<mesos::v1::CommandInfo>& commandInfo = None(),
+      const Option<mesos::v1::ContainerInfo>& containerInfo = None())
+  {
+    v1::agent::Call call;
+
+    switch (std::get<1>(GetParam())) {
+      case ContainerLaunchType::NORMAL: {
+        call.set_type(v1::agent::Call::LAUNCH_NESTED_CONTAINER);
+
+        call.mutable_launch_nested_container()->mutable_container_id()
+          ->CopyFrom(containerId);
+
+        call.mutable_launch_nested_container()->mutable_command()
+          ->set_value(SLEEP_COMMAND(100));
+
+        if (commandInfo.isSome()) {
+          call.mutable_launch_nested_container()->mutable_command()
+            ->CopyFrom(commandInfo.get());
+        }
+
+        if (containerInfo.isSome()) {
+          call.mutable_launch_nested_container()->mutable_container()
+            ->CopyFrom(containerInfo.get());
+        }
+        break;
+      }
+
+      case ContainerLaunchType::STANDALONE: {
+        call.set_type(v1::agent::Call::LAUNCH_CONTAINER);
+
+        call.mutable_launch_container()->mutable_container_id()
+          ->CopyFrom(containerId);
+
+        call.mutable_launch_container()->mutable_command()
+          ->set_value(SLEEP_COMMAND(100));
+
+        if (commandInfo.isSome()) {
+          call.mutable_launch_container()->mutable_command()
+            ->CopyFrom(commandInfo.get());
+        }
+
+        if (containerInfo.isSome()) {
+          call.mutable_launch_container()->mutable_container()
+            ->CopyFrom(containerInfo.get());
+        }
+        break;
+      }
+    }
+
+    return post(slave, call);
+  }
+
+  // Helper function to wait for a nested container to terminate,
+  // based on the second test parameter:
+  //   * NORMAL mode uses the (deprecated) `WAIT_NESTED_CONTAINER` API.
+  //   * STANDALONE mode uses the `WAIT_CONTAINER` API.
+  Future<http::Response> waitNestedContainer(
+      const process::PID<slave::Slave>& slave,
+      const v1::ContainerID& containerId)
+  {
+    v1::agent::Call call;
+
+    switch (std::get<1>(GetParam())) {
+      case ContainerLaunchType::NORMAL: {
+        call.set_type(v1::agent::Call::WAIT_NESTED_CONTAINER);
+
+        call.mutable_wait_nested_container()->mutable_container_id()
+          ->CopyFrom(containerId);
+        break;
+      }
+
+      case ContainerLaunchType::STANDALONE: {
+        call.set_type(v1::agent::Call::WAIT_CONTAINER);
+
+        call.mutable_wait_container()->mutable_container_id()
+          ->CopyFrom(containerId);
+        break;
+      }
+    }
+
+    return post(slave, call);
+  }
+
+  // Helper function to check the response to a `WAIT_[NESTED_]CONTAINER`
+  // call according to the given signal or lack of signal.
+  // The expected response type is based on the second test parameter.
+  // See `waitNestedContainer`.
+  bool checkWaitContainerResponse(
+      const Future<v1::agent::Response>& response,
+      const Option<int>& signal)
+  {
+    switch (std::get<1>(GetParam())) {
+      case ContainerLaunchType::NORMAL: {
+        if (response->type() != v1::agent::Response::WAIT_NESTED_CONTAINER) {
+          return false;
+        }
+
+        if (signal.isSome()) {
+          if (!response->wait_nested_container().has_exit_status()) {
+            return false;
+          }
+
+          if (response->wait_nested_container().exit_status() != signal.get()) {
+            return false;
+          }
+        }
+        break;
+      }
+
+      case ContainerLaunchType::STANDALONE: {
+        if (response->type() != v1::agent::Response::WAIT_CONTAINER) {
+          return false;
+        }
+
+        if (signal.isSome()) {
+          if (!response->wait_container().has_exit_status()) {
+            return false;
+          }
+
+          if (response->wait_container().exit_status() != signal.get()) {
+            return false;
+          }
+        }
+        break;
+      }
+    }
+
+    return true;
+  }
+
+  // Helper function to kill a nested container, based on the second
+  // test parameter:
+  //   * NORMAL mode uses the (deprecated) `KILL_NESTED_CONTAINER` API.
+  //   * STANDALONE mode uses the `KILL_CONTAINER` API.
+  Future<http::Response> killNestedContainer(
+      const process::PID<slave::Slave>& slave,
+      const v1::ContainerID& containerId)
+  {
+    v1::agent::Call call;
+
+    switch (std::get<1>(GetParam())) {
+      case ContainerLaunchType::NORMAL: {
+        call.set_type(v1::agent::Call::KILL_NESTED_CONTAINER);
+
+        call.mutable_kill_nested_container()->mutable_container_id()
+          ->CopyFrom(containerId);
+        break;
+      }
+
+      case ContainerLaunchType::STANDALONE: {
+        call.set_type(v1::agent::Call::KILL_CONTAINER);
+
+        call.mutable_kill_container()->mutable_container_id()
+          ->CopyFrom(containerId);
+        break;
+      }
+    }
+
+    return post(slave, call);
+  }
+
+protected:
+  virtual void TearDown()
+  {
+    if (normal.isSome()) {
+      normal->driver->stop();
+      normal->driver->join();
+
+      normal = None();
+    }
+
+    MesosTest::TearDown();
+  }
+
+private:
+  struct NormalParentContainerDependencies
+  {
+    NormalParentContainerDependencies(
+        const process::PID<master::Master>& master)
+      : scheduler(new MockScheduler())
+    {
+      // Enable checkpointing for the framework.
+      FrameworkInfo framework = DEFAULT_FRAMEWORK_INFO;
+      framework.set_checkpoint(true);
+
+      driver.reset(new MesosSchedulerDriver(
+          scheduler.get(),
+          framework,
+          master,
+          DEFAULT_CREDENTIAL));
+    }
+
+    // NOTE: These need to be pointers due to how each mock's copy constructor
+    // is explicitly deleted (for good reason) and because `Option` requires
+    // said copy constructors for certain operations.
+    Owned<MockScheduler> scheduler;
+    Owned<MesosSchedulerDriver> driver;
+  };
+
+  Option<NormalParentContainerDependencies> normal;
+};
+
+
+INSTANTIATE_TEST_CASE_P(
+    ParentChildContainerTypeAndContentType,
+    AgentContainerAPITest,
+    ::testing::Combine(
+      ::testing::Values(
+        ContainerLaunchType::NORMAL,
+        ContainerLaunchType::STANDALONE),
+      ::testing::Values(
+        ContainerLaunchType::NORMAL,
+        ContainerLaunchType::STANDALONE),
+      ::testing::Values(
+        ContentType::JSON,
+        ContentType::PROTOBUF),
+      ::testing::Values(
+        make_tuple(
+            string("posix/cpu,posix/mem"),
+            string("posix"),
+            string()),
+        make_tuple(
+            string("cgroups/cpu,cgroups/mem,filesystem/linux,namespaces/pid"),
+            string("linux"),
+            string("ROOT_CGROUPS_")))));
+
+
+// This test runs through the basic workflow of launching a nested container,
+// killing the nested container, and retrieving the exit status (SIGKILL).
+TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentContainerAPITest, NestedContainerLaunch)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.launcher = std::get<1>(std::get<3>(GetParam()));
+  slaveFlags.isolation = std::get<0>(std::get<3>(GetParam()));
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), slaveFlags);
+  ASSERT_SOME(slave);
+
+  Try<v1::ContainerID> parentContainerId =
+    launchParentContainer(master.get()->pid, slave.get()->pid);
+
+  ASSERT_SOME(parentContainerId);
+
+  // Launch a nested container and wait for it to finish.
+  v1::ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+  containerId.mutable_parent()->CopyFrom(parentContainerId.get());
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::OK().status,
+      launchNestedContainer(slave.get()->pid, containerId));
+
+  Future<v1::agent::Response> wait =
+    deserialize(waitNestedContainer(slave.get()->pid, containerId));
+
+  EXPECT_TRUE(wait.isPending());
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::OK().status,
+      killNestedContainer(slave.get()->pid, containerId));
+
+  AWAIT_READY(wait);
+  EXPECT_TRUE(checkWaitContainerResponse(wait, SIGKILL));
+}
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/e5323f01/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 4e2cbe1..5fc89be 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -5119,124 +5119,6 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentAPITest, NestedContainerLaunchFalse)
 }
 
 
-TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentAPITest, NestedContainerLaunch)
-{
-  ContentType contentType = GetParam();
-
-  Clock::pause();
-
-  Try<Owned<cluster::Master>> master = StartMaster();
-  ASSERT_SOME(master);
-
-  Owned<MasterDetector> detector = master.get()->createDetector();
-
-  MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  TestContainerizer containerizer(&exec);
-
-  slave::Flags agentFlags = CreateSlaveFlags();
-  Try<Owned<cluster::Slave>> slave =
-    StartSlave(detector.get(), &containerizer, agentFlags);
-
-  ASSERT_SOME(slave);
-
-  MockScheduler sched;
-  MesosSchedulerDriver driver(
-      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
-
-  EXPECT_CALL(sched, registered(&driver, _, _));
-
-  EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 0.1, 32, "*"))
-    .WillRepeatedly(Return()); // Ignore subsequent offers.
-
-  Future<Nothing> executorRegistered;
-  EXPECT_CALL(exec, registered(_, _, _, _))
-    .WillOnce(FutureSatisfy(&executorRegistered));
-
-  EXPECT_CALL(exec, launchTask(_, _));
-
-  driver.start();
-
-  // Trigger authentication and registration for the agent.
-  Clock::advance(agentFlags.authentication_backoff_factor);
-  Clock::advance(agentFlags.registration_backoff_factor);
-
-  AWAIT_READY(executorRegistered);
-
-  Future<hashset<ContainerID>> containerIds = containerizer.containers();
-  AWAIT_READY(containerIds);
-  ASSERT_EQ(1u, containerIds->size());
-
-  // Launch a nested container and wait for it to finish.
-  v1::ContainerID containerId;
-  containerId.set_value(UUID::random().toString());
-  containerId.mutable_parent()->set_value(containerIds->begin()->value());
-
-  {
-    v1::agent::Call call;
-    call.set_type(v1::agent::Call::LAUNCH_NESTED_CONTAINER);
-
-    call.mutable_launch_nested_container()->mutable_container_id()
-      ->CopyFrom(containerId);
-
-    Future<http::Response> response = http::post(
-      slave.get()->pid,
-      "api/v1",
-      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
-      serialize(contentType, call),
-      stringify(contentType));
-
-    AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::OK().status, response);
-  }
-
-  Future<v1::agent::Response> wait;
-
-  {
-    v1::agent::Call call;
-    call.set_type(v1::agent::Call::WAIT_NESTED_CONTAINER);
-
-    call.mutable_wait_nested_container()->mutable_container_id()
-      ->CopyFrom(containerId);
-
-    wait = post(slave.get()->pid, call, contentType);
-
-    Clock::settle();
-
-    EXPECT_TRUE(wait.isPending());
-  }
-
-  // Now kill the nested container.
-  {
-    v1::agent::Call call;
-    call.set_type(v1::agent::Call::KILL_NESTED_CONTAINER);
-
-    call.mutable_kill_nested_container()->mutable_container_id()
-      ->CopyFrom(containerId);
-
-    Future<http::Response> response = http::post(
-      slave.get()->pid,
-      "api/v1",
-      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
-      serialize(contentType, call),
-      stringify(contentType));
-
-    AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::OK().status, response);
-  }
-
-  AWAIT_READY(wait);
-  ASSERT_EQ(v1::agent::Response::WAIT_NESTED_CONTAINER, wait->type());
-
-  // The test containerizer sets exit status to 0 when destroyed.
-  EXPECT_EQ(0, wait->wait_nested_container().exit_status());
-
-  EXPECT_CALL(exec, shutdown(_))
-    .Times(AtMost(1));
-
-  driver.stop();
-  driver.join();
-}
-
-
 TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentAPITest, TwoLevelNestedContainerLaunch)
 {
   ContentType contentType = GetParam();


[07/15] mesos git commit: Implemented Standalone Container API.

Posted by jo...@apache.org.
Implemented Standalone Container API.

The Standalone and Nested Container APIs are very similar.
This commit combines the two API implementations by adding a
translation function (i.e. `launchNestedContainer` and
`launchContainer`) which unpacks the V1 protobuf into fields
which can be passed into a common function (i.e. `_launchContainer`).

The common functions authorize based on the type of container being
launched and it is possible to use both Standalone and Nested
Container APIs interchangably for nested containers.

This approach is somewhat messy for for the `WAIT_(NESTED_)CONTAINER`
calls, as these methods require different return protobufs based on
the original call.

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


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

Branch: refs/heads/master
Commit: 9086ecef4070b126b106c3f03b5763ce6ca444b4
Parents: 2d7ec26
Author: Joseph Wu <jo...@apache.org>
Authored: Wed Sep 6 14:53:05 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 17:16:13 2017 -0800

----------------------------------------------------------------------
 src/slave/http.cpp | 647 ++++++++++++++++++++++++++++++++----------------
 src/slave/http.hpp |  44 +++-
 2 files changed, 472 insertions(+), 219 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9086ecef/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 22cdac9..e4fa4df 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -599,6 +599,18 @@ Future<Response> Http::_api(
 
     case mesos::agent::Call::ATTACH_CONTAINER_OUTPUT:
       return attachContainerOutput(call, mediaTypes, principal);
+
+    case mesos::agent::Call::LAUNCH_CONTAINER:
+      return launchContainer(call, mediaTypes.accept, principal);
+
+    case mesos::agent::Call::WAIT_CONTAINER:
+      return waitContainer(call, mediaTypes.accept, principal);
+
+    case mesos::agent::Call::KILL_CONTAINER:
+      return killContainer(call, mediaTypes.accept, principal);
+
+    case mesos::agent::Call::REMOVE_CONTAINER:
+      return removeContainer(call, mediaTypes.accept, principal);
   }
 
   UNREACHABLE();
@@ -2350,76 +2362,114 @@ Future<Response> Http::launchNestedContainer(
   LOG(INFO) << "Processing LAUNCH_NESTED_CONTAINER call for container '"
             << call.launch_nested_container().container_id() << "'";
 
-  Future<Owned<ObjectApprover>> approver;
+  Future<Owned<AuthorizationAcceptor>> authorizer =
+    AuthorizationAcceptor::create(
+        principal, slave->authorizer, authorization::LAUNCH_NESTED_CONTAINER);
 
-  if (slave->authorizer.isSome()) {
-    Option<authorization::Subject> subject = createSubject(principal);
+  return authorizer
+    .then(defer(
+        slave->self(),
+        [=](const Owned<AuthorizationAcceptor>& authorizer) {
+          return _launchContainer(
+              call.launch_nested_container().container_id(),
+              call.launch_nested_container().command(),
+              None(),
+              call.launch_nested_container().has_container()
+                ? call.launch_nested_container().container()
+                : Option<ContainerInfo>::none(),
+              ContainerClass::DEFAULT,
+              acceptType,
+              authorizer);
+        }));
+}
 
-    approver = slave->authorizer.get()->getObjectApprover(
-        subject, authorization::LAUNCH_NESTED_CONTAINER);
-  } else {
-    approver = Owned<ObjectApprover>(new AcceptingObjectApprover());
-  }
 
-  return approver
-    .then(defer(slave->self(), [=](const Owned<ObjectApprover>& approver) {
-      return _launchNestedContainer(
-          call.launch_nested_container().container_id(),
-          call.launch_nested_container().command(),
-          call.launch_nested_container().has_container()
-            ? call.launch_nested_container().container()
-            : Option<ContainerInfo>::none(),
-          ContainerClass::DEFAULT,
-          acceptType,
-          approver);
-    }));
+Future<Response> Http::launchContainer(
+    const mesos::agent::Call& call,
+    ContentType acceptType,
+    const Option<Principal>& principal) const
+{
+  CHECK_EQ(mesos::agent::Call::LAUNCH_CONTAINER, call.type());
+  CHECK(call.has_launch_container());
+
+  Future<Owned<AuthorizationAcceptor>> authorizer =
+    AuthorizationAcceptor::create(
+        principal,
+        slave->authorizer,
+        call.launch_container().container_id().has_parent()
+          ? authorization::LAUNCH_NESTED_CONTAINER
+          : authorization::LAUNCH_STANDALONE_CONTAINER);
+
+  return authorizer
+    .then(defer(
+        slave->self(),
+        [=](const Owned<AuthorizationAcceptor>& authorizer)
+          -> Future<Response> {
+          return _launchContainer(
+              call.launch_container().container_id(),
+              call.launch_container().command(),
+              call.launch_container().resources(),
+              call.launch_container().has_container()
+                ? call.launch_container().container()
+                : Option<ContainerInfo>::none(),
+              ContainerClass::DEFAULT,
+              acceptType,
+              authorizer);
+        }));
 }
 
 
-Future<Response> Http::_launchNestedContainer(
+Future<Response> Http::_launchContainer(
     const ContainerID& containerId,
     const CommandInfo& commandInfo,
+    const Option<Resources>& resources,
     const Option<ContainerInfo>& containerInfo,
     const Option<ContainerClass>& containerClass,
     ContentType acceptType,
-    const Owned<ObjectApprover>& approver) const
+    const Owned<AuthorizationAcceptor>& authorizer) const
 {
+  Option<string> user;
+
+  // Attempt to get the executor associated with this ContainerID.
+  // We only expect to get the executor when launching a nested container
+  // under a container launched via a scheduler. In other cases, we are
+  // launching a standalone container (possibly nested).
   Executor* executor = slave->getExecutor(containerId);
   if (executor == nullptr) {
-    return NotFound("Container " + stringify(containerId) + " cannot be found");
-  }
-
-  Framework* framework = slave->getFramework(executor->frameworkId);
-  CHECK_NOTNULL(framework);
+    if (!authorizer->accept()) {
+      return Forbidden();
+    }
+  } else {
+    Framework* framework = slave->getFramework(executor->frameworkId);
+    CHECK_NOTNULL(framework);
 
-  Try<bool> approved = approver->approved(
-      ObjectApprover::Object(
-          executor->info,
-          framework->info,
-          commandInfo,
-          containerId));
+    if (!authorizer->accept(
+            executor->info, framework->info, commandInfo, containerId)) {
+      return Forbidden();
+    }
 
-  if (approved.isError()) {
-    return Failure(approved.error());
-  } else if (!approved.get()) {
-    return Forbidden();
+    // By default, we use the executor's user.
+    // The CommandInfo can override it, if specified.
+    user = executor->user;
   }
 
-  // By default, we use the executor's user.
-  // The command user overrides it if specified.
-  Option<string> user = executor->user;
+  ContainerConfig containerConfig;
+  containerConfig.mutable_command_info()->CopyFrom(commandInfo);
 
 #ifndef __WINDOWS__
-  if (commandInfo.has_user()) {
-    user = commandInfo.user();
-  }
-#endif
+  if (slave->flags.switch_user) {
+    if (commandInfo.has_user()) {
+      user = commandInfo.user();
+    }
 
-  ContainerConfig containerConfig;
-  containerConfig.mutable_command_info()->CopyFrom(commandInfo);
+    if (user.isSome()) {
+      containerConfig.set_user(user.get());
+    }
+  }
+#endif // __WINDOWS__
 
-  if (user.isSome()) {
-    containerConfig.set_user(user.get());
+  if (resources.isSome()) {
+    containerConfig.mutable_resources()->CopyFrom(resources.get());
   }
 
   if (containerInfo.isSome()) {
@@ -2430,6 +2480,38 @@ Future<Response> Http::_launchNestedContainer(
     containerConfig.set_container_class(containerClass.get());
   }
 
+  // For standalone top-level containers, supply a sandbox directory.
+  if (!containerId.has_parent()) {
+    const string directory =
+      slave::paths::getContainerPath(slave->flags.work_dir, containerId);
+
+    // NOTE: The below partially mirrors logic executed before the agent calls
+    // `containerizer->launch`. See `slave::paths::createExecutorDirectory`.
+    Try<Nothing> mkdir = os::mkdir(directory);
+    if (mkdir.isError()) {
+      return InternalServerError(
+          "Failed to create sandbox directory: " + mkdir.error());
+    }
+
+// `os::chown()` is not available on Windows.
+#ifndef __WINDOWS__
+    if (containerConfig.has_user()) {
+      Try<Nothing> chown = os::chown(containerConfig.user(), directory);
+      if (chown.isError()) {
+        // Attempt to clean up, but since we've already failed to chown,
+        // we don't check the return value here.
+        os::rmdir(directory);
+
+        return InternalServerError(
+            "Failed to chown sandbox directory '" +
+            directory + "':" + chown.error());
+      }
+    }
+#endif // __WINDOWS__
+
+    containerConfig.set_directory(directory);
+  }
+
   Future<bool> launched = slave->containerizer->launch(
       containerId,
       containerConfig,
@@ -2450,7 +2532,7 @@ Future<Response> Http::_launchNestedContainer(
         return;
       }
 
-      LOG(WARNING) << "Failed to launch nested container "
+      LOG(WARNING) << "Failed to launch container "
                    << containerId << ": "
                    << (launch.isFailed() ? launch.failure() : "discarded");
 
@@ -2460,7 +2542,7 @@ Future<Response> Http::_launchNestedContainer(
             return;
           }
 
-          LOG(ERROR) << "Failed to destroy nested container "
+          LOG(ERROR) << "Failed to destroy container "
                      << containerId << " after launch failure: "
                      << (destroy.isFailed() ? destroy.failure() : "discarded");
         });
@@ -2487,86 +2569,147 @@ Future<Response> Http::waitNestedContainer(
   LOG(INFO) << "Processing WAIT_NESTED_CONTAINER call for container '"
             << call.wait_nested_container().container_id() << "'";
 
-  Future<Owned<ObjectApprover>> approver;
+  Future<Owned<AuthorizationAcceptor>> authorizer =
+    AuthorizationAcceptor::create(
+        principal, slave->authorizer, authorization::WAIT_NESTED_CONTAINER);
 
-  if (slave->authorizer.isSome()) {
-    Option<authorization::Subject> subject = createSubject(principal);
+  return authorizer
+    .then(defer(
+        slave->self(),
+        [=](const Owned<AuthorizationAcceptor>& authorizer) {
+          return _waitContainer(
+              call.wait_nested_container().container_id(),
+              acceptType,
+              authorizer,
+              true);
+        }));
+}
 
-    approver = slave->authorizer.get()->getObjectApprover(
-        subject, authorization::WAIT_NESTED_CONTAINER);
+
+Future<Response> Http::waitContainer(
+    const mesos::agent::Call& call,
+    ContentType acceptType,
+    const Option<Principal>& principal) const
+{
+  CHECK_EQ(mesos::agent::Call::WAIT_CONTAINER, call.type());
+  CHECK(call.has_wait_container());
+
+  Future<Owned<AuthorizationAcceptor>> authorizer =
+    AuthorizationAcceptor::create(
+        principal,
+        slave->authorizer,
+        call.wait_container().container_id().has_parent()
+          ? authorization::WAIT_NESTED_CONTAINER
+          : authorization::WAIT_STANDALONE_CONTAINER);
+
+  return authorizer
+    .then(defer(
+        slave->self(),
+        [=](const Owned<AuthorizationAcceptor>& authorizer) {
+          return _waitContainer(
+              call.wait_container().container_id(),
+              acceptType,
+              authorizer,
+              false);
+        }));
+}
+
+
+Future<Response> Http::_waitContainer(
+    const ContainerID& containerId,
+    ContentType acceptType,
+    const Owned<AuthorizationAcceptor>& authorizer,
+    const bool deprecated) const
+{
+  // Attempt to get the executor associated with this ContainerID.
+  // We only expect to get the executor when waiting upon a nested container
+  // under a container launched via a scheduler. In other cases, we are
+  // waiting on a standalone container (possibly nested).
+  Executor* executor = slave->getExecutor(containerId);
+  if (executor == nullptr) {
+    if (!authorizer->accept()) {
+      return Forbidden();
+    }
   } else {
-    approver = Owned<ObjectApprover>(new AcceptingObjectApprover());
+    Framework* framework = slave->getFramework(executor->frameworkId);
+    CHECK_NOTNULL(framework);
+
+    if (!authorizer->accept(
+            executor->info,
+            framework->info,
+            containerId)) {
+      return Forbidden();
+    }
   }
 
-  return approver.then(defer(slave->self(),
-    [this, call, acceptType](const Owned<ObjectApprover>& waitApprover)
-        -> Future<Response> {
-      const ContainerID& containerId =
-        call.wait_nested_container().container_id();
-
-      Executor* executor = slave->getExecutor(containerId);
-      if (executor == nullptr) {
+  return slave->containerizer->wait(containerId)
+    .then([=](const Option<ContainerTermination>& termination) -> Response {
+      if (termination.isNone()) {
         return NotFound(
             "Container " + stringify(containerId) + " cannot be found");
       }
 
-      Framework* framework = slave->getFramework(executor->frameworkId);
-      CHECK_NOTNULL(framework);
+      mesos::agent::Response response;
 
-      Try<bool> approved = waitApprover->approved(
-          ObjectApprover::Object(
-              executor->info,
-              framework->info,
-              containerId));
+      // The response object depends on which API was originally used
+      // to make this call.
+      if (deprecated) {
+        response.set_type(mesos::agent::Response::WAIT_NESTED_CONTAINER);
 
-      if (approved.isError()) {
-        return Failure(approved.error());
-      } else if (!approved.get()) {
-        return Forbidden();
-      }
+        mesos::agent::Response::WaitNestedContainer* waitNestedContainer =
+          response.mutable_wait_nested_container();
 
-      Future<Option<mesos::slave::ContainerTermination>> wait =
-        slave->containerizer->wait(containerId);
+        if (termination->has_status()) {
+          waitNestedContainer->set_exit_status(termination->status());
+        }
 
-      return wait
-        .then([containerId, acceptType](
-            const Option<ContainerTermination>& termination) -> Response {
-          if (termination.isNone()) {
-            return NotFound("Container " + stringify(containerId) +
-                            " cannot be found");
-          }
+        if (termination->has_state()) {
+          waitNestedContainer->set_state(termination->state());
+        }
 
-          mesos::agent::Response response;
-          response.set_type(mesos::agent::Response::WAIT_NESTED_CONTAINER);
+        if (termination->has_reason()) {
+          waitNestedContainer->set_reason(termination->reason());
+        }
 
-          mesos::agent::Response::WaitNestedContainer* waitNestedContainer =
-            response.mutable_wait_nested_container();
+        if (!termination->limited_resources().empty()) {
+          waitNestedContainer->mutable_limitation()->mutable_resources()
+            ->CopyFrom(termination->limited_resources());
+        }
 
-          if (termination->has_status()) {
-            waitNestedContainer->set_exit_status(termination->status());
-          }
+        if (termination->has_message()) {
+          waitNestedContainer->set_message(termination->message());
+        }
+      } else {
+        response.set_type(mesos::agent::Response::WAIT_CONTAINER);
 
-          if (termination->has_state()) {
-            waitNestedContainer->set_state(termination->state());
-          }
+        mesos::agent::Response::WaitContainer* waitContainer =
+          response.mutable_wait_container();
 
-          if (termination->has_reason()) {
-            waitNestedContainer->set_reason(termination->reason());
-          }
+        if (termination->has_status()) {
+          waitContainer->set_exit_status(termination->status());
+        }
 
-          if (!termination->limited_resources().empty()) {
-            waitNestedContainer->mutable_limitation()->mutable_resources()
-              ->CopyFrom(termination->limited_resources());
-          }
+        if (termination->has_state()) {
+          waitContainer->set_state(termination->state());
+        }
 
-          if (termination->has_message()) {
-            waitNestedContainer->set_message(termination->message());
-          }
+        if (termination->has_reason()) {
+          waitContainer->set_reason(termination->reason());
+        }
 
-          return OK(serialize(acceptType, evolve(response)),
-                    stringify(acceptType));
-        });
-    }));
+        if (!termination->limited_resources().empty()) {
+          waitContainer->mutable_limitation()->mutable_resources()
+            ->CopyFrom(termination->limited_resources());
+        }
+
+        if (termination->has_message()) {
+          waitContainer->set_message(termination->message());
+        }
+      }
+
+      return OK(serialize(acceptType, evolve(response)),
+                stringify(acceptType));
+    });
 }
 
 
@@ -2581,61 +2724,101 @@ Future<Response> Http::killNestedContainer(
   LOG(INFO) << "Processing KILL_NESTED_CONTAINER call for container '"
             << call.kill_nested_container().container_id() << "'";
 
-  Future<Owned<ObjectApprover>> approver;
-
-  if (slave->authorizer.isSome()) {
-    Option<authorization::Subject> subject = createSubject(principal);
+  Future<Owned<AuthorizationAcceptor>> authorizer =
+    AuthorizationAcceptor::create(
+        principal, slave->authorizer, authorization::KILL_NESTED_CONTAINER);
 
-    approver = slave->authorizer.get()->getObjectApprover(
-        subject, authorization::KILL_NESTED_CONTAINER);
-  } else {
-    approver = Owned<ObjectApprover>(new AcceptingObjectApprover());
+  // SIGKILL is used by default if a signal is not specified.
+  int signal = SIGKILL;
+  if (call.kill_nested_container().has_signal()) {
+    signal = call.kill_nested_container().signal();
   }
 
-  return approver.then(defer(slave->self(),
-    [this, call](const Owned<ObjectApprover>& killApprover)
-        -> Future<Response> {
-      const ContainerID& containerId =
-        call.kill_nested_container().container_id();
+  return authorizer
+    .then(defer(
+        slave->self(),
+        [=](const Owned<AuthorizationAcceptor>& authorizer) {
+          return _killContainer(
+              call.kill_nested_container().container_id(),
+              signal,
+              acceptType,
+              authorizer);
+        }));
+}
 
-      // SIGKILL is used by default if a signal is not specified.
-      int signal = SIGKILL;
-      if (call.kill_nested_container().has_signal()) {
-        signal = call.kill_nested_container().signal();
-      }
 
-      Executor* executor = slave->getExecutor(containerId);
-      if (executor == nullptr) {
-        return NotFound(
-            "Container " + stringify(containerId) + " cannot be found");
-      }
+Future<Response> Http::killContainer(
+    const mesos::agent::Call& call,
+    ContentType acceptType,
+    const Option<Principal>& principal) const
+{
+  CHECK_EQ(mesos::agent::Call::KILL_CONTAINER, call.type());
+  CHECK(call.has_kill_container());
 
-      Framework* framework = slave->getFramework(executor->frameworkId);
-      CHECK_NOTNULL(framework);
+  Future<Owned<AuthorizationAcceptor>> authorizer =
+    AuthorizationAcceptor::create(
+        principal,
+        slave->authorizer,
+        call.kill_container().container_id().has_parent()
+          ? authorization::KILL_NESTED_CONTAINER
+          : authorization::KILL_STANDALONE_CONTAINER);
 
-      Try<bool> approved = killApprover->approved(
-          ObjectApprover::Object(
-              executor->info,
-              framework->info,
-              containerId));
+  // SIGKILL is used by default if a signal is not specified.
+  int signal = SIGKILL;
+  if (call.kill_container().has_signal()) {
+    signal = call.kill_container().signal();
+  }
 
-      if (approved.isError()) {
-        return Failure(approved.error());
-      } else if (!approved.get()) {
-        return Forbidden();
-      }
+  return authorizer
+    .then(defer(
+        slave->self(),
+        [=](const Owned<AuthorizationAcceptor>& authorizer) {
+          return _killContainer(
+              call.kill_container().container_id(),
+              signal,
+              acceptType,
+              authorizer);
+        }));
+}
 
-      Future<bool> kill = slave->containerizer->kill(containerId, signal);
 
-      return kill
-        .then([containerId](bool found) -> Response {
-          if (!found) {
-            return NotFound("Container '" + stringify(containerId) + "'"
-                            " cannot be found (or is already killed)");
-          }
-          return OK();
-        });
-    }));
+Future<Response> Http::_killContainer(
+    const ContainerID& containerId,
+    const int signal,
+    ContentType acceptType,
+    const Owned<AuthorizationAcceptor>& authorizer) const
+{
+  // Attempt to get the executor associated with this ContainerID.
+  // We only expect to get the executor when killing a nested container
+  // under a container launched via a scheduler. In other cases, we are
+  // killing a standalone container (possibly nested).
+  Executor* executor = slave->getExecutor(containerId);
+  if (executor == nullptr) {
+    if (!authorizer->accept()) {
+      return Forbidden();
+    }
+  } else {
+    Framework* framework = slave->getFramework(executor->frameworkId);
+    CHECK_NOTNULL(framework);
+
+    if (!authorizer->accept(
+            executor->info,
+            framework->info,
+            containerId)) {
+      return Forbidden();
+    }
+  }
+
+  Future<bool> kill = slave->containerizer->kill(containerId, signal);
+
+  return kill
+    .then([containerId](bool found) -> Response {
+      if (!found) {
+        return NotFound("Container '" + stringify(containerId) + "'"
+                        " cannot be found (or is already killed)");
+      }
+      return OK();
+    });
 }
 
 
@@ -2650,56 +2833,88 @@ Future<Response> Http::removeNestedContainer(
   LOG(INFO) << "Processing REMOVE_NESTED_CONTAINER call for container '"
             << call.remove_nested_container().container_id() << "'";
 
-  Future<Owned<ObjectApprover>> approver;
+  Future<Owned<AuthorizationAcceptor>> authorizer =
+    AuthorizationAcceptor::create(
+        principal, slave->authorizer, authorization::REMOVE_NESTED_CONTAINER);
 
-  if (slave->authorizer.isSome()) {
-    Option<authorization::Subject> subject = createSubject(principal);
+  return authorizer
+    .then(defer(
+        slave->self(),
+        [=](const Owned<AuthorizationAcceptor>& authorizer) {
+          return _removeContainer(
+              call.remove_nested_container().container_id(),
+              acceptType,
+              authorizer);
+        }));
+}
 
-    approver = slave->authorizer.get()->getObjectApprover(
-        subject, authorization::REMOVE_NESTED_CONTAINER);
-  } else {
-    approver = Owned<ObjectApprover>(new AcceptingObjectApprover());
-  }
 
-  return approver.then(defer(slave->self(),
-    [this, call](const Owned<ObjectApprover>& removeApprover)
-        -> Future<Response> {
-      const ContainerID& containerId =
-        call.remove_nested_container().container_id();
+Future<Response> Http::removeContainer(
+    const mesos::agent::Call& call,
+    ContentType acceptType,
+    const Option<Principal>& principal) const
+{
+  CHECK_EQ(mesos::agent::Call::REMOVE_CONTAINER, call.type());
+  CHECK(call.has_remove_container());
 
-      Executor* executor = slave->getExecutor(containerId);
-      if (executor == nullptr) {
-        return OK();
-      }
+  Future<Owned<AuthorizationAcceptor>> authorizer =
+    AuthorizationAcceptor::create(
+        principal,
+        slave->authorizer,
+        call.remove_container().container_id().has_parent()
+          ? authorization::REMOVE_NESTED_CONTAINER
+          : authorization::REMOVE_STANDALONE_CONTAINER);
 
-      Framework* framework = slave->getFramework(executor->frameworkId);
-      CHECK_NOTNULL(framework);
+  return authorizer
+    .then(defer(
+        slave->self(),
+        [=](const Owned<AuthorizationAcceptor>& authorizer) {
+          return _removeContainer(
+              call.remove_container().container_id(),
+              acceptType,
+              authorizer);
+        }));
+}
 
-      Try<bool> approved = removeApprover->approved(
-          ObjectApprover::Object(
-              executor->info,
-              framework->info,
-              containerId));
 
-      if (approved.isError()) {
-        return Failure(approved.error());
-      } else if (!approved.get()) {
-        return Forbidden();
-      }
+Future<Response> Http::_removeContainer(
+    const ContainerID& containerId,
+    ContentType acceptType,
+    const Owned<AuthorizationAcceptor>& authorizer) const
+{
+  // Attempt to get the executor associated with this ContainerID.
+  // We only expect to get the executor when removing a nested container
+  // under a container launched via a scheduler. In other cases, we are
+  // removing a standalone container (possibly nested).
+  Executor* executor = slave->getExecutor(containerId);
+  if (executor == nullptr) {
+    if (!authorizer->accept()) {
+      return Forbidden();
+    }
+  } else {
+    Framework* framework = slave->getFramework(executor->frameworkId);
+    CHECK_NOTNULL(framework);
+
+    if (!authorizer->accept(
+            executor->info,
+            framework->info,
+            containerId)) {
+      return Forbidden();
+    }
+  }
 
-      Future<Nothing> remove = slave->containerizer->remove(containerId);
+  Future<Nothing> remove = slave->containerizer->remove(containerId);
 
-      return remove.then(
-          [containerId](const Future<Nothing>& result) -> Response {
-            if (result.isFailed()) {
-              LOG(ERROR) << "Failed to remove nested container " << containerId
-                         << ": " << result.failure();
-              return InternalServerError(result.failure());
-            }
+  return remove
+    .then([=](const Future<Nothing>& result) -> Response {
+      if (result.isFailed()) {
+        LOG(ERROR) << "Failed to remove container " << containerId
+                   << ": " << result.failure();
+        return InternalServerError(result.failure());
+      }
 
-            return OK();
-          });
-    }));
+      return OK();
+    });
 }
 
 
@@ -2872,29 +3087,27 @@ Future<Response> Http::launchNestedContainerSession(
   LOG(INFO) << "Processing LAUNCH_NESTED_CONTAINER_SESSION call for container '"
             << call.launch_nested_container_session().container_id() << "'";
 
-  Future<Owned<ObjectApprover>> approver;
-
-  if (slave->authorizer.isSome()) {
-    Option<authorization::Subject> subject = createSubject(principal);
-
-    approver = slave->authorizer.get()->getObjectApprover(
-        subject, authorization::LAUNCH_NESTED_CONTAINER_SESSION);
-  } else {
-    approver = Owned<ObjectApprover>(new AcceptingObjectApprover());
-  }
+  Future<Owned<AuthorizationAcceptor>> authorizer =
+    AuthorizationAcceptor::create(
+        principal,
+        slave->authorizer,
+        authorization::LAUNCH_NESTED_CONTAINER_SESSION);
 
-  Future<Response> response = approver
-    .then(defer(slave->self(), [=](const Owned<ObjectApprover>& approver) {
-      return _launchNestedContainer(
-          call.launch_nested_container_session().container_id(),
-          call.launch_nested_container_session().command(),
-          call.launch_nested_container_session().has_container()
-            ? call.launch_nested_container_session().container()
-            : Option<ContainerInfo>::none(),
-          ContainerClass::DEBUG,
-          mediaTypes.accept,
-          approver);
-    }));
+  Future<Response> response = authorizer
+    .then(defer(
+        slave->self(),
+        [=](const Owned<AuthorizationAcceptor>& authorizer) {
+          return _launchContainer(
+              call.launch_nested_container_session().container_id(),
+              call.launch_nested_container_session().command(),
+              None(),
+              call.launch_nested_container_session().has_container()
+                ? call.launch_nested_container_session().container()
+                : Option<ContainerInfo>::none(),
+              ContainerClass::DEBUG,
+              mediaTypes.accept,
+              authorizer);
+        }));
 
   // Helper to destroy the container.
   auto destroy = [this](const ContainerID& containerId) {
@@ -2906,7 +3119,7 @@ Future<Response> Http::launchNestedContainerSession(
   };
 
   // If `response` has failed or is not `OK`, the container will be
-  // destroyed by `_launchNestedContainer`.
+  // destroyed by `_launchContainer`.
   return response
     .then(defer(slave->self(),
                 [=](const Response& response) -> Future<Response> {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9086ecef/src/slave/http.hpp
----------------------------------------------------------------------
diff --git a/src/slave/http.hpp b/src/slave/http.hpp
index 44a95de..a51831c 100644
--- a/src/slave/http.hpp
+++ b/src/slave/http.hpp
@@ -28,6 +28,8 @@
 
 #include <mesos/authorizer/authorizer.hpp>
 
+#include "common/http.hpp"
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -213,29 +215,67 @@ private:
       ContentType acceptType,
       const Option<process::http::authentication::Principal>& principal) const;
 
-  process::Future<process::http::Response> _launchNestedContainer(
+  process::Future<process::http::Response> launchContainer(
+      const mesos::agent::Call& call,
+      ContentType acceptType,
+      const Option<process::http::authentication::Principal>& principal) const;
+
+  process::Future<process::http::Response> _launchContainer(
       const ContainerID& containerId,
       const CommandInfo& commandInfo,
+      const Option<Resources>& resources,
       const Option<ContainerInfo>& containerInfo,
       const Option<mesos::slave::ContainerClass>& containerClass,
       ContentType acceptType,
-      const process::Owned<ObjectApprover>& approver) const;
+      const process::Owned<AuthorizationAcceptor>& authorizer) const;
 
   process::Future<process::http::Response> waitNestedContainer(
       const mesos::agent::Call& call,
       ContentType acceptType,
       const Option<process::http::authentication::Principal>& principal) const;
 
+  process::Future<process::http::Response> waitContainer(
+      const mesos::agent::Call& call,
+      ContentType acceptType,
+      const Option<process::http::authentication::Principal>& principal) const;
+
+  process::Future<process::http::Response> _waitContainer(
+      const ContainerID& containerId,
+      ContentType acceptType,
+      const process::Owned<AuthorizationAcceptor>& authorizer,
+      const bool deprecated) const;
+
   process::Future<process::http::Response> killNestedContainer(
       const mesos::agent::Call& call,
       ContentType acceptType,
       const Option<process::http::authentication::Principal>& principal) const;
 
+  process::Future<process::http::Response> killContainer(
+      const mesos::agent::Call& call,
+      ContentType acceptType,
+      const Option<process::http::authentication::Principal>& principal) const;
+
+  process::Future<process::http::Response> _killContainer(
+      const ContainerID& containerId,
+      const int signal,
+      ContentType acceptType,
+      const process::Owned<AuthorizationAcceptor>& authorizer) const;
+
   process::Future<process::http::Response> removeNestedContainer(
       const mesos::agent::Call& call,
       ContentType acceptType,
       const Option<process::http::authentication::Principal>& principal) const;
 
+  process::Future<process::http::Response> removeContainer(
+      const mesos::agent::Call& call,
+      ContentType acceptType,
+      const Option<process::http::authentication::Principal>& principal) const;
+
+  process::Future<process::http::Response> _removeContainer(
+      const ContainerID& containerId,
+      ContentType acceptType,
+      const process::Owned<AuthorizationAcceptor>& authorizer) const;
+
   process::Future<process::http::Response> launchNestedContainerSession(
       const mesos::agent::Call& call,
       const RequestMediaTypes& mediaTypes,


[04/15] mesos git commit: Added recovery logic for standalone containers.

Posted by jo...@apache.org.
Added recovery logic for standalone containers.

Although there is no way to launch standalone containers yet,
this commit outlines the expected layout of container metadata
which should be populated when launching standalone containers.

The layout is fairly simple, as standalone containers have no
framework, executor, or tasks to worry about.  The sandbox directory
will live under a new top-level directory `containers` and there is
no metadata to checkpoint at the moment.

The containerizer will checkpoint a marker file (in the runtime
directory) so that it knows to recover all standalone containers.

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


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

Branch: refs/heads/master
Commit: 14145780bd056e973bee674a4fa6a519336fcf6a
Parents: 69a7551
Author: Joseph Wu <jo...@apache.org>
Authored: Mon Jun 5 18:21:22 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 16:58:40 2017 -0800

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 59 +++++++++++++++-----
 src/slave/containerizer/mesos/paths.cpp         | 20 +++++++
 src/slave/containerizer/mesos/paths.hpp         | 16 ++++++
 src/slave/paths.cpp                             | 16 ++++++
 src/slave/paths.hpp                             | 11 ++++
 5 files changed, 109 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/14145780/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 100e3bb..c5ca76e 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -661,9 +661,10 @@ Future<Nothing> MesosContainerizerProcess::recover(
 {
   LOG(INFO) << "Recovering containerizer";
 
-  // Gather the executor run states that we will attempt to recover.
+  // Gather the container states that we will attempt to recover.
   list<ContainerState> recoverable;
   if (state.isSome()) {
+    // Gather the latest run of checkpointed executors.
     foreachvalue (const FrameworkState& framework, state.get().frameworks) {
       foreachvalue (const ExecutorState& executor, framework.executors) {
         if (executor.info.isNone()) {
@@ -743,11 +744,9 @@ Future<Nothing> MesosContainerizerProcess::recover(
     }
   }
 
-  // Recover the executor containers from 'SlaveState'.
-  hashset<ContainerID> alive;
+  // Recover the containers from 'SlaveState'.
   foreach (const ContainerState& state, recoverable) {
     const ContainerID& containerId = state.container_id();
-    alive.insert(containerId);
 
     // Contruct the structure for containers from the 'SlaveState'
     // first, to maintain the children list in the container.
@@ -767,6 +766,13 @@ Future<Nothing> MesosContainerizerProcess::recover(
   hashset<ContainerID> orphans;
 
   // Recover the containers from the runtime directory.
+  //
+  // NOTE: The returned vector guarantees that parent containers
+  // will always appear before their child containers (if any).
+  // This is particularly important for containers nested underneath
+  // standalone containers, because standalone containers are only
+  // added to the list of recoverable containers in the following loop,
+  // whereas normal parent containers are added in the prior loop.
   Try<vector<ContainerID>> containerIds =
     containerizer::paths::getContainerIds(flags.runtime_dir);
 
@@ -781,7 +787,7 @@ Future<Nothing> MesosContainerizerProcess::recover(
   // that we aggregate with any orphans that get returned from
   // calling `launcher->recover`.
   foreach (const ContainerID& containerId, containerIds.get()) {
-    if (alive.contains(containerId)) {
+    if (containers_.contains(containerId)) {
       continue;
     }
 
@@ -806,7 +812,11 @@ Future<Nothing> MesosContainerizerProcess::recover(
       return Failure("Failed to get container pid: " + pid.error());
     }
 
-    // Determine the sandbox if this is a nested container.
+    // Determine the sandbox if this is a nested or standalone container.
+    const bool isStandaloneContainer =
+      containerizer::paths::isStandaloneContainer(
+          flags.runtime_dir, containerId);
+
     Option<string> directory;
     if (containerId.has_parent()) {
       const ContainerID& rootContainerId =
@@ -818,6 +828,8 @@ Future<Nothing> MesosContainerizerProcess::recover(
             containers_[rootContainerId]->directory.get(),
             containerId);
       }
+    } else if (isStandaloneContainer) {
+      directory = slave::paths::getContainerPath(flags.work_dir, containerId);
     }
 
     Owned<Container> container(new Container());
@@ -838,19 +850,22 @@ Future<Nothing> MesosContainerizerProcess::recover(
 
     containers_[containerId] = container;
 
-    // Add recoverable nested containers to the list of 'ContainerState'.
-    //
     // TODO(klueska): The final check in the if statement makes sure
     // that this container was not marked for forcible destruction on
     // recover. We currently only support 'destroy-on-recovery'
     // semantics for nested `DEBUG` containers. If we ever support it
     // on other types of containers, we may need duplicate this logic
     // elsewhere.
-    if (containerId.has_parent() &&
-        alive.contains(protobuf::getRootContainerId(containerId)) &&
-        pid.isSome() &&
-        !containerizer::paths::getContainerForceDestroyOnRecovery(
-            flags.runtime_dir, containerId)) {
+    const bool isRecoverableNestedContainer =
+      containerId.has_parent() &&
+      (containers_.contains(protobuf::getRootContainerId(containerId))) &&
+      pid.isSome() &&
+      !containerizer::paths::getContainerForceDestroyOnRecovery(
+          flags.runtime_dir, containerId);
+
+    // Add recoverable nested containers or standalone containers
+    // to the list of 'ContainerState'.
+    if (isRecoverableNestedContainer || isStandaloneContainer) {
       CHECK_SOME(directory);
       ContainerState state =
         protobuf::slave::createContainerState(
@@ -1168,6 +1183,24 @@ Future<bool> MesosContainerizerProcess::launch(
     }
   }
 
+  // If we are launching a standalone container, checkpoint a file to
+  // mark it as a standalone container. Nested containers launched
+  // under a standalone container are treated as nested containers
+  // (_not_ as both standalone and nested containers).
+  if (!containerId.has_parent() &&
+      !containerConfig.has_task_info() &&
+      !containerConfig.has_executor_info()) {
+    const string path =
+      containerizer::paths::getStandaloneContainerMarkerPath(
+          flags.runtime_dir, containerId);
+
+    Try<Nothing> checkpointed = slave::state::checkpoint(path, "");
+    if (checkpointed.isError()) {
+      return Failure(
+          "Failed to checkpoint file to mark container as standalone");
+    }
+  }
+
   Owned<Container> container(new Container());
   container->state = PROVISIONING;
   container->config = containerConfig;

http://git-wip-us.apache.org/repos/asf/mesos/blob/14145780/src/slave/containerizer/mesos/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/paths.cpp b/src/slave/containerizer/mesos/paths.cpp
index 0c61c20..23f1fee 100644
--- a/src/slave/containerizer/mesos/paths.cpp
+++ b/src/slave/containerizer/mesos/paths.cpp
@@ -284,6 +284,26 @@ Result<ContainerTermination> getContainerTermination(
 }
 
 
+string getStandaloneContainerMarkerPath(
+    const string& runtimeDir,
+    const ContainerID& containerId)
+{
+  return path::join(
+      getRuntimePath(runtimeDir, containerId),
+      STANDALONE_MARKER_FILE);
+}
+
+
+bool isStandaloneContainer(
+    const string& runtimeDir,
+    const ContainerID& containerId)
+{
+  const string path = getStandaloneContainerMarkerPath(runtimeDir, containerId);
+
+  return os::exists(path);
+}
+
+
 Try<vector<ContainerID>> getContainerIds(const string& runtimeDir)
 {
   lambda::function<Try<vector<ContainerID>>(const Option<ContainerID>&)> helper;

http://git-wip-us.apache.org/repos/asf/mesos/blob/14145780/src/slave/containerizer/mesos/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/paths.hpp b/src/slave/containerizer/mesos/paths.hpp
index a03f15e..7b67ccf 100644
--- a/src/slave/containerizer/mesos/paths.hpp
+++ b/src/slave/containerizer/mesos/paths.hpp
@@ -59,6 +59,7 @@ namespace paths {
 //           |   |-- socket
 //           |-- launch_info
 //           |-- pid
+//           |-- standalone.marker
 //           |-- status
 //           |-- termination
 
@@ -71,6 +72,7 @@ constexpr char FORCE_DESTROY_ON_RECOVERY_FILE[] = "force_destroy_on_recovery";
 constexpr char IO_SWITCHBOARD_DIRECTORY[] = "io_switchboard";
 constexpr char CONTAINER_DIRECTORY[] = "containers";
 constexpr char CONTAINER_LAUNCH_INFO_FILE[] = "launch_info";
+constexpr char STANDALONE_MARKER_FILE[] = "standalone.marker";
 
 
 enum Mode
@@ -166,6 +168,20 @@ Result<mesos::slave::ContainerTermination> getContainerTermination(
     const ContainerID& containerId);
 
 
+// The helper method to get the standalone container marker path.
+std::string getStandaloneContainerMarkerPath(
+    const std::string& runtimeDir,
+    const ContainerID& containerId);
+
+
+// The helper method to check if the given container is a standalone
+// container or not. This is determined by the existence (or not) of
+// a marker file in the container's runtime metadata directory.
+bool isStandaloneContainer(
+    const std::string& runtimeDir,
+    const ContainerID& containerId);
+
+
 // The helper method to list all container IDs (including nested
 // containers) from the container runtime directory. The order of
 // returned vector is a result of pre-ordering walk (i.e., parent

http://git-wip-us.apache.org/repos/asf/mesos/blob/14145780/src/slave/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/paths.cpp b/src/slave/paths.cpp
index fd54652..b03ffee 100644
--- a/src/slave/paths.cpp
+++ b/src/slave/paths.cpp
@@ -66,6 +66,7 @@ const char RESOURCES_INFO_FILE[] = "resources.info";
 const char RESOURCES_TARGET_FILE[] = "resources.target";
 
 
+const char CONTAINERS_DIR[] = "containers";
 const char SLAVES_DIR[] = "slaves";
 const char FRAMEWORKS_DIR[] = "frameworks";
 const char EXECUTORS_DIR[] = "executors";
@@ -157,6 +158,21 @@ string getSlavePath(
 }
 
 
+Try<list<string>> getContainerPaths(
+    const string& rootDir)
+{
+  return fs::list(path::join(rootDir, CONTAINERS_DIR, "*"));
+}
+
+
+string getContainerPath(
+    const string& rootDir,
+    const ContainerID& containerId)
+{
+  return path::join(rootDir, CONTAINERS_DIR, stringify(containerId));
+}
+
+
 string getSlaveInfoPath(
     const string& rootDir,
     const SlaveID& slaveId)

http://git-wip-us.apache.org/repos/asf/mesos/blob/14145780/src/slave/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/paths.hpp b/src/slave/paths.hpp
index f000508..66dfa45 100644
--- a/src/slave/paths.hpp
+++ b/src/slave/paths.hpp
@@ -50,6 +50,8 @@ namespace paths {
 // The file system layout is as follows:
 //
 //   root ('--work_dir' flag)
+//   |-- containers
+//   |   |-- <container_id> (sandbox)
 //   |-- slaves
 //   |   |-- latest (symlink)
 //   |   |-- <slave_id>
@@ -137,6 +139,15 @@ std::string getSlavePath(
     const SlaveID& slaveId);
 
 
+Try<std::list<std::string>> getContainerPaths(
+    const std::string& rootDir);
+
+
+std::string getContainerPath(
+    const std::string& rootDir,
+    const ContainerID& containerId);
+
+
 Try<std::list<std::string>> getFrameworkPaths(
     const std::string& rootDir,
     const SlaveID& slaveId);


[12/15] mesos git commit: Changed failure response of LAUNCH containers API.

Posted by jo...@apache.org.
Changed failure response of LAUNCH containers API.

By default, the HTTP handlers in libprocess will translate failures
(of returned Futures) into '500 Internal Server Error'.  This commit
only changes the `LAUNCH_NESTED_CONTAINER` and `LAUNCH_CONTAINER` APIs
to return '400 Bad Request' instead when the container launch fails,
as it is more likely for the failure to be a user-input error.

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


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

Branch: refs/heads/master
Commit: 89258c04497af293b9b3aeac2e83a73d6b1141fb
Parents: fd4b9af
Author: Joseph Wu <jo...@apache.org>
Authored: Mon Oct 16 10:44:01 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Nov 14 17:16:21 2017 -0800

----------------------------------------------------------------------
 src/slave/http.cpp                      |  5 ++++
 src/tests/agent_container_api_tests.cpp | 37 ++++++++++++++++++++++++++++
 2 files changed, 42 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/89258c04/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index e4fa4df..ff6d21d 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -2554,6 +2554,11 @@ Future<Response> Http::_launchContainer(
         return BadRequest("The provided ContainerInfo is not supported");
       }
       return OK();
+    })
+    .repair([](const Future<Response>& launch) {
+      // NOTE: Failures are automatically translated into 500 Internal Server
+      // Errors, but a launch failure is likely due to user input.
+      return BadRequest(launch.failure());
     });
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/89258c04/src/tests/agent_container_api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/agent_container_api_tests.cpp b/src/tests/agent_container_api_tests.cpp
index 6b9bace..531ec73 100644
--- a/src/tests/agent_container_api_tests.cpp
+++ b/src/tests/agent_container_api_tests.cpp
@@ -623,6 +623,43 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentContainerAPITest, NestedContainerNotFound)
 }
 
 
+// This test runs tries to launch a nested container that fails upon
+// launch (rather than validation) and expects a 400 Bad Request in response.
+TEST_P_TEMP_DISABLED_ON_WINDOWS(
+    AgentContainerAPITest, NestedContainerFailLaunch)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.launcher = std::get<1>(std::get<3>(GetParam()));
+  slaveFlags.isolation = std::get<0>(std::get<3>(GetParam()));
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), slaveFlags);
+  ASSERT_SOME(slave);
+
+  Try<v1::ContainerID> parentContainerId =
+    launchParentContainer(master.get()->pid, slave.get()->pid);
+
+  ASSERT_SOME(parentContainerId);
+
+  // Launch a nested container that needs to fetch a URI that
+  // doesn't exist. The launch should therefore fail.
+  v1::ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+  containerId.mutable_parent()->CopyFrom(parentContainerId.get());
+
+  mesos::v1::CommandInfo commandInfo;
+  commandInfo.add_uris()->set_value("This file doesn't exist");
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      http::BadRequest().status,
+      launchNestedContainer(slave.get()->pid, containerId, commandInfo));
+}
+
+
 // This test attempts to give invalid ContainerInfo when launching a
 // nested container. The invalid nested container LAUNCH call is expected
 // to give a 400 Bad Request, but the parent container should be otherwise