You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by bm...@apache.org on 2016/10/06 02:00:58 UTC

[1/4] mesos git commit: Added a TODO for cleaning up the `Mesos` library lifetime issues.

Repository: mesos
Updated Branches:
  refs/heads/master 5e722cec4 -> 558613cc7


Added a TODO for cleaning up the `Mesos` library lifetime issues.

The `Mesos` library may invoke the provided callbacks after `Mesos`
is destructed. This has necessitated callers to bind a shared pointer
to the executor when creating the callbacks. However, this should not
be necessary, and this adds a TODO to describe how to eliminate this
issue.

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


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

Branch: refs/heads/master
Commit: 9b7a2a5f36571d00a812eacf97fc1d5433ebcffa
Parents: 5e722ce
Author: Benjamin Mahler <bm...@apache.org>
Authored: Wed Oct 5 12:14:50 2016 -0700
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Wed Oct 5 18:34:22 2016 -0700

----------------------------------------------------------------------
 src/tests/mesos.hpp | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9b7a2a5f/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index a05e3e0..8d2d7f4 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -1238,6 +1238,13 @@ protected:
   }
 
 private:
+  // TODO(bmahler): This is a shared pointer because the `Mesos`
+  // library copies the pointer into callbacks that can execute
+  // after `Mesos` is destructed. We can avoid this by ensuring
+  // that `~Mesos()` blocks until deferred callbacks are cleared
+  // (merely grabbing the `process::Mutex` lock is sufficient).
+  // The `Mesos` library can also provide a `Future<Nothing> stop()`
+  // to allow callers to wait for all events to be flushed.
   std::shared_ptr<MockHTTPExecutor<Mesos, Event>> executor;
 };
 


[3/4] mesos git commit: Store the user in the Executor struct in agent.

Posted by bm...@apache.org.
Store the user in the Executor struct in agent.

This was split out from https://reviews.apache.org/r/52569/.


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

Branch: refs/heads/master
Commit: 797d722c0e469d3039d1320ba0acc8348bf5030a
Parents: 0ce6361
Author: Benjamin Mahler <bm...@apache.org>
Authored: Wed Oct 5 18:39:30 2016 -0700
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Wed Oct 5 18:39:39 2016 -0700

----------------------------------------------------------------------
 src/slave/slave.cpp | 18 ++++++++++++++++--
 src/slave/slave.hpp |  6 ++++++
 2 files changed, 22 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/797d722c/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index fba0895..d30001b 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -6131,7 +6131,13 @@ Executor* Framework::launchExecutor(
       user);
 
   Executor* executor = new Executor(
-      slave, id(), executorInfo, containerId, directory, info.checkpoint());
+      slave,
+      id(),
+      executorInfo,
+      containerId,
+      directory,
+      user,
+      info.checkpoint());
 
   if (executor->checkpoint) {
     executor->checkpointExecutor();
@@ -6346,7 +6352,13 @@ void Framework::recoverExecutor(const ExecutorState& state)
       slave->flags.work_dir, slave->info.id(), id(), state.id, latest);
 
   Executor* executor = new Executor(
-      slave, id(), state.info.get(), latest, directory, info.checkpoint());
+      slave,
+      id(),
+      state.info.get(),
+      latest,
+      directory,
+      info.user(),
+      info.checkpoint());
 
   // Recover the libprocess PID if possible for PID based executors.
   if (run.get().http.isSome()) {
@@ -6443,6 +6455,7 @@ Executor::Executor(
     const ExecutorInfo& _info,
     const ContainerID& _containerId,
     const string& _directory,
+    const Option<string>& _user,
     bool _checkpoint)
   : state(REGISTERING),
     slave(_slave),
@@ -6451,6 +6464,7 @@ Executor::Executor(
     frameworkId(_frameworkId),
     containerId(_containerId),
     directory(_directory),
+    user(_user),
     checkpoint(_checkpoint),
     http(None()),
     pid(None()),

http://git-wip-us.apache.org/repos/asf/mesos/blob/797d722c/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 9beefa1..e7a6e2c 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -828,6 +828,7 @@ struct Executor
       const ExecutorInfo& info,
       const ContainerID& containerId,
       const std::string& directory,
+      const Option<std::string>& user,
       bool checkpoint);
 
   ~Executor();
@@ -909,6 +910,11 @@ struct Executor
 
   const std::string directory;
 
+  // The sandbox will be owned by this user and the executor will
+  // run as this user. This can be set to None when --switch_user
+  // is false or when compiled for Windows.
+  const Option<std::string> user;
+
   const bool checkpoint;
 
   // An Executor can either be connected via HTTP or by libprocess


[4/4] mesos git commit: Updated the nested container launch to correctly determine the user.

Posted by bm...@apache.org.
Updated the nested container launch to correctly determine the user.

This locates the executor that corresponds to the parent container,
and uses the user that was used to launch the executor.

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


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

Branch: refs/heads/master
Commit: 558613cc72248b633bb5e26ef93708abca8ccbf0
Parents: 797d722
Author: Benjamin Mahler <bm...@apache.org>
Authored: Wed Oct 5 18:40:13 2016 -0700
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Wed Oct 5 18:50:56 2016 -0700

----------------------------------------------------------------------
 src/slave/http.cpp      |  33 +++++++++++-
 src/tests/api_tests.cpp | 125 +++++++++++++++++++++++++++----------------
 2 files changed, 112 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/558613cc/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index bdafe1d..79061c3 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -1930,6 +1930,37 @@ Future<Response> Slave::Http::launchNestedContainer(
         " but 'launch_nested_container.container_id.parent.parent' is set");
   }
 
+  // Locate the executor (for now we just loop since we don't
+  // index based on container id and this likely won't have a
+  // significant performance impact due to the low number of
+  // executors per-agent).
+  Executor* executor = nullptr;
+  foreachvalue (Framework* framework, slave->frameworks) {
+    foreachvalue (Executor* executor_, framework->executors) {
+      if (executor_->containerId == containerId.parent()) {
+        executor = executor_;
+        break;
+      }
+    }
+  }
+
+  // Return a "Bad Request" here rather than "Not Found" since
+  // the executor needs to set parent to its container id.
+  if (executor == nullptr) {
+    return BadRequest("Unable to locate executor for parent container"
+                      " " + stringify(containerId.parent()));
+  }
+
+  // By default, we use the executor's user.
+  // The command user overrides it if specified.
+  Option<string> user = executor->user;
+
+#ifndef __WINDOWS__
+  if (call.launch_nested_container().command().has_user()) {
+    user = call.launch_nested_container().command().user();
+  }
+#endif
+
   // TODO(gilbert): The sandbox directory and user are incorrect,
   // Please update it.
   Future<bool> launched = slave->containerizer->launch(
@@ -1938,7 +1969,7 @@ Future<Response> Slave::Http::launchNestedContainer(
       call.launch_nested_container().has_container()
         ? call.launch_nested_container().container()
         : Option<ContainerInfo>::none(),
-      None(),
+      user,
       slave->info.id());
 
   // TODO(bmahler): The containerizers currently require that

http://git-wip-us.apache.org/repos/asf/mesos/blob/558613cc/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 7b0ad3c..24fefbd 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -3330,31 +3330,54 @@ TEST_P(AgentAPITest, NestedContainerLaunchFalse)
 
   Clock::pause();
 
-  StandaloneMasterDetector detector;
-  MockContainerizer mockContainerizer;
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
 
-  EXPECT_CALL(mockContainerizer, recover(_))
-    .WillOnce(Return(Future<Nothing>(Nothing())));
+  Owned<MasterDetector> detector = master.get()->createDetector();
 
-  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+  MockExecutor exec(DEFAULT_EXECUTOR_ID);
+  TestContainerizer containerizer(&exec);
 
   Try<Owned<cluster::Slave>> slave =
-    StartSlave(&detector, &mockContainerizer);
+    StartSlave(detector.get(), &containerizer);
 
   ASSERT_SOME(slave);
 
-  // Wait for the agent to finish recovery.
-  AWAIT_READY(__recover);
-  Clock::settle();
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .Times(1);
+
+  Future<vector<Offer>> offers;
+  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(_, _))
+    .Times(1);
+
+  driver.start();
+
+  AWAIT_READY(executorRegistered);
+
+  Future<hashset<ContainerID>> containerIds = containerizer.containers();
+  AWAIT_READY(containerIds);
+  EXPECT_EQ(1u, containerIds->size());
 
   // Try to launch an "unsupported" container.
   v1::ContainerID containerId;
   containerId.set_value(UUID::random().toString());
-  containerId.mutable_parent()->set_value(UUID::random().toString());
+  containerId.mutable_parent()->set_value(containerIds->begin()->value());
 
   {
     // Return false here to indicate "unsupported".
-    EXPECT_CALL(mockContainerizer, launch(_, _, _, _, _))
+    EXPECT_CALL(containerizer, launch(_, _, _, _, _))
       .WillOnce(Return(Future<bool>(false)));
 
     v1::agent::Call call;
@@ -3373,10 +3396,11 @@ TEST_P(AgentAPITest, NestedContainerLaunchFalse)
     AWAIT_EXPECT_RESPONSE_STATUS_EQ(BadRequest().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>()));
+  EXPECT_CALL(exec, shutdown(_))
+    .Times(AtMost(1));
+
+  driver.stop();
+  driver.join();
 }
 
 
@@ -3429,32 +3453,52 @@ TEST_P(AgentAPITest, NestedContainerLaunch)
 
   Clock::pause();
 
-  StandaloneMasterDetector detector;
-  MockContainerizer mockContainerizer;
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
 
-  EXPECT_CALL(mockContainerizer, recover(_))
-    .WillOnce(Return(Future<Nothing>(Nothing())));
+  Owned<MasterDetector> detector = master.get()->createDetector();
 
-  Future<Nothing> __recover = FUTURE_DISPATCH(_, &Slave::__recover);
+  MockExecutor exec(DEFAULT_EXECUTOR_ID);
+  TestContainerizer containerizer(&exec);
 
   Try<Owned<cluster::Slave>> slave =
-    StartSlave(&detector, &mockContainerizer);
+    StartSlave(detector.get(), &containerizer);
 
   ASSERT_SOME(slave);
 
-  // Wait for the agent to finish recovery.
-  AWAIT_READY(__recover);
-  Clock::settle();
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .Times(1);
+
+  Future<vector<Offer>> offers;
+  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(_, _))
+    .Times(1);
+
+  driver.start();
+
+  AWAIT_READY(executorRegistered);
+
+  Future<hashset<ContainerID>> containerIds = containerizer.containers();
+  AWAIT_READY(containerIds);
+  EXPECT_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(UUID::random().toString());
+  containerId.mutable_parent()->set_value(containerIds->begin()->value());
 
   {
-    EXPECT_CALL(mockContainerizer, launch(_, _, _, _, _))
-      .WillOnce(Return(Future<bool>(true)));
-
     v1::agent::Call call;
     call.set_type(v1::agent::Call::LAUNCH_NESTED_CONTAINER);
 
@@ -3471,13 +3515,9 @@ TEST_P(AgentAPITest, NestedContainerLaunch)
     AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
   }
 
-  Promise<Option<ContainerTermination>> waitPromise;
   Future<v1::agent::Response> wait;
 
   {
-    EXPECT_CALL(mockContainerizer, wait(_))
-      .WillOnce(Return(waitPromise.future()));
-
     v1::agent::Call call;
     call.set_type(v1::agent::Call::WAIT_NESTED_CONTAINER);
 
@@ -3493,9 +3533,6 @@ TEST_P(AgentAPITest, NestedContainerLaunch)
 
   // Now kill the nested container.
   {
-    EXPECT_CALL(mockContainerizer, destroy(_))
-      .WillOnce(Return(Future<bool>(true)));
-
     v1::agent::Call call;
     call.set_type(v1::agent::Call::KILL_NESTED_CONTAINER);
 
@@ -3512,19 +3549,17 @@ TEST_P(AgentAPITest, NestedContainerLaunch)
     AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
   }
 
-  ContainerTermination termination;
-  termination.set_status(1);
-
-  waitPromise.set(Option<ContainerTermination>::some(termination));
-
   AWAIT_READY(wait);
   ASSERT_EQ(v1::agent::Response::WAIT_NESTED_CONTAINER, wait->type());
-  EXPECT_EQ(1, wait->wait_nested_container().exit_status());
 
-  // 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>()));
+  // 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();
 }
 
 } // namespace tests {


[2/4] mesos git commit: Added support for nested containers to the test containerizer.

Posted by bm...@apache.org.
Added support for nested containers to the test containerizer.

In the process, the maps that were stored were simplified.

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


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

Branch: refs/heads/master
Commit: 0ce6361bfa17d0d75f1cae60800bbc35a69eb449
Parents: 9b7a2a5
Author: Benjamin Mahler <bm...@apache.org>
Authored: Wed Oct 5 12:15:23 2016 -0700
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Wed Oct 5 18:36:57 2016 -0700

----------------------------------------------------------------------
 src/tests/containerizer.cpp | 178 ++++++++++++++++++++++++++-------------
 src/tests/containerizer.hpp |  50 +++++++++--
 2 files changed, 162 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0ce6361b/src/tests/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.cpp b/src/tests/containerizer.cpp
index 89c5400..3affa0f 100644
--- a/src/tests/containerizer.cpp
+++ b/src/tests/containerizer.cpp
@@ -44,15 +44,21 @@ TestContainerizer::TestContainerizer(
     const ExecutorID& executorId,
     const shared_ptr<MockV1HTTPExecutor>& executor)
 {
-  v1Executors[executorId] = executor;
+  executors[executorId] = Owned<ExecutorData>(new ExecutorData());
+  executors.at(executorId)->v1ExecutorMock = executor;
+
   setup();
 }
 
 
 TestContainerizer::TestContainerizer(
     const hashmap<ExecutorID, Executor*>& _executors)
-  : executors(_executors)
 {
+  foreachpair (const ExecutorID& executorId, Executor* executor, _executors) {
+    executors[executorId] = Owned<ExecutorData>(new ExecutorData());
+    executors.at(executorId)->executor = executor;
+  }
+
   setup();
 }
 
@@ -61,14 +67,18 @@ TestContainerizer::TestContainerizer(
     const ExecutorID& executorId,
     Executor* executor)
 {
-  executors[executorId] = executor;
+  executors[executorId] = Owned<ExecutorData>(new ExecutorData());
+  executors.at(executorId)->executor = executor;
+
   setup();
 }
 
 
 TestContainerizer::TestContainerizer(MockExecutor* executor)
 {
-  executors[executor->id] = executor;
+  executors[executor->id] = Owned<ExecutorData>(new ExecutorData());
+  executors.at(executor->id)->executor = executor;
+
   setup();
 }
 
@@ -81,11 +91,12 @@ TestContainerizer::TestContainerizer()
 
 TestContainerizer::~TestContainerizer()
 {
-  foreachvalue (const Owned<MesosExecutorDriver>& driver, drivers) {
-    driver->stop();
-    driver->join();
+  foreachvalue (const Owned<ExecutorData>& data, executors) {
+    if (data->driver.get() != nullptr) {
+      data->driver->stop();
+      data->driver->join();
+    }
   }
-  drivers.clear();
 }
 
 
@@ -99,22 +110,20 @@ Future<bool> TestContainerizer::_launch(
     const map<string, string>& environment,
     bool checkpoint)
 {
-  CHECK(!drivers.contains(containerId))
-    << "Failed to launch executor '" << executorInfo.executor_id()
-    << "' of framework " << executorInfo.framework_id()
+  CHECK(!containers_.contains(containerId))
+    << "Failed to launch container " << containerId
+    << " for executor '" << executorInfo.executor_id() << "'"
+    << " of framework " << executorInfo.framework_id()
     << " because it is already launched";
 
-  CHECK(executors.contains(executorInfo.executor_id()) ||
-        v1Executors.contains(executorInfo.executor_id()))
-    << "Failed to launch executor '" << executorInfo.executor_id()
-    << "' of framework " << executorInfo.framework_id()
+  CHECK(executors.contains(executorInfo.executor_id()))
+    << "Failed to launch executor '" << executorInfo.executor_id() << "'"
+    << " of framework " << executorInfo.framework_id()
     << " because it is unknown to the containerizer";
 
-  // Store mapping from (frameworkId, executorId) -> containerId to facilitate
-  // easy destroy from tests.
-  std::pair<FrameworkID, ExecutorID> key(executorInfo.framework_id(),
-                                         executorInfo.executor_id());
-  containers_[key] = containerId;
+  containers_[containerId] = Owned<ContainerData>(new ContainerData());
+  containers_.at(containerId)->executorId = executorInfo.executor_id();
+  containers_.at(containerId)->frameworkId = executorInfo.framework_id();
 
   // We need to synchronize all reads and writes to the environment as this is
   // global state.
@@ -153,21 +162,17 @@ Future<bool> TestContainerizer::_launch(
 
     os::setenv("MESOS_LOCAL", "1");
 
-    if (executors.contains(executorInfo.executor_id())) {
-      Executor* executor = executors[executorInfo.executor_id()];
+    const Owned<ExecutorData>& executorData =
+      executors.at(executorInfo.executor_id());
 
-      Owned<MesosExecutorDriver> driver(new MesosExecutorDriver(executor));
-      drivers[containerId] = driver;
-
-      driver->start();
+    if (executorData->executor != nullptr) {
+      executorData->driver = Owned<MesosExecutorDriver>(
+          new MesosExecutorDriver(executorData->executor));
+      executorData->driver->start();
     } else {
-      shared_ptr<MockV1HTTPExecutor> executor =
-        v1Executors[executorInfo.executor_id()];
-
-      Owned<executor::TestV1Mesos> mesos(
-          new executor::TestV1Mesos(ContentType::PROTOBUF, executor));
-
-      v1Libraries[containerId] = mesos;
+      shared_ptr<MockV1HTTPExecutor> executor = executorData->v1ExecutorMock;
+      executorData->v1Library = Owned<executor::TestV1Mesos>(
+        new executor::TestV1Mesos(ContentType::PROTOBUF, executor));
     }
 
     os::unsetenv("MESOS_LOCAL");
@@ -186,10 +191,24 @@ Future<bool> TestContainerizer::_launch(
     }
   }
 
-  promises[containerId] =
-    Owned<Promise<ContainerTermination>>(
-      new Promise<ContainerTermination>());
+  return true;
+}
+
 
+Future<bool> TestContainerizer::_launch(
+    const ContainerID& containerId,
+    const CommandInfo& commandInfo,
+    const Option<ContainerInfo>& containerInfo,
+    const Option<string>& user,
+    const SlaveID& slaveId)
+{
+  CHECK(!containers_.contains(containerId))
+    << "Failed to launch nested container " << containerId
+    << " because it is already launched";
+
+  containers_[containerId] = Owned<ContainerData>(new ContainerData());
+
+  // No-op for now.
   return true;
 }
 
@@ -199,11 +218,11 @@ Future<Option<ContainerTermination>> TestContainerizer::_wait(
 {
   // An unknown container is possible for tests where we "drop" the
   // 'launch' in order to verify recovery still works correctly.
-  if (!promises.contains(containerId)) {
+  if (!containers_.contains(containerId)) {
     return None();
   }
 
-  return promises.at(containerId)->future()
+  return containers_.at(containerId)->termination.future()
     .then(Option<ContainerTermination>::some);
 }
 
@@ -212,46 +231,65 @@ Future<bool> TestContainerizer::destroy(
     const FrameworkID& frameworkId,
     const ExecutorID& executorId)
 {
-  std::pair<FrameworkID, ExecutorID> key(frameworkId, executorId);
-  if (!containers_.contains(key)) {
-    LOG(WARNING) << "Ignoring destroy of unknown container for executor '"
-                 << executorId << "' of framework " << frameworkId;
+  Option<ContainerID> containerId = None();
+
+  foreachpair (const ContainerID& containerId_,
+               const Owned<ContainerData>& container,
+               containers_) {
+    if (container->frameworkId == frameworkId &&
+        container->executorId == executorId) {
+      containerId = containerId_;
+    }
+  }
+
+  if (containerId.isNone()) {
+    LOG(WARNING) << "Ignoring destroy of unknown container"
+                 << " for executor '" << executorId << "'"
+                 << " of framework " << frameworkId;
     return false;
   }
 
-  return _destroy(containers_.at(key));
+  return _destroy(containerId.get());
 }
 
 
 Future<bool> TestContainerizer::_destroy(const ContainerID& containerId)
 {
-  if (drivers.contains(containerId)) {
-    Owned<MesosExecutorDriver> driver = drivers.at(containerId);
-    driver->stop();
-    driver->join();
-    drivers.erase(containerId);
+  if (!containers_.contains(containerId)) {
+    return false;
   }
 
-  if (v1Libraries.contains(containerId)) {
-    v1Libraries.erase(containerId);
-  }
+  const Owned<ContainerData>& containerData = containers_.at(containerId);
 
-  if (promises.contains(containerId)) {
-    ContainerTermination termination;
-    termination.set_message("Killed executor");
-    termination.set_status(0);
+  if (containerData->executorId.isSome()) {
+    CHECK(executors.contains(containerData->executorId.get()));
 
-    promises.at(containerId)->set(termination);
-    promises.erase(containerId);
+    const Owned<ExecutorData>& executorData =
+      executors.at(containerData->executorId.get());
+
+    if (executorData->driver.get() != nullptr) {
+      executorData->driver->stop();
+      executorData->driver->join();
+    }
+
+    executors.erase(containerData->executorId.get());
   }
 
+  ContainerTermination termination;
+  termination.set_message("Killed executor");
+  termination.set_status(0);
+
+  containerData->termination.set(termination);
+
+  containers_.erase(containerId);
+
   return true;
 }
 
 
 Future<hashset<ContainerID>> TestContainerizer::containers()
 {
-  return promises.keys();
+  return containers_.keys();
 }
 
 
@@ -280,8 +318,30 @@ void TestContainerizer::setup()
   EXPECT_CALL(*this, update(_, _))
     .WillRepeatedly(Return(Nothing()));
 
+  Future<bool> (TestContainerizer::*_launch)(
+      const ContainerID& containerId,
+      const Option<TaskInfo>& taskInfo,
+      const ExecutorInfo& executorInfo,
+      const string& directory,
+      const Option<string>& user,
+      const SlaveID& slaveId,
+      const map<string, string>& environment,
+      bool checkpoint) =
+    &TestContainerizer::_launch;
+
   EXPECT_CALL(*this, launch(_, _, _, _, _, _, _, _))
-    .WillRepeatedly(Invoke(this, &TestContainerizer::_launch));
+    .WillRepeatedly(Invoke(this, _launch));
+
+  Future<bool> (TestContainerizer::*_launchNested)(
+      const ContainerID& containerId,
+      const CommandInfo& commandInfo,
+      const Option<ContainerInfo>& containerInfo,
+      const Option<string>& user,
+      const SlaveID& slaveId) =
+    &TestContainerizer::_launch;
+
+  EXPECT_CALL(*this, launch(_, _, _, _, _))
+    .WillRepeatedly(Invoke(this, _launchNested));
 
   EXPECT_CALL(*this, wait(_))
     .WillRepeatedly(Invoke(this, &TestContainerizer::_wait));

http://git-wip-us.apache.org/repos/asf/mesos/blob/0ce6361b/src/tests/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.hpp b/src/tests/containerizer.hpp
index 3cf418a..0ffba73 100644
--- a/src/tests/containerizer.hpp
+++ b/src/tests/containerizer.hpp
@@ -57,6 +57,8 @@ class MockExecutor;
 class TestContainerizer : public slave::Containerizer
 {
 public:
+  // TODO(bmahler): These constructors assume that ExecutorIDs are
+  // unique across FrameworkIDs, which is not the case.
   TestContainerizer(
       const ExecutorID& executorId,
       const std::shared_ptr<MockV1HTTPExecutor>& executor);
@@ -89,6 +91,15 @@ public:
           const std::map<std::string, std::string>&,
           bool checkpoint));
 
+  MOCK_METHOD5(
+      launch,
+      process::Future<bool>(
+          const ContainerID& containerId,
+          const CommandInfo& commandInfo,
+          const Option<ContainerInfo>& containerInfo,
+          const Option<std::string>& user,
+          const SlaveID& slaveId));
+
   MOCK_METHOD2(
       update,
       process::Future<Nothing>(const ContainerID&, const Resources&));
@@ -120,6 +131,7 @@ private:
   void setup();
 
   // Default implementations of mock methods.
+
   process::Future<bool> _launch(
       const ContainerID& containerId,
       const Option<TaskInfo>& taskInfo,
@@ -130,19 +142,43 @@ private:
       const std::map<std::string, std::string>& environment,
       bool checkpoint);
 
+  process::Future<bool> _launch(
+      const ContainerID& containerId,
+      const CommandInfo& commandInfo,
+      const Option<ContainerInfo>& containerInfo,
+      const Option<std::string>& user,
+      const SlaveID& slaveId);
+
   process::Future<Option<mesos::slave::ContainerTermination>> _wait(
       const ContainerID& containerId) const;
 
   process::Future<bool> _destroy(const ContainerID& containerID);
 
-  hashmap<ExecutorID, Executor*> executors;
-  hashmap<ExecutorID, std::shared_ptr<MockV1HTTPExecutor>> v1Executors;
+  struct ContainerData
+  {
+    Option<ExecutorID> executorId;
+    Option<FrameworkID> frameworkId;
+
+    process::Promise<mesos::slave::ContainerTermination> termination;
+  };
+
+  hashmap<ContainerID, process::Owned<ContainerData>> containers_;
+
+  struct ExecutorData
+  {
+    // Pre-HTTP executors.
+    Executor* executor;
+    process::Owned<MesosExecutorDriver> driver;
+
+    // HTTP executors. Note that `mesos::v1::executor::Mesos`
+    // requires that we provide it a shared pointer to the executor.
+    std::shared_ptr<MockV1HTTPExecutor> v1ExecutorMock;
+    process::Owned<executor::TestV1Mesos> v1Library;
+  };
 
-  hashmap<std::pair<FrameworkID, ExecutorID>, ContainerID> containers_;
-  hashmap<ContainerID, process::Owned<MesosExecutorDriver>> drivers;
-  hashmap<ContainerID, process::Owned<executor::TestV1Mesos>> v1Libraries;
-  hashmap<ContainerID, process::Owned<
-      process::Promise<mesos::slave::ContainerTermination>>> promises;
+  // TODO(bmahler): The test containerizer currently assumes that
+  // executor IDs are unique across frameworks (see the constructors).
+  hashmap<ExecutorID, process::Owned<ExecutorData>> executors;
 };
 
 } // namespace tests {