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/11/04 20:28:24 UTC

[1/2] mesos git commit: Fixed the TestContainerizer to be thread-safe.

Repository: mesos
Updated Branches:
  refs/heads/master 6a22d4d95 -> 8f976a8ef


Fixed the TestContainerizer to be thread-safe.

The TestContainerizer is currently not backed by a Process and
does not do any explicit synchronization and so is not thread safe.

Most tests currently cannot trip the concurrency issues, but
this surfaced recently in MESOS-6544.

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


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

Branch: refs/heads/master
Commit: 250cfcb7bbc4ece56fb43da2cd35aea102d7a289
Parents: 6a22d4d
Author: Benjamin Mahler <bm...@apache.org>
Authored: Thu Nov 3 23:06:12 2016 -0700
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Fri Nov 4 13:28:12 2016 -0700

----------------------------------------------------------------------
 src/tests/containerizer.cpp | 657 ++++++++++++++++++++++++++-------------
 src/tests/containerizer.hpp |  58 ++--
 2 files changed, 471 insertions(+), 244 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/250cfcb7/src/tests/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.cpp b/src/tests/containerizer.cpp
index c220866..32bdb87 100644
--- a/src/tests/containerizer.cpp
+++ b/src/tests/containerizer.cpp
@@ -40,273 +40,358 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
-TestContainerizer::TestContainerizer(
-    const ExecutorID& executorId,
-    const shared_ptr<v1::MockHTTPExecutor>& executor)
-{
-  executors[executorId] = Owned<ExecutorData>(new ExecutorData());
-  executors.at(executorId)->v1ExecutorMock = executor;
-
-  setup();
-}
-
 
-TestContainerizer::TestContainerizer(
-    const hashmap<ExecutorID, Executor*>& _executors)
+class TestContainerizerProcess
+  : public process::Process<TestContainerizerProcess>
 {
-  foreachpair (const ExecutorID& executorId, Executor* executor, _executors) {
+public:
+  TestContainerizerProcess() {}
+
+  TestContainerizerProcess(
+      const ExecutorID& executorId,
+      const std::shared_ptr<v1::MockHTTPExecutor>& executor)
+  {
     executors[executorId] = Owned<ExecutorData>(new ExecutorData());
-    executors.at(executorId)->executor = executor;
+    executors.at(executorId)->v1ExecutorMock = executor;
   }
 
-  setup();
-}
+  TestContainerizerProcess(
+      const hashmap<ExecutorID, Executor*>& _executors)
+  {
+    foreachpair (const ExecutorID& executorId, Executor* executor, _executors) {
+      executors[executorId] = Owned<ExecutorData>(new ExecutorData());
+      executors.at(executorId)->executor = executor;
+    }
+  }
 
+  virtual ~TestContainerizerProcess()
+  {
+    foreachvalue (const Owned<ExecutorData>& data, executors) {
+      if (data->driver.get() != nullptr) {
+        data->driver->stop();
+        data->driver->join();
+      }
+    }
+  }
 
-TestContainerizer::TestContainerizer(
-    const ExecutorID& executorId,
-    Executor* executor)
-{
-  executors[executorId] = Owned<ExecutorData>(new ExecutorData());
-  executors.at(executorId)->executor = executor;
+  Future<Nothing> recover(
+      const Option<slave::state::SlaveState>& state)
+  {
+    return Nothing();
+  }
 
-  setup();
-}
+  Future<bool> 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)
+  {
+    CHECK(!terminatedContainers.contains(containerId))
+      << "Failed to launch nested container " << containerId
+      << " for executor '" << executorInfo.executor_id() << "'"
+      << " of framework " << executorInfo.framework_id()
+      << " because this ContainerID is being re-used with"
+      << " a previously terminated container";
+
+    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()))
+      << "Failed to launch executor '" << executorInfo.executor_id() << "'"
+      << " of framework " << executorInfo.framework_id()
+      << " because it is unknown to the containerizer";
+
+    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.
+    //
+    // TODO(jmlvanre): Even this is not sufficient, as other aspects
+    // of the code may read an environment variable while we are
+    // manipulating it. The better solution is to pass the environment
+    // variables into the fork, or to set them on the command line.
+    // See MESOS-3475.
+    static std::mutex mutex;
+
+    synchronized(mutex) {
+      // Since the constructor for `MesosExecutorDriver` reads
+      // environment variables to load flags, even it needs to
+      // be within this synchronization section.
+      //
+      // Prepare additional environment variables for the executor.
+      // TODO(benh): Need to get flags passed into the TestContainerizer
+      // in order to properly use here.
+      slave::Flags flags;
+      flags.recovery_timeout = Duration::zero();
+
+      // We need to save the original set of environment variables so we
+      // can reset the environment after calling 'driver->start()' below.
+      hashmap<string, string> original = os::environment();
+
+      foreachpair (const string& name, const string variable, environment) {
+        os::setenv(name, variable);
+      }
 
+      // TODO(benh): Can this be removed and done exclusively in the
+      // 'executorEnvironment()' function? There are other places in the
+      // code where we do this as well and it's likely we can do this once
+      // in 'executorEnvironment()'.
+      foreach (const Environment::Variable& variable,
+               executorInfo.command().environment().variables()) {
+        os::setenv(variable.name(), variable.value());
+      }
 
-TestContainerizer::TestContainerizer(MockExecutor* executor)
-{
-  executors[executor->id] = Owned<ExecutorData>(new ExecutorData());
-  executors.at(executor->id)->executor = executor;
+      os::setenv("MESOS_LOCAL", "1");
 
-  setup();
-}
+      const Owned<ExecutorData>& executorData =
+        executors.at(executorInfo.executor_id());
 
+      if (executorData->executor != nullptr) {
+        executorData->driver = Owned<MesosExecutorDriver>(
+            new MesosExecutorDriver(executorData->executor));
+        executorData->driver->start();
+      } else {
+        shared_ptr<v1::MockHTTPExecutor> executor =
+          executorData->v1ExecutorMock;
+        executorData->v1Library = Owned<v1::executor::TestMesos>(
+          new v1::executor::TestMesos(ContentType::PROTOBUF, executor));
+      }
 
-TestContainerizer::TestContainerizer()
-{
-  setup();
-}
+      os::unsetenv("MESOS_LOCAL");
 
+      // Unset the environment variables we set by resetting them to their
+      // original values and also removing any that were not part of the
+      // original environment.
+      foreachpair (const string& name, const string& value, original) {
+        os::setenv(name, value);
+      }
 
-TestContainerizer::~TestContainerizer()
-{
-  foreachvalue (const Owned<ExecutorData>& data, executors) {
-    if (data->driver.get() != nullptr) {
-      data->driver->stop();
-      data->driver->join();
+      foreachkey (const string& name, environment) {
+        if (!original.contains(name)) {
+          os::unsetenv(name);
+        }
+      }
     }
+
+    return true;
   }
-}
 
+  Future<bool> launch(
+      const ContainerID& containerId,
+      const CommandInfo& commandInfo,
+      const Option<ContainerInfo>& containerInfo,
+      const Option<string>& user,
+      const SlaveID& slaveId)
+  {
+    CHECK(!terminatedContainers.contains(containerId))
+      << "Failed to launch nested container " << containerId
+      << " because this ContainerID is being re-used with"
+      << " a previously terminated container";
 
-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)
-{
-  CHECK(!terminatedContainers.contains(containerId))
-    << "Failed to launch nested container " << containerId
-    << " for executor '" << executorInfo.executor_id() << "'"
-    << " of framework " << executorInfo.framework_id()
-    << " because this ContainerID is being re-used with"
-    << " a previously terminated container";
-
-  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()))
-    << "Failed to launch executor '" << executorInfo.executor_id() << "'"
-    << " of framework " << executorInfo.framework_id()
-    << " because it is unknown to the containerizer";
-
-  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.
-  // TODO(jmlvanre): Even this is not sufficient, as other aspects of the code
-  // may read an environment variable while we are manipulating it. The better
-  // solution is to pass the environment variables into the fork, or to set them
-  // on the command line. See MESOS-3475.
-  static std::mutex mutex;
-
-  synchronized(mutex) {
-    // Since the constructor for `MesosExecutorDriver` reads environment
-    // variables to load flags, even it needs to be within this synchronization
-    // section.
-    // Prepare additional environment variables for the executor.
-    // TODO(benh): Need to get flags passed into the TestContainerizer
-    // in order to properly use here.
-    slave::Flags flags;
-    flags.recovery_timeout = Duration::zero();
-
-    // We need to save the original set of environment variables so we
-    // can reset the environment after calling 'driver->start()' below.
-    hashmap<string, string> original = os::environment();
-
-    foreachpair (const string& name, const string variable, environment) {
-      os::setenv(name, variable);
-    }
+    CHECK(!containers_.contains(containerId))
+      << "Failed to launch nested container " << containerId
+      << " because it is already launched";
 
-    // TODO(benh): Can this be removed and done exclusively in the
-    // 'executorEnvironment()' function? There are other places in the
-    // code where we do this as well and it's likely we can do this once
-    // in 'executorEnvironment()'.
-    foreach (const Environment::Variable& variable,
-             executorInfo.command().environment().variables()) {
-      os::setenv(variable.name(), variable.value());
-    }
+    containers_[containerId] = Owned<ContainerData>(new ContainerData());
 
-    os::setenv("MESOS_LOCAL", "1");
+    // No-op for now.
+    return true;
+  }
 
-    const Owned<ExecutorData>& executorData =
-      executors.at(executorInfo.executor_id());
+  Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources)
+  {
+    return Nothing();
+  }
 
-    if (executorData->executor != nullptr) {
-      executorData->driver = Owned<MesosExecutorDriver>(
-          new MesosExecutorDriver(executorData->executor));
-      executorData->driver->start();
-    } else {
-      shared_ptr<v1::MockHTTPExecutor> executor = executorData->v1ExecutorMock;
-      executorData->v1Library = Owned<v1::executor::TestMesos>(
-        new v1::executor::TestMesos(ContentType::PROTOBUF, executor));
-    }
+  Future<ResourceStatistics> usage(
+      const ContainerID& containerId)
+  {
+    return ResourceStatistics();
+  }
 
-    os::unsetenv("MESOS_LOCAL");
+  Future<ContainerStatus> status(
+      const ContainerID& containerId)
+  {
+    return ContainerStatus();
+  }
 
-    // Unset the environment variables we set by resetting them to their
-    // original values and also removing any that were not part of the
-    // original environment.
-    foreachpair (const string& name, const string& value, original) {
-      os::setenv(name, value);
+  Future<Option<mesos::slave::ContainerTermination>> wait(
+      const ContainerID& containerId)
+  {
+    if (terminatedContainers.contains(containerId)) {
+      return terminatedContainers.at(containerId);
     }
 
-    foreachkey (const string& name, environment) {
-      if (!original.contains(name)) {
-        os::unsetenv(name);
-      }
+    // An unknown container is possible for tests where we "drop" the
+    // 'launch' in order to verify recovery still works correctly.
+    if (!containers_.contains(containerId)) {
+      return None();
     }
+
+    return containers_.at(containerId)->termination.future()
+      .then(Option<ContainerTermination>::some);
   }
 
-  return true;
-}
+  Future<bool> destroy(
+      const ContainerID& containerId)
+  {
+    if (!containers_.contains(containerId)) {
+      return false;
+    }
 
+    const Owned<ContainerData>& containerData = containers_.at(containerId);
 
-Future<bool> TestContainerizer::_launch(
-    const ContainerID& containerId,
-    const CommandInfo& commandInfo,
-    const Option<ContainerInfo>& containerInfo,
-    const Option<string>& user,
-    const SlaveID& slaveId)
-{
-  CHECK(!terminatedContainers.contains(containerId))
-    << "Failed to launch nested container " << containerId
-    << " because this ContainerID is being re-used with"
-    << " a previously terminated container";
+    if (containerData->executorId.isSome()) {
+      CHECK(executors.contains(containerData->executorId.get()));
 
-  CHECK(!containers_.contains(containerId))
-    << "Failed to launch nested container " << containerId
-    << " because it is already launched";
+      const Owned<ExecutorData>& executorData =
+        executors.at(containerData->executorId.get());
 
-  containers_[containerId] = Owned<ContainerData>(new ContainerData());
+      if (executorData->driver.get() != nullptr) {
+        executorData->driver->stop();
+        executorData->driver->join();
+      }
 
-  // No-op for now.
-  return true;
-}
+      executors.erase(containerData->executorId.get());
+    }
 
+    ContainerTermination termination;
+    termination.set_message("Killed executor");
+    termination.set_status(0);
 
-Future<Option<ContainerTermination>> TestContainerizer::_wait(
-    const ContainerID& containerId) const
-{
-  if (terminatedContainers.contains(containerId)) {
-    return terminatedContainers.at(containerId);
-  }
+    containerData->termination.set(termination);
 
-  // An unknown container is possible for tests where we "drop" the
-  // 'launch' in order to verify recovery still works correctly.
-  if (!containers_.contains(containerId)) {
-    return None();
-  }
+    containers_.erase(containerId);
+    terminatedContainers[containerId] = termination;
 
-  return containers_.at(containerId)->termination.future()
-    .then(Option<ContainerTermination>::some);
-}
+    return true;
+  }
 
+  // Additional destroy method for testing because we won't know the
+  // ContainerID created for each container.
+  Future<bool> destroy(
+      const FrameworkID& frameworkId,
+      const ExecutorID& executorId)
+  {
+    Option<ContainerID> containerId = None();
+
+    foreachpair (const ContainerID& containerId_,
+                 const Owned<ContainerData>& container,
+                 containers_) {
+      if (container->frameworkId == frameworkId &&
+          container->executorId == executorId) {
+        containerId = containerId_;
+      }
+    }
 
-Future<bool> TestContainerizer::destroy(
-    const FrameworkID& frameworkId,
-    const ExecutorID& executorId)
-{
-  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(containerId.get());
   }
 
-  if (containerId.isNone()) {
-    LOG(WARNING) << "Ignoring destroy of unknown container"
-                 << " for executor '" << executorId << "'"
-                 << " of framework " << frameworkId;
-    return false;
+  Future<hashset<ContainerID>> containers()
+  {
+    return containers_.keys();
   }
 
-  return _destroy(containerId.get());
-}
+private:
+  struct ContainerData
+  {
+    Option<ExecutorID> executorId;
+    Option<FrameworkID> frameworkId;
 
+    process::Promise<mesos::slave::ContainerTermination> termination;
+  };
 
-Future<bool> TestContainerizer::_destroy(const ContainerID& containerId)
-{
-  if (!containers_.contains(containerId)) {
-    return false;
-  }
+  // We also store the terminated containers to allow callers to
+  // "reap" the termination if a container is already destroyed.
+  // This mimics the behavior of the mesos containerizer.
+  hashmap<ContainerID, process::Owned<ContainerData>> containers_;
+  hashmap<ContainerID, mesos::slave::ContainerTermination> terminatedContainers;
 
-  const Owned<ContainerData>& containerData = containers_.at(containerId);
+  struct ExecutorData
+  {
+    // Pre-HTTP executors.
+    Executor* executor;
+    process::Owned<MesosExecutorDriver> driver;
 
-  if (containerData->executorId.isSome()) {
-    CHECK(executors.contains(containerData->executorId.get()));
+    // HTTP executors. Note that `mesos::v1::executor::Mesos`
+    // requires that we provide it a shared pointer to the executor.
+    shared_ptr<v1::MockHTTPExecutor> v1ExecutorMock;
+    process::Owned<v1::executor::TestMesos> v1Library;
+  };
 
-    const Owned<ExecutorData>& executorData =
-      executors.at(containerData->executorId.get());
+  // TODO(bmahler): The test containerizer currently assumes that
+  // executor IDs are unique across frameworks (see the constructors).
+  hashmap<ExecutorID, process::Owned<ExecutorData>> executors;
+};
 
-    if (executorData->driver.get() != nullptr) {
-      executorData->driver->stop();
-      executorData->driver->join();
-    }
 
-    executors.erase(containerData->executorId.get());
-  }
+TestContainerizer::TestContainerizer(
+    const ExecutorID& executorId,
+    const shared_ptr<v1::MockHTTPExecutor>& executor)
+  : process(new TestContainerizerProcess(executorId, executor))
+{
+  spawn(process.get());
+  setup();
+}
 
-  ContainerTermination termination;
-  termination.set_message("Killed executor");
-  termination.set_status(0);
 
-  containerData->termination.set(termination);
+TestContainerizer::TestContainerizer(
+    const hashmap<ExecutorID, Executor*>& executors)
+  : process(new TestContainerizerProcess(executors))
+{
+  spawn(process.get());
+  setup();
+}
 
-  containers_.erase(containerId);
-  terminatedContainers[containerId] = termination;
 
-  return true;
+TestContainerizer::TestContainerizer(
+    const ExecutorID& executorId,
+    Executor* executor)
+  : process(new TestContainerizerProcess({{executorId, executor}}))
+{
+  spawn(process.get());
+  setup();
 }
 
 
-Future<hashset<ContainerID>> TestContainerizer::containers()
+TestContainerizer::TestContainerizer(MockExecutor* executor)
+  : process(new TestContainerizerProcess({{executor->id, executor}}))
+{
+  spawn(process.get());
+  setup();
+}
+
+
+TestContainerizer::TestContainerizer()
+  : process(new TestContainerizerProcess())
+{
+  spawn(process.get());
+  setup();
+}
+
+
+TestContainerizer::~TestContainerizer()
 {
-  return containers_.keys();
+  terminate(process.get());
+  process::wait(process.get());
 }
 
 
@@ -324,16 +409,16 @@ void TestContainerizer::setup()
   // place.
 
   EXPECT_CALL(*this, recover(_))
-    .WillRepeatedly(Return(Nothing()));
+    .WillRepeatedly(Invoke(this, &TestContainerizer::_recover));
 
   EXPECT_CALL(*this, usage(_))
-    .WillRepeatedly(Return(ResourceStatistics()));
+    .WillRepeatedly(Invoke(this, &TestContainerizer::_usage));
 
   EXPECT_CALL(*this, status(_))
-    .WillRepeatedly(Return(ContainerStatus()));
+    .WillRepeatedly(Invoke(this, &TestContainerizer::_status));
 
   EXPECT_CALL(*this, update(_, _))
-    .WillRepeatedly(Return(Nothing()));
+    .WillRepeatedly(Invoke(this, &TestContainerizer::_update));
 
   Future<bool> (TestContainerizer::*_launch)(
       const ContainerID& containerId,
@@ -367,6 +452,158 @@ void TestContainerizer::setup()
     .WillRepeatedly(Invoke(this, &TestContainerizer::_destroy));
 }
 
+
+Future<Nothing> TestContainerizer::_recover(
+    const Option<slave::state::SlaveState>& state)
+{
+  return dispatch(
+    process.get(),
+    &TestContainerizerProcess::recover,
+    state);
+}
+
+
+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)
+{
+  // Need to disambiguate for the compiler.
+  Future<bool> (TestContainerizerProcess::*launch)(
+      const ContainerID&,
+      const Option<TaskInfo>&,
+      const ExecutorInfo&,
+      const string&,
+      const Option<string>&,
+      const SlaveID&,
+      const map<string, string>&,
+      bool) = &TestContainerizerProcess::launch;
+
+  return dispatch(
+      process.get(),
+      launch,
+      containerId,
+      taskInfo,
+      executorInfo,
+      directory,
+      user,
+      slaveId,
+      environment,
+      checkpoint);
+}
+
+
+Future<bool> TestContainerizer::_launch(
+    const ContainerID& containerId,
+    const CommandInfo& commandInfo,
+    const Option<ContainerInfo>& containerInfo,
+    const Option<string>& user,
+    const SlaveID& slaveId)
+{
+  // Need to disambiguate for the compiler.
+  Future<bool> (TestContainerizerProcess::*launch)(
+      const ContainerID&,
+      const CommandInfo&,
+      const Option<ContainerInfo>&,
+      const Option<string>&,
+      const SlaveID&) = &TestContainerizerProcess::launch;
+
+  return dispatch(
+      process.get(),
+      launch,
+      containerId,
+      commandInfo,
+      containerInfo,
+      user,
+      slaveId);
+}
+
+
+Future<Nothing> TestContainerizer::_update(
+    const ContainerID& containerId,
+    const Resources& resources)
+{
+  return dispatch(
+      process.get(),
+      &TestContainerizerProcess::update,
+      containerId,
+      resources);
+}
+
+
+Future<ResourceStatistics> TestContainerizer::_usage(
+    const ContainerID& containerId)
+{
+  return dispatch(
+    process.get(),
+    &TestContainerizerProcess::usage,
+    containerId);
+}
+
+
+Future<ContainerStatus> TestContainerizer::_status(
+    const ContainerID& containerId)
+{
+  return dispatch(
+      process.get(),
+      &TestContainerizerProcess::status,
+      containerId);
+}
+
+
+Future<Option<mesos::slave::ContainerTermination>> TestContainerizer::_wait(
+    const ContainerID& containerId)
+{
+  return dispatch(
+      process.get(),
+      &TestContainerizerProcess::wait,
+      containerId);
+}
+
+
+Future<bool> TestContainerizer::_destroy(
+    const ContainerID& containerId)
+{
+  // Need to disambiguate for the compiler.
+  Future<bool> (TestContainerizerProcess::*destroy)(
+      const ContainerID&) = &TestContainerizerProcess::destroy;
+
+  return dispatch(
+      process.get(),
+      destroy,
+      containerId);
+}
+
+
+Future<bool> TestContainerizer::destroy(
+    const FrameworkID& frameworkId,
+    const ExecutorID& executorId)
+{
+  // Need to disambiguate for the compiler.
+  Future<bool> (TestContainerizerProcess::*destroy)(
+      const FrameworkID&,
+      const ExecutorID&) = &TestContainerizerProcess::destroy;
+
+  return dispatch(
+      process.get(),
+      destroy,
+      frameworkId,
+      executorId);
+}
+
+
+Future<hashset<ContainerID>> TestContainerizer::containers()
+{
+  return dispatch(
+      process.get(),
+      &TestContainerizerProcess::containers);
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/250cfcb7/src/tests/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.hpp b/src/tests/containerizer.hpp
index d418ece..080b8fa 100644
--- a/src/tests/containerizer.hpp
+++ b/src/tests/containerizer.hpp
@@ -53,6 +53,8 @@ namespace tests {
 
 // Forward declaration.
 class MockExecutor;
+class TestContainerizerProcess;
+
 
 class TestContainerizer : public slave::Containerizer
 {
@@ -130,7 +132,12 @@ public:
 private:
   void setup();
 
-  // Default implementations of mock methods.
+  // The following functions act as a level of indirection to
+  // perform the dispatch while still allowing the above to be
+  // mock functions.
+
+  process::Future<Nothing> _recover(
+      const Option<slave::state::SlaveState>& state);
 
   process::Future<bool> _launch(
       const ContainerID& containerId,
@@ -149,40 +156,23 @@ private:
       const Option<std::string>& user,
       const SlaveID& slaveId);
 
+  process::Future<Nothing> _update(
+      const ContainerID& containerId,
+      const Resources& resources);
+
+  process::Future<ResourceStatistics> _usage(
+      const ContainerID& containerId);
+
+  process::Future<ContainerStatus> _status(
+      const ContainerID& containerId);
+
   process::Future<Option<mesos::slave::ContainerTermination>> _wait(
-      const ContainerID& containerId) const;
-
-  process::Future<bool> _destroy(const ContainerID& containerID);
-
-  struct ContainerData
-  {
-    Option<ExecutorID> executorId;
-    Option<FrameworkID> frameworkId;
-
-    process::Promise<mesos::slave::ContainerTermination> termination;
-  };
-
-  // We also store the terminated containers to allow callers to
-  // "reap" the termination if a container is already destroyed.
-  // This mimics the behavior of the mesos containerizer.
-  hashmap<ContainerID, process::Owned<ContainerData>> containers_;
-  hashmap<ContainerID, mesos::slave::ContainerTermination> terminatedContainers;
-
-  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<v1::MockHTTPExecutor> v1ExecutorMock;
-    process::Owned<v1::executor::TestMesos> v1Library;
-  };
-
-  // TODO(bmahler): The test containerizer currently assumes that
-  // executor IDs are unique across frameworks (see the constructors).
-  hashmap<ExecutorID, process::Owned<ExecutorData>> executors;
+      const ContainerID& containerId);
+
+  process::Future<bool> _destroy(
+      const ContainerID& containerId);
+
+  process::Owned<TestContainerizerProcess> process;
 };
 
 } // namespace tests {


[2/2] mesos git commit: Avoid 'using namespace' clause in the test containerizer.

Posted by bm...@apache.org.
Avoid 'using namespace' clause in the test containerizer.

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


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

Branch: refs/heads/master
Commit: 8f976a8ef737f20b1c3c482f5cce81d4e70521ac
Parents: 250cfcb
Author: Benjamin Mahler <bm...@apache.org>
Authored: Thu Nov 3 23:08:06 2016 -0700
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Fri Nov 4 13:28:13 2016 -0700

----------------------------------------------------------------------
 src/tests/containerizer.cpp | 35 ++++++++++++++++++-----------------
 1 file changed, 18 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8f976a8e/src/tests/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.cpp b/src/tests/containerizer.cpp
index 32bdb87..c3bcb85 100644
--- a/src/tests/containerizer.cpp
+++ b/src/tests/containerizer.cpp
@@ -22,7 +22,8 @@
 
 #include "tests/mesos.hpp"
 
-using namespace process;
+using process::Future;
+using process::Owned;
 
 using std::map;
 using std::shared_ptr;
@@ -348,7 +349,7 @@ TestContainerizer::TestContainerizer(
     const shared_ptr<v1::MockHTTPExecutor>& executor)
   : process(new TestContainerizerProcess(executorId, executor))
 {
-  spawn(process.get());
+  process::spawn(process.get());
   setup();
 }
 
@@ -357,7 +358,7 @@ TestContainerizer::TestContainerizer(
     const hashmap<ExecutorID, Executor*>& executors)
   : process(new TestContainerizerProcess(executors))
 {
-  spawn(process.get());
+  process::spawn(process.get());
   setup();
 }
 
@@ -367,7 +368,7 @@ TestContainerizer::TestContainerizer(
     Executor* executor)
   : process(new TestContainerizerProcess({{executorId, executor}}))
 {
-  spawn(process.get());
+  process::spawn(process.get());
   setup();
 }
 
@@ -375,7 +376,7 @@ TestContainerizer::TestContainerizer(
 TestContainerizer::TestContainerizer(MockExecutor* executor)
   : process(new TestContainerizerProcess({{executor->id, executor}}))
 {
-  spawn(process.get());
+  process::spawn(process.get());
   setup();
 }
 
@@ -383,14 +384,14 @@ TestContainerizer::TestContainerizer(MockExecutor* executor)
 TestContainerizer::TestContainerizer()
   : process(new TestContainerizerProcess())
 {
-  spawn(process.get());
+  process::spawn(process.get());
   setup();
 }
 
 
 TestContainerizer::~TestContainerizer()
 {
-  terminate(process.get());
+  process::terminate(process.get());
   process::wait(process.get());
 }
 
@@ -456,7 +457,7 @@ void TestContainerizer::setup()
 Future<Nothing> TestContainerizer::_recover(
     const Option<slave::state::SlaveState>& state)
 {
-  return dispatch(
+  return process::dispatch(
     process.get(),
     &TestContainerizerProcess::recover,
     state);
@@ -484,7 +485,7 @@ Future<bool> TestContainerizer::_launch(
       const map<string, string>&,
       bool) = &TestContainerizerProcess::launch;
 
-  return dispatch(
+  return process::dispatch(
       process.get(),
       launch,
       containerId,
@@ -513,7 +514,7 @@ Future<bool> TestContainerizer::_launch(
       const Option<string>&,
       const SlaveID&) = &TestContainerizerProcess::launch;
 
-  return dispatch(
+  return process::dispatch(
       process.get(),
       launch,
       containerId,
@@ -528,7 +529,7 @@ Future<Nothing> TestContainerizer::_update(
     const ContainerID& containerId,
     const Resources& resources)
 {
-  return dispatch(
+  return process::dispatch(
       process.get(),
       &TestContainerizerProcess::update,
       containerId,
@@ -539,7 +540,7 @@ Future<Nothing> TestContainerizer::_update(
 Future<ResourceStatistics> TestContainerizer::_usage(
     const ContainerID& containerId)
 {
-  return dispatch(
+  return process::dispatch(
     process.get(),
     &TestContainerizerProcess::usage,
     containerId);
@@ -549,7 +550,7 @@ Future<ResourceStatistics> TestContainerizer::_usage(
 Future<ContainerStatus> TestContainerizer::_status(
     const ContainerID& containerId)
 {
-  return dispatch(
+  return process::dispatch(
       process.get(),
       &TestContainerizerProcess::status,
       containerId);
@@ -559,7 +560,7 @@ Future<ContainerStatus> TestContainerizer::_status(
 Future<Option<mesos::slave::ContainerTermination>> TestContainerizer::_wait(
     const ContainerID& containerId)
 {
-  return dispatch(
+  return process::dispatch(
       process.get(),
       &TestContainerizerProcess::wait,
       containerId);
@@ -573,7 +574,7 @@ Future<bool> TestContainerizer::_destroy(
   Future<bool> (TestContainerizerProcess::*destroy)(
       const ContainerID&) = &TestContainerizerProcess::destroy;
 
-  return dispatch(
+  return process::dispatch(
       process.get(),
       destroy,
       containerId);
@@ -589,7 +590,7 @@ Future<bool> TestContainerizer::destroy(
       const FrameworkID&,
       const ExecutorID&) = &TestContainerizerProcess::destroy;
 
-  return dispatch(
+  return process::dispatch(
       process.get(),
       destroy,
       frameworkId,
@@ -599,7 +600,7 @@ Future<bool> TestContainerizer::destroy(
 
 Future<hashset<ContainerID>> TestContainerizer::containers()
 {
-  return dispatch(
+  return process::dispatch(
       process.get(),
       &TestContainerizerProcess::containers);
 }