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/05/26 01:40:52 UTC

[01/16] mesos git commit: Combined Mesos containerizer's launch methods.

Repository: mesos
Updated Branches:
  refs/heads/master 51da8c4bf -> 74c61ceba


Combined Mesos containerizer's launch methods.

This simplifies the container launch path by removing combining
the nested and non-nested container code paths into one.

The Mesos containerizer was originally translating the two
`containerizer->launch` entrypoints into a common method (also
called `launch`).  The previous commits moved this translation
logic into the caller (i.e. the Agent).

The end result has some slight changes:
  * It is now possible for the Agent to specify some more combinations
    of `ContainerConfig`.  For example, specifying a TaskInfo with
    a DEBUG-class container.  Or a nested container with Resources.
    We may need to add extra validation around this.
  * The `bool checkpoint` argument was replaced with a `Option<string>`
    which optionally contains an absolute path.  This allows us to
    remove the `SlaveID` field.

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


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

Branch: refs/heads/master
Commit: 17ffb97ae7eda78edf85b204eba35bc59649a479
Parents: af21bb7
Author: Joseph Wu <jo...@apache.org>
Authored: Mon Apr 10 17:20:46 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:06 2017 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 354 ++++++-------------
 src/slave/containerizer/mesos/containerizer.hpp |  45 +--
 2 files changed, 111 insertions(+), 288 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/17ffb97a/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 199202a..f3e6210 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -493,63 +493,16 @@ Future<Nothing> MesosContainerizer::recover(
 
 Future<bool> MesosContainerizer::launch(
     const ContainerID& containerId,
-    const Option<TaskInfo>& taskInfo,
-    const ExecutorInfo& executorInfo,
-    const string& directory,
-    const Option<string>& user,
-    const SlaveID& slaveId,
+    const ContainerConfig& containerConfig,
     const map<string, string>& environment,
-    bool checkpoint)
+    const Option<std::string>& pidCheckpointPath)
 {
-  // Need to disambiguate for the compiler.
-  Future<bool> (MesosContainerizerProcess::*launch)(
-      const ContainerID&,
-      const Option<TaskInfo>&,
-      const ExecutorInfo&,
-      const string&,
-      const Option<string>&,
-      const SlaveID&,
-      const map<string, string>&,
-      bool) = &MesosContainerizerProcess::launch;
-
   return dispatch(process.get(),
-                  launch,
+                  &MesosContainerizerProcess::launch,
                   containerId,
-                  taskInfo,
-                  executorInfo,
-                  directory,
-                  user,
-                  slaveId,
+                  containerConfig,
                   environment,
-                  checkpoint);
-}
-
-
-Future<bool> MesosContainerizer::launch(
-    const ContainerID& containerId,
-    const CommandInfo& commandInfo,
-    const Option<ContainerInfo>& containerInfo,
-    const Option<string>& user,
-    const SlaveID& slaveId,
-    const Option<ContainerClass>& containerClass)
-{
-  // Need to disambiguate for the compiler.
-  Future<bool> (MesosContainerizerProcess::*launch)(
-      const ContainerID&,
-      const CommandInfo&,
-      const Option<ContainerInfo>&,
-      const Option<string>&,
-      const SlaveID&,
-      const Option<ContainerClass>&) = &MesosContainerizerProcess::launch;
-
-  return dispatch(process.get(),
-                  launch,
-                  containerId,
-                  commandInfo,
-                  containerInfo,
-                  user,
-                  slaveId,
-                  containerClass);
+                  pidCheckpointPath);
 }
 
 
@@ -991,99 +944,117 @@ Future<Nothing> MesosContainerizerProcess::__recover(
 }
 
 
-// Launching an executor involves the following steps:
+// Launching an container involves the following steps:
 // 1. Call prepare on each isolator.
-// 2. Fork the executor. The forked child is blocked from exec'ing until it has
-//    been isolated.
-// 3. Isolate the executor. Call isolate with the pid for each isolator.
-// 4. Fetch the executor.
-// 5. Exec the executor. The forked child is signalled to continue. It will
-//    first execute any preparation commands from isolators and then exec the
-//    executor.
+// 2. Fork a helper process. The forked helper is blocked from exec'ing
+//    until it has been isolated.
+// 3. Isolate the helper's pid; e.g. call `isolate` for each isolator.
+// 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(
     const ContainerID& containerId,
-    const Option<TaskInfo>& taskInfo,
-    const ExecutorInfo& _executorInfo,
-    const string& directory,
-    const Option<string>& user,
-    const SlaveID& slaveId,
+    const ContainerConfig& _containerConfig,
     const map<string, string>& environment,
-    bool checkpoint)
+    const Option<std::string>& pidCheckpointPath)
 {
-  CHECK(!containerId.has_parent());
-
   if (containers_.contains(containerId)) {
-    return Failure("Container already started");
+    return Failure(
+        (containerId.has_parent() ? "Nested container" : "Container") +
+        stringify(containerId) + " already started");
   }
 
-  if (taskInfo.isSome() &&
-      taskInfo.get().has_container() &&
-      taskInfo.get().container().type() != ContainerInfo::MESOS) {
+  if (_containerConfig.has_container_info() &&
+      _containerConfig.container_info().type() != ContainerInfo::MESOS) {
     return false;
   }
 
-  // NOTE: We make a copy of the executor info because we may mutate
-  // it with default container info.
-  ExecutorInfo executorInfo = _executorInfo;
+  // NOTE: We make a copy of the ContainerConfig because we may need
+  // to modify it based on the parent container (for nested containers).
+  ContainerConfig containerConfig = _containerConfig;
 
-  if (executorInfo.has_container() &&
-      executorInfo.container().type() != ContainerInfo::MESOS) {
-    return false;
-  }
+  // For nested containers, we must perform some extra validation
+  // (i.e. does the parent exist?) and create the sandbox directory
+  // based on the parent's sandbox.
+  if (containerId.has_parent()) {
+    if (containerConfig.has_task_info() ||
+        containerConfig.has_executor_info()) {
+      return Failure(
+          "Nested containers may not supply a TaskInfo/ExecutorInfo");
+    }
 
-  // Add the default container info to the executor info.
-  // TODO(jieyu): Rename the flag to be default_mesos_container_info.
-  if (!executorInfo.has_container() &&
-      flags.default_container_info.isSome()) {
-    executorInfo.mutable_container()->CopyFrom(
-        flags.default_container_info.get());
-  }
+    if (pidCheckpointPath.isSome()) {
+      return Failure("Nested containers may not be checkpointed");
+    }
 
-  LOG(INFO) << "Starting container " << containerId
-            << " for executor '" << executorInfo.executor_id()
-            << "' of framework " << executorInfo.framework_id();
+    const ContainerID& parentContainerId = containerId.parent();
 
-  ContainerConfig containerConfig;
-  containerConfig.mutable_executor_info()->CopyFrom(executorInfo);
-  containerConfig.mutable_command_info()->CopyFrom(executorInfo.command());
-  containerConfig.mutable_resources()->CopyFrom(executorInfo.resources());
-  containerConfig.set_directory(directory);
+    if (!containers_.contains(parentContainerId)) {
+      return Failure(
+          "Parent container " + stringify(parentContainerId) +
+          " does not exist");
+    }
 
-  if (user.isSome()) {
-    containerConfig.set_user(user.get());
-  }
+    if (containers_[parentContainerId]->state == DESTROYING) {
+      return Failure(
+          "Parent container " + stringify(parentContainerId) +
+          " is in 'DESTROYING' state");
+    }
 
-  if (taskInfo.isSome()) {
-    // Command task case.
-    containerConfig.mutable_task_info()->CopyFrom(taskInfo.get());
+    const ContainerID rootContainerId =
+      protobuf::getRootContainerId(containerId);
 
-    if (taskInfo->has_container()) {
-      ContainerInfo* containerInfo = containerConfig.mutable_container_info();
-      containerInfo->CopyFrom(taskInfo->container());
+    CHECK(containers_.contains(rootContainerId));
+    if (containers_[rootContainerId]->directory.isNone()) {
+      return Failure(
+          "Unexpected empty sandbox directory for root container " +
+          stringify(rootContainerId));
     }
-  } else {
-    // Other cases.
-    if (executorInfo.has_container()) {
-      ContainerInfo* containerInfo = containerConfig.mutable_container_info();
-      containerInfo->CopyFrom(executorInfo.container());
+
+    const string directory = containerizer::paths::getSandboxPath(
+        containers_[rootContainerId]->directory.get(),
+        containerId);
+
+    Try<Nothing> mkdir = os::mkdir(directory);
+    if (mkdir.isError()) {
+      return Failure(
+          "Failed to create nested sandbox directory '" +
+          directory + "': " + mkdir.error());
     }
-  }
 
-  return launch(containerId,
-                containerConfig,
-                environment,
-                slaveId,
-                checkpoint);
-}
+#ifndef __WINDOWS__
+    if (containerConfig.has_user()) {
+      LOG(INFO) << "Trying to chown '" << directory << "' to user '"
+                << containerConfig.user() << "'";
+
+      Try<Nothing> chown = os::chown(containerConfig.user(), directory);
+      if (chown.isError()) {
+        LOG(WARNING)
+          << "Failed to chown sandbox directory '" << directory
+          << "'. This may be due to attempting to run the container "
+          << "as a nonexistent user on the agent; see the description"
+          << " for the `--switch_user` flag for more information: "
+          << chown.error();
+      }
+    }
+#endif // __WINDOWS__
 
+    // Modify the sandbox directory in the ContainerConfig.
+    // TODO(josephw): Should we validate that this value
+    // is not set for nested containers?
+    containerConfig.set_directory(directory);
+
+    // TODO(jieyu): This is currently best effort. After the agent fails
+    // over, 'executor_info' won't be set in root parent container's
+    // 'config'. Consider populating 'executor_info' in recover path.
+    if (containers_[rootContainerId]->config.has_executor_info()) {
+      containerConfig.mutable_executor_info()->CopyFrom(
+          containers_[rootContainerId]->config.executor_info());
+    }
+  }
+
+  LOG(INFO) << "Starting container " << containerId;
 
-Future<bool> MesosContainerizerProcess::launch(
-    const ContainerID& containerId,
-    const ContainerConfig& containerConfig,
-    const map<string, string>& environment,
-    const SlaveID& slaveId,
-    bool checkpoint)
-{
   // Before we launch the container, we first create the container
   // runtime directory to hold internal checkpoint information about
   // the container.
@@ -1146,8 +1117,7 @@ Future<bool> MesosContainerizerProcess::launch(
                   containerId,
                   lambda::_1,
                   environment,
-                  slaveId,
-                  checkpoint));
+                  pidCheckpointPath));
   }
 
   container->provisioning = provisioner->provision(
@@ -1166,8 +1136,7 @@ Future<bool> MesosContainerizerProcess::launch(
                     containerId,
                     lambda::_1,
                     environment,
-                    slaveId,
-                    checkpoint));
+                    pidCheckpointPath));
     }));
 }
 
@@ -1251,8 +1220,7 @@ Future<Nothing> MesosContainerizerProcess::prepare(
 
 
 Future<Nothing> MesosContainerizerProcess::fetch(
-    const ContainerID& containerId,
-    const SlaveID& slaveId)
+    const ContainerID& containerId)
 {
   if (!containers_.contains(containerId)) {
     return Failure("Container destroyed during isolating");
@@ -1270,18 +1238,13 @@ Future<Nothing> MesosContainerizerProcess::fetch(
 
   const string directory = container->config.directory();
 
-  Option<string> user;
-  if (container->config.has_user()) {
-    user = container->config.user();
-  }
-
   return fetcher->fetch(
       containerId,
       container->config.command_info(),
       directory,
-      user,
-      slaveId,
-      flags)
+      container->config.has_user()
+        ? container->config.user()
+        : Option<string>::none())
     .then([=]() -> Future<Nothing> {
       if (HookManager::hooksAvailable()) {
         HookManager::slavePostFetchHook(containerId, directory);
@@ -1295,8 +1258,7 @@ Future<bool> MesosContainerizerProcess::_launch(
     const ContainerID& containerId,
     const Option<ContainerIO>& containerIO,
     const map<string, string>& environment,
-    const SlaveID& slaveId,
-    bool checkpoint)
+    const Option<std::string>& pidCheckpointPath)
 {
   if (!containers_.contains(containerId)) {
     return Failure("Container destroyed during preparing");
@@ -1756,23 +1718,16 @@ Future<bool> MesosContainerizerProcess::_launch(
   container->pid = pid;
 
   // Checkpoint the forked pid if requested by the agent.
-  if (checkpoint) {
-    const string& path = slave::paths::getForkedPidPath(
-        slave::paths::getMetaRootDir(flags.work_dir),
-        slaveId,
-        container->config.executor_info().framework_id(),
-        container->config.executor_info().executor_id(),
-        containerId);
-
+  if (pidCheckpointPath.isSome()) {
     LOG(INFO) << "Checkpointing container's forked pid " << pid
-              << " to '" << path << "'";
+              << " to '" << pidCheckpointPath.get() << "'";
 
     Try<Nothing> checkpointed =
-      slave::state::checkpoint(path, stringify(pid));
+      slave::state::checkpoint(pidCheckpointPath.get(), stringify(pid));
 
     if (checkpointed.isError()) {
       LOG(ERROR) << "Failed to checkpoint container's forked pid to '"
-                 << path << "': " << checkpointed.error();
+                 << pidCheckpointPath.get() << "': " << checkpointed.error();
 
       return Failure("Could not checkpoint container's pid");
     }
@@ -1809,8 +1764,7 @@ Future<bool> MesosContainerizerProcess::_launch(
   return isolate(containerId, pid)
     .then(defer(self(),
                 &Self::fetch,
-                containerId,
-                slaveId))
+                containerId))
     .then(defer(self(), &Self::exec, containerId, pipes[1]))
     .onAny([pipes]() { os::close(pipes[0]); })
     .onAny([pipes]() { os::close(pipes[1]); });
@@ -1903,104 +1857,6 @@ Future<bool> MesosContainerizerProcess::exec(
 }
 
 
-Future<bool> MesosContainerizerProcess::launch(
-    const ContainerID& containerId,
-    const CommandInfo& commandInfo,
-    const Option<ContainerInfo>& containerInfo,
-    const Option<string>& user,
-    const SlaveID& slaveId,
-    const Option<ContainerClass>& containerClass)
-{
-  CHECK(containerId.has_parent());
-
-  if (containers_.contains(containerId)) {
-    return Failure(
-        "Nested container " + stringify(containerId) + " already started");
-  }
-
-  const ContainerID& parentContainerId = containerId.parent();
-  if (!containers_.contains(parentContainerId)) {
-    return Failure(
-        "Parent container " + stringify(parentContainerId) +
-        " does not exist");
-  }
-
-  if (containers_[parentContainerId]->state == DESTROYING) {
-    return Failure(
-        "Parent container " + stringify(parentContainerId) +
-        " is in 'DESTROYING' state");
-  }
-
-  LOG(INFO) << "Starting nested container " << containerId;
-
-  const ContainerID rootContainerId = protobuf::getRootContainerId(containerId);
-
-  CHECK(containers_.contains(rootContainerId));
-  if (containers_[rootContainerId]->directory.isNone()) {
-    return Failure(
-        "Unexpected empty sandbox directory for root container " +
-        stringify(rootContainerId));
-  }
-
-  const string directory = containerizer::paths::getSandboxPath(
-      containers_[rootContainerId]->directory.get(),
-      containerId);
-
-  Try<Nothing> mkdir = os::mkdir(directory);
-  if (mkdir.isError()) {
-    return Failure(
-        "Failed to create nested sandbox directory '" +
-        directory + "': " + mkdir.error());
-  }
-
-#ifndef __WINDOWS__
-  if (user.isSome()) {
-    LOG(INFO) << "Trying to chown '" << directory << "' to user '"
-              << user.get() << "'";
-
-    Try<Nothing> chown = os::chown(user.get(), directory);
-    if (chown.isError()) {
-      LOG(WARNING) << "Failed to chown sandbox directory '" << directory
-                   << "'. This may be due to attempting to run the container "
-                   << "as a nonexistent user on the agent; see the description"
-                   << " for the `--switch_user` flag for more information: "
-                   << chown.error();
-    }
-  }
-#endif // __WINDOWS__
-
-  ContainerConfig containerConfig;
-  containerConfig.mutable_command_info()->CopyFrom(commandInfo);
-  containerConfig.set_directory(directory);
-
-  if (user.isSome()) {
-    containerConfig.set_user(user.get());
-  }
-
-  if (containerInfo.isSome()) {
-    containerConfig.mutable_container_info()->CopyFrom(containerInfo.get());
-  }
-
-  if (containerClass.isSome()) {
-    containerConfig.set_container_class(containerClass.get());
-  }
-
-  // TODO(jieyu): This is currently best effort. After the agent fails
-  // over, 'executor_info' won't be set in root parent container's
-  // 'config'. Consider populating 'executor_info' in recover path.
-  if (containers_[rootContainerId]->config.has_executor_info()) {
-    containerConfig.mutable_executor_info()->CopyFrom(
-        containers_[rootContainerId]->config.executor_info());
-  }
-
-  return launch(containerId,
-                containerConfig,
-                map<string, string>(),
-                slaveId,
-                false);
-}
-
-
 Future<Connection> MesosContainerizerProcess::attach(
     const ContainerID& containerId)
 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ffb97a/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index d767031..ea06919 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -80,21 +80,9 @@ public:
 
   virtual process::Future<bool> launch(
       const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
+      const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,
-      bool checkpoint);
-
-  virtual process::Future<bool> launch(
-      const ContainerID& containerId,
-      const CommandInfo& commandInfo,
-      const Option<ContainerInfo>& containerInfo,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
-      const Option<mesos::slave::ContainerClass>& containerClass = None());
+      const Option<std::string>& pidCheckpointPath);
 
   virtual process::Future<process::http::Connection> attach(
       const ContainerID& containerId);
@@ -153,21 +141,9 @@ public:
 
   virtual process::Future<bool> launch(
       const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
+      const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,
-      bool checkpoint);
-
-  virtual process::Future<bool> launch(
-      const ContainerID& containerId,
-      const CommandInfo& commandInfo,
-      const Option<ContainerInfo>& containerInfo,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
-      const Option<mesos::slave::ContainerClass>& containerClass);
+      const Option<std::string>& pidCheckpointPath);
 
   virtual process::Future<process::http::Connection> attach(
       const ContainerID& containerId);
@@ -230,22 +206,13 @@ private:
       const Option<ProvisionInfo>& provisionInfo);
 
   process::Future<Nothing> fetch(
-      const ContainerID& containerId,
-      const SlaveID& slaveId);
-
-  process::Future<bool> launch(
-      const ContainerID& containerId,
-      const mesos::slave::ContainerConfig& containerConfig,
-      const std::map<std::string, std::string>& environment,
-      const SlaveID& slaveId,
-      bool checkpoint);
+      const ContainerID& containerId);
 
   process::Future<bool> _launch(
       const ContainerID& containerId,
       const Option<mesos::slave::ContainerIO>& containerIO,
       const std::map<std::string, std::string>& environment,
-      const SlaveID& slaveId,
-      bool checkpoint);
+      const Option<std::string>& pidCheckpointPath);
 
   process::Future<bool> isolate(
       const ContainerID& containerId,


[04/16] mesos git commit: Updated mesos-local work directory structure.

Posted by jo...@apache.org.
Updated mesos-local work directory structure.

This combines the `--work_dir` and `--runtime_dir` flags for the
mesos-local helper (**not** the Agent flags, which have the same name).
Only the `--work_dir` will remain.

The new directory structure will be the following:

  work_dir (mesos-local flag)
  |-- agents
  |   |-- 0
  |   |   |-- fetch (Agent --fetcher_cache_dir flag)
  |   |   |-- run   (Agent --runtime_dir flag)
  |   |   |-- work  (Agent --work_dir flag)
  |   |-- 1
  |   |   ...
  |-- master

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


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

Branch: refs/heads/master
Commit: 7a8f864046ef1f668427d431aeb571cd723745c4
Parents: 145896b
Author: Joseph Wu <jo...@apache.org>
Authored: Mon May 15 11:20:43 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:06 2017 -0700

----------------------------------------------------------------------
 src/local/flags.hpp | 43 +++++++++++++++++++------------------------
 src/local/local.cpp | 10 ++++------
 2 files changed, 23 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7a8f8640/src/local/flags.hpp
----------------------------------------------------------------------
diff --git a/src/local/flags.hpp b/src/local/flags.hpp
index 0cef5ac..37eba7b 100644
--- a/src/local/flags.hpp
+++ b/src/local/flags.hpp
@@ -32,34 +32,30 @@ class Flags : public virtual logging::Flags
 public:
   Flags()
   {
-    // `work_dir` is passed from here to the agents/master.
-    // This is necessary because `work_dir` is a required flag
-    // in agents/master and without this, the load call for their
-    // flags will spit out an error unless they have an env
-    // variable for the `work_dir` explicitly set.
-    // Since local mode is used strictly for non-production
-    // purposes, it is the one case where we deem it acceptable
-    // to set a default value for `work_dir`.
     add(&Flags::work_dir,
         "work_dir",
         "Path of the master/agent work directory. This is where the\n"
-        "persistent information of the cluster will be stored.\n"
-        "Note that locations like `/tmp` which are cleaned\n"
-        "automatically are not suitable for the work directory\n"
-        "when running in production, since long-running masters\n"
-        "and agents could lose data when cleanup occurs.\n"
-        "(Example: `/var/lib/mesos`)",
+        "persistent information of the cluster will be stored.\n\n"
+        "NOTE: Locations like `/tmp` which are cleaned automatically\n"
+        "are not suitable for the work directory when running in\n"
+        "production, since long-running masters and agents could lose\n"
+        "data when cleanup occurs. Local mode is used explicitly for\n"
+        "non-production purposes, so this is the only case where having\n"
+        "a default `work_dir` flag is acceptable.\n"
+        "(Example: `/var/lib/mesos`)\n\n"
+        "Individual work directories for each master and agent will be\n"
+        "nested underneath the given work directory:\n"
+        "root (`work_dir` flag)\n"
+        "|-- agents\n"
+        "|   |-- 0\n"
+        "|   |   |-- fetch (--fetcher_cache_dir)\n"
+        "|   |   |-- run   (--runtime_dir)\n"
+        "|   |   |-- work  (--work_dir)\n"
+        "|   |-- 1\n"
+        "|   |   ...\n"
+        "|-- master",
         path::join(os::temp(), "mesos", "work"));
 
-    add(&Flags::runtime_dir,
-        "runtime_dir",
-        "Path of the agent runtime directory. This is where runtime\n"
-        "data is stored by an agent that it needs to persist across\n"
-        "crashes (but not across reboots). This directory will be\n"
-        "cleared on reboot.\n"
-        "(Example: `/var/run/mesos`)",
-        path::join(os::temp(), "mesos", "runtime"));
-
     add(&Flags::num_slaves,
         "num_slaves",
         "Number of agents to launch for local cluster",
@@ -67,7 +63,6 @@ public:
   }
 
   std::string work_dir;
-  std::string runtime_dir;
   int num_slaves;
 };
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/7a8f8640/src/local/local.cpp
----------------------------------------------------------------------
diff --git a/src/local/local.cpp b/src/local/local.cpp
index ad35723..99c1b08 100644
--- a/src/local/local.cpp
+++ b/src/local/local.cpp
@@ -364,17 +364,15 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
     // environment variables explicitly set.
     map<string, string> propagatedFlags;
 
-    // Use a different work directory for each agent.
+    // Use a different work/runtime/fetcher-cache directory for each agent.
     propagatedFlags["work_dir"] =
-      path::join(flags.work_dir, "agents", stringify(i));
+      path::join(flags.work_dir, "agents", stringify(i), "work");
 
-    // Use a different runtime directory for each agent.
     propagatedFlags["runtime_dir"] =
-      path::join(flags.runtime_dir, "agents", stringify(i));
+      path::join(flags.work_dir, "agents", stringify(i), "run");
 
-    // Use a different fetcher cache directory for each agent.
     propagatedFlags["fetcher_cache_dir"] =
-      path::join(os::temp(), "mesos", "fetch", "agents", stringify(i));
+      path::join(flags.work_dir, "agents", stringify(i), "fetch");
 
     slave::Flags slaveFlags;
     Try<flags::Warnings> load = slaveFlags.load(


[14/16] mesos git commit: Updated Docker containerizer tests to use the old naming scheme.

Posted by jo...@apache.org.
Updated Docker containerizer tests to use the old naming scheme.

This changes test expectations of Docker container names back to the
0.22 scheme and simultaneously updates the tests to the new
containerizer interface.

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


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

Branch: refs/heads/master
Commit: c66f7e2a0838b765af330e2a31225cd59dacc0fb
Parents: b194d5a
Author: Joseph Wu <jo...@apache.org>
Authored: Thu May 4 12:45:01 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:07 2017 -0700

----------------------------------------------------------------------
 .../docker_containerizer_tests.cpp              | 317 +++++++++----------
 1 file changed, 153 insertions(+), 164 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c66f7e2a/src/tests/containerizer/docker_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/docker_containerizer_tests.cpp b/src/tests/containerizer/docker_containerizer_tests.cpp
index 4eef399..84e062a 100644
--- a/src/tests/containerizer/docker_containerizer_tests.cpp
+++ b/src/tests/containerizer/docker_containerizer_tests.cpp
@@ -62,6 +62,7 @@ using mesos::internal::slave::Slave;
 
 using mesos::master::detector::MasterDetector;
 
+using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLogger;
 using mesos::slave::ContainerTermination;
 
@@ -84,12 +85,9 @@ namespace tests {
 class DockerContainerizerTest : public MesosTest
 {
 public:
-  static string containerName(
-      const SlaveID& slaveId,
-      const ContainerID& containerId)
+  static string containerName(const ContainerID& containerId)
   {
-    return slave::DOCKER_NAME_PREFIX + slaveId.value() +
-      slave::DOCKER_NAME_SEPERATOR + containerId.value();
+    return slave::DOCKER_NAME_PREFIX + containerId.value();
   }
 
   enum ContainerState
@@ -100,12 +98,11 @@ public:
 
   static bool exists(
       const process::Shared<Docker>& docker,
-      const SlaveID& slaveId,
       const ContainerID& containerId,
       ContainerState state = ContainerState::EXISTS)
   {
     Duration waited = Duration::zero();
-    string expectedName = containerName(slaveId, containerId);
+    string expectedName = containerName(containerId);
 
     do {
       Future<Docker::Container> inspect = docker->inspect(expectedName);
@@ -185,7 +182,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -226,8 +223,6 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor)
 
   const Offer& offer = offers.get()[0];
 
-  const SlaveID& slaveId = offer.slave_id();
-
   TaskInfo task;
   task.set_name("");
   task.mutable_task_id()->set_value("1");
@@ -256,7 +251,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor)
   task.mutable_executor()->CopyFrom(executorInfo);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -275,7 +270,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor)
   AWAIT_READY_FOR(statusFinished, Seconds(60));
   EXPECT_EQ(TASK_FINISHED, statusFinished->state());
 
-  ASSERT_TRUE(exists(docker, slaveId, containerId.get()));
+  ASSERT_TRUE(exists(docker, containerId.get()));
 
   Future<Option<ContainerTermination>> termination =
     dockerContainerizer.wait(containerId.get());
@@ -287,7 +282,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor)
   EXPECT_SOME(termination.get());
 
   ASSERT_FALSE(
-    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING));
 }
 
 
@@ -313,7 +308,7 @@ TEST_F(DockerContainerizerTest, DISABLED_ROOT_DOCKER_Launch_Executor_Bridged)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -354,8 +349,6 @@ TEST_F(DockerContainerizerTest, DISABLED_ROOT_DOCKER_Launch_Executor_Bridged)
 
   const Offer& offer = offers.get()[0];
 
-  const SlaveID& slaveId = offer.slave_id();
-
   TaskInfo task;
   task.set_name("");
   task.mutable_task_id()->set_value("1");
@@ -385,7 +378,7 @@ TEST_F(DockerContainerizerTest, DISABLED_ROOT_DOCKER_Launch_Executor_Bridged)
   task.mutable_executor()->CopyFrom(executorInfo);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -404,7 +397,7 @@ TEST_F(DockerContainerizerTest, DISABLED_ROOT_DOCKER_Launch_Executor_Bridged)
   AWAIT_READY_FOR(statusFinished, Seconds(60));
   EXPECT_EQ(TASK_FINISHED, statusFinished->state());
 
-  ASSERT_TRUE(exists(docker, slaveId, containerId.get()));
+  ASSERT_TRUE(exists(docker, containerId.get()));
 
   Future<Option<ContainerTermination>> termination =
     dockerContainerizer.wait(containerId.get());
@@ -416,7 +409,7 @@ TEST_F(DockerContainerizerTest, DISABLED_ROOT_DOCKER_Launch_Executor_Bridged)
   EXPECT_SOME(termination.get());
 
   ASSERT_FALSE(
-    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING));
 }
 #endif // __linux__
 
@@ -433,7 +426,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -474,8 +467,6 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch)
 
   const Offer& offer = offers.get()[0];
 
-  const SlaveID& slaveId = offer.slave_id();
-
   TaskInfo task;
   task.set_name("");
   task.mutable_task_id()->set_value("1");
@@ -497,7 +488,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -557,7 +548,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch)
   ASSERT_EQ(1, statusRunning->container_status().network_infos().size());
   EXPECT_EQ(1, statusRunning->container_status().network_infos(0).ip_addresses().size()); // NOLINT(whitespace/line_length)
 
-  ASSERT_TRUE(exists(docker, slaveId, containerId.get()));
+  ASSERT_TRUE(exists(docker, containerId.get()));
 
   Future<Option<ContainerTermination>> termination =
     dockerContainerizer.wait(containerId.get());
@@ -569,7 +560,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch)
   EXPECT_SOME(termination.get());
 
   ASSERT_FALSE(
-    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING));
 }
 
 
@@ -585,7 +576,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Kill)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -626,8 +617,6 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Kill)
 
   const Offer& offer = offers.get()[0];
 
-  const SlaveID& slaveId = offer.slave_id();
-
   TaskInfo task;
   task.set_name("");
   task.mutable_task_id()->set_value("1");
@@ -649,7 +638,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Kill)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -665,7 +654,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Kill)
   EXPECT_EQ(TASK_RUNNING, statusRunning->state());
 
   ASSERT_TRUE(
-    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING));
 
   Future<TaskStatus> statusKilled;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
@@ -687,7 +676,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Kill)
   EXPECT_EQ(0, termination.get()->status());
 
   ASSERT_FALSE(
-    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING));
 
   driver.stop();
   driver.join();
@@ -708,7 +697,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_TaskKillingCapability)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -756,8 +745,6 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_TaskKillingCapability)
 
   const Offer& offer = offers.get()[0];
 
-  const SlaveID& slaveId = offer.slave_id();
-
   TaskInfo task;
   task.set_name("");
   task.mutable_task_id()->set_value("1");
@@ -779,7 +766,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_TaskKillingCapability)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -795,7 +782,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_TaskKillingCapability)
   EXPECT_EQ(TASK_RUNNING, statusRunning->state());
 
   ASSERT_TRUE(
-    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING));
 
   Future<TaskStatus> statusKilling, statusKilled;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
@@ -817,7 +804,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_TaskKillingCapability)
   EXPECT_SOME(termination.get());
 
   ASSERT_FALSE(
-    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING));
 
   driver.stop();
   driver.join();
@@ -838,7 +825,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Usage)
 
   Shared<Docker> docker(mockDocker);
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -901,7 +888,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Usage)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -984,7 +971,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Update)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -1025,8 +1012,6 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Update)
 
   const Offer& offer = offers.get()[0];
 
-  const SlaveID& slaveId = offer.slave_id();
-
   TaskInfo task;
   task.set_name("");
   task.mutable_task_id()->set_value("1");
@@ -1048,7 +1033,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Update)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -1066,9 +1051,9 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Update)
   EXPECT_EQ(TASK_RUNNING, statusRunning->state());
 
   ASSERT_TRUE(
-    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING));
 
-  string name = containerName(slaveId, containerId.get());
+  string name = containerName(containerId.get());
 
   Future<Docker::Container> inspect = docker->inspect(name);
 
@@ -1148,7 +1133,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Recover)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -1168,8 +1153,8 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Recover)
   ContainerID reapedContainerId;
   reapedContainerId.set_value(UUID::random().toString());
 
-  string container1 = containerName(slaveId, containerId);
-  string container2 = containerName(slaveId, reapedContainerId);
+  string container1 = containerName(containerId);
+  string container2 = containerName(reapedContainerId);
 
   // Clean up artifacts if containers still exists.
   ASSERT_TRUE(docker->rm(container1, true).await(Seconds(30)));
@@ -1213,9 +1198,9 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Recover)
   Future<Option<int>> orphanRun = docker->run(orphanOptions.get());
 
   ASSERT_TRUE(
-    exists(docker, slaveId, containerId, ContainerState::RUNNING));
+    exists(docker, containerId, ContainerState::RUNNING));
   ASSERT_TRUE(
-    exists(docker, slaveId, reapedContainerId, ContainerState::RUNNING));
+    exists(docker, reapedContainerId, ContainerState::RUNNING));
 
   Future<Docker::Container> inspect = docker->inspect(container2);
   AWAIT_READY(inspect);
@@ -1280,7 +1265,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_KillOrphanContainers)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -1296,19 +1281,16 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_KillOrphanContainers)
   SlaveID slaveId;
   slaveId.set_value("s1");
 
-  SlaveID oldSlaveId;
-  oldSlaveId.set_value("old-agent-id");
-
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
 
   ContainerID orphanContainerId;
   orphanContainerId.set_value(UUID::random().toString());
 
-  string container1 = containerName(slaveId, containerId);
+  string container1 = containerName(containerId);
 
   // Start the orphan container with the old slave id.
-  string container2 = containerName(oldSlaveId, orphanContainerId);
+  string container2 = containerName(orphanContainerId);
 
   // Clean up artifacts if containers still exists.
   ASSERT_TRUE(docker->rm(container1, true).await(Seconds(30)));
@@ -1353,10 +1335,10 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_KillOrphanContainers)
   Future<Option<int>> orphanRun = docker->run(orphanOptions.get());
 
   ASSERT_TRUE(
-    exists(docker, slaveId, containerId, ContainerState::RUNNING));
+    exists(docker, containerId, ContainerState::RUNNING));
 
   ASSERT_TRUE(
-    exists(docker, oldSlaveId, orphanContainerId, ContainerState::RUNNING));
+    exists(docker, orphanContainerId, ContainerState::RUNNING));
 
   SlaveState slaveState;
   slaveState.id = slaveId;
@@ -1403,7 +1385,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_KillOrphanContainers)
 
   AWAIT_READY(termination2);
   EXPECT_NONE(termination2.get());
-  ASSERT_FALSE(exists(docker, oldSlaveId, orphanContainerId));
+  ASSERT_FALSE(exists(docker, orphanContainerId));
 
   AWAIT_EXPECT_WEXITSTATUS_EQ(128 + SIGKILL, orphanRun);
 }
@@ -1420,7 +1402,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_SkipRecoverNonDocker)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -1482,7 +1464,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_SkipRecoverMalformedUUID)
   slave::Flags flags = CreateSlaveFlags();
   flags.docker_kill_orphans = true;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -1500,7 +1482,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_SkipRecoverMalformedUUID)
   ContainerID containerId;
   containerId.set_value("malformedUUID");
 
-  string container = containerName(slaveId, containerId);
+  string container = containerName(containerId);
 
   // Clean up container if it still exists.
   ASSERT_TRUE(docker->rm(container, true).await(Seconds(30)));
@@ -1528,7 +1510,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_SkipRecoverMalformedUUID)
   Future<Option<int>> run = docker->run(runOptions.get());
 
   ASSERT_TRUE(
-    exists(docker, slaveId, containerId, ContainerState::RUNNING));
+    exists(docker, containerId, ContainerState::RUNNING));
 
   SlaveState slaveState;
   slaveState.id = slaveId;
@@ -1550,7 +1532,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_SkipRecoverMalformedUUID)
 
   // The container should still exist and should not get killed
   // by containerizer recovery.
-  ASSERT_TRUE(exists(docker, slaveId, containerId));
+  ASSERT_TRUE(exists(docker, containerId));
 }
 
 
@@ -1570,7 +1552,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchWithPersistentVolumes)
   slave::Flags flags = CreateSlaveFlags();
   flags.resources = "cpu:2;mem:2048;disk(role1):2048";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -1614,8 +1596,6 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchWithPersistentVolumes)
 
   const Offer& offer = offers.get()[0];
 
-  const SlaveID& slaveId = offer.slave_id();
-
   Resource volume = createPersistentVolume(
     Megabytes(64),
     "role1",
@@ -1653,10 +1633,10 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchWithPersistentVolumes)
   filters.set_refuse_seconds(0);
 
   Future<ContainerID> containerId;
-  Future<string> directory;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  Future<ContainerConfig> containerConfig;
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
-                    FutureArg<3>(&directory),
+                    FutureArg<1>(&containerConfig),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
 
@@ -1673,7 +1653,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchWithPersistentVolumes)
       filters);
 
   AWAIT_READY_FOR(containerId, Seconds(60));
-  AWAIT_READY(directory);
+  AWAIT_READY(containerConfig);
   AWAIT_READY_FOR(statusRunning, Seconds(60));
   EXPECT_EQ(TASK_RUNNING, statusRunning->state());
 
@@ -1690,7 +1670,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchWithPersistentVolumes)
   EXPECT_SOME(termination.get());
 
   ASSERT_FALSE(
-    exists(docker, slaveId, containerId.get(), ContainerState::RUNNING));
+    exists(docker, containerId.get(), ContainerState::RUNNING));
 
   const string& volumePath = getPersistentVolumePath(
       flags.work_dir,
@@ -1703,8 +1683,9 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchWithPersistentVolumes)
 
   // Verify that the persistent volume is unmounted.
   foreach (const fs::MountInfoTable::Entry& entry, table->entries) {
-    EXPECT_FALSE(
-        strings::contains(entry.target, path::join(directory.get(), "path1")));
+    EXPECT_FALSE(strings::contains(
+        entry.target,
+        path::join(containerConfig->directory(), "path1")));
   }
 }
 
@@ -1724,7 +1705,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_RecoverPersistentVolumes)
   slave::Flags flags = CreateSlaveFlags();
   flags.resources = "cpu:2;mem:2048;disk(role1):2048";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -1806,10 +1787,10 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_RecoverPersistentVolumes)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  Future<string> directory;
-  EXPECT_CALL(*dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  Future<ContainerConfig> containerConfig;
+  EXPECT_CALL(*dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
-                    FutureArg<3>(&directory),
+                    FutureArg<1>(&containerConfig),
                     Invoke(dockerContainerizer.get(),
                            &MockDockerContainerizer::_launch)));
 
@@ -1824,7 +1805,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_RecoverPersistentVolumes)
       filters);
 
   AWAIT_READY_FOR(containerId, Seconds(60));
-  AWAIT_READY(directory);
+  AWAIT_READY(containerConfig);
   AWAIT_READY_FOR(statusRunning, Seconds(60));
   EXPECT_EQ(TASK_RUNNING, statusRunning->state());
 
@@ -1863,8 +1844,9 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_RecoverPersistentVolumes)
   // Verify that the recovered container's persistent volume is
   // unmounted.
   foreach (const fs::MountInfoTable::Entry& entry, table->entries) {
-    EXPECT_FALSE(
-        strings::contains(entry.target, path::join(directory.get(), "path1")));
+    EXPECT_FALSE(strings::contains(
+        entry.target,
+        path::join(containerConfig->directory(), "path1")));
   }
 
   driver.stop();
@@ -1887,7 +1869,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_RecoverOrphanedPersistentVolumes)
   slave::Flags flags = CreateSlaveFlags();
   flags.resources = "cpu:2;mem:2048;disk(role1):2048";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -1969,10 +1951,10 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_RecoverOrphanedPersistentVolumes)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  Future<string> directory;
-  EXPECT_CALL(*dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  Future<ContainerConfig> containerConfig;
+  EXPECT_CALL(*dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
-                    FutureArg<3>(&directory),
+                    FutureArg<1>(&containerConfig),
                     Invoke(dockerContainerizer.get(),
                            &MockDockerContainerizer::_launch)));
 
@@ -1987,7 +1969,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_RecoverOrphanedPersistentVolumes)
       filters);
 
   AWAIT_READY_FOR(containerId, Seconds(60));
-  AWAIT_READY(directory);
+  AWAIT_READY(containerConfig);
   AWAIT_READY_FOR(statusRunning, Seconds(60));
   EXPECT_EQ(TASK_RUNNING, statusRunning->state());
 
@@ -2028,8 +2010,9 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_RecoverOrphanedPersistentVolumes)
   // Verify that the orphaned container's persistent volume is
   // unmounted.
   foreach (const fs::MountInfoTable::Entry& entry, table->entries) {
-    EXPECT_FALSE(
-        strings::contains(entry.target, path::join(directory.get(), "path1")));
+    EXPECT_FALSE(strings::contains(
+        entry.target,
+        path::join(containerConfig->directory(), "path1")));
   }
 
   driver.stop();
@@ -2037,7 +2020,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_RecoverOrphanedPersistentVolumes)
 
   slave->reset();
 
-  EXPECT_FALSE(exists(docker, offer.slave_id(), containerId.get()));
+  EXPECT_FALSE(exists(docker, containerId.get()));
 }
 #endif // __linux__
 
@@ -2054,7 +2037,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Logs)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -2132,10 +2115,10 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Logs)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  Future<string> directory;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  Future<ContainerConfig> containerConfig;
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
-                    FutureArg<3>(&directory),
+                    FutureArg<1>(&containerConfig),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
 
@@ -2149,7 +2132,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Logs)
   driver.launchTasks(offers.get()[0].id(), {task});
 
   AWAIT_READY_FOR(containerId, Seconds(60));
-  AWAIT_READY(directory);
+  AWAIT_READY(containerConfig);
   AWAIT_READY_FOR(statusRunning, Seconds(60));
   EXPECT_EQ(TASK_RUNNING, statusRunning->state());
   AWAIT_READY_FOR(statusFinished, Seconds(60));
@@ -2157,7 +2140,9 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Logs)
 
   // Now check that the proper output is in stderr and stdout (which
   // might also contain other things, hence the use of a UUID).
-  Try<string> read = os::read(path::join(directory.get(), "stderr"));
+  Try<string> read =
+    os::read(path::join(containerConfig->directory(), "stderr"));
+
   ASSERT_SOME(read);
 
   vector<string> lines = strings::split(read.get(), "\n");
@@ -2165,7 +2150,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Logs)
   EXPECT_TRUE(containsLine(lines, "err" + uuid));
   EXPECT_FALSE(containsLine(lines, "out" + uuid));
 
-  read = os::read(path::join(directory.get(), "stdout"));
+  read = os::read(path::join(containerConfig->directory(), "stdout"));
   ASSERT_SOME(read);
 
   lines = strings::split(read.get(), "\n");
@@ -2192,7 +2177,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -2264,10 +2249,10 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  Future<string> directory;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  Future<ContainerConfig> containerConfig;
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
-                    FutureArg<3>(&directory),
+                    FutureArg<1>(&containerConfig),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
 
@@ -2281,13 +2266,15 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD)
   driver.launchTasks(offers.get()[0].id(), {task});
 
   AWAIT_READY_FOR(containerId, Seconds(60));
-  AWAIT_READY(directory);
+  AWAIT_READY(containerConfig);
   AWAIT_READY_FOR(statusRunning, Seconds(60));
   EXPECT_EQ(TASK_RUNNING, statusRunning->state());
   AWAIT_READY_FOR(statusFinished, Seconds(60));
   EXPECT_EQ(TASK_FINISHED, statusFinished->state());
 
-  Try<string> read = os::read(path::join(directory.get(), "stdout"));
+  Try<string> read =
+    os::read(path::join(containerConfig->directory(), "stdout"));
+
   ASSERT_SOME(read);
 
   vector<string> lines = strings::split(read.get(), "\n");
@@ -2297,7 +2284,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD)
   // command from the image which is 'inky'.
   EXPECT_TRUE(containsLine(lines, "inky"));
 
-  read = os::read(path::join(directory.get(), "stderr"));
+  read = os::read(path::join(containerConfig->directory(), "stderr"));
   ASSERT_SOME(read);
 
   lines = strings::split(read.get(), "\n");
@@ -2323,7 +2310,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD_Override)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -2397,10 +2384,10 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD_Override)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  Future<string> directory;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  Future<ContainerConfig> containerConfig;
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
-                    FutureArg<3>(&directory),
+                    FutureArg<1>(&containerConfig),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
 
@@ -2414,14 +2401,16 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD_Override)
   driver.launchTasks(offers.get()[0].id(), {task});
 
   AWAIT_READY_FOR(containerId, Seconds(60));
-  AWAIT_READY(directory);
+  AWAIT_READY(containerConfig);
   AWAIT_READY_FOR(statusRunning, Seconds(60));
   EXPECT_EQ(TASK_RUNNING, statusRunning->state());
   AWAIT_READY_FOR(statusFinished, Seconds(60));
   EXPECT_EQ(TASK_FINISHED, statusFinished->state());
 
   // Now check that the proper output is in stderr and stdout.
-  Try<string> read = os::read(path::join(directory.get(), "stdout"));
+  Try<string> read =
+    os::read(path::join(containerConfig->directory(), "stdout"));
+
   ASSERT_SOME(read);
 
   vector<string> lines = strings::split(read.get(), "\n");
@@ -2432,7 +2421,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD_Override)
   EXPECT_TRUE(containsLine(lines, uuid));
   EXPECT_FALSE(containsLine(lines, "inky"));
 
-  read = os::read(path::join(directory.get(), "stderr"));
+  read = os::read(path::join(containerConfig->directory(), "stderr"));
   ASSERT_SOME(read);
 
   lines = strings::split(read.get(), "\n");
@@ -2459,7 +2448,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD_Args)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -2534,10 +2523,10 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD_Args)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  Future<string> directory;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  Future<ContainerConfig> containerConfig;
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
-                    FutureArg<3>(&directory),
+                    FutureArg<1>(&containerConfig),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
 
@@ -2551,14 +2540,16 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD_Args)
   driver.launchTasks(offers.get()[0].id(), {task});
 
   AWAIT_READY_FOR(containerId, Seconds(60));
-  AWAIT_READY(directory);
+  AWAIT_READY(containerConfig);
   AWAIT_READY_FOR(statusRunning, Seconds(60));
   EXPECT_EQ(TASK_RUNNING, statusRunning->state());
   AWAIT_READY_FOR(statusFinished, Seconds(60));
   EXPECT_EQ(TASK_FINISHED, statusFinished->state());
 
   // Now check that the proper output is in stderr and stdout.
-  Try<string> read = os::read(path::join(directory.get(), "stdout"));
+  Try<string> read =
+    os::read(path::join(containerConfig->directory(), "stdout"));
+
   ASSERT_SOME(read);
 
   vector<string> lines = strings::split(read.get(), "\n");
@@ -2569,7 +2560,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Default_CMD_Args)
   EXPECT_TRUE(containsLine(lines, uuid));
   EXPECT_FALSE(containsLine(lines, "inky"));
 
-  read = os::read(path::join(directory.get(), "stderr"));
+  read = os::read(path::join(containerConfig->directory(), "stderr"));
   ASSERT_SOME(read);
 
   lines = strings::split(read.get(), "\n");
@@ -2597,7 +2588,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_SlaveRecoveryTaskContainer)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // This is owned by the containerizer, so we'll need one per containerizer.
   Try<ContainerLogger*> logger =
@@ -2665,7 +2656,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_SlaveRecoveryTaskContainer)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(*dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(*dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(dockerContainerizer.get(),
                            &MockDockerContainerizer::_launch)));
@@ -2720,7 +2711,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_SlaveRecoveryTaskContainer)
   AWAIT_READY(status);
   ASSERT_EQ(TASK_RUNNING, status->state());
 
-  ASSERT_TRUE(exists(docker, offer.slave_id(), containerId.get()));
+  ASSERT_TRUE(exists(docker, containerId.get()));
 
   Future<Option<ContainerTermination>> termination =
     dockerContainerizer->wait(containerId.get());
@@ -2758,7 +2749,7 @@ TEST_F(DockerContainerizerTest,
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // This is owned by the containerizer, so we'll need one per containerizer.
   Try<ContainerLogger*> logger =
@@ -2833,10 +2824,8 @@ TEST_F(DockerContainerizerTest,
   task.mutable_executor()->CopyFrom(executorInfo);
 
   Future<ContainerID> containerId;
-  Future<SlaveID> slaveId;
-  EXPECT_CALL(*dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(*dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
-                    FutureArg<5>(&slaveId),
                     Invoke(dockerContainerizer.get(),
                            &MockDockerContainerizer::_launch)));
 
@@ -2863,7 +2852,6 @@ TEST_F(DockerContainerizerTest,
   driver.launchTasks(offers.get()[0].id(), {task});
 
   AWAIT_READY(containerId);
-  AWAIT_READY(slaveId);
 
   AWAIT_READY(executorLaunched);
   AWAIT_READY(statusUpdateMessage1);
@@ -2908,7 +2896,7 @@ TEST_F(DockerContainerizerTest,
   AWAIT_READY(status);
   ASSERT_EQ(TASK_RUNNING, status->state());
 
-  ASSERT_TRUE(exists(docker, slaveId.get(), containerId.get()));
+  ASSERT_TRUE(exists(docker, containerId.get()));
 
   driver.stop();
   driver.join();
@@ -2932,7 +2920,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_NC_PortMapping)
   slave::Flags flags = CreateSlaveFlags();
   flags.resources = "cpus:1;mem:1024;ports:[10000-10000]";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -3011,10 +2999,10 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_NC_PortMapping)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  Future<string> directory;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  Future<ContainerConfig> containerConfig;
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
-                    FutureArg<3>(&directory),
+                    FutureArg<1>(&containerConfig),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
 
@@ -3028,13 +3016,12 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_NC_PortMapping)
   driver.launchTasks(offers.get()[0].id(), {task});
 
   AWAIT_READY_FOR(containerId, Seconds(60));
-  AWAIT_READY(directory);
+  AWAIT_READY(containerConfig);
   AWAIT_READY_FOR(statusRunning, Seconds(60));
   EXPECT_EQ(TASK_RUNNING, statusRunning->state());
 
   ASSERT_TRUE(
     exists(docker,
-           offer.slave_id(),
            containerId.get(),
            ContainerState::RUNNING));
 
@@ -3051,7 +3038,9 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_NC_PortMapping)
   EXPECT_EQ(TASK_FINISHED, statusFinished->state());
 
   // Now check that the proper output is in stdout.
-  Try<string> read = os::read(path::join(directory.get(), "stdout"));
+  Try<string> read =
+    os::read(path::join(containerConfig->directory(), "stdout"));
+
   ASSERT_SOME(read);
 
   const vector<string> lines = strings::split(read.get(), "\n");
@@ -3087,7 +3076,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchSandboxWithColon)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -3149,7 +3138,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchSandboxWithColon)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -3165,7 +3154,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchSandboxWithColon)
   AWAIT_READY_FOR(statusRunning, Seconds(60));
   EXPECT_EQ(TASK_RUNNING, statusRunning->state());
 
-  ASSERT_TRUE(exists(docker, offer.slave_id(), containerId.get()));
+  ASSERT_TRUE(exists(docker, containerId.get()));
 
   Future<Option<ContainerTermination>> termination =
     dockerContainerizer.wait(containerId.get());
@@ -3190,7 +3179,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DestroyWhileFetching)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -3213,14 +3202,14 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DestroyWhileFetching)
   Future<Nothing> fetch;
 
   // We want to pause the fetch call to simulate a long fetch time.
-  EXPECT_CALL(*process, fetch(_, _))
+  EXPECT_CALL(*process, fetch(_))
     .WillOnce(DoAll(FutureSatisfy(&fetch),
                     Return(promise.future())));
 
   Owned<MasterDetector> detector = master.get()->createDetector();
 
   Try<Owned<cluster::Slave>> slave =
-    StartSlave(detector.get(), &dockerContainerizer);
+    StartSlave(detector.get(), &dockerContainerizer, flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -3270,7 +3259,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DestroyWhileFetching)
     .WillOnce(FutureArg<1>(&statusFailed));
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -3307,7 +3296,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DestroyWhilePulling)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -3327,7 +3316,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DestroyWhilePulling)
       (Owned<DockerContainerizerProcess>(process)));
 
   Future<Nothing> fetch;
-  EXPECT_CALL(*process, fetch(_, _))
+  EXPECT_CALL(*process, fetch(_))
     .WillOnce(DoAll(FutureSatisfy(&fetch),
                     Return(Nothing())));
 
@@ -3340,7 +3329,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DestroyWhilePulling)
   Owned<MasterDetector> detector = master.get()->createDetector();
 
   Try<Owned<cluster::Slave>> slave =
-    StartSlave(detector.get(), &dockerContainerizer);
+    StartSlave(detector.get(), &dockerContainerizer, flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -3390,7 +3379,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DestroyWhilePulling)
     .WillOnce(FutureArg<1>(&statusFailed));
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -3422,7 +3411,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DestroyUnknownContainer)
 {
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<DockerContainerizer*> create =
     DockerContainerizer::create(flags, &fetcher);
@@ -3453,7 +3442,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_ExecutorCleanupWhenLaunchFailed)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -3475,7 +3464,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_ExecutorCleanupWhenLaunchFailed)
   Owned<MasterDetector> detector = master.get()->createDetector();
 
   Try<Owned<cluster::Slave>> slave =
-    StartSlave(detector.get(), &dockerContainerizer);
+    StartSlave(detector.get(), &dockerContainerizer, flags);
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
@@ -3528,7 +3517,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_ExecutorCleanupWhenLaunchFailed)
     .WillOnce(FutureArg<1>(&statusGone));
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -3565,7 +3554,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_FetchFailure)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -3587,7 +3576,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_FetchFailure)
   Owned<MasterDetector> detector = master.get()->createDetector();
 
   Try<Owned<cluster::Slave>> slave =
-    StartSlave(detector.get(), &dockerContainerizer);
+    StartSlave(detector.get(), &dockerContainerizer, flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -3636,12 +3625,12 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_FetchFailure)
     .WillOnce(FutureArg<1>(&statusFailed));
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
 
-  EXPECT_CALL(*process, fetch(_, _))
+  EXPECT_CALL(*process, fetch(_))
     .WillOnce(Return(Failure("some error from fetch")));
 
   driver.launchTasks(offers.get()[0].id(), {task});
@@ -3676,7 +3665,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerPullFailure)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -3698,7 +3687,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerPullFailure)
   Owned<MasterDetector> detector = master.get()->createDetector();
 
   Try<Owned<cluster::Slave>> slave =
-    StartSlave(detector.get(), &dockerContainerizer);
+    StartSlave(detector.get(), &dockerContainerizer, flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -3747,7 +3736,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerPullFailure)
     .WillOnce(FutureArg<1>(&statusFailed));
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -3787,7 +3776,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerInspectDiscard)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -3818,7 +3807,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerInspectDiscard)
   Owned<MasterDetector> detector = master.get()->createDetector();
 
   Try<Owned<cluster::Slave>> slave =
-    StartSlave(detector.get(), &dockerContainerizer);
+    StartSlave(detector.get(), &dockerContainerizer, flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
@@ -3875,7 +3864,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_DockerInspectDiscard)
     .WillOnce(FutureArg<1>(&statusFailed));
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -3906,7 +3895,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_WaitUnknownContainer)
 {
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<DockerContainerizer*> create =
     DockerContainerizer::create(flags, &fetcher);
@@ -3941,7 +3930,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_NoTransitionFromKillingToRunning)
 
   slave::Flags agentFlags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(agentFlags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(agentFlags.container_logger);
@@ -4021,7 +4010,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_NoTransitionFromKillingToRunning)
   task.mutable_kill_policy()->CopyFrom(killPolicy);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(containerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(containerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&containerizer,
                            &MockDockerContainerizer::_launch)));
@@ -4088,7 +4077,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_CGROUPS_CFS_CgroupsEnableCFS)
   flags.cgroups_enable_cfs = true;
   flags.resources = "cpus:1;mem:128";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -4150,7 +4139,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_CGROUPS_CFS_CgroupsEnableCFS)
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(dockerContainerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&dockerContainerizer,
                            &MockDockerContainerizer::_launch)));
@@ -4169,7 +4158,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_CGROUPS_CFS_CgroupsEnableCFS)
 
   // Find cgroups cpu hierarchy of the container and verifies
   // quota is set.
-  string name = containerName(offer.slave_id(), containerId.get());
+  string name = containerName(containerId.get());
   Future<Docker::Container> inspect = docker->inspect(name);
   AWAIT_READY(inspect);
 


[10/16] mesos git commit: Changed naming of Docker containers to the pre-0.23 scheme.

Posted by jo...@apache.org.
Changed naming of Docker containers to the pre-0.23 scheme.

For the most part, the choice of naming scheme is cosmetic.
All versions of the Docker containerizer will consider containers
starting with "mesos-" as containers started by Mesos.
The Docker containerizer does not consider the `SlaveID` at all
when recovering.

This naming change also tweaks the recovery logic to store the
name of the recovered container, so that future calls to the Docker
CLI use the actual name of the container, rather than a name generated
based on the ContainerID.

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


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

Branch: refs/heads/master
Commit: 55d7c9f76b61ef574e9d686960b280a05c1afc5d
Parents: 7015097
Author: Joseph Wu <jo...@apache.org>
Authored: Thu Apr 27 13:57:15 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:07 2017 -0700

----------------------------------------------------------------------
 src/slave/containerizer/docker.cpp | 66 ++++++++++++++++++++-------------
 src/slave/containerizer/docker.hpp | 24 ++++++------
 2 files changed, 53 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/55d7c9f7/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index 2480f59..9418e01 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -121,13 +121,15 @@ Option<ContainerID> parse(const Docker::Container& container)
   }
 
   if (name.isSome()) {
-    // For Mesos version < 0.23.0, the docker container name format
-    // was DOCKER_NAME_PREFIX + containerId, and starting with 0.23.0
-    // it is changed to DOCKER_NAME_PREFIX + slaveId +
-    // DOCKER_NAME_SEPERATOR + containerId.
+    // For Mesos versions 0.23 to 1.3 (inclusive), the docker
+    // container name format was:
+    //   DOCKER_NAME_PREFIX + SlaveID + DOCKER_NAME_SEPERATOR + ContainerID.
+    //
+    // In versions <= 0.22 or >= 1.4, the name format is:
+    //   DOCKER_NAME_PREFIX + ContainerID.
+    //
     // To be backward compatible during upgrade, we still have to
-    // support the previous format.
-    // TODO(tnachen): Remove this check after deprecation cycle.
+    // support all formats.
     if (!strings::contains(name.get(), DOCKER_NAME_SEPERATOR)) {
       ContainerID id;
       id.set_value(name.get());
@@ -383,7 +385,7 @@ DockerContainerizerProcess::Container::create(
     // from a hook. This hook is called after `Container::create`.
     ::mesos::internal::docker::Flags dockerExecutorFlags = dockerFlags(
       flags,
-      Container::name(slaveId, stringify(id)),
+      Container::name(id),
       containerWorkdir,
       None());
 
@@ -923,16 +925,18 @@ Future<Nothing> DockerContainerizerProcess::_recover(
     const list<Docker::Container>& _containers)
 {
   if (state.isSome()) {
-    // Although the slave checkpoints executor pids, before 0.23
-    // docker containers without custom executors didn't record the
-    // container type in the executor info, therefore the Docker
-    // containerizer doesn't know if it should recover that container
-    // as it could be launched from another containerizer. The
-    // workaround is to reconcile running Docker containers and see
-    // if we can find an known container that matches the
-    // checkpointed container id.
-    // TODO(tnachen): Remove this explicit reconciliation 0.24.
-    hashset<ContainerID> existingContainers;
+    // This mapping of ContainerIDs to running Docker container names
+    // is established for two reasons:
+    //   * Docker containers launched by Mesos versions prior to 0.23
+    //     did not checkpoint the container type, so the Docker
+    //     Containerizer does not know if it should recover that
+    //     container or not.
+    //   * The naming scheme of Docker containers changed in Mesos
+    //     versions 0.23 and 1.4. The Docker Containerizer code needs
+    //     to use the name of the container when interacting with the
+    //     Docker CLI, rather than generating the container name
+    //     based on the current version's scheme.
+    hashmap<ContainerID, string> existingContainers;
 
     // Tracks all the task containers that launched an executor in
     // a docker container.
@@ -941,7 +945,13 @@ Future<Nothing> DockerContainerizerProcess::_recover(
     foreach (const Docker::Container& container, _containers) {
       Option<ContainerID> id = parse(container);
       if (id.isSome()) {
-        existingContainers.insert(id.get());
+        // NOTE: The container name returned by `docker inspect` may
+        // sometimes be prefixed with a forward slash. While this is
+        // technically part of the container name, subsequent calls
+        // to the Docker CLI do not expect the prefix.
+        existingContainers[id.get()] = strings::remove(
+            container.name, "/", strings::PREFIX);
+
         if (strings::contains(container.name, ".executor")) {
           executorContainers.insert(id.get());
         }
@@ -1023,6 +1033,10 @@ Future<Nothing> DockerContainerizerProcess::_recover(
         container->launchesExecutorContainer =
           executorContainers.contains(containerId);
 
+        if (existingContainers.contains(containerId)) {
+          container->containerName = existingContainers.at(containerId);
+        }
+
         pid_t pid = run.get().forkedPid.get();
 
         container->status.set(process::reap(pid));
@@ -1183,7 +1197,7 @@ Future<bool> DockerContainerizerProcess::launch(
     f = HookManager::slavePreLaunchDockerTaskExecutorDecorator(
         taskInfo,
         executorInfo,
-        container.get()->name(),
+        container.get()->containerName,
         container.get()->directory,
         flags.sandbox_directory,
         container.get()->environment)
@@ -1299,7 +1313,7 @@ Future<bool> DockerContainerizerProcess::_launch(
       }));
   }
 
-  string containerName = container->name();
+  string containerName = container->containerName;
 
   if (container->executorName().isSome()) {
     // Launch the container with the executor name as we expect the
@@ -1539,7 +1553,7 @@ Future<pid_t> DockerContainerizerProcess::launchExecutorProcess(
     // Prepare the flags to pass to the mesos docker executor process.
     ::mesos::internal::docker::Flags launchFlags = dockerFlags(
         flags,
-        container->name(),
+        container->containerName,
         container->directory,
         container->taskEnvironment);
 
@@ -1662,7 +1676,7 @@ Future<Nothing> DockerContainerizerProcess::update(
     return __update(containerId, _resources, container->pid.get());
   }
 
-  return docker->inspect(containers_.at(containerId)->name())
+  return docker->inspect(containers_.at(containerId)->containerName)
     .then(defer(self(), &Self::_update, containerId, _resources, lambda::_1));
 #else
   return Nothing();
@@ -1909,7 +1923,7 @@ Future<ResourceStatistics> DockerContainerizerProcess::usage(
     return collectUsage(container->pid.get());
   }
 
-  return docker->inspect(container->name())
+  return docker->inspect(container->containerName)
     .then(defer(
       self(),
       [this, containerId, collectUsage]
@@ -2257,7 +2271,7 @@ void DockerContainerizerProcess::_destroy(
     // container should be destroyed forcefully.
     // The `after` fallback should remain as a precaution against the docker
     // stop command hanging.
-    docker->stop(container->name(), flags.docker_stop_timeout)
+    docker->stop(container->containerName, flags.docker_stop_timeout)
       .after(
           flags.docker_stop_timeout + DOCKER_FORCE_KILL_TIMEOUT,
           defer(self(), &Self::destroyTimeout, containerId, lambda::_1))
@@ -2303,7 +2317,7 @@ void DockerContainerizerProcess::__destroy(
       flags.docker_remove_delay,
       self(),
       &Self::remove,
-      container->name(),
+      container->containerName,
       container->executorName());
 
     delete container;
@@ -2374,7 +2388,7 @@ void DockerContainerizerProcess::____destroy(
     flags.docker_remove_delay,
     self(),
     &Self::remove,
-    container->name(),
+    container->containerName,
     container->executorName());
 
   delete container;

http://git-wip-us.apache.org/repos/asf/mesos/blob/55d7c9f7/src/slave/containerizer/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.hpp b/src/slave/containerizer/docker.hpp
index 44efa44..2ed8e1c 100644
--- a/src/slave/containerizer/docker.hpp
+++ b/src/slave/containerizer/docker.hpp
@@ -45,8 +45,9 @@ namespace slave {
 // created by Mesos from those created manually.
 extern const std::string DOCKER_NAME_PREFIX;
 
-// Separator used to compose docker container name, which is made up
-// of slave ID and container ID.
+// Separator used to compose docker container name, which consists
+// of the name prefix, ContainerID, and possibly the SlaveID depending
+// on the version of Mesos used to create the container.
 extern const std::string DOCKER_NAME_SEPERATOR;
 
 // Directory that stores all the symlinked sandboxes that is mapped
@@ -315,10 +316,9 @@ private:
         bool checkpoint,
         const Flags& flags);
 
-    static std::string name(const SlaveID& slaveId, const std::string& id)
+    static std::string name(const ContainerID& id)
     {
-      return DOCKER_NAME_PREFIX + slaveId.value() + DOCKER_NAME_SEPERATOR +
-        stringify(id);
+      return DOCKER_NAME_PREFIX + stringify(id);
     }
 
     Container(const ContainerID& id)
@@ -348,6 +348,7 @@ private:
         checkpoint(checkpoint),
         symlinked(symlinked),
         flags(flags),
+        containerName(name(id)),
         launchesExecutorContainer(launchesExecutorContainer)
     {
       // NOTE: The task's resources are included in the executor's
@@ -392,15 +393,10 @@ private:
       }
     }
 
-    std::string name()
-    {
-      return name(slaveId, stringify(id));
-    }
-
     Option<std::string> executorName()
     {
       if (launchesExecutorContainer) {
-        return name() + DOCKER_NAME_SEPERATOR + "executor";
+        return containerName + DOCKER_NAME_SEPERATOR + "executor";
       } else {
         return None();
       }
@@ -477,6 +473,12 @@ private:
     bool symlinked;
     const Flags flags;
 
+    // The string used to refer to this container via the Docker CLI.
+    // This name is either computed by concatenating the DOCKER_NAME_PREFIX
+    // and the ContainerID; or during recovery, by taking the recovered
+    // container's name.
+    std::string containerName;
+
     // Promise for future returned from wait().
     process::Promise<mesos::slave::ContainerTermination> termination;
 


[13/16] mesos git commit: Added test helpers to tranlate to ContainerConfig.

Posted by jo...@apache.org.
Added test helpers to tranlate to ContainerConfig.

This adds a helper that emulates the logic in the Agent's launch path.
Now that the Containerizer takes a ContainerConfig instead of
a TaskInfo/ExecutorInfo/Directory/SlaveID/User, tests that call
the containerizer directly will need to translate to
ContainerConfig.

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


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

Branch: refs/heads/master
Commit: 012e6a26b2d4debe8bd32fb8337d86005217e24b
Parents: 46da722
Author: Joseph Wu <jo...@apache.org>
Authored: Mon May 1 17:12:14 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:07 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/mesos/blob/012e6a26/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 9b04a40..48072a9 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -1083,6 +1083,67 @@ inline CommandInfo createCommandInfo(
 }
 
 
+// Almost a direct snippet of code at the bottom of `Slave::launchExecutor`.
+inline mesos::slave::ContainerConfig createContainerConfig(
+    const Option<TaskInfo>& taskInfo,
+    const ExecutorInfo& executorInfo,
+    const std::string& sandboxDirectory,
+    const Option<std::string>& user = None())
+{
+  mesos::slave::ContainerConfig containerConfig;
+  containerConfig.mutable_executor_info()->CopyFrom(executorInfo);
+  containerConfig.mutable_command_info()->CopyFrom(executorInfo.command());
+  containerConfig.mutable_resources()->CopyFrom(executorInfo.resources());
+  containerConfig.set_directory(sandboxDirectory);
+
+  if (user.isSome()) {
+    containerConfig.set_user(user.get());
+  }
+
+  if (taskInfo.isSome()) {
+    containerConfig.mutable_task_info()->CopyFrom(taskInfo.get());
+
+    if (taskInfo.get().has_container()) {
+      containerConfig.mutable_container_info()
+        ->CopyFrom(taskInfo.get().container());
+    }
+  } else {
+    if (executorInfo.has_container()) {
+      containerConfig.mutable_container_info()
+        ->CopyFrom(executorInfo.container());
+    }
+  }
+
+  return containerConfig;
+}
+
+
+// Almost a direct snippet of code in `Slave::Http::_launchNestedContainer`.
+inline mesos::slave::ContainerConfig createContainerConfig(
+    const CommandInfo& commandInfo,
+    const Option<ContainerInfo>& containerInfo = None(),
+    const Option<mesos::slave::ContainerClass>& containerClass = None(),
+    const Option<std::string>& user = None())
+{
+  mesos::slave::ContainerConfig containerConfig;
+  containerConfig.mutable_command_info()->CopyFrom(commandInfo);
+
+  if (user.isSome()) {
+    containerConfig.set_user(user.get());
+  }
+
+  if (containerInfo.isSome()) {
+    containerConfig.mutable_container_info()->CopyFrom(containerInfo.get());
+  }
+
+  if (containerClass.isSome()) {
+    containerConfig.set_container_class(containerClass.get());
+  }
+
+  return containerConfig;
+}
+
+
 template <typename... Args>
 inline Image createDockerImage(Args&&... args)
 {


[15/16] mesos git commit: Updated fetcher tests to reflect changed interfaces.

Posted by jo...@apache.org.
Updated fetcher tests to reflect changed interfaces.

There are two primary changes to the tests:
  * The arguments passed to the fetcher's methods no longer contain
    the Flags or SlaveID.  So a couple of the mock expectations need
    to be tweaked.
  * Flags are now passed into the fetcher at construction.  Some of
    the tests modify the flags after construction, so the order of
    operations needed to be flipped.

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


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

Branch: refs/heads/master
Commit: 391ced51552f4917fd3fe33f1003391ce7d01f75
Parents: c66f7e2
Author: Joseph Wu <jo...@apache.org>
Authored: Wed May 10 17:16:49 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:08 2017 -0700

----------------------------------------------------------------------
 src/tests/fetcher_cache_tests.cpp |  91 ++++++++++------------
 src/tests/fetcher_tests.cpp       | 135 +++++++++++++--------------------
 2 files changed, 94 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/391ced51/src/tests/fetcher_cache_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fetcher_cache_tests.cpp b/src/tests/fetcher_cache_tests.cpp
index c6ba79a..1edd15c 100644
--- a/src/tests/fetcher_cache_tests.cpp
+++ b/src/tests/fetcher_cache_tests.cpp
@@ -153,7 +153,6 @@ protected:
   string assetsDirectory;
   string commandPath;
   string archivePath;
-  string cacheDirectory;
 
   Owned<cluster::Master> master;
   Owned<cluster::Slave> slave;
@@ -199,9 +198,6 @@ void FetcherCacheTest::SetUp()
   ASSERT_SOME(_master);
   master = _master.get();
 
-  fetcherProcess = new MockFetcherProcess();
-  fetcher.reset(new Fetcher(Owned<FetcherProcess>(fetcherProcess)));
-
   FrameworkInfo frameworkInfo;
   frameworkInfo.set_name("default");
   frameworkInfo.set_checkpoint(true);
@@ -282,6 +278,9 @@ void FetcherCacheTest::TearDown()
 // available for all testing as possible.
 void FetcherCacheTest::startSlave()
 {
+  fetcherProcess = new MockFetcherProcess(flags);
+  fetcher.reset(new Fetcher(Owned<FetcherProcess>(fetcherProcess)));
+
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags, true, fetcher.get());
 
@@ -300,9 +299,6 @@ void FetcherCacheTest::startSlave()
 
   AWAIT_READY(slaveRegisteredMessage);
   slaveId = slaveRegisteredMessage->slave_id();
-
-  cacheDirectory =
-    slave::paths::getSlavePath(flags.fetcher_cache_dir, slaveId);
 }
 
 
@@ -517,7 +513,7 @@ Try<vector<FetcherCacheTest::Task>> FetcherCacheTest::launchTasks(
   // When _fetch() is called, notify us by satisfying a promise that
   // a task has passed the code stretch in which it competes for cache
   // entries.
-  EXPECT_CALL(*fetcherProcess, _fetch(_, _, _, _, _, _))
+  EXPECT_CALL(*fetcherProcess, _fetch(_, _, _, _, _))
     .WillRepeatedly(
         DoAll(SatisfyOne(&fetchContentionWaypoints),
               Invoke(fetcherProcess, &MockFetcherProcess::unmocked__fetch)));
@@ -620,8 +616,8 @@ TEST_F(FetcherCacheTest, LocalUncached)
   AWAIT_READY(awaitFinished(task.get()));
 
   EXPECT_EQ(0u, fetcherProcess->cacheSize());
-  ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-  EXPECT_EQ(0u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+  ASSERT_SOME(fetcherProcess->cacheFiles());
+  EXPECT_EQ(0u, fetcherProcess->cacheFiles()->size());
 
   const string path = path::join(task->runDirectory.string(), COMMAND_NAME);
   EXPECT_TRUE(isExecutable(path));
@@ -657,8 +653,8 @@ TEST_F(FetcherCacheTest, LocalCached)
     EXPECT_TRUE(os::exists(path + taskName(i)));
 
     EXPECT_EQ(1u, fetcherProcess->cacheSize());
-    ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-    EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+    ASSERT_SOME(fetcherProcess->cacheFiles());
+    EXPECT_EQ(1u, fetcherProcess->cacheFiles()->size());
   }
 }
 
@@ -686,8 +682,8 @@ TEST_F(FetcherCacheTest, CachedCustomFilename)
   AWAIT_READY(awaitFinished(task.get()));
 
   EXPECT_EQ(1u, fetcherProcess->cacheSize());
-  ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-  EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+  ASSERT_SOME(fetcherProcess->cacheFiles());
+  EXPECT_EQ(1u, fetcherProcess->cacheFiles()->size());
 
   // Verify that the downloaded executable lives at our custom output path.
   const string executablePath = path::join(
@@ -728,8 +724,8 @@ TEST_F(FetcherCacheTest, CachedCustomOutputFileWithSubdirectory)
   AWAIT_READY(awaitFinished(task.get()));
 
   EXPECT_EQ(1u, fetcherProcess->cacheSize());
-  ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-  EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+  ASSERT_SOME(fetcherProcess->cacheFiles());
+  EXPECT_EQ(1u, fetcherProcess->cacheFiles()->size());
 
   // Verify that the downloaded executable lives at our custom output file
   // path.
@@ -769,7 +765,7 @@ TEST_F(FetcherCacheTest, CachedFallback)
 
   // Bring back the asset just before running mesos-fetcher to fetch it.
   Future<FetcherInfo> fetcherInfo;
-  EXPECT_CALL(*fetcherProcess, run(_, _, _, _, _))
+  EXPECT_CALL(*fetcherProcess, run(_, _, _, _))
     .WillOnce(DoAll(FutureArg<3>(&fetcherInfo),
                     InvokeWithoutArgs(this,
                                       &FetcherCacheTest::setupCommandFileAsset),
@@ -792,8 +788,8 @@ TEST_F(FetcherCacheTest, CachedFallback)
             fetcherInfo->items(0).action());
 
   EXPECT_EQ(0u, fetcherProcess->cacheSize());
-  ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-  EXPECT_EQ(0u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+  ASSERT_SOME(fetcherProcess->cacheFiles());
+  EXPECT_EQ(0u, fetcherProcess->cacheFiles()->size());
 }
 
 
@@ -829,8 +825,8 @@ TEST_F(FetcherCacheTest, LocalUncachedExtract)
   EXPECT_TRUE(os::exists(path + taskName(index)));
 
   EXPECT_EQ(0u, fetcherProcess->cacheSize());
-  ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-  EXPECT_EQ(0u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+  ASSERT_SOME(fetcherProcess->cacheFiles());
+  EXPECT_EQ(0u, fetcherProcess->cacheFiles()->size());
 }
 
 
@@ -864,8 +860,8 @@ TEST_F(FetcherCacheTest, LocalCachedExtract)
     EXPECT_TRUE(os::exists(path + taskName(i)));
 
     EXPECT_EQ(1u, fetcherProcess->cacheSize());
-    ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-    EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+    ASSERT_SOME(fetcherProcess->cacheFiles());
+    EXPECT_EQ(1u, fetcherProcess->cacheFiles()->size());
   }
 }
 
@@ -1013,8 +1009,8 @@ TEST_F(FetcherCacheHttpTest, HttpCachedSerialized)
     EXPECT_TRUE(os::exists(path + taskName(i)));
 
     EXPECT_EQ(1u, fetcherProcess->cacheSize());
-    ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-    EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+    ASSERT_SOME(fetcherProcess->cacheFiles());
+    EXPECT_EQ(1u, fetcherProcess->cacheFiles()->size());
 
     // 2 requests: 1 for content-length, 1 for download.
     EXPECT_EQ(2u, httpServer->countCommandRequests);
@@ -1078,8 +1074,8 @@ TEST_F(FetcherCacheHttpTest, HttpCachedConcurrent)
   AWAIT_READY(awaitFinished(tasks.get()));
 
   EXPECT_EQ(1u, fetcherProcess->cacheSize());
-  ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-  EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+  ASSERT_SOME(fetcherProcess->cacheFiles());
+  EXPECT_EQ(1u, fetcherProcess->cacheFiles()->size());
 
   // HTTP requests regarding the archive asset as follows. Archive
   // "content-length" requests: 1, archive file downloads: 2.
@@ -1187,8 +1183,8 @@ TEST_F(FetcherCacheHttpTest, HttpMixed)
   AWAIT_READY(awaitFinished(tasks.get()));
 
   EXPECT_EQ(1u, fetcherProcess->cacheSize());
-  ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-  EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+  ASSERT_SOME(fetcherProcess->cacheFiles());
+  EXPECT_EQ(1u, fetcherProcess->cacheFiles()->size());
 
   // HTTP requests regarding the command asset as follows. Command
   // "content-length" requests: 0, command file downloads: 3.
@@ -1274,8 +1270,8 @@ TEST_F(FetcherCacheHttpTest, DISABLED_HttpCachedRecovery)
     EXPECT_TRUE(os::exists(path + taskName(i)));
 
     EXPECT_EQ(1u, fetcherProcess->cacheSize());
-    ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-    EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+    ASSERT_SOME(fetcherProcess->cacheFiles());
+    EXPECT_EQ(1u, fetcherProcess->cacheFiles()->size());
 
     // content-length requests: 1
     // downloads: 1
@@ -1290,7 +1286,7 @@ TEST_F(FetcherCacheHttpTest, DISABLED_HttpCachedRecovery)
 
   // Don't reuse the old fetcher, which has stale state after
   // stopping the slave.
-  Fetcher fetcher2;
+  Fetcher fetcher2(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher2);
@@ -1311,9 +1307,6 @@ TEST_F(FetcherCacheHttpTest, DISABLED_HttpCachedRecovery)
   // Wait until the containerizer is updated.
   AWAIT_READY(update);
 
-  // Recovery must have cleaned the cache by now.
-  EXPECT_FALSE(os::exists(cacheDirectory));
-
   // Repeat of the above to see if it works the same.
   for (size_t i = 0; i < 3; i++) {
     CommandInfo::URI uri;
@@ -1336,8 +1329,8 @@ TEST_F(FetcherCacheHttpTest, DISABLED_HttpCachedRecovery)
     EXPECT_TRUE(os::exists(path + taskName(i)));
 
     EXPECT_EQ(1u, fetcherProcess->cacheSize());
-    ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-    EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+    ASSERT_SOME(fetcherProcess->cacheFiles());
+    EXPECT_EQ(1u, fetcherProcess->cacheFiles()->size());
 
     // content-length requests: 1
     // downloads: 1
@@ -1388,13 +1381,13 @@ TEST_F(FetcherCacheTest, SimpleEviction)
 
     if (i < countCacheEntries) {
       EXPECT_EQ(i + 1, fetcherProcess->cacheSize());
-      ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-      EXPECT_EQ(i+1u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+      ASSERT_SOME(fetcherProcess->cacheFiles());
+      EXPECT_EQ(i+1u, fetcherProcess->cacheFiles()->size());
     } else {
       EXPECT_EQ(countCacheEntries, fetcherProcess->cacheSize());
-      ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
+      ASSERT_SOME(fetcherProcess->cacheFiles());
       EXPECT_EQ(countCacheEntries,
-                fetcherProcess->cacheFiles(slaveId, flags)->size());
+                fetcherProcess->cacheFiles()->size());
     }
   }
 }
@@ -1422,7 +1415,7 @@ TEST_F(FetcherCacheTest, FallbackFromEviction)
   Future<FetcherInfo> fetcherInfo0;
   Future<FetcherInfo> fetcherInfo1;
   Future<FetcherInfo> fetcherInfo2;
-  EXPECT_CALL(*fetcherProcess, run(_, _, _, _, _))
+  EXPECT_CALL(*fetcherProcess, run(_, _, _, _))
     .WillOnce(DoAll(FutureArg<3>(&fetcherInfo0),
                     Invoke(fetcherProcess,
                            &MockFetcherProcess::unmocked_run)))
@@ -1476,8 +1469,8 @@ TEST_F(FetcherCacheTest, FallbackFromEviction)
   ASSERT_EQ(Bytes(growth), fetcherProcess->availableCacheSpace());
 
   EXPECT_EQ(1u, fetcherProcess->cacheSize());
-  ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-  EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+  ASSERT_SOME(fetcherProcess->cacheFiles());
+  EXPECT_EQ(1u, fetcherProcess->cacheFiles()->size());
 
 
   // Task 1:
@@ -1523,8 +1516,8 @@ TEST_F(FetcherCacheTest, FallbackFromEviction)
   ASSERT_EQ(Bytes(0u), fetcherProcess->availableCacheSpace());
 
   EXPECT_EQ(1u, fetcherProcess->cacheSize());
-  ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-  EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+  ASSERT_SOME(fetcherProcess->cacheFiles());
+  EXPECT_EQ(1u, fetcherProcess->cacheFiles()->size());
 
 
   // Task 2:
@@ -1568,8 +1561,8 @@ TEST_F(FetcherCacheTest, FallbackFromEviction)
             fetcherInfo2->items(0).action());
 
   EXPECT_EQ(1u, fetcherProcess->cacheSize());
-  ASSERT_SOME(fetcherProcess->cacheFiles(slaveId, flags));
-  EXPECT_EQ(1u, fetcherProcess->cacheFiles(slaveId, flags)->size());
+  ASSERT_SOME(fetcherProcess->cacheFiles());
+  EXPECT_EQ(1u, fetcherProcess->cacheFiles()->size());
 }
 
 // Tests LRU cache eviction strategy.
@@ -1628,7 +1621,7 @@ TEST_F(FetcherCacheTest, RemoveLRUCacheEntries)
   // FetcherProcess::cacheFiles returns all cache files that are in the cache
   // directory. We expect cmd1 and cmd2 to be there, cmd0 should have been
   // evicted.
-  Try<list<Path>> cacheFiles = fetcherProcess->cacheFiles(slaveId, flags);
+  Try<list<Path>> cacheFiles = fetcherProcess->cacheFiles();
   ASSERT_SOME(cacheFiles);
 
   bool cmd1Found = false;

http://git-wip-us.apache.org/repos/asf/mesos/blob/391ced51/src/tests/fetcher_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fetcher_tests.cpp b/src/tests/fetcher_tests.cpp
index 27ea724..b412415 100644
--- a/src/tests/fetcher_tests.cpp
+++ b/src/tests/fetcher_tests.cpp
@@ -93,11 +93,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, FileURI)
   CommandInfo::URI* uri = commandInfo.add_uris();
   uri->set_value("file://" + testFile);
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
   AWAIT_READY(fetch);
 
   EXPECT_TRUE(os::exists(localFile));
@@ -131,16 +130,13 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, ROOT_RootProtectedFileURI)
   CommandInfo::URI* uri = commandInfo.add_uris();
   uri->set_value("file://" + testFile);
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   AWAIT_FAILED(fetcher.fetch(
       containerId,
       commandInfo,
       os::getcwd(),
-      None(),
-      slaveId,
-      flags));
+      None()));
 }
 #endif // __WINDOWS__
 
@@ -165,11 +161,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, CustomOutputFileSubdirectory)
   uri->set_value("file://" + testFile);
   uri->set_output_file(customOutputFile);
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
   AWAIT_READY(fetch);
 
   EXPECT_TRUE(os::exists(localFile));
@@ -201,11 +196,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, AbsoluteCustomSubdirectoryFails)
   uri->set_value("file://" + testFile);
   uri->set_output_file(customOutputFile);
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
   AWAIT_FAILED(fetch);
 
   EXPECT_FALSE(os::exists(localFile));
@@ -238,11 +232,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, InvalidUser)
   CommandInfo::URI* uri = commandInfo.add_uris();
   uri->set_value("file://" + testFile);
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
   AWAIT_FAILED(fetch);
 
   // See FetcherProcess::fetch(), the message must mention "chown" in
@@ -272,11 +265,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, NonExistingFile)
   CommandInfo::URI* uri = commandInfo.add_uris();
   uri->set_value("file://" + testFile);
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
   AWAIT_FAILED(fetch);
 
   // See FetcherProcess::run().
@@ -298,11 +290,10 @@ TEST_F(FetcherTest, MalformedURI)
   CommandInfo::URI* uri = commandInfo.add_uris();
   uri->set_value("lala://nopath");
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
   AWAIT_FAILED(fetch);
 
   // See Fetcher::basename().
@@ -330,11 +321,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, AbsoluteFilePath)
   CommandInfo::URI* uri = commandInfo.add_uris();
   uri->set_value(testPath);
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
   AWAIT_READY(fetch);
 
   EXPECT_TRUE(os::exists(localFile));
@@ -361,22 +351,22 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, RelativeFilePath)
   CommandInfo::URI* uri = commandInfo.add_uris();
   uri->set_value("test");
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher badFetcher(flags);
 
   // The first run must fail, because we have not set frameworks_home yet.
 
-  Future<Nothing> fetch1 = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+  Future<Nothing> fetch1 = badFetcher.fetch(
+      containerId, commandInfo, os::getcwd(), None());
   AWAIT_FAILED(fetch1);
 
   EXPECT_FALSE(os::exists(localFile));
 
   // The next run must succeed due to this flag.
   flags.frameworks_home = fromDir;
+  Fetcher goodFetcher(flags);
 
-  Future<Nothing> fetch2 = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+  Future<Nothing> fetch2 = goodFetcher.fetch(
+      containerId, commandInfo, os::getcwd(), None());
   AWAIT_READY(fetch2);
 
   EXPECT_TRUE(os::exists(localFile));
@@ -442,14 +432,13 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, OSNetUriTest)
   CommandInfo::URI* uri = commandInfo.add_uris();
   uri->set_value(stringify(url));
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   EXPECT_CALL(*http.process, test(_))
     .WillOnce(Return(http::OK()));
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
 
   AWAIT_READY(fetch);
 
@@ -489,15 +478,14 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, OSNetUriSpaceTest)
   // Add whitespace characters to the beginning of the URL.
   uri->set_value("\r\n\t " + stringify(url));
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   // Verify that the intended endpoint is hit.
   EXPECT_CALL(*http.process, test(_))
     .WillOnce(Return(http::OK()));
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
 
   AWAIT_READY(fetch);
 
@@ -525,11 +513,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, FileLocalhostURI)
   CommandInfo::URI* uri = commandInfo.add_uris();
   uri->set_value(path::join("file://localhost", testFile));
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
   AWAIT_READY(fetch);
 
   EXPECT_TRUE(os::exists(localFile));
@@ -557,11 +544,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, NoExtractNotExecutable)
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
   AWAIT_READY(fetch);
 
   string basename = Path(path.get()).basename();
@@ -596,11 +582,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, NoExtractExecutable)
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
 
   AWAIT_READY(fetch);
 
@@ -647,11 +632,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, ExtractNotExecutable)
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
 
   AWAIT_READY(fetch);
 
@@ -699,11 +683,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, ExtractTar)
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
 
   AWAIT_READY(fetch);
 
@@ -737,11 +720,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, ExtractGzipFile)
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
 
   AWAIT_READY(fetch);
 
@@ -785,16 +767,13 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, UNZIP_ExtractFile)
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
       containerId,
       commandInfo,
       os::getcwd(),
-      None(),
-      slaveId,
-      flags);
+      None());
 
   AWAIT_READY(fetch);
 
@@ -839,16 +818,13 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, UNZIP_ExtractInvalidFile)
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
       containerId,
       commandInfo,
       os::getcwd(),
-      None(),
-      slaveId,
-      flags);
+      None());
 
   AWAIT_FAILED(fetch);
 
@@ -897,16 +873,13 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
       containerId,
       commandInfo,
       os::getcwd(),
-      None(),
-      slaveId,
-      flags);
+      None());
 
   AWAIT_READY(fetch);
 
@@ -941,11 +914,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, UseCustomOutputFile)
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
 
   AWAIT_READY(fetch);
 
@@ -981,11 +953,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, CustomGzipOutputFile)
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
 
   AWAIT_READY(fetch);
 
@@ -1072,11 +1043,10 @@ TEST_F(FetcherTest, HdfsURI)
   CommandInfo::URI* uri = commandInfo.add_uris();
   uri->set_value(path::join("hdfs://localhost", testFile));
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
 
   AWAIT_READY(fetch);
 
@@ -1126,11 +1096,10 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(FetcherTest, SSLEnvironmentSpillover)
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
 
-  Fetcher fetcher;
-  SlaveID slaveId;
+  Fetcher fetcher(flags);
 
   Future<Nothing> fetch = fetcher.fetch(
-      containerId, commandInfo, os::getcwd(), None(), slaveId, flags);
+      containerId, commandInfo, os::getcwd(), None());
 
   // The mesos-fetcher runnable will fail initializing libprocess if
   // the SSL environment spilled over. Such failure would cause it to


[16/16] mesos git commit: Updated all tests that use Containerizer::launch(...).

Posted by jo...@apache.org.
Updated all tests that use Containerizer::launch(...).

This sweeps through the unit tests and translates all existing
tests that use the two variants of Containerizer::launch(...)
(i.e. nested and non-nested) and translates the arguments to the
new interface.  Some of the  fetcher interface changes are also
addressed in this sweep.

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


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

Branch: refs/heads/master
Commit: 74c61cebacb0c24e0a16a708d7ba958b29aeebf3
Parents: 391ced5
Author: Joseph Wu <jo...@apache.org>
Authored: Thu May 4 16:56:56 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:08 2017 -0700

----------------------------------------------------------------------
 src/tests/api_tests.cpp                         |  16 +--
 src/tests/check_tests.cpp                       |  12 +-
 src/tests/command_executor_tests.cpp            |   2 +-
 src/tests/container_logger_tests.cpp            |  10 +-
 .../containerizer/cgroups_isolator_tests.cpp    |  20 +--
 src/tests/containerizer/cni_isolator_tests.cpp  |   2 +-
 .../composing_containerizer_tests.cpp           |  34 ++---
 src/tests/containerizer/cpu_isolator_tests.cpp  |   4 +-
 .../docker_volume_isolator_tests.cpp            |   6 +-
 .../environment_secret_isolator_tests.cpp       |   2 +-
 src/tests/containerizer/isolator_tests.cpp      |  28 ++---
 .../linux_filesystem_isolator_tests.cpp         | 126 ++++++++-----------
 .../containerizer/memory_isolator_tests.cpp     |   2 +-
 .../containerizer/memory_pressure_tests.cpp     |   4 +-
 src/tests/containerizer/port_mapping_tests.cpp  |  10 +-
 .../volume_image_isolator_tests.cpp             |  35 +++---
 .../volume_secret_isolator_tests.cpp            |  17 +--
 src/tests/containerizer/xfs_quota_tests.cpp     |  13 +-
 src/tests/disk_quota_tests.cpp                  |   6 +-
 src/tests/health_check_tests.cpp                |  22 ++--
 src/tests/hook_tests.cpp                        |  28 ++---
 src/tests/master_tests.cpp                      |   4 +-
 src/tests/slave_recovery_tests.cpp              |  68 +++++-----
 src/tests/slave_tests.cpp                       |  40 +++---
 24 files changed, 233 insertions(+), 278 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index faf3242..97a8cc9 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -3623,7 +3623,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentAPITest, NestedContainerLaunchFalse)
 
   {
     // Return false here to indicate "unsupported".
-    EXPECT_CALL(containerizer, launch(_, _, _, _, _, _))
+    EXPECT_CALL(containerizer, launch(_, _, _, _))
       .WillOnce(Return(Future<bool>(false)));
 
     v1::agent::Call call;
@@ -4055,7 +4055,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentAPITest, LaunchNestedContainerSession)
   ASSERT_SOME(master);
 
   slave::Flags flags = CreateSlaveFlags();
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, false, &fetcher);
@@ -4167,7 +4167,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   ASSERT_SOME(master);
 
   slave::Flags flags = CreateSlaveFlags();
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> mesosContainerizer =
     MesosContainerizer::create(flags, false, &fetcher);
@@ -4272,7 +4272,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   ASSERT_SOME(master);
 
   slave::Flags flags = CreateSlaveFlags();
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, false, &fetcher);
@@ -4389,7 +4389,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   ASSERT_SOME(master);
 
   slave::Flags flags = CreateSlaveFlags();
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, false, &fetcher);
@@ -4693,7 +4693,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
     acl->mutable_users()->set_type(mesos::ACL::Entity::NONE);
   }
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, false, &fetcher);
@@ -5056,7 +5056,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(AgentAPIStreamingTest,
   ASSERT_SOME(master);
 
   slave::Flags flags = CreateSlaveFlags();
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, false, &fetcher);
@@ -5300,7 +5300,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   ASSERT_SOME(master);
 
   slave::Flags flags = CreateSlaveFlags();
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, false, &fetcher);

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/check_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/check_tests.cpp b/src/tests/check_tests.cpp
index 68aa29a..d85d1c3 100644
--- a/src/tests/check_tests.cpp
+++ b/src/tests/check_tests.cpp
@@ -1311,7 +1311,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // We have to explicitly create a `Containerizer` in non-local mode,
   // because `LaunchNestedContainerSession` (used by command checks)
@@ -1505,7 +1505,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // We have to explicitly create a `Containerizer` in non-local mode,
   // because `LaunchNestedContainerSession` (used by command checks)
@@ -1666,7 +1666,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // We have to explicitly create a `Containerizer` in non-local mode,
   // because `LaunchNestedContainerSession` (used by command checks)
@@ -1811,7 +1811,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // We have to explicitly create a `Containerizer` in non-local mode,
   // because `LaunchNestedContainerSession` (used by command checks)
@@ -1979,7 +1979,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(DefaultExecutorCheckTest, CommandCheckTimeout)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // We have to explicitly create a `Containerizer` in non-local mode,
   // because `LaunchNestedContainerSession` (used by command checks)
@@ -2127,7 +2127,7 @@ TEST_F(DefaultExecutorCheckTest, CommandCheckAndHealthCheckNoShadowing)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // We have to explicitly create a `Containerizer` in non-local mode,
   // because `LaunchNestedContainerSession` (used by command checks)

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/command_executor_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/command_executor_tests.cpp b/src/tests/command_executor_tests.cpp
index da4b653..382606a 100644
--- a/src/tests/command_executor_tests.cpp
+++ b/src/tests/command_executor_tests.cpp
@@ -353,7 +353,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(HTTPCommandExecutorTest, TerminateWithACK)
   slave::Flags flags = CreateSlaveFlags();
   flags.http_command_executor = true;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, false, &fetcher);

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/container_logger_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/container_logger_tests.cpp b/src/tests/container_logger_tests.cpp
index 28436b6..2ebff80 100644
--- a/src/tests/container_logger_tests.cpp
+++ b/src/tests/container_logger_tests.cpp
@@ -147,7 +147,7 @@ TEST_F(ContainerLoggerTest, DefaultToSandbox)
   // We'll need access to these flags later.
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // We use an actual containerizer + executor since we want something to run.
   Try<MesosContainerizer*> _containerizer =
@@ -248,7 +248,7 @@ TEST_F(ContainerLoggerTest, LOGROTATE_RotateInSandbox)
   // Use the non-default container logger that rotates logs.
   flags.container_logger = LOGROTATE_CONTAINER_LOGGER_NAME;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // We use an actual containerizer + executor since we want something to run.
   Try<MesosContainerizer*> _containerizer =
@@ -398,7 +398,7 @@ TEST_F(ContainerLoggerTest, LOGROTATE_CustomRotateOptions)
   // Use the non-default container logger that rotates logs.
   flags.container_logger = LOGROTATE_CONTAINER_LOGGER_NAME;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // We use an actual containerizer + executor since we want something to run.
   Try<MesosContainerizer*> _containerizer =
@@ -505,7 +505,7 @@ TEST_F(ContainerLoggerTest, LOGROTATE_ModuleFDOwnership)
   // Use the non-default container logger that rotates logs.
   flags.container_logger = LOGROTATE_CONTAINER_LOGGER_NAME;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // We use an actual containerizer + executor since we want something to run.
   Try<MesosContainerizer*> _containerizer =
@@ -629,7 +629,7 @@ TEST_P(UserContainerLoggerTest, ROOT_LOGROTATE_RotateWithSwitchUserTrueOrFalse)
       flags.work_dir, S_IRWXU | S_IRGRP | S_IXGRP | S_IROTH | S_IXOTH);
   ASSERT_SOME(chmod);
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // We use an actual containerizer + executor since we want something to run.
   Try<MesosContainerizer*> _containerizer =

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/containerizer/cgroups_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/cgroups_isolator_tests.cpp b/src/tests/containerizer/cgroups_isolator_tests.cpp
index 4e1d027..6724211 100644
--- a/src/tests/containerizer/cgroups_isolator_tests.cpp
+++ b/src/tests/containerizer/cgroups_isolator_tests.cpp
@@ -119,7 +119,7 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_PERF_NET_CLS_UserCgroup)
     "docker/runtime,"
     "filesystem/linux";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -244,7 +244,7 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_RevocableCpu)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "cgroups/cpu";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -359,7 +359,7 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_CFS_EnableCfs)
   // Enable CFS to cap CPU utilization.
   flags.cgroups_enable_cfs = true;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -568,7 +568,7 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_PidsAndTids)
   flags.isolation = "cgroups/cpu";
   flags.cgroups_cpu_enable_pids_and_tids_count = true;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -753,7 +753,7 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_NET_CLS_Isolate)
   flags.cgroups_net_cls_primary_handle = stringify(primary);
   flags.cgroups_net_cls_secondary_handles = "0xffff,0xffff";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -880,7 +880,7 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_NET_CLS_ContainerStatus)
   flags.cgroups_net_cls_primary_handle = stringify(0x0012);
   flags.cgroups_net_cls_secondary_handles = "0x0011,0x0012";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -976,7 +976,7 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_PERF_Sample)
   flags.perf_interval = Milliseconds(500);
   flags.isolation = "cgroups/perf_event";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -1087,7 +1087,7 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_PERF_PerfForward)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "cgroups/cpu,cgroups/mem";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -1248,7 +1248,7 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_MemoryForward)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "cgroups/cpu";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -1404,7 +1404,7 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_MemoryBackward)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "cgroups/cpu,cgroups/mem";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/containerizer/cni_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/cni_isolator_tests.cpp b/src/tests/containerizer/cni_isolator_tests.cpp
index 565e58a..3a5f4eb 100644
--- a/src/tests/containerizer/cni_isolator_tests.cpp
+++ b/src/tests/containerizer/cni_isolator_tests.cpp
@@ -289,7 +289,7 @@ TEST_F(CniIsolatorTest, ROOT_VerifyCheckpointedInfo)
   flags.network_cni_plugins_dir = cniPluginDir;
   flags.network_cni_config_dir = cniConfigDir;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/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 d7fd621..61e47e9 100644
--- a/src/tests/containerizer/composing_containerizer_tests.cpp
+++ b/src/tests/containerizer/composing_containerizer_tests.cpp
@@ -85,7 +85,7 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
 
   Promise<bool> launchPromise;
 
-  EXPECT_CALL(*mockContainerizer1, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(*mockContainerizer1, launch(_, _, _, _))
     .WillOnce(Return(launchPromise.future()));
 
   Future<Nothing> destroy;
@@ -96,13 +96,9 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
 
   Future<bool> launched = containerizer.launch(
       containerId,
-      taskInfo,
-      executorInfo,
-      "dir",
-      "user",
-      slaveId,
+      createContainerConfig(taskInfo, executorInfo, "dir", "user"),
       environment,
-      false);
+      None());
 
   Resources resources = Resources::parse("cpus:1;mem:256").get();
 
@@ -110,7 +106,7 @@ TEST_F(ComposingContainerizerTest, DestroyDuringUnsupportedLaunchLoop)
 
   Future<bool> destroyed = containerizer.destroy(containerId);
 
-  EXPECT_CALL(*mockContainerizer2, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(*mockContainerizer2, launch(_, _, _, _))
     .Times(0);
 
   // We make sure the destroy is being called on the first containerizer.
@@ -155,7 +151,7 @@ TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
 
   Promise<bool> launchPromise;
 
-  EXPECT_CALL(*mockContainerizer1, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(*mockContainerizer1, launch(_, _, _, _))
     .WillOnce(Return(launchPromise.future()));
 
   Future<Nothing> destroy;
@@ -166,13 +162,9 @@ TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
 
   Future<bool> launched = containerizer.launch(
       containerId,
-      taskInfo,
-      executorInfo,
-      "dir",
-      "user",
-      slaveId,
+      createContainerConfig(taskInfo, executorInfo, "dir", "user"),
       environment,
-      false);
+      None());
 
   Resources resources = Resources::parse("cpus:1;mem:256").get();
 
@@ -180,7 +172,7 @@ TEST_F(ComposingContainerizerTest, DestroyDuringSupportedLaunchLoop)
 
   Future<bool> destroyed = containerizer.destroy(containerId);
 
-  EXPECT_CALL(*mockContainerizer2, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(*mockContainerizer2, launch(_, _, _, _))
     .Times(0);
 
   // We make sure the destroy is being called on the first containerizer.
@@ -221,7 +213,7 @@ TEST_F(ComposingContainerizerTest, DestroyAfterLaunchLoop)
 
   Promise<bool> launchPromise;
 
-  EXPECT_CALL(*mockContainerizer1, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(*mockContainerizer1, launch(_, _, _, _))
     .WillOnce(Return(launchPromise.future()));
 
   Future<Nothing> destroy;
@@ -232,13 +224,9 @@ TEST_F(ComposingContainerizerTest, DestroyAfterLaunchLoop)
 
   Future<bool> launched = containerizer.launch(
       containerId,
-      taskInfo,
-      executorInfo,
-      "dir",
-      "user",
-      slaveId,
+      createContainerConfig(taskInfo, executorInfo, "dir", "user"),
       environment,
-      false);
+      None());
 
   Resources resources = Resources::parse("cpus:1;mem:256").get();
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/containerizer/cpu_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/cpu_isolator_tests.cpp b/src/tests/containerizer/cpu_isolator_tests.cpp
index 3c6f748..0e86318 100644
--- a/src/tests/containerizer/cpu_isolator_tests.cpp
+++ b/src/tests/containerizer/cpu_isolator_tests.cpp
@@ -72,7 +72,7 @@ TEST_P(CpuIsolatorTest, ROOT_UserCpuUsage)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = GetParam();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -163,7 +163,7 @@ TEST_P(CpuIsolatorTest, ROOT_SystemCpuUsage)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = GetParam();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/containerizer/docker_volume_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/docker_volume_isolator_tests.cpp b/src/tests/containerizer/docker_volume_isolator_tests.cpp
index b47a6b5..866af61 100644
--- a/src/tests/containerizer/docker_volume_isolator_tests.cpp
+++ b/src/tests/containerizer/docker_volume_isolator_tests.cpp
@@ -151,6 +151,8 @@ protected:
       const slave::Flags& flags,
       const Owned<DriverClient>& mockClient)
   {
+    fetcher.reset(new Fetcher(flags));
+
     Try<Isolator*> linuxIsolator_ =
       LinuxFilesystemIsolatorProcess::create(flags);
 
@@ -199,7 +201,7 @@ protected:
     Try<MesosContainerizer*> containerizer = MesosContainerizer::create(
         flags,
         true,
-        &fetcher,
+        fetcher.get(),
         std::move(launcher),
         provisioner->share(),
         {std::move(linuxIsolator),
@@ -214,7 +216,7 @@ protected:
   }
 
 private:
-  Fetcher fetcher;
+  Owned<Fetcher> fetcher;
 };
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/containerizer/environment_secret_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/environment_secret_isolator_tests.cpp b/src/tests/containerizer/environment_secret_isolator_tests.cpp
index 6190040..b034cee 100644
--- a/src/tests/containerizer/environment_secret_isolator_tests.cpp
+++ b/src/tests/containerizer/environment_secret_isolator_tests.cpp
@@ -54,7 +54,7 @@ TEST_F(EnvironmentSecretIsolatorTest, ResolveSecret)
 
   mesos::internal::slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
   Try<SecretResolver*> secretResolver = SecretResolver::create();
   EXPECT_SOME(secretResolver);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/containerizer/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index 355e15f..f3c541c 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -71,8 +71,10 @@ public:
     slave::Flags flags = CreateSlaveFlags();
     flags.isolation = isolation;
 
+    fetcher.reset(new Fetcher(flags));
+
     Try<MesosContainerizer*> _containerizer =
-      MesosContainerizer::create(flags, false, &fetcher);
+      MesosContainerizer::create(flags, false, fetcher.get());
 
     if (_containerizer.isError()) {
       return Error(_containerizer.error());
@@ -94,7 +96,7 @@ public:
   }
 
   string directory;
-  Fetcher fetcher;
+  Owned<Fetcher> fetcher;
   ContainerID containerId;
 };
 
@@ -111,13 +113,12 @@ TEST_F(NamespacesIsolatorTest, ROOT_PidNamespace)
 
   process::Future<bool> launch = containerizer.get()->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", command),
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", command),
+          directory),
       std::map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(launch);
   ASSERT_TRUE(launch.get());
@@ -178,13 +179,12 @@ TEST_F(NamespacesIsolatorTest, ROOT_IPCNamespace)
 
   process::Future<bool> launch = containerizer.get()->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", command),
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", command),
+          directory),
       std::map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(launch);
   ASSERT_TRUE(launch.get());

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/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 70a0dce..803758c 100644
--- a/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
+++ b/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
@@ -69,11 +69,7 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
-class LinuxFilesystemIsolatorTest : public MesosTest
-{
-protected:
-  Fetcher fetcher;
-};
+class LinuxFilesystemIsolatorTest : public MesosTest {};
 
 
 // This test verifies that the root filesystem of the container is
@@ -89,6 +85,8 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ChangeRootFilesystem)
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -110,13 +108,9 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ChangeRootFilesystem)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(launch);
 
@@ -142,6 +136,8 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_Metrics)
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -164,13 +160,9 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_Metrics)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(launch);
 
@@ -205,6 +197,8 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromSandbox)
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -228,13 +222,9 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromSandbox)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(launch);
 
@@ -263,6 +253,8 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromHost)
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -289,13 +281,9 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromHost)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(launch);
 
@@ -322,6 +310,8 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_FileVolumeFromHost)
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -348,13 +338,9 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_FileVolumeFromHost)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY_FOR(launch, Seconds(60));
 
@@ -381,6 +367,8 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromHostSandboxMountPoint)
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -407,13 +395,9 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromHostSandboxMountPoint)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(launch);
 
@@ -440,6 +424,8 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_FileVolumeFromHostSandboxMountPoint)
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -466,13 +452,9 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_FileVolumeFromHostSandboxMountPoint)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY_FOR(launch, Seconds(60));
 
@@ -498,6 +480,8 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithRootFilesystem)
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -533,13 +517,9 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithRootFilesystem)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(launch);
 
@@ -567,6 +547,8 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithoutRootFilesystem)
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -600,13 +582,9 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithoutRootFilesystem)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(launch);
 
@@ -635,6 +613,8 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_MultipleContainers)
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -674,13 +654,9 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_MultipleContainers)
 
   Future<bool> launch1 = containerizer->launch(
       containerId1,
-      None(),
-      executor1,
-      directory1,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor1, directory1),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(launch1);
 
@@ -695,13 +671,9 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_MultipleContainers)
 
   Future<bool> launch2 = containerizer->launch(
       containerId2,
-      None(),
-      executor2,
-      directory2,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor2, directory2),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(launch1);
 
@@ -819,6 +791,8 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeMountPointCleanup)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "filesystem/linux";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -852,13 +826,9 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeMountPointCleanup)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(launch);
 
@@ -1193,6 +1163,8 @@ TEST_F(LinuxFilesystemIsolatorMesosTest,
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/containerizer/memory_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/memory_isolator_tests.cpp b/src/tests/containerizer/memory_isolator_tests.cpp
index 0df4aa8..b7b7acd 100644
--- a/src/tests/containerizer/memory_isolator_tests.cpp
+++ b/src/tests/containerizer/memory_isolator_tests.cpp
@@ -72,7 +72,7 @@ TEST_P(MemoryIsolatorTest, ROOT_MemUsage)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = GetParam();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/containerizer/memory_pressure_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/memory_pressure_tests.cpp b/src/tests/containerizer/memory_pressure_tests.cpp
index c4ad779..8a43c4f 100644
--- a/src/tests/containerizer/memory_pressure_tests.cpp
+++ b/src/tests/containerizer/memory_pressure_tests.cpp
@@ -84,7 +84,7 @@ TEST_F(MemoryPressureMesosTest, CGROUPS_ROOT_Statistics)
   // We only care about memory cgroup for this test.
   flags.isolation = "cgroups/mem";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -201,7 +201,7 @@ TEST_F(MemoryPressureMesosTest, CGROUPS_ROOT_SlaveRecovery)
   // We only care about memory cgroup for this test.
   flags.isolation = "cgroups/mem";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/containerizer/port_mapping_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/port_mapping_tests.cpp b/src/tests/containerizer/port_mapping_tests.cpp
index d062f2f..16e015a 100644
--- a/src/tests/containerizer/port_mapping_tests.cpp
+++ b/src/tests/containerizer/port_mapping_tests.cpp
@@ -1849,8 +1849,6 @@ public:
     ContainerizerTest<MesosContainerizer>::TearDown();
   }
 
-  Fetcher fetcher;
-
   // Name of the host eth0 and lo.
   string eth0;
   string lo;
@@ -1875,6 +1873,8 @@ TEST_F(PortMappingMesosTest, CGROUPS_ROOT_RecoverMixedContainers)
   // we create in this test. Also, this will bypass MESOS-2554.
   slaveFlags.isolation = "cgroups/cpu,cgroups/mem";
 
+  Fetcher fetcher(slaveFlags);
+
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(slaveFlags, true, &fetcher);
 
@@ -2049,6 +2049,8 @@ TEST_F(PortMappingMesosTest, CGROUPS_ROOT_CleanUpOrphan)
   // NOTE: We add 'cgroups/cpu,cgroups/mem' to bypass MESOS-2554.
   flags.isolation = "cgroups/cpu,cgroups/mem,network/port_mapping";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -2174,6 +2176,8 @@ TEST_F(PortMappingMesosTest, ROOT_NetworkNamespaceHandleSymlink)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "network/port_mapping";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -2262,6 +2266,8 @@ TEST_F(PortMappingMesosTest, CGROUPS_ROOT_RecoverMixedKnownAndUnKnownOrphans)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "network/port_mapping";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/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 ad18844..2f91730 100644
--- a/src/tests/containerizer/volume_image_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_image_isolator_tests.cpp
@@ -61,7 +61,6 @@ protected:
   }
 
   bool nesting;
-  Fetcher fetcher;
 };
 
 
@@ -85,6 +84,8 @@ TEST_P(VolumeImageIsolatorTest, ROOT_ImageInVolumeWithoutRootFilesystem)
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -114,13 +115,9 @@ TEST_P(VolumeImageIsolatorTest, ROOT_ImageInVolumeWithoutRootFilesystem)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -133,10 +130,9 @@ TEST_P(VolumeImageIsolatorTest, ROOT_ImageInVolumeWithoutRootFilesystem)
 
     launch = containerizer->launch(
         nestedContainerId,
-        command,
-        container,
-        None(),
-        SlaveID());
+        createContainerConfig(command, container),
+        map<string, string>(),
+        None());
 
     AWAIT_ASSERT_TRUE(launch);
 
@@ -176,6 +172,8 @@ TEST_P(VolumeImageIsolatorTest, ROOT_ImageInVolumeWithRootFilesystem)
   flags.docker_store_dir = path::join(sandbox.get(), "store");
   flags.image_providers = "docker";
 
+  Fetcher fetcher(flags);
+
   Try<MesosContainerizer*> create =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -207,13 +205,9 @@ TEST_P(VolumeImageIsolatorTest, ROOT_ImageInVolumeWithRootFilesystem)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -226,10 +220,9 @@ TEST_P(VolumeImageIsolatorTest, ROOT_ImageInVolumeWithRootFilesystem)
 
     launch = containerizer->launch(
         nestedContainerId,
-        command,
-        container,
-        None(),
-        SlaveID());
+        createContainerConfig(command, container),
+        map<string, string>(),
+        None());
 
     AWAIT_ASSERT_TRUE(launch);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/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 073c392..a55af95 100644
--- a/src/tests/containerizer/volume_secret_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_secret_isolator_tests.cpp
@@ -159,7 +159,7 @@ TEST_P(VolumeSecretIsolatorTest, ROOT_SecretInVolumeWithRootFilesystem)
     flags.image_providers = "docker";
   }
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<SecretResolver*> secretResolver = SecretResolver::create();
   EXPECT_SOME(secretResolver);
@@ -206,13 +206,9 @@ TEST_P(VolumeSecretIsolatorTest, ROOT_SecretInVolumeWithRootFilesystem)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory),
       map<string, string>(),
-      false);
+      None());
 
   if (expectedContainerLaunchStatus == CONTAINER_LAUNCH_FAILURE) {
     AWAIT_FAILED(launch);
@@ -232,10 +228,9 @@ TEST_P(VolumeSecretIsolatorTest, ROOT_SecretInVolumeWithRootFilesystem)
 
   launch = containerizer->launch(
       nestedContainerId,
-      nestedCommand,
-      containerInfo,
-      None(),
-      state.id);
+      createContainerConfig(nestedCommand, containerInfo),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/containerizer/xfs_quota_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/xfs_quota_tests.cpp b/src/tests/containerizer/xfs_quota_tests.cpp
index b33fe8b..c220a6b 100644
--- a/src/tests/containerizer/xfs_quota_tests.cpp
+++ b/src/tests/containerizer/xfs_quota_tests.cpp
@@ -510,10 +510,11 @@ TEST_F(ROOT_XFS_QuotaTest, ResourceStatistics)
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  Fetcher fetcher;
-  Owned<MasterDetector> detector = master.get()->createDetector();
   slave::Flags flags = CreateSlaveFlags();
 
+  Fetcher fetcher(flags);
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -606,12 +607,12 @@ TEST_F(ROOT_XFS_QuotaTest, ResourceStatisticsNoEnforce)
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
-  Fetcher fetcher;
-  Owned<MasterDetector> detector = master.get()->createDetector();
-
   slave::Flags flags = CreateSlaveFlags();
   flags.enforce_container_disk_quota = false;
 
+  Fetcher fetcher(flags);
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
 
@@ -707,7 +708,7 @@ TEST_F(ROOT_XFS_QuotaTest, NoCheckpointRecovery)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/disk_quota_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/disk_quota_tests.cpp b/src/tests/disk_quota_tests.cpp
index a0a7093..3bf0508 100644
--- a/src/tests/disk_quota_tests.cpp
+++ b/src/tests/disk_quota_tests.cpp
@@ -354,7 +354,7 @@ TEST_F(DiskQuotaTest, NoQuotaEnforcement)
   flags.container_disk_watch_interval = Milliseconds(1);
   flags.enforce_container_disk_quota = false;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -455,7 +455,7 @@ TEST_F(DiskQuotaTest, ResourceStatistics)
   // the 'du' subprocess.
   flags.container_disk_watch_interval = Milliseconds(1);
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -609,7 +609,7 @@ TEST_F(DiskQuotaTest, SlaveRecovery)
   flags.isolation = "posix/cpu,posix/mem,disk/du";
   flags.container_disk_watch_interval = Milliseconds(1);
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/health_check_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/health_check_tests.cpp b/src/tests/health_check_tests.cpp
index 4e339bf..7917a22 100644
--- a/src/tests/health_check_tests.cpp
+++ b/src/tests/health_check_tests.cpp
@@ -545,7 +545,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(HealthCheckTest, ROOT_DOCKER_DockerHealthyTask)
 
   slave::Flags agentFlags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(agentFlags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(agentFlags.container_logger);
@@ -597,7 +597,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(HealthCheckTest, ROOT_DOCKER_DockerHealthyTask)
       containerInfo);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(containerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(containerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&containerizer,
                            &MockDockerContainerizer::_launch)));
@@ -818,7 +818,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 
   slave::Flags agentFlags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(agentFlags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(agentFlags.container_logger);
@@ -889,7 +889,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
       containerInfo);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(containerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(containerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&containerizer,
                            &MockDockerContainerizer::_launch)));
@@ -1841,7 +1841,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 
   slave::Flags agentFlags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(agentFlags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(agentFlags.container_logger);
@@ -1906,7 +1906,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   task.mutable_health_check()->CopyFrom(healthCheck);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(containerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(containerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&containerizer,
                            &MockDockerContainerizer::_launch)));
@@ -1972,7 +1972,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 
   slave::Flags agentFlags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(agentFlags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(agentFlags.container_logger);
@@ -2039,7 +2039,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   task.mutable_health_check()->CopyFrom(healthCheck);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(containerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(containerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&containerizer,
                            &MockDockerContainerizer::_launch)));
@@ -2108,7 +2108,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 
   slave::Flags agentFlags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(agentFlags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(agentFlags.container_logger);
@@ -2173,7 +2173,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   task.mutable_health_check()->CopyFrom(healthCheck);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(containerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(containerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&containerizer,
                            &MockDockerContainerizer::_launch)));
@@ -2247,7 +2247,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   flags.acls = acls;
 #endif // USE_SSL_SOCKET
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // We have to explicitly create a `Containerizer` in non-local mode,
   // because `LaunchNestedContainerSession` (used by command health

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/hook_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hook_tests.cpp b/src/tests/hook_tests.cpp
index f051fae..04edbf8 100644
--- a/src/tests/hook_tests.cpp
+++ b/src/tests/hook_tests.cpp
@@ -287,10 +287,12 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
     VerifySlaveExecutorEnvironmentDecorator)
 {
   const string& directory = os::getcwd(); // We're inside a temporary sandbox.
-  Fetcher fetcher;
+
+  slave::Flags flags = CreateSlaveFlags();
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
-    MesosContainerizer::create(CreateSlaveFlags(), false, &fetcher);
+    MesosContainerizer::create(flags, false, &fetcher);
 
   ASSERT_SOME(_containerizer);
   Owned<MesosContainerizer> containerizer(_containerizer.get());
@@ -305,7 +307,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   SlaveID slaveId = SlaveID();
 
   std::map<string, string> environment = executorEnvironment(
-      CreateSlaveFlags(),
+      flags,
       executorInfo,
       directory,
       slaveId,
@@ -318,13 +320,9 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   // command validates the hook.
   process::Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executorInfo,
-      directory,
-      None(),
-      slaveId,
+      createContainerConfig(None(), executorInfo, directory),
       environment,
-      false);
+      None());
 
   AWAIT_READY(launch);
   ASSERT_TRUE(launch.get());
@@ -652,7 +650,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -704,7 +702,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   task.mutable_container()->CopyFrom(containerInfo);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(containerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(containerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&containerizer,
                            &MockDockerContainerizer::_launch)));
@@ -761,7 +759,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -856,7 +854,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);
@@ -921,7 +919,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   tasks.push_back(task);
 
   Future<ContainerID> containerId;
-  EXPECT_CALL(containerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(containerizer, launch(_, _, _, _))
     .WillOnce(DoAll(FutureArg<0>(&containerId),
                     Invoke(&containerizer,
                            &MockDockerContainerizer::_launch)));
@@ -982,7 +980,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(HookTest, ROOT_DOCKER_VerifySlavePostFetchHook)
 
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<ContainerLogger*> logger =
     ContainerLogger::create(flags.container_logger);

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index 1dfe5fd..bacd44d 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -6353,7 +6353,7 @@ TEST_F(MasterTest, AgentRestartNoReregister)
   slave::Flags agentFlags = CreateSlaveFlags();
   agentFlags.credential = None();
 
-  mesos::internal::slave::Fetcher fetcher;
+  mesos::internal::slave::Fetcher fetcher(agentFlags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(agentFlags, true, &fetcher);
@@ -6550,7 +6550,7 @@ TEST_F(MasterTest, AgentRestartNoReregisterRateLimit)
   ASSERT_SOME(master);
 
   slave::Flags agentFlags = CreateSlaveFlags();
-  mesos::internal::slave::Fetcher fetcher;
+  mesos::internal::slave::Fetcher fetcher(agentFlags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(agentFlags, true, &fetcher);

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index 0aa87f5..aedc60a 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -175,7 +175,7 @@ TYPED_TEST(SlaveRecoveryTest, RecoverSlaveState)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -355,7 +355,7 @@ TYPED_TEST(SlaveRecoveryTest, RecoverStatusUpdateManager)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -438,7 +438,7 @@ TYPED_TEST(SlaveRecoveryTest, DISABLED_ReconnectHTTPExecutor)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.http_command_executor = true;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -536,7 +536,7 @@ TYPED_TEST(SlaveRecoveryTest, DISABLED_ROOT_CGROUPS_ReconnectDefaultExecutor)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.authenticate_http_readwrite = false;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -724,7 +724,7 @@ TYPED_TEST(SlaveRecoveryTest, ReconnectExecutor)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -814,7 +814,7 @@ TYPED_TEST(SlaveRecoveryTest, DISABLED_RecoverUnregisteredHTTPExecutor)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.http_command_executor = true;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -930,7 +930,7 @@ TYPED_TEST(SlaveRecoveryTest, RecoverUnregisteredExecutor)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -1042,7 +1042,7 @@ TYPED_TEST(SlaveRecoveryTest, KillTaskUnregisteredExecutor)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -1147,7 +1147,7 @@ TYPED_TEST(SlaveRecoveryTest, RecoverTerminatedHTTPExecutor)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.http_command_executor = true;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -1294,7 +1294,7 @@ TYPED_TEST(SlaveRecoveryTest, RecoverTerminatedExecutor)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -1424,7 +1424,7 @@ TYPED_TEST(SlaveRecoveryTest, DISABLED_RecoveryTimeout)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.recovery_timeout = Milliseconds(1);
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -1516,7 +1516,7 @@ TYPED_TEST(SlaveRecoveryTest, RecoverCompletedExecutor)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -1606,7 +1606,7 @@ TYPED_TEST(SlaveRecoveryTest, DISABLED_CleanupHTTPExecutor)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.http_command_executor = true;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -1711,7 +1711,7 @@ TYPED_TEST(SlaveRecoveryTest, CleanupExecutor)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -1812,7 +1812,7 @@ TYPED_TEST(SlaveRecoveryTest, RemoveNonCheckpointingFramework)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -1925,7 +1925,7 @@ TYPED_TEST(SlaveRecoveryTest, NonCheckpointingFramework)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -2021,7 +2021,7 @@ TYPED_TEST(SlaveRecoveryTest, DISABLED_KillTaskWithHTTPExecutor)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.http_command_executor = true;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -2140,7 +2140,7 @@ TYPED_TEST(SlaveRecoveryTest, KillTask)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -2260,7 +2260,7 @@ TYPED_TEST(SlaveRecoveryTest, Reboot)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.strict = false;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -2407,7 +2407,7 @@ TYPED_TEST(SlaveRecoveryTest, GCExecutor)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.strict = false;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -2541,7 +2541,7 @@ TYPED_TEST(SlaveRecoveryTest, ShutdownSlave)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -2660,7 +2660,7 @@ TYPED_TEST(SlaveRecoveryTest, ShutdownSlaveSIGUSR1)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -2766,7 +2766,7 @@ TYPED_TEST(SlaveRecoveryTest, RegisterDisconnectedSlave)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -2885,7 +2885,7 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileKillTask)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -2985,7 +2985,7 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileShutdownFramework)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -3127,7 +3127,7 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileTasksMissingFromSlave)
 
   EXPECT_CALL(allocator, addSlave(_, _, _, _, _, _));
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -3296,7 +3296,7 @@ TYPED_TEST(SlaveRecoveryTest, SchedulerFailover)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -3446,7 +3446,7 @@ TYPED_TEST(SlaveRecoveryTest, MasterFailover)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -3585,7 +3585,7 @@ TYPED_TEST(SlaveRecoveryTest, MultipleFrameworks)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<TypeParam*> _containerizer = TypeParam::create(flags, true, &fetcher);
   ASSERT_SOME(_containerizer);
@@ -3772,7 +3772,7 @@ TYPED_TEST(SlaveRecoveryTest, MultipleSlaves)
   // cgroups isolation is involved.
   flags1.isolation = "filesystem/posix,posix/mem,posix/cpu";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags1);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
 
@@ -3954,7 +3954,7 @@ TYPED_TEST(SlaveRecoveryTest, RestartBeforeContainerizerLaunch)
 
   // Expect the launch but don't do anything.
   Future<Nothing> launch;
-  EXPECT_CALL(containerizer1, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(containerizer1, launch(_, _, _, _))
     .WillOnce(DoAll(FutureSatisfy(&launch),
                     Return(Future<bool>())));
 
@@ -4018,7 +4018,7 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, ResourceStatistics)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -4119,7 +4119,7 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, CGROUPS_ROOT_PidNamespaceForward)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.isolation = "cgroups/cpu,cgroups/mem";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -4223,7 +4223,7 @@ TEST_F(MesosContainerizerSlaveRecoveryTest, CGROUPS_ROOT_PidNamespaceBackward)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.isolation = "cgroups/cpu,cgroups/mem,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);

http://git-wip-us.apache.org/repos/asf/mesos/blob/74c61ceb/src/tests/slave_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_tests.cpp b/src/tests/slave_tests.cpp
index b56b014..68ed8e9 100644
--- a/src/tests/slave_tests.cpp
+++ b/src/tests/slave_tests.cpp
@@ -95,6 +95,7 @@ using mesos::internal::protobuf::createLabel;
 using mesos::master::detector::MasterDetector;
 using mesos::master::detector::StandaloneMasterDetector;
 
+using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerTermination;
 
 using mesos::v1::scheduler::Call;
@@ -327,7 +328,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(SlaveTest, ShutdownUnregisteredExecutor)
   // be created.
   flags.isolation = "posix/cpu,posix/mem";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, false, &fetcher);
@@ -439,7 +440,7 @@ TEST_F(SlaveTest, ExecutorTimeoutCausedBySlowFetch)
   slave::Flags flags = CreateSlaveFlags();
   flags.hadoop_home = hadoopPath;
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer = MesosContainerizer::create(
       flags, true, &fetcher);
@@ -628,7 +629,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(SlaveTest, CommandTaskWithArguments)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "posix/cpu,posix/mem";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, false, &fetcher);
@@ -909,7 +910,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(SlaveTest, ROOT_LaunchTaskInfoWithContainerInfo)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "posix/cpu,posix/mem";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, false, &fetcher);
@@ -963,13 +964,9 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(SlaveTest, ROOT_LaunchTaskInfoWithContainerInfo)
   slaveID.set_value(UUID::random().toString());
   Future<bool> launch = containerizer->launch(
       containerId,
-      task,
-      executor,
-      sandbox.get(),
-      "nobody",
-      slaveID,
+      createContainerConfig(task, executor, sandbox.get(), "nobody"),
       map<string, string>(),
-      false);
+      None());
   AWAIT_READY(launch);
 
   // TODO(spikecurtis): With agent capabilities (MESOS-3362), the
@@ -996,7 +993,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "posix/cpu,posix/mem";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, false, &fetcher);
@@ -1096,7 +1093,7 @@ TEST_F(SlaveTest, DISABLED_ROOT_RunTaskWithCommandInfoWithUser)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "posix/cpu,posix/mem";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, false, &fetcher);
@@ -1423,7 +1420,7 @@ TEST_F(SlaveTest, MetricsSlaveLaunchErrors)
   JSON::Object snapshot = Metrics();
   EXPECT_EQ(0, snapshot.values["slave/container_launch_errors"]);
 
-  EXPECT_CALL(containerizer, launch(_, _, _, _, _, _, _, _))
+  EXPECT_CALL(containerizer, launch(_, _, _, _))
     .WillOnce(Return(Failure("Injected failure")));
 
   Future<TaskStatus> failureUpdate;
@@ -4908,7 +4905,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(SlaveTest, HTTPSchedulerSlaveRestart)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -6624,9 +6621,9 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(SlaveTest, DefaultExecutorCommandInfo)
   AWAIT_READY(offers);
   EXPECT_NE(0, offers->offers().size());
 
-  Future<ExecutorInfo> executorInfo_;
-  EXPECT_CALL(containerizer, launch(_, _, _, _, _, _, _, _))
-    .WillOnce(DoAll(FutureArg<2>(&executorInfo_),
+  Future<ContainerConfig> containerConfig;
+  EXPECT_CALL(containerizer, launch(_, _, _, _))
+    .WillOnce(DoAll(FutureArg<1>(&containerConfig),
                     Return(Future<bool>())));
 
   const v1::Offer& offer = offers->offers(0);
@@ -6658,13 +6655,16 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(SlaveTest, DefaultExecutorCommandInfo)
     mesos.send(call);
   }
 
-  AWAIT_READY(executorInfo_);
+  AWAIT_READY(containerConfig);
 
   // TODO(anand): Add a `strings::contains()` check to ensure
   // `MESOS_DEFAULT_EXECUTOR` is present in the command when
   // we add the executable for default executor.
-  ASSERT_TRUE(executorInfo_->has_command());
-  EXPECT_EQ(frameworkInfo.user(), executorInfo_->command().user());
+  ASSERT_TRUE(containerConfig->has_executor_info());
+  ASSERT_TRUE(containerConfig->executor_info().has_command());
+  EXPECT_EQ(
+      frameworkInfo.user(),
+      containerConfig->executor_info().command().user());
 }
 
 


[12/16] mesos git commit: Addressed a TODO about checkpointing in tests.

Posted by jo...@apache.org.
Addressed a TODO about checkpointing in tests.

Checkpointing only plays a role in tests when the containerizer
is destroyed and subsequently recovered.  This commit updates a
couple of test files to the new containerizer interface and
also turns checkpointing _off_ in tests which do not perform any
containerizer recovery.

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


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

Branch: refs/heads/master
Commit: b194d5a62e44711d0147ba5a65d8f0e7cd6c367d
Parents: 012e6a2
Author: Joseph Wu <jo...@apache.org>
Authored: Tue May 2 16:51:43 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:07 2017 -0700

----------------------------------------------------------------------
 .../containerizer/io_switchboard_tests.cpp      |  45 +-
 .../containerizer/mesos_containerizer_tests.cpp | 247 ++++------
 .../nested_mesos_containerizer_tests.cpp        | 483 +++++++++----------
 .../volume_sandbox_path_isolator_tests.cpp      |  30 +-
 4 files changed, 361 insertions(+), 444 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b194d5a6/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 f5c2157..777f76a 100644
--- a/src/tests/containerizer/io_switchboard_tests.cpp
+++ b/src/tests/containerizer/io_switchboard_tests.cpp
@@ -711,7 +711,7 @@ TEST_F(IOSwitchboardTest, ContainerAttach)
   flags.launcher = "posix";
   flags.isolation = "posix/cpu";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -744,13 +744,9 @@ TEST_F(IOSwitchboardTest, ContainerAttach)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executorInfo,
-      directory.get(),
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executorInfo, directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -777,7 +773,7 @@ TEST_F(IOSwitchboardTest, OutputRedirectionWithTTY)
   flags.launcher = "posix";
   flags.isolation = "posix/cpu";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -813,13 +809,9 @@ TEST_F(IOSwitchboardTest, OutputRedirectionWithTTY)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executorInfo,
-      directory.get(),
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executorInfo, directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -842,7 +834,7 @@ TEST_F(IOSwitchboardTest, KillSwitchboardContainerDestroyed)
   flags.launcher = "posix";
   flags.isolation = "posix/cpu";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -871,13 +863,9 @@ TEST_F(IOSwitchboardTest, KillSwitchboardContainerDestroyed)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executorInfo,
-      directory.get(),
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executorInfo, directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -887,11 +875,12 @@ TEST_F(IOSwitchboardTest, KillSwitchboardContainerDestroyed)
 
   launch = containerizer->launch(
       childContainerId,
-      createCommandInfo("sleep 1000"),
-      None(),
-      None(),
-      state.id,
-      mesos::slave::ContainerClass::DEBUG);
+      createContainerConfig(
+          createCommandInfo("sleep 1000"),
+          None(),
+          mesos::slave::ContainerClass::DEBUG),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -937,7 +926,7 @@ TEST_F(IOSwitchboardTest, DISABLED_RecoverThenKillSwitchboardContainerDestroyed)
   flags.launcher = "posix";
   flags.isolation = "posix/cpu";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
 
@@ -1058,7 +1047,7 @@ TEST_F(IOSwitchboardTest, ContainerAttachAfterSlaveRestart)
   flags.launcher = "posix";
   flags.isolation = "posix/cpu";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/b194d5a6/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 13e0f7e..d3a2bd7 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -106,7 +106,7 @@ TEST_F(MesosContainerizerTest, Launch)
   flags.launcher = "posix";
   flags.isolation = "posix/cpu";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -130,13 +130,12 @@ TEST_F(MesosContainerizerTest, Launch)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "exit 42", "cpus:1"),
-      directory.get(),
-      None(),
-      SlaveID(),
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "exit 42", "cpus:1"),
+          directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -155,7 +154,7 @@ TEST_F(MesosContainerizerTest, Destroy)
   flags.launcher = "posix";
   flags.isolation = "posix/cpu";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -179,13 +178,12 @@ TEST_F(MesosContainerizerTest, Destroy)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
-      directory.get(),
-      None(),
-      SlaveID(),
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+          directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -208,7 +206,7 @@ TEST_F(MesosContainerizerTest, StatusWithContainerID)
   flags.launcher = "posix";
   flags.isolation = "posix/cpu";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -232,13 +230,12 @@ TEST_F(MesosContainerizerTest, StatusWithContainerID)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
-      directory.get(),
-      None(),
-      SlaveID(),
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+          directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -264,7 +261,6 @@ public:
   // Construct a MesosContainerizer with MockIsolator(s) which return
   // the provided ContainerLaunchInfo for Isolator::prepare.
   Try<MesosContainerizer*> createContainerizer(
-      Fetcher* fetcher,
       const vector<Option<ContainerLaunchInfo>>& launchInfos)
   {
     vector<Owned<Isolator>> isolators;
@@ -281,6 +277,8 @@ public:
     slave::Flags flags = CreateSlaveFlags();
     flags.launcher = "posix";
 
+    fetcher.reset(new Fetcher(flags));
+
     Try<Launcher*> launcher_ = SubprocessLauncher::create(flags);
     if (launcher_.isError()) {
       return Error(launcher_.error());
@@ -296,37 +294,33 @@ public:
     return MesosContainerizer::create(
         flags,
         true,
-        fetcher,
+        fetcher.get(),
         std::move(launcher),
         provisioner->share(),
         isolators);
   }
 
   Try<MesosContainerizer*> createContainerizer(
-      Fetcher* fetcher,
       const Option<ContainerLaunchInfo>& launchInfo)
   {
     vector<Option<ContainerLaunchInfo>> launchInfos = {launchInfo};
-    return createContainerizer(fetcher, launchInfos);
+    return createContainerizer(launchInfos);
   }
+
+private:
+  Owned<Fetcher> fetcher;
 };
 
 
 // The isolator has a prepare command that succeeds.
 TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptSucceeds)
 {
-  string directory = os::getcwd(); // We're inside a temporary sandbox.
-  string file = path::join(directory, "child.script.executed");
-
-  Fetcher fetcher;
+  string file = path::join(sandbox.get(), "child.script.executed");
 
   ContainerLaunchInfo launchInfo;
   launchInfo.add_pre_exec_commands()->set_value("touch " + file);
 
-  Try<MesosContainerizer*> create = createContainerizer(
-      &fetcher,
-      launchInfo);
-
+  Try<MesosContainerizer*> create = createContainerizer(launchInfo);
   ASSERT_SOME(create);
 
   Owned<MesosContainerizer> containerizer(create.get());
@@ -336,13 +330,12 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptSucceeds)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "exit 0"),
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "exit 0"),
+          sandbox.get()),
       map<string, string>(),
-      false);
+      None());
 
   // Wait until the launch completes.
   AWAIT_READY(launch);
@@ -368,19 +361,13 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptSucceeds)
 // The isolator has a prepare command that fails.
 TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptFails)
 {
-  string directory = os::getcwd(); // We're inside a temporary sandbox.
-  string file = path::join(directory, "child.script.executed");
-
-  Fetcher fetcher;
+  string file = path::join(sandbox.get(), "child.script.executed");
 
   ContainerLaunchInfo launchInfo;
   launchInfo.add_pre_exec_commands()->set_value(
       "touch " + file + " && exit 1");
 
-  Try<MesosContainerizer*> create = createContainerizer(
-      &fetcher,
-      launchInfo);
-
+  Try<MesosContainerizer*> create = createContainerizer(launchInfo);
   ASSERT_SOME(create);
 
   Owned<MesosContainerizer> containerizer(create.get());
@@ -390,13 +377,12 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptFails)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "exit 0"),
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "exit 0"),
+          sandbox.get()),
       map<string, string>(),
-      false);
+      None());
 
   // Wait until the launch completes.
   AWAIT_READY(launch);
@@ -424,9 +410,8 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptFails)
 // launch should fail from the failing prepare command.
 TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
 {
-  string directory = os::getcwd(); // We're inside a temporary sandbox.
-  string file1 = path::join(directory, "child.script.executed.1");
-  string file2 = path::join(directory, "child.script.executed.2");
+  string file1 = path::join(sandbox.get(), "child.script.executed.1");
+  string file2 = path::join(sandbox.get(), "child.script.executed.2");
 
   vector<Option<ContainerLaunchInfo>> launchInfos;
 
@@ -441,12 +426,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
   launch2.add_pre_exec_commands()->set_value("touch " + file2 + " && exit 1");
   launchInfos.push_back(launch2);
 
-  Fetcher fetcher;
-
-  Try<MesosContainerizer*> create = createContainerizer(
-      &fetcher,
-      launchInfos);
-
+  Try<MesosContainerizer*> create = createContainerizer(launchInfos);
   ASSERT_SOME(create);
 
   Owned<MesosContainerizer> containerizer(create.get());
@@ -456,13 +436,12 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "exit 0"),
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "exit 0"),
+          sandbox.get()),
       map<string, string>(),
-      false);
+      None());
 
   // Wait until the launch completes.
   AWAIT_READY(launch);
@@ -496,10 +475,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ExecutorEnvironmentVariable)
   net::IP ip = net::IP(INADDR_LOOPBACK);
   os::setenv("LIBPROCESS_IP", stringify(ip));
 
-  string directory = os::getcwd(); // We're inside a temporary sandbox.
-  string file = path::join(directory, "child.script.executed");
-
-  Fetcher fetcher;
+  string file = path::join(sandbox.get(), "child.script.executed");
 
   ContainerLaunchInfo launchInfo;
 
@@ -509,10 +485,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ExecutorEnvironmentVariable)
   variable->set_name("TEST_ENVIRONMENT");
   variable->set_value(file);
 
-  Try<MesosContainerizer*> create = createContainerizer(
-      &fetcher,
-      launchInfo);
-
+  Try<MesosContainerizer*> create = createContainerizer(launchInfo);
   ASSERT_SOME(create);
 
   Owned<MesosContainerizer> containerizer(create.get());
@@ -534,7 +507,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ExecutorEnvironmentVariable)
   map<string, string> environment = executorEnvironment(
       flags,
       executorInfo,
-      directory,
+      sandbox.get(),
       slaveId,
       PID<Slave>(),
       None(),
@@ -542,13 +515,9 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ExecutorEnvironmentVariable)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executorInfo,
-      directory,
-      None(),
-      slaveId,
+      createContainerConfig(None(), executorInfo, sandbox.get()),
       environment,
-      false);
+      None());
 
   // Wait until the launch completes.
   AWAIT_READY(launch);
@@ -583,11 +552,9 @@ class MesosContainerizerExecuteTest : public MesosTest {};
 
 TEST_F(MesosContainerizerExecuteTest, IoRedirection)
 {
-  string directory = os::getcwd(); // We're inside a temporary sandbox.
-
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   // Use local=false so std{err,out} are redirected to files.
   Try<MesosContainerizer*> create = MesosContainerizer::create(
@@ -609,13 +576,12 @@ TEST_F(MesosContainerizerExecuteTest, IoRedirection)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", command),
-      directory,
-      None(),
-      SlaveID(),
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", command),
+          sandbox.get()),
       map<string, string>(),
-      false);
+      None());
 
   // Wait for the launch to complete.
   AWAIT_READY(launch);
@@ -634,11 +600,11 @@ TEST_F(MesosContainerizerExecuteTest, IoRedirection)
   // Check that std{err, out} was redirected.
   // NOTE: Fetcher uses GLOG, which outputs extra information to
   // stderr.
-  Try<string> stderr = os::read(path::join(directory, "stderr"));
+  Try<string> stderr = os::read(path::join(sandbox.get(), "stderr"));
   ASSERT_SOME(stderr);
   EXPECT_TRUE(strings::contains(stderr.get(), errMsg));
 
-  EXPECT_SOME_EQ(outMsg + "\n", os::read(path::join(directory, "stdout")));
+  EXPECT_SOME_EQ(outMsg + "\n", os::read(path::join(sandbox.get(), "stdout")));
 }
 
 
@@ -646,12 +612,10 @@ TEST_F(MesosContainerizerExecuteTest, IoRedirection)
 // are owned by the task user.
 TEST_F(MesosContainerizerExecuteTest, ROOT_SandboxFileOwnership)
 {
-  string sandbox = os::getcwd(); // We're inside a temporary sandbox.
-
   slave::Flags flags;
   flags.launcher_dir = getLauncherDir();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, false, &fetcher);
@@ -669,13 +633,9 @@ TEST_F(MesosContainerizerExecuteTest, ROOT_SandboxFileOwnership)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      sandbox,
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, sandbox.get()),
       map<string, string>(),
-      false);
+      None());
 
   // Wait for the launch to complete.
   AWAIT_READY(launch);
@@ -685,12 +645,12 @@ TEST_F(MesosContainerizerExecuteTest, ROOT_SandboxFileOwnership)
 
   // Verify that stdout is owned by the task user.
   struct stat s;
-  string stdoutPath = path::join(sandbox, "stdout");
+  string stdoutPath = path::join(sandbox.get(), "stdout");
   EXPECT_EQ(0, ::stat(stdoutPath.c_str(), &s));
   EXPECT_EQ(uid.get(), s.st_uid);
 
   // Verify that stderr is owned by the task user.
-  string stderrPath = path::join(sandbox, "stderr");
+  string stderrPath = path::join(sandbox.get(), "stderr");
   EXPECT_EQ(0, ::stat(stderrPath.c_str(), &s));
   EXPECT_EQ(uid.get(), s.st_uid);
 
@@ -720,7 +680,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhileFetching)
 
   Owned<Launcher> launcher(launcher_.get());
 
-  MockFetcherProcess* mockFetcherProcess = new MockFetcherProcess();
+  MockFetcherProcess* mockFetcherProcess = new MockFetcherProcess(flags);
   Owned<FetcherProcess> fetcherProcess(mockFetcherProcess);
   Fetcher fetcher(fetcherProcess);
 
@@ -743,7 +703,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhileFetching)
   Promise<Nothing> promise;
 
   // Letting exec hang to simulate a long fetch.
-  EXPECT_CALL(*mockFetcherProcess, _fetch(_, _, _, _, _, _))
+  EXPECT_CALL(*mockFetcherProcess, _fetch(_, _, _, _, _))
     .WillOnce(DoAll(FutureSatisfy(&fetch),
                     Return(promise.future())));
 
@@ -756,13 +716,12 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhileFetching)
 
   containerizer->launch(
       containerId,
-      taskInfo,
-      createExecutorInfo("executor", "exit 0"),
-      os::getcwd(),
-      None(),
-      SlaveID(),
+      createContainerConfig(
+          taskInfo,
+          createExecutorInfo("executor", "exit 0"),
+          sandbox.get()),
       map<string, string>(),
-      false);
+      None());
 
   Future<Option<ContainerTermination>> wait =
     containerizer->wait(containerId);
@@ -799,7 +758,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
     .WillOnce(DoAll(FutureSatisfy(&prepare),
                     Return(promise.future())));
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<Owned<Provisioner>> provisioner = Provisioner::create(flags);
   ASSERT_SOME(provisioner);
@@ -825,13 +784,12 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
 
   containerizer->launch(
       containerId,
-      taskInfo,
-      createExecutorInfo("executor", "exit 0"),
-      os::getcwd(),
-      None(),
-      SlaveID(),
+      createContainerConfig(
+          taskInfo,
+          createExecutorInfo("executor", "exit 0"),
+          sandbox.get()),
       map<string, string>(),
-      false);
+      None());
 
   Future<Option<ContainerTermination>> wait =
     containerizer->wait(containerId);
@@ -864,7 +822,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyUnknownContainer)
 {
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -925,7 +883,7 @@ TEST_F(MesosContainerizerProvisionerTest, ProvisionFailed)
   EXPECT_CALL(*provisioner, destroy(_))
     .WillOnce(Return(true));
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer = MesosContainerizer::create(
       flags,
@@ -963,13 +921,9 @@ TEST_F(MesosContainerizerProvisionerTest, ProvisionFailed)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      taskInfo,
-      executorInfo,
-      os::getcwd(),
-      None(),
-      SlaveID(),
+      createContainerConfig(taskInfo, executorInfo, sandbox.get()),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(provision);
 
@@ -1013,7 +967,7 @@ TEST_F(MesosContainerizerProvisionerTest, DestroyWhileProvisioning)
   EXPECT_CALL(*provisioner, destroy(_))
     .WillOnce(Return(true));
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer = MesosContainerizer::create(
       flags,
@@ -1051,13 +1005,9 @@ TEST_F(MesosContainerizerProvisionerTest, DestroyWhileProvisioning)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      taskInfo,
-      executorInfo,
-      os::getcwd(),
-      None(),
-      SlaveID(),
+      createContainerConfig(taskInfo, executorInfo, sandbox.get()),
       map<string, string>(),
-      false);
+      None());
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -1108,7 +1058,7 @@ TEST_F(MesosContainerizerProvisionerTest, IsolatorCleanupBeforePrepare)
   EXPECT_CALL(*isolator, cleanup(_))
     .Times(0);
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer = MesosContainerizer::create(
       flags,
@@ -1146,13 +1096,9 @@ TEST_F(MesosContainerizerProvisionerTest, IsolatorCleanupBeforePrepare)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      taskInfo,
-      executorInfo,
-      os::getcwd(),
-      None(),
-      SlaveID(),
+      createContainerConfig(taskInfo, executorInfo, sandbox.get()),
       map<string, string>(),
-      false);
+      None());
 
   Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
 
@@ -1198,7 +1144,7 @@ TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
 
   Owned<Launcher> launcher(testLauncher);
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<Owned<Provisioner>> provisioner = Provisioner::create(flags);
   ASSERT_SOME(provisioner);
@@ -1228,13 +1174,12 @@ TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      taskInfo,
-      createExecutorInfo("executor", "sleep 1000"),
-      os::getcwd(),
-      None(),
-      SlaveID(),
+      createContainerConfig(
+          taskInfo,
+          createExecutorInfo("executor", "sleep 1000"),
+          sandbox.get()),
       map<string, string>(),
-      false);
+      None());
 
   AWAIT_READY(launch);
 
@@ -1271,7 +1216,7 @@ class MesosContainerizerRecoverTest : public MesosTest {};
 TEST_F(MesosContainerizerRecoverTest, SkipRecoverNonMesosContainers)
 {
   slave::Flags flags = CreateSlaveFlags();
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -1371,7 +1316,7 @@ TEST_F(MesosContainerizerWaitTest, WaitUnknownContainer)
 {
   slave::Flags flags = CreateSlaveFlags();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,

http://git-wip-us.apache.org/repos/asf/mesos/blob/b194d5a6/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 d47e9ae..e466434 100644
--- a/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/nested_mesos_containerizer_tests.cpp
@@ -173,7 +173,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNested)
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -197,13 +197,12 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNested)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+          directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -214,10 +213,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNested)
 
   launch = containerizer->launch(
       nestedContainerId,
-      createCommandInfo("exit 42"),
-      None(),
-      None(),
-      state.id);
+      createContainerConfig(createCommandInfo("exit 42")),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -249,7 +247,7 @@ TEST_F(NestedMesosContainerizerTest,
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -282,13 +280,14 @@ TEST_F(NestedMesosContainerizerTest,
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      slave::paths::getForkedPidPath(
+          slave::paths::getMetaRootDir(flags.work_dir),
+          state.id,
+          executor.framework_id(),
+          executor.executor_id(),
+          containerId));
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -307,11 +306,12 @@ TEST_F(NestedMesosContainerizerTest,
 
     Future<bool> launchNested = containerizer->launch(
         nestedContainerId,
-        createCommandInfo("exit $" + envKey),
-        None(),
-        None(),
-        state.id,
-        ContainerClass::DEBUG);
+        createContainerConfig(
+            createCommandInfo("exit $" + envKey),
+            None(),
+            ContainerClass::DEBUG),
+        map<string, string>(),
+        None());
 
     AWAIT_ASSERT_TRUE(launchNested);
 
@@ -340,11 +340,12 @@ TEST_F(NestedMesosContainerizerTest,
 
     Future<bool> launchNested = containerizer->launch(
         nestedContainerId,
-        nestedCommand,
-        None(),
-        None(),
-        state.id,
-        ContainerClass::DEBUG);
+        createContainerConfig(
+            nestedCommand,
+            None(),
+            ContainerClass::DEBUG),
+        map<string, string>(),
+        None());
 
     AWAIT_ASSERT_TRUE(launchNested);
 
@@ -390,11 +391,12 @@ TEST_F(NestedMesosContainerizerTest,
 
     Future<bool> launchNested = containerizer->launch(
         nestedContainerId,
-        createCommandInfo("exit $" + envKey),
-        None(),
-        None(),
-        state.id,
-        ContainerClass::DEBUG);
+        createContainerConfig(
+            createCommandInfo("exit $" + envKey),
+            None(),
+            ContainerClass::DEBUG),
+        map<string, string>(),
+        None());
 
     AWAIT_ASSERT_TRUE(launchNested);
 
@@ -428,7 +430,7 @@ TEST_F(NestedMesosContainerizerTest,
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -472,13 +474,12 @@ TEST_F(NestedMesosContainerizerTest,
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(
+          None(),
+          executor,
+          directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -509,11 +510,12 @@ TEST_F(NestedMesosContainerizerTest,
 
     Future<bool> launchNested = containerizer->launch(
         nestedContainerId,
-        nestedCommand,
-        None(),
-        None(),
-        state.id,
-        ContainerClass::DEBUG);
+        createContainerConfig(
+            nestedCommand,
+            None(),
+            ContainerClass::DEBUG),
+        map<string, string>(),
+        None());
 
     AWAIT_ASSERT_TRUE(launchNested);
 
@@ -549,7 +551,7 @@ TEST_F(NestedMesosContainerizerTest,
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -596,13 +598,17 @@ TEST_F(NestedMesosContainerizerTest,
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(
+          None(),
+          executor,
+          directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      slave::paths::getForkedPidPath(
+          slave::paths::getMetaRootDir(flags.work_dir),
+          state.id,
+          executor.framework_id(),
+          executor.executor_id(),
+          containerId));
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -626,11 +632,12 @@ TEST_F(NestedMesosContainerizerTest,
 
     Future<bool> launchNested = containerizer->launch(
         nestedContainerId,
-        createCommandInfo("ls " + filename),
-        None(),
-        None(),
-        state.id,
-        ContainerClass::DEBUG);
+        createContainerConfig(
+            createCommandInfo("ls " + filename),
+            None(),
+            ContainerClass::DEBUG),
+        map<string, string>(),
+        None());
 
     AWAIT_ASSERT_TRUE(launchNested);
 
@@ -675,11 +682,12 @@ TEST_F(NestedMesosContainerizerTest,
 
     Future<bool> launchNested = containerizer->launch(
         nestedContainerId,
-        createCommandInfo("ls " + filename),
-        None(),
-        None(),
-        state.id,
-        ContainerClass::DEBUG);
+        createContainerConfig(
+            createCommandInfo("ls " + filename),
+            None(),
+            ContainerClass::DEBUG),
+        map<string, string>(),
+        None());
 
     AWAIT_ASSERT_TRUE(launchNested);
 
@@ -711,7 +719,7 @@ TEST_F(NestedMesosContainerizerTest,
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -735,13 +743,12 @@ TEST_F(NestedMesosContainerizerTest,
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+          directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -764,14 +771,13 @@ TEST_F(NestedMesosContainerizerTest,
   // 6) `ps`
   launch = containerizer->launch(
       nestedContainerId,
-      createCommandInfo(
+      createContainerConfig(createCommandInfo(
           "PS_LINES=`ps | wc -l`;"
           "if [ ${PS_LINES} -ne 6 ]; then"
           "  exit ${PS_LINES};"
-          "fi;"),
-      None(),
-      None(),
-      state.id);
+          "fi;")),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -790,15 +796,16 @@ TEST_F(NestedMesosContainerizerTest,
   // We expect to see much more than 6 lines of output from `ps`.
   launch = containerizer->launch(
       nestedContainerId,
-      createCommandInfo(
-          "PS_LINES=`ps | wc -l`;"
-          "if [ ${PS_LINES} -le 6 ]; then"
-          "  exit ${PS_LINES};"
-          "fi;"),
-      None(),
-      None(),
-      state.id,
-      ContainerClass::DEBUG);
+      createContainerConfig(
+          createCommandInfo(
+              "PS_LINES=`ps | wc -l`;"
+              "if [ ${PS_LINES} -le 6 ]; then"
+              "  exit ${PS_LINES};"
+              "fi;"),
+          None(),
+          ContainerClass::DEBUG),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -832,7 +839,7 @@ TEST_F(NestedMesosContainerizerTest,
 
   Owned<MasterDetector> detector = master.get()->createDetector();
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> _containerizer =
     MesosContainerizer::create(flags, true, &fetcher);
@@ -914,15 +921,16 @@ TEST_F(NestedMesosContainerizerTest,
   // image (but not on the host filesystem).
   Future<bool> launch = containerizer->launch(
       nestedContainerId,
-      createCommandInfo(
-          "LINES=`ls -la /etc/alpine-release | wc -l`;"
-          "if [ ${LINES} -ne 1 ]; then"
-          "  exit 1;"
-          "fi;"),
-      None(),
-      None(),
-      statusRunning->slave_id(),
-      ContainerClass::DEBUG);
+      createContainerConfig(
+          createCommandInfo(
+              "LINES=`ls -la /etc/alpine-release | wc -l`;"
+              "if [ ${LINES} -ne 1 ]; then"
+              "  exit 1;"
+              "fi;"),
+          None(),
+          ContainerClass::DEBUG),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -946,7 +954,7 @@ TEST_F(NestedMesosContainerizerTest,
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -975,13 +983,14 @@ TEST_F(NestedMesosContainerizerTest,
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory.get(),
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      slave::paths::getForkedPidPath(
+          slave::paths::getMetaRootDir(flags.work_dir),
+          state.id,
+          executor.framework_id(),
+          executor.executor_id(),
+          containerId));
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -998,11 +1007,12 @@ TEST_F(NestedMesosContainerizerTest,
 
   launch = containerizer->launch(
       nestedContainerId,
-      createCommandInfo("sleep 1000"),
-      None(),
-      None(),
-      state.id,
-      ContainerClass::DEBUG);
+      createContainerConfig(
+          createCommandInfo("sleep 1000"),
+          None(),
+          ContainerClass::DEBUG),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -1062,7 +1072,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyNested)
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -1086,13 +1096,12 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyNested)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+          directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -1103,10 +1112,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyNested)
 
   launch = containerizer->launch(
       nestedContainerId,
-      createCommandInfo("sleep 1000"),
-      None(),
-      None(),
-      state.id);
+      createContainerConfig(createCommandInfo("sleep 1000")),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -1141,7 +1149,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyParent)
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -1165,13 +1173,12 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyParent)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+          directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -1182,10 +1189,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_DestroyParent)
 
   launch = containerizer->launch(
       nestedContainerId,
-      createCommandInfo("sleep 1000"),
-      None(),
-      None(),
-      state.id);
+      createContainerConfig(createCommandInfo("sleep 1000")),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -1218,7 +1224,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentExit)
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -1260,13 +1266,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentExit)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   close(pipes[0]); // We're never going to read.
 
@@ -1279,10 +1281,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentExit)
 
   launch = containerizer->launch(
       nestedContainerId,
-      createCommandInfo("sleep 1000"),
-      None(),
-      None(),
-      state.id);
+      createContainerConfig(createCommandInfo("sleep 1000")),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -1317,7 +1318,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentSigterm)
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -1361,13 +1362,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentSigterm)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -1380,10 +1377,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_ParentSigterm)
 
   launch = containerizer->launch(
       nestedContainerId,
-      createCommandInfo("sleep 1000"),
-      None(),
-      None(),
-      state.id);
+      createContainerConfig(createCommandInfo("sleep 1000")),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -1427,7 +1423,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNested)
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -1456,13 +1452,14 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNested)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory.get(),
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      slave::paths::getForkedPidPath(
+          slave::paths::getMetaRootDir(flags.work_dir),
+          state.id,
+          executor.framework_id(),
+          executor.executor_id(),
+          containerId));
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -1479,10 +1476,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNested)
 
   launch = containerizer->launch(
       nestedContainerId,
-      createCommandInfo("sleep 1000"),
-      None(),
-      None(),
-      state.id);
+      createContainerConfig(createCommandInfo("sleep 1000")),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -1557,7 +1553,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverLauncherOrphans)
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -1608,7 +1604,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNestedLauncherOrphans)
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -1637,13 +1633,14 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_RecoverNestedLauncherOrphans)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory.get(),
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      slave::paths::getForkedPidPath(
+          slave::paths::getMetaRootDir(flags.work_dir),
+          state.id,
+          executor.framework_id(),
+          executor.executor_id(),
+          containerId));
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -1730,7 +1727,7 @@ TEST_F(NestedMesosContainerizerTest,
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -1802,7 +1799,7 @@ TEST_F(NestedMesosContainerizerTest,
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -1831,13 +1828,14 @@ TEST_F(NestedMesosContainerizerTest,
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory.get(),
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      slave::paths::getForkedPidPath(
+          slave::paths::getMetaRootDir(flags.work_dir),
+          state.id,
+          executor.framework_id(),
+          executor.executor_id(),
+          containerId));
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -1941,7 +1939,7 @@ TEST_F(NestedMesosContainerizerTest,
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -1970,13 +1968,14 @@ TEST_F(NestedMesosContainerizerTest,
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory.get(),
-      None(),
-      SlaveID(),
+      createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      slave::paths::getForkedPidPath(
+          slave::paths::getMetaRootDir(flags.work_dir),
+          state.id,
+          executor.framework_id(),
+          executor.executor_id(),
+          containerId));
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -1993,10 +1992,9 @@ TEST_F(NestedMesosContainerizerTest,
 
   launch = containerizer->launch(
       nestedContainerId1,
-      createCommandInfo("sleep 1000"),
-      None(),
-      None(),
-      state.id);
+      createContainerConfig(createCommandInfo("sleep 1000")),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -2101,7 +2099,7 @@ TEST_F(NestedMesosContainerizerTest,
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -2187,7 +2185,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_WaitAfterDestroy)
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -2198,8 +2196,6 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_WaitAfterDestroy)
 
   Owned<MesosContainerizer> containerizer(create.get());
 
-  SlaveID slaveId = SlaveID();
-
   // Launch a top-level container.
   ContainerID containerId;
   containerId.set_value(UUID::random().toString());
@@ -2209,13 +2205,12 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_WaitAfterDestroy)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
-      directory.get(),
-      None(),
-      slaveId,
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+          directory.get()),
       map<string, string>(),
-      true);
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -2226,10 +2221,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_WaitAfterDestroy)
 
   launch = containerizer->launch(
       nestedContainerId,
-      createCommandInfo("exit 42"),
-      None(),
-      None(),
-      slaveId);
+      createContainerConfig(createCommandInfo("exit 42")),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -2278,7 +2272,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_AgentEnvironmentNotLeaked)
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -2302,13 +2296,12 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_AgentEnvironmentNotLeaked)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+          directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -2336,10 +2329,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_AgentEnvironmentNotLeaked)
 
   launch = containerizer->launch(
       nestedContainerId,
-      command,
-      None(),
-      None(),
-      state.id);
+      createContainerConfig(command),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -2368,7 +2360,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNestedThreeLevels)
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -2392,13 +2384,12 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNestedThreeLevels)
 
   Future<bool> launch = containerizer->launch(
       level1ContainerId,
-      None(),
-      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+          directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -2408,10 +2399,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNestedThreeLevels)
 
   launch = containerizer->launch(
       level2ContainerId,
-      createCommandInfo("sleep 1000"),
-      None(),
-      None(),
-      state.id);
+      createContainerConfig(createCommandInfo("sleep 1000")),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -2421,10 +2411,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_LaunchNestedThreeLevels)
 
   launch = containerizer->launch(
       level3ContainerId,
-      createCommandInfo("exit 42"),
-      None(),
-      None(),
-      state.id);
+      createContainerConfig(createCommandInfo("exit 42")),
+      map<string, string>(),
+      None());
 
   Future<Option<ContainerTermination>> wait =
     containerizer->wait(level3ContainerId);
@@ -2451,7 +2440,7 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_Remove)
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -2475,13 +2464,12 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_Remove)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+          directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -2492,10 +2480,9 @@ TEST_F(NestedMesosContainerizerTest, ROOT_CGROUPS_Remove)
 
   launch = containerizer->launch(
       nestedContainerId,
-      createCommandInfo("true"),
-      None(),
-      None(),
-      state.id);
+      createContainerConfig(createCommandInfo("true")),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -2542,7 +2529,7 @@ TEST_F(NestedMesosContainerizerTest,
   flags.launcher = "linux";
   flags.isolation = "cgroups/cpu,filesystem/linux,namespaces/pid";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -2566,13 +2553,12 @@ TEST_F(NestedMesosContainerizerTest,
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      createExecutorInfo("executor", "sleep 1000", "cpus:1"),
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(
+          None(),
+          createExecutorInfo("executor", "sleep 1000", "cpus:1"),
+          directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to test not checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -2583,10 +2569,9 @@ TEST_F(NestedMesosContainerizerTest,
 
   launch = containerizer->launch(
       nestedContainerId,
-      createCommandInfo("true"),
-      None(),
-      None(),
-      state.id);
+      createContainerConfig(createCommandInfo("true")),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/b194d5a6/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 5f4e382..3228b9a 100644
--- a/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
+++ b/src/tests/containerizer/volume_sandbox_path_isolator_tests.cpp
@@ -54,7 +54,7 @@ TEST_F(VolumeSandboxPathIsolatorTest, SharedVolume)
   slave::Flags flags = CreateSlaveFlags();
   flags.isolation = "volume/sandbox_path";
 
-  Fetcher fetcher;
+  Fetcher fetcher(flags);
 
   Try<MesosContainerizer*> create = MesosContainerizer::create(
       flags,
@@ -80,13 +80,9 @@ TEST_F(VolumeSandboxPathIsolatorTest, SharedVolume)
 
   Future<bool> launch = containerizer->launch(
       containerId,
-      None(),
-      executor,
-      directory.get(),
-      None(),
-      state.id,
+      createContainerConfig(None(), executor, directory.get()),
       map<string, string>(),
-      true); // TODO(benh): Ever want to check not-checkpointing?
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -110,10 +106,11 @@ TEST_F(VolumeSandboxPathIsolatorTest, SharedVolume)
 
   launch = containerizer->launch(
       nestedContainerId1,
-      createCommandInfo("touch parent/file; sleep 1000"),
-      containerInfo,
-      None(),
-      state.id);
+      createContainerConfig(
+          createCommandInfo("touch parent/file; sleep 1000"),
+          containerInfo),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 
@@ -123,11 +120,12 @@ TEST_F(VolumeSandboxPathIsolatorTest, SharedVolume)
 
   launch = containerizer->launch(
       nestedContainerId2,
-      createCommandInfo(
-        "while true; do if [ -f parent/file ]; then exit 0; fi; done"),
-      containerInfo,
-      None(),
-      state.id);
+      createContainerConfig(
+          createCommandInfo(
+            "while true; do if [ -f parent/file ]; then exit 0; fi; done"),
+          containerInfo),
+      map<string, string>(),
+      None());
 
   AWAIT_ASSERT_TRUE(launch);
 


[09/16] mesos git commit: Combined launch paths in ComposingContainerizer.

Posted by jo...@apache.org.
Combined launch paths in ComposingContainerizer.

This eliminates one of the entrypoints for launching a container
in the `ComposingContainerizer`.  Nested and non-nested containers
are still launched the same way, where nested containers get launched
using their root container's containerizer.

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


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

Branch: refs/heads/master
Commit: 7015097559189b4497d3888d53f2f4ad825f2ea8
Parents: 17ffb97
Author: Joseph Wu <jo...@apache.org>
Authored: Mon Apr 10 18:17:27 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:07 2017 -0700

----------------------------------------------------------------------
 src/slave/containerizer/composing.cpp | 179 ++++++++---------------------
 src/slave/containerizer/composing.hpp |  19 +--
 2 files changed, 55 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/70150975/src/slave/containerizer/composing.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.cpp b/src/slave/containerizer/composing.cpp
index 0b6c76b..c6b6ddf 100644
--- a/src/slave/containerizer/composing.cpp
+++ b/src/slave/containerizer/composing.cpp
@@ -43,6 +43,7 @@ using std::string;
 using std::vector;
 
 using mesos::slave::ContainerClass;
+using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerTermination;
 
 namespace mesos {
@@ -66,21 +67,9 @@ public:
 
   Future<bool> launch(
       const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const string& directory,
-      const Option<string>& user,
-      const SlaveID& slaveId,
+      const ContainerConfig& config,
       const map<string, string>& environment,
-      bool checkpoint);
-
-  Future<bool> launch(
-      const ContainerID& containerId,
-      const CommandInfo& commandInfo,
-      const Option<ContainerInfo>& containerInfo,
-      const Option<string>& user,
-      const SlaveID& slaveId,
-      const Option<ContainerClass>& containerClass);
+      const Option<std::string>& pidCheckpointPath);
 
   Future<http::Connection> attach(
       const ContainerID& containerId);
@@ -114,16 +103,13 @@ private:
 
   Future<bool> _launch(
       const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const string& directory,
-      const Option<string>& user,
-      const SlaveID& slaveId,
+      const ContainerConfig& config,
       const map<string, string>& environment,
-      bool checkpoint,
+      const Option<std::string>& pidCheckpointPath,
       vector<Containerizer*>::iterator containerizer,
       bool launched);
 
+  // Continuation for nested containers.
   Future<bool> _launch(
       const ContainerID& containerId,
       bool launched);
@@ -184,43 +170,16 @@ Future<Nothing> ComposingContainerizer::recover(
 
 Future<bool> ComposingContainerizer::launch(
     const ContainerID& containerId,
-    const Option<TaskInfo>& taskInfo,
-    const ExecutorInfo& executorInfo,
-    const string& directory,
-    const Option<string>& user,
-    const SlaveID& slaveId,
+    const ContainerConfig& containerConfig,
     const map<string, string>& environment,
-    bool checkpoint)
+    const Option<std::string>& pidCheckpointPath)
 {
   return dispatch(process,
                   &ComposingContainerizerProcess::launch,
                   containerId,
-                  taskInfo,
-                  executorInfo,
-                  directory,
-                  user,
-                  slaveId,
+                  containerConfig,
                   environment,
-                  checkpoint);
-}
-
-
-Future<bool> ComposingContainerizer::launch(
-    const ContainerID& containerId,
-    const CommandInfo& commandInfo,
-    const Option<ContainerInfo>& containerInfo,
-    const Option<string>& user,
-    const SlaveID& slaveId,
-    const Option<ContainerClass>& containerClass)
-{
-  return dispatch(process,
-                  &ComposingContainerizerProcess::launch,
-                  containerId,
-                  commandInfo,
-                  containerInfo,
-                  user,
-                  slaveId,
-                  containerClass);
+                  pidCheckpointPath);
 }
 
 
@@ -351,13 +310,9 @@ Future<Nothing> ComposingContainerizerProcess::___recover()
 
 Future<bool> ComposingContainerizerProcess::_launch(
     const ContainerID& containerId,
-    const Option<TaskInfo>& taskInfo,
-    const ExecutorInfo& executorInfo,
-    const string& directory,
-    const Option<string>& user,
-    const SlaveID& slaveId,
+    const ContainerConfig& containerConfig,
     const map<string, string>& environment,
-    bool checkpoint,
+    const Option<std::string>& pidCheckpointPath,
     vector<Containerizer*>::iterator containerizer,
     bool launched)
 {
@@ -424,24 +379,16 @@ Future<bool> ComposingContainerizerProcess::_launch(
 
   return (*containerizer)->launch(
       containerId,
-      taskInfo,
-      executorInfo,
-      directory,
-      user,
-      slaveId,
+      containerConfig,
       environment,
-      checkpoint)
+      pidCheckpointPath)
     .then(defer(
         self(),
         &Self::_launch,
         containerId,
-        taskInfo,
-        executorInfo,
-        directory,
-        user,
-        slaveId,
+        containerConfig,
         environment,
-        checkpoint,
+        pidCheckpointPath,
         containerizer,
         lambda::_1));
 }
@@ -449,85 +396,61 @@ Future<bool> ComposingContainerizerProcess::_launch(
 
 Future<bool> ComposingContainerizerProcess::launch(
     const ContainerID& containerId,
-    const Option<TaskInfo>& taskInfo,
-    const ExecutorInfo& executorInfo,
-    const string& directory,
-    const Option<string>& user,
-    const SlaveID& slaveId,
+    const ContainerConfig& containerConfig,
     const map<string, string>& environment,
-    bool checkpoint)
+    const Option<std::string>& pidCheckpointPath)
 {
   if (containers_.contains(containerId)) {
     return Failure("Duplicate container found");
   }
 
-  // Try each containerizer. If none of them handle the
-  // TaskInfo/ExecutorInfo then return a Failure.
-  vector<Containerizer*>::iterator containerizer = containerizers_.begin();
-
   Container* container = new Container();
   container->state = LAUNCHING;
-  container->containerizer = *containerizer;
   containers_[containerId] = container;
 
-  return (*containerizer)->launch(
-      containerId,
-      taskInfo,
-      executorInfo,
-      directory,
-      user,
-      slaveId,
-      environment,
-      checkpoint)
-    .then(defer(self(),
-                &Self::_launch,
-                containerId,
-                taskInfo,
-                executorInfo,
-                directory,
-                user,
-                slaveId,
-                environment,
-                checkpoint,
-                containerizer,
-                lambda::_1));
-}
+  // For nested containers, use the containerizer that launched the
+  // root container. This code path uses a different continuation
+  // function because there is no need to try other containerizers.
+  if (containerId.has_parent()) {
+    ContainerID rootContainerId = protobuf::getRootContainerId(containerId);
+    if (!containers_.contains(rootContainerId)) {
+      return Failure(
+          "Root container " + stringify(rootContainerId) + " not found");
+    }
 
+    Containerizer* containerizer =
+      containers_.at(rootContainerId)->containerizer;
 
-Future<bool> ComposingContainerizerProcess::launch(
-          const ContainerID& containerId,
-          const CommandInfo& commandInfo,
-          const Option<ContainerInfo>& containerInfo,
-          const Option<string>& user,
-          const SlaveID& slaveId,
-          const Option<ContainerClass>& containerClass)
-{
-  ContainerID rootContainerId = protobuf::getRootContainerId(containerId);
+    container->containerizer = containerizer;
 
-  if (!containers_.contains(rootContainerId)) {
-    return Failure(
-        "Root container " + stringify(rootContainerId) + " not found");
+    return containerizer->launch(
+        containerId,
+        containerConfig,
+        environment,
+        pidCheckpointPath)
+      .then(defer(self(),
+                  &Self::_launch,
+                  containerId,
+                  lambda::_1));
   }
 
-  // Use the containerizer that launched the root container to launch
-  // the nested container.
-  Containerizer* containerizer = containers_.at(rootContainerId)->containerizer;
-
-  Container* container = new Container();
-  container->state = LAUNCHING;
-  container->containerizer = containerizer;
-  containers_[containerId] = container;
+  // Try each containerizer. If none of them handle the
+  // TaskInfo/ExecutorInfo then return a Failure.
+  vector<Containerizer*>::iterator containerizer = containerizers_.begin();
+  container->containerizer = *containerizer;
 
-  return containerizer->launch(
+  return (*containerizer)->launch(
       containerId,
-      commandInfo,
-      containerInfo,
-      user,
-      slaveId,
-      containerClass)
+      containerConfig,
+      environment,
+      pidCheckpointPath)
     .then(defer(self(),
                 &Self::_launch,
                 containerId,
+                containerConfig,
+                environment,
+                pidCheckpointPath,
+                containerizer,
                 lambda::_1));
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/70150975/src/slave/containerizer/composing.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.hpp b/src/slave/containerizer/composing.hpp
index 8e04bfe..bef6d88 100644
--- a/src/slave/containerizer/composing.hpp
+++ b/src/slave/containerizer/composing.hpp
@@ -26,11 +26,12 @@
 #include <process/http.hpp>
 #include <process/process.hpp>
 
-#include <stout/hashmap.hpp>
 #include <stout/hashset.hpp>
 #include <stout/option.hpp>
 #include <stout/try.hpp>
 
+#include "slave/containerizer/containerizer.hpp"
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -55,21 +56,9 @@ public:
 
   virtual process::Future<bool> launch(
       const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
+      const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,
-      bool checkpoint);
-
-  virtual process::Future<bool> launch(
-      const ContainerID& containerId,
-      const CommandInfo& commandInfo,
-      const Option<ContainerInfo>& containerInfo,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
-      const Option<mesos::slave::ContainerClass>& containerClass = None());
+      const Option<std::string>& pidCheckpointPath);
 
   virtual process::Future<process::http::Connection> attach(
       const ContainerID& containerId);


[08/16] mesos git commit: Updated test mocks per interface changes.

Posted by jo...@apache.org.
Updated test mocks per interface changes.

This updates the mocks for the containerizers and fetcher after
the interface changes.  For the containerizers, the two launch methods
have been combined into a single method.

The fetcher now has an argument in its constructor and no longer
takes some arguments (SlaveID and Flags) in its methods.

Launching nested containers via the test containerizer was
originally a no-op, so similar behavior is implemented in the
combined method.

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


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

Branch: refs/heads/master
Commit: 46da722e72fa06133fd0a5130abc34a6edcbbf57
Parents: 66070eb
Author: Joseph Wu <jo...@apache.org>
Authored: Mon May 1 13:59:32 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:07 2017 -0700

----------------------------------------------------------------------
 src/tests/cluster.cpp                          |   2 +-
 src/tests/containerizer.cpp                    | 154 +++++---------------
 src/tests/containerizer.hpp                    |  36 +----
 src/tests/containerizer/mock_containerizer.hpp |  20 +--
 src/tests/mesos.cpp                            |  19 +--
 src/tests/mesos.hpp                            |  18 +--
 src/tests/mock_docker.cpp                      |   2 +-
 src/tests/mock_docker.hpp                      |  42 ++----
 8 files changed, 72 insertions(+), 221 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/46da722e/src/tests/cluster.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cluster.cpp b/src/tests/cluster.cpp
index a4f57e0..d657da6 100644
--- a/src/tests/cluster.cpp
+++ b/src/tests/cluster.cpp
@@ -413,7 +413,7 @@ Try<process::Owned<Slave>> Slave::start(
     slave->containerizer = containerizer.get();
   } else {
     // Create a new fetcher.
-    slave->fetcher.reset(new slave::Fetcher());
+    slave->fetcher.reset(new slave::Fetcher(flags));
 
     Try<slave::Containerizer*> _containerizer =
       slave::Containerizer::create(flags, true, slave->fetcher.get());

http://git-wip-us.apache.org/repos/asf/mesos/blob/46da722e/src/tests/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.cpp b/src/tests/containerizer.cpp
index 548da3a..1d2b639 100644
--- a/src/tests/containerizer.cpp
+++ b/src/tests/containerizer.cpp
@@ -37,6 +37,7 @@ using testing::Invoke;
 using testing::Return;
 
 using mesos::slave::ContainerClass;
+using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerTermination;
 
 using mesos::v1::executor::Mesos;
@@ -87,35 +88,42 @@ public:
 
   Future<bool> launch(
       const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const string& directory,
-      const Option<string>& user,
-      const SlaveID& slaveId,
+      const ContainerConfig& containerConfig,
       const map<string, string>& environment,
-      bool checkpoint)
+      const Option<string>& pidCheckpointPath)
   {
     CHECK(!terminatedContainers.contains(containerId))
       << "Failed to launch nested container " << containerId
-      << " for executor '" << executorInfo.executor_id() << "'"
-      << " of framework " << executorInfo.framework_id()
+      << " for executor '" << containerConfig.executor_info().executor_id()
+      << "' of framework " << containerConfig.executor_info().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()
+      << " for executor '" << containerConfig.executor_info().executor_id()
+      << "' of framework " << containerConfig.executor_info().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()
+    containers_[containerId] = Owned<ContainerData>(new ContainerData());
+
+    if (containerId.has_parent()) {
+      // Launching a nested container via the test containerizer is a
+      // no-op for now.
+      return true;
+    }
+
+    CHECK(executors.contains(containerConfig.executor_info().executor_id()))
+      << "Failed to launch executor '"
+      << containerConfig.executor_info().executor_id()
+      << "' of framework " << containerConfig.executor_info().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();
+    containers_.at(containerId)->executorId =
+      containerConfig.executor_info().executor_id();
+
+    containers_.at(containerId)->frameworkId =
+      containerConfig.executor_info().framework_id();
 
     // We need to synchronize all reads and writes to the environment
     // as this is global state.
@@ -151,14 +159,15 @@ public:
       // 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()) {
+               containerConfig.executor_info()
+                 .command().environment().variables()) {
         os::setenv(variable.name(), variable.value());
       }
 
       os::setenv("MESOS_LOCAL", "1");
 
       const Owned<ExecutorData>& executorData =
-        executors.at(executorInfo.executor_id());
+        executors.at(containerConfig.executor_info().executor_id());
 
       if (executorData->executor != nullptr) {
         executorData->driver = Owned<MesosExecutorDriver>(
@@ -190,29 +199,6 @@ public:
     return true;
   }
 
-  Future<bool> launch(
-      const ContainerID& containerId,
-      const CommandInfo& commandInfo,
-      const Option<ContainerInfo>& containerInfo,
-      const Option<string>& user,
-      const SlaveID& slaveId,
-      const Option<ContainerClass>& containerClass)
-  {
-    CHECK(!terminatedContainers.contains(containerId))
-      << "Failed to launch nested container " << containerId
-      << " because this ContainerID is being re-used with"
-      << " a previously terminated container";
-
-    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;
-  }
-
   Future<Nothing> update(
       const ContainerID& containerId,
       const Resources& resources)
@@ -220,7 +206,6 @@ public:
     return Nothing();
   }
 
-
   Future<Connection> attach(
       const ContainerID& containerId)
   {
@@ -433,30 +418,8 @@ void TestContainerizer::setup()
   EXPECT_CALL(*this, update(_, _))
     .WillRepeatedly(Invoke(this, &TestContainerizer::_update));
 
-  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, _launch));
-
-  Future<bool> (TestContainerizer::*_launchNested)(
-      const ContainerID& containerId,
-      const CommandInfo& commandInfo,
-      const Option<ContainerInfo>& containerInfo,
-      const Option<string>& user,
-      const SlaveID& slaveId,
-      const Option<ContainerClass>&) = &TestContainerizer::_launch;
-
-  EXPECT_CALL(*this, launch(_, _, _, _, _, _))
-    .WillRepeatedly(Invoke(this, _launchNested));
+  EXPECT_CALL(*this, launch(_, _, _, _))
+    .WillRepeatedly(Invoke(this, &TestContainerizer::_launch));
 
   EXPECT_CALL(*this, attach(_))
     .WillRepeatedly(Invoke(this, &TestContainerizer::_attach));
@@ -481,66 +444,17 @@ Future<Nothing> TestContainerizer::_recover(
 
 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 ContainerConfig& containerConfig,
     const map<string, string>& environment,
-    bool checkpoint)
+    const Option<string>& pidCheckpointPath)
 {
-  // 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 process::dispatch(
       process.get(),
-      launch,
+      &TestContainerizerProcess::launch,
       containerId,
-      taskInfo,
-      executorInfo,
-      directory,
-      user,
-      slaveId,
+      containerConfig,
       environment,
-      checkpoint);
-}
-
-
-Future<bool> TestContainerizer::_launch(
-    const ContainerID& containerId,
-    const CommandInfo& commandInfo,
-    const Option<ContainerInfo>& containerInfo,
-    const Option<string>& user,
-    const SlaveID& slaveId,
-    const Option<ContainerClass>& containerClass)
-{
-  // Need to disambiguate for the compiler.
-  Future<bool> (TestContainerizerProcess::*launch)(
-      const ContainerID&,
-      const CommandInfo&,
-      const Option<ContainerInfo>&,
-      const Option<string>&,
-      const SlaveID&,
-      const Option<ContainerClass>& containerClass) =
-        &TestContainerizerProcess::launch;
-
-  return process::dispatch(
-      process.get(),
-      launch,
-      containerId,
-      commandInfo,
-      containerInfo,
-      user,
-      slaveId,
-      containerClass);
+      pidCheckpointPath);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/46da722e/src/tests/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.hpp b/src/tests/containerizer.hpp
index 63fe236..4bd40c3 100644
--- a/src/tests/containerizer.hpp
+++ b/src/tests/containerizer.hpp
@@ -84,27 +84,13 @@ public:
       recover,
       process::Future<Nothing>(const Option<slave::state::SlaveState>&));
 
-  MOCK_METHOD8(
+  MOCK_METHOD4(
       launch,
       process::Future<bool>(
           const ContainerID&,
-          const Option<TaskInfo>&,
-          const ExecutorInfo&,
-          const std::string&,
-          const Option<std::string>&,
-          const SlaveID&,
+          const mesos::slave::ContainerConfig&,
           const std::map<std::string, std::string>&,
-          bool checkpoint));
-
-  MOCK_METHOD6(
-      launch,
-      process::Future<bool>(
-          const ContainerID& containerId,
-          const CommandInfo& commandInfo,
-          const Option<ContainerInfo>& containerInfo,
-          const Option<std::string>& user,
-          const SlaveID& slaveId,
-          const Option<mesos::slave::ContainerClass>& containerClass));
+          const Option<std::string>&));
 
   MOCK_METHOD1(
       attach,
@@ -150,21 +136,9 @@ private:
 
   process::Future<bool> _launch(
       const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
+      const mesos::slave::ContainerConfig& containerConfig,
       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,
-      const Option<mesos::slave::ContainerClass>& containerClass = None());
+      const Option<std::string>& pidCheckpointPath);
 
   process::Future<process::http::Connection> _attach(
       const ContainerID& containerId);

http://git-wip-us.apache.org/repos/asf/mesos/blob/46da722e/src/tests/containerizer/mock_containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/mock_containerizer.hpp b/src/tests/containerizer/mock_containerizer.hpp
index ca0ae05..0adcb01 100644
--- a/src/tests/containerizer/mock_containerizer.hpp
+++ b/src/tests/containerizer/mock_containerizer.hpp
@@ -46,27 +46,13 @@ public:
       process::Future<Nothing>(
           const Option<slave::state::SlaveState>&));
 
-  MOCK_METHOD8(
+  MOCK_METHOD4(
       launch,
       process::Future<bool>(
           const ContainerID&,
-          const Option<TaskInfo>&,
-          const ExecutorInfo&,
-          const std::string&,
-          const Option<std::string>&,
-          const SlaveID&,
+          const mesos::slave::ContainerConfig&,
           const std::map<std::string, std::string>&,
-          bool));
-
-  MOCK_METHOD6(
-      launch,
-      process::Future<bool>(
-          const ContainerID&,
-          const CommandInfo&,
-          const Option<ContainerInfo>&,
-          const Option<std::string>&,
-          const SlaveID&,
-          const Option<mesos::slave::ContainerClass>&));
+          const Option<std::string>&));
 
   MOCK_METHOD1(
       attach,

http://git-wip-us.apache.org/repos/asf/mesos/blob/46da722e/src/tests/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index 714a520..a6ddb45 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -506,12 +506,13 @@ MockExecutor::MockExecutor(const ExecutorID& _id) : id(_id) {}
 MockExecutor::~MockExecutor() {}
 
 
-MockFetcherProcess::MockFetcherProcess()
+MockFetcherProcess::MockFetcherProcess(const slave::Flags& flags)
+  : slave::FetcherProcess(flags)
 {
   // Set up default behaviors, calling the original methods.
-  EXPECT_CALL(*this, _fetch(_, _, _, _, _, _))
+  EXPECT_CALL(*this, _fetch(_, _, _, _, _))
     .WillRepeatedly(Invoke(this, &MockFetcherProcess::unmocked__fetch));
-  EXPECT_CALL(*this, run(_, _, _, _, _))
+  EXPECT_CALL(*this, run(_, _, _, _))
     .WillRepeatedly(Invoke(this, &MockFetcherProcess::unmocked_run));
 }
 
@@ -552,16 +553,14 @@ Future<Nothing> MockFetcherProcess::unmocked__fetch(
     const ContainerID& containerId,
     const string& sandboxDirectory,
     const string& cacheDirectory,
-    const Option<string>& user,
-    const slave::Flags& flags)
+    const Option<string>& user)
 {
   return slave::FetcherProcess::_fetch(
       entries,
       containerId,
       sandboxDirectory,
       cacheDirectory,
-      user,
-      flags);
+      user);
 }
 
 
@@ -569,15 +568,13 @@ Future<Nothing> MockFetcherProcess::unmocked_run(
     const ContainerID& containerId,
     const string& sandboxDirectory,
     const Option<string>& user,
-    const FetcherInfo& info,
-    const slave::Flags& flags)
+    const FetcherInfo& info)
 {
   return slave::FetcherProcess::run(
       containerId,
       sandboxDirectory,
       user,
-      info,
-      flags);
+      info);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/46da722e/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 3c57f25..9b04a40 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -2098,10 +2098,10 @@ using MockHTTPExecutor = tests::executor::MockHTTPExecutor<
 class MockFetcherProcess : public slave::FetcherProcess
 {
 public:
-  MockFetcherProcess();
+  MockFetcherProcess(const slave::Flags& flags);
   virtual ~MockFetcherProcess();
 
-  MOCK_METHOD6(_fetch, process::Future<Nothing>(
+  MOCK_METHOD5(_fetch, process::Future<Nothing>(
       const hashmap<
           CommandInfo::URI,
           Option<process::Future<std::shared_ptr<Cache::Entry>>>>&
@@ -2109,8 +2109,7 @@ public:
       const ContainerID& containerId,
       const std::string& sandboxDirectory,
       const std::string& cacheDirectory,
-      const Option<std::string>& user,
-      const slave::Flags& flags));
+      const Option<std::string>& user));
 
   process::Future<Nothing> unmocked__fetch(
       const hashmap<
@@ -2120,22 +2119,19 @@ public:
       const ContainerID& containerId,
       const std::string& sandboxDirectory,
       const std::string& cacheDirectory,
-      const Option<std::string>& user,
-      const slave::Flags& flags);
+      const Option<std::string>& user);
 
-  MOCK_METHOD5(run, process::Future<Nothing>(
+  MOCK_METHOD4(run, process::Future<Nothing>(
       const ContainerID& containerId,
       const std::string& sandboxDirectory,
       const Option<std::string>& user,
-      const mesos::fetcher::FetcherInfo& info,
-      const slave::Flags& flags));
+      const mesos::fetcher::FetcherInfo& info));
 
   process::Future<Nothing> unmocked_run(
       const ContainerID& containerId,
       const std::string& sandboxDirectory,
       const Option<std::string>& user,
-      const mesos::fetcher::FetcherInfo& info,
-      const slave::Flags& flags);
+      const mesos::fetcher::FetcherInfo& info);
 };
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/46da722e/src/tests/mock_docker.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mock_docker.cpp b/src/tests/mock_docker.cpp
index 81a14ca..0ed6386 100644
--- a/src/tests/mock_docker.cpp
+++ b/src/tests/mock_docker.cpp
@@ -98,7 +98,7 @@ MockDockerContainerizerProcess::MockDockerContainerizerProcess(
   : slave::DockerContainerizerProcess(
       flags, fetcher, logger, docker, nvidia)
 {
-  EXPECT_CALL(*this, fetch(_, _))
+  EXPECT_CALL(*this, fetch(_))
     .WillRepeatedly(Invoke(this, &MockDockerContainerizerProcess::_fetch));
 
   EXPECT_CALL(*this, pull(_))

http://git-wip-us.apache.org/repos/asf/mesos/blob/46da722e/src/tests/mock_docker.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mock_docker.hpp b/src/tests/mock_docker.hpp
index f58211d..5987364 100644
--- a/src/tests/mock_docker.hpp
+++ b/src/tests/mock_docker.hpp
@@ -153,24 +153,20 @@ public:
     // NOTE: See TestContainerizer::setup for why we use
     // 'EXPECT_CALL' and 'WillRepeatedly' here instead of
     // 'ON_CALL' and 'WillByDefault'.
-    EXPECT_CALL(*this, launch(_, _, _, _, _, _, _, _))
+    EXPECT_CALL(*this, launch(_, _, _, _))
       .WillRepeatedly(Invoke(this, &MockDockerContainerizer::_launch));
 
     EXPECT_CALL(*this, update(_, _))
       .WillRepeatedly(Invoke(this, &MockDockerContainerizer::_update));
   }
 
-  MOCK_METHOD8(
+  MOCK_METHOD4(
       launch,
       process::Future<bool>(
           const ContainerID&,
-          const Option<TaskInfo>&,
-          const ExecutorInfo&,
-          const std::string&,
-          const Option<std::string>&,
-          const SlaveID&,
+          const mesos::slave::ContainerConfig&,
           const std::map<std::string, std::string>&,
-          bool checkpoint));
+          const Option<std::string>&));
 
   MOCK_METHOD2(
       update,
@@ -182,23 +178,15 @@ public:
   // use &slave::DockerContainerizer::launch with 'Invoke').
   process::Future<bool> _launch(
       const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
+      const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,
-      bool checkpoint)
+      const Option<std::string>& pidCheckpointPath)
   {
     return slave::DockerContainerizer::launch(
         containerId,
-        taskInfo,
-        executorInfo,
-        directory,
-        user,
-        slaveId,
+        containerConfig,
         environment,
-        checkpoint);
+        pidCheckpointPath);
   }
 
   process::Future<Nothing> _update(
@@ -226,21 +214,17 @@ public:
 
   virtual ~MockDockerContainerizerProcess();
 
-  MOCK_METHOD2(
+  MOCK_METHOD1(
       fetch,
-      process::Future<Nothing>(
-          const ContainerID& containerId,
-          const SlaveID& slaveId));
+      process::Future<Nothing>(const ContainerID&));
 
   MOCK_METHOD1(
       pull,
-      process::Future<Nothing>(const ContainerID& containerId));
+      process::Future<Nothing>(const ContainerID&));
 
-  process::Future<Nothing> _fetch(
-      const ContainerID& containerId,
-      const SlaveID& slaveId)
+  process::Future<Nothing> _fetch(const ContainerID& containerId)
   {
-    return slave::DockerContainerizerProcess::fetch(containerId, slaveId);
+    return slave::DockerContainerizerProcess::fetch(containerId);
   }
 
   process::Future<Nothing> _pull(const ContainerID& containerId)


[05/16] mesos git commit: Added some comments to ContainerConfig protobuf.

Posted by jo...@apache.org.
Added some comments to ContainerConfig protobuf.

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


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

Branch: refs/heads/master
Commit: a6a7a38107d6f6b429d8564e91564b901e35eaf4
Parents: 0709ce3
Author: Joseph Wu <jo...@apache.org>
Authored: Mon May 15 14:41:35 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:06 2017 -0700

----------------------------------------------------------------------
 include/mesos/slave/containerizer.proto | 20 ++++++++++++++++++--
 1 file changed, 18 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a6a7a381/include/mesos/slave/containerizer.proto
----------------------------------------------------------------------
diff --git a/include/mesos/slave/containerizer.proto b/include/mesos/slave/containerizer.proto
index 0f96334..03a983e 100644
--- a/include/mesos/slave/containerizer.proto
+++ b/include/mesos/slave/containerizer.proto
@@ -105,18 +105,32 @@ message ContainerRecoverInfo {
  */
 message ContainerConfig {
   // The executor associated with this container.
+  // This field is blank when launching a nested or standalone container.
+  // When `task_info` is specified, this field will always be present,
+  // but will contain a default ExecutorInfo generated by the Agent.
   optional ExecutorInfo executor_info = 8;
 
-  // The task (optional) associated with this container.
+  // The task associated with this container.
+  // This field is only specified when launching a command task.
+  // Tasks are not passed to isolators when run via a custom executor.
   optional TaskInfo task_info = 9;
 
   // The command used to launch the container.
+  // When `executor_info` is specified, this will always be a copy of
+  // the CommandInfo in the ExecutorInfo.
   required CommandInfo command_info = 11;
 
   // The information about the container.
+  // When either `task_info` or `executor_info` are specified, this will
+  // always be a copy of the ContainerInfo in those fields (or None if
+  // neither specify a ContainerInfo). When both `task_info` and
+  // `executor_info` are specified, the ContainerInfo will be the
+  // same across all fields.
   optional ContainerInfo container_info = 12;
 
   // Resources associated with the container during launch.
+  // When either `task_info` or `executor_info` are specified, this will
+  // always be the sum of the Resources within those fields.
   repeated Resource resources = 13;
 
   // The class of container being launched. Used by isolators to
@@ -126,7 +140,9 @@ message ContainerConfig {
   // The work directory for the container in the host filesystem.
   required string directory = 3;
 
-  // The user the task will be run as.
+  // The user that should be used to run the `command_info`.
+  // The sandbox directory and any artifacts from the Mesos fetcher will
+  // be made accessible to this user.
   optional string user = 4;
 
   // NOTE: 'rootfs' and 'docker' below are for the executor in custom


[06/16] mesos git commit: Changed the fetcher cache directory.

Posted by jo...@apache.org.
Changed the fetcher cache directory.

The fetcher cache directory was historically located (by default)
in `/tmp/mesos/fetch`.  The agent flag `--fetcher_cache_dir` could
be used to change this value.

The fetcher would create a subdirectory underneath `/tmp/mesos/fetch`
for each `SlaveID`.  This was done because multiple agents can run on
the same node.  If all the agents use the same default fetcher cache
directory, they will collide and cause unpredictable results.
As a result, the `SlaveID` needed to be passed into the fetcher
after the agent recovers and/or registers with the master, because
that is when the `SlaveID` is determined.

This changes the default fetcher cache directory to
`/tmp/mesos/fetch`.  The `SlaveID` subdirectory has been removed.

This change, while techically a breaking change, is safe because of
how the fetcher uses this directory.  Upon starting up, the fetcher
"recovers" by clearing this directory.  Although the subdirectory
has been removed, the fetcher still clears the fetcher cache
on startup.

This change will only cause breakages if multiple agents are run
with the same `--fetcher_cache_dir`.  In this case, each agent
will delete the fetcher caches of all the other agents.

---

With the removal of the `SlaveID` field in the fetcher's methods,
it is no longer necessary to pass in the `SlaveID` or agent Flags
at agent recovery time.  Instead, the flags can be passed in during
the fetcher's construction.

Similarly, the fetcher's "recovery" (clearing the fetcher cache)
can be done immediately upon construction, which simplifies the
code slightly.

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


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

Branch: refs/heads/master
Commit: 145896bcf0dc0f2d53dd6836a99a0397e8ac13ae
Parents: 51da8c4
Author: Joseph Wu <jo...@apache.org>
Authored: Mon Apr 10 18:24:15 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:06 2017 -0700

----------------------------------------------------------------------
 src/local/local.cpp                 |  6 ++-
 src/slave/containerizer/fetcher.cpp | 87 ++++++++------------------------
 src/slave/containerizer/fetcher.hpp | 37 +++++---------
 src/slave/flags.cpp                 | 17 +++----
 src/slave/main.cpp                  |  2 +-
 src/slave/slave.cpp                 | 15 ------
 6 files changed, 48 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/145896bc/src/local/local.cpp
----------------------------------------------------------------------
diff --git a/src/local/local.cpp b/src/local/local.cpp
index 3f4150b..ad35723 100644
--- a/src/local/local.cpp
+++ b/src/local/local.cpp
@@ -372,6 +372,10 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
     propagatedFlags["runtime_dir"] =
       path::join(flags.runtime_dir, "agents", stringify(i));
 
+    // Use a different fetcher cache directory for each agent.
+    propagatedFlags["fetcher_cache_dir"] =
+      path::join(os::temp(), "mesos", "fetch", "agents", stringify(i));
+
     slave::Flags slaveFlags;
     Try<flags::Warnings> load = slaveFlags.load(
         propagatedFlags,
@@ -407,7 +411,7 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
 
     garbageCollectors->push_back(new GarbageCollector());
     statusUpdateManagers->push_back(new StatusUpdateManager(slaveFlags));
-    fetchers->push_back(new Fetcher());
+    fetchers->push_back(new Fetcher(slaveFlags));
 
     Try<ResourceEstimator*> resourceEstimator =
       ResourceEstimator::create(slaveFlags.resource_estimator);

http://git-wip-us.apache.org/repos/asf/mesos/blob/145896bc/src/slave/containerizer/fetcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/fetcher.cpp b/src/slave/containerizer/fetcher.cpp
index a910fea..770cad3 100644
--- a/src/slave/containerizer/fetcher.cpp
+++ b/src/slave/containerizer/fetcher.cpp
@@ -36,8 +36,6 @@
 
 #include "hdfs/hdfs.hpp"
 
-#include "slave/slave.hpp"
-
 #include "slave/containerizer/fetcher.hpp"
 
 using std::list;
@@ -68,8 +66,15 @@ static const string FILE_URI_LOCALHOST = "file://localhost";
 static const string CACHE_FILE_NAME_PREFIX = "c";
 
 
-Fetcher::Fetcher() : process(new FetcherProcess())
+Fetcher::Fetcher(const Flags& flags) : process(new FetcherProcess(flags))
 {
+  if (os::exists(flags.fetcher_cache_dir)) {
+    Try<Nothing> rmdir = os::rmdir(flags.fetcher_cache_dir, true);
+    CHECK_SOME(rmdir)
+      << "Could not delete fetcher cache directory '"
+      << flags.fetcher_cache_dir << "': " + rmdir.error();
+  }
+
   spawn(process.get());
 }
 
@@ -88,34 +93,6 @@ Fetcher::~Fetcher()
 }
 
 
-Try<Nothing> Fetcher::recover(const SlaveID& slaveId, const Flags& flags)
-{
-  // Good enough for now, simple, least-effort recovery.
-  VLOG(1) << "Clearing fetcher cache";
-
-  string cacheDirectory = paths::getSlavePath(flags.fetcher_cache_dir, slaveId);
-  Result<string> path = os::realpath(cacheDirectory);
-  if (path.isError()) {
-    LOG(ERROR) << "Malformed fetcher cache directory path '" << cacheDirectory
-               << "', error: " + path.error();
-
-    return Error(path.error());
-  }
-
-  if (path.isSome() && os::exists(path.get())) {
-    Try<Nothing> rmdir = os::rmdir(path.get(), true);
-    if (rmdir.isError()) {
-      LOG(ERROR) << "Could not delete fetcher cache directory '"
-                 << cacheDirectory << "', error: " + rmdir.error();
-
-      return rmdir;
-    }
-  }
-
-  return Nothing();
-}
-
-
 Try<string> Fetcher::basename(const string& uri)
 {
   // TODO(bernd-mesos): full URI parsing, then move this to stout.
@@ -258,18 +235,14 @@ Future<Nothing> Fetcher::fetch(
     const ContainerID& containerId,
     const CommandInfo& commandInfo,
     const string& sandboxDirectory,
-    const Option<string>& user,
-    const SlaveID& slaveId,
-    const Flags& flags)
+    const Option<string>& user)
 {
   return dispatch(process.get(),
                   &FetcherProcess::fetch,
                   containerId,
                   commandInfo,
                   sandboxDirectory,
-                  user,
-                  slaveId,
-                  flags);
+                  user);
 }
 
 
@@ -346,9 +319,7 @@ Future<Nothing> FetcherProcess::fetch(
     const ContainerID& containerId,
     const CommandInfo& commandInfo,
     const string& sandboxDirectory,
-    const Option<string>& user,
-    const SlaveID& slaveId,
-    const Flags& flags)
+    const Option<string>& user)
 {
   VLOG(1) << "Starting to fetch URIs for container: " << containerId
           << ", directory: " << sandboxDirectory;
@@ -368,7 +339,7 @@ Future<Nothing> FetcherProcess::fetch(
     commandUser = commandInfo.user();
   }
 
-  string cacheDirectory = paths::getSlavePath(flags.fetcher_cache_dir, slaveId);
+  string cacheDirectory = flags.fetcher_cache_dir;
   if (commandUser.isSome()) {
     // Segregating per-user cache directories.
     cacheDirectory = path::join(cacheDirectory, commandUser.get());
@@ -436,8 +407,7 @@ Future<Nothing> FetcherProcess::fetch(
                 containerId,
                 sandboxDirectory,
                 cacheDirectory,
-                commandUser,
-                flags);
+                commandUser);
 }
 
 
@@ -447,8 +417,7 @@ Future<Nothing> FetcherProcess::_fetch(
     const ContainerID& containerId,
     const string& sandboxDirectory,
     const string& cacheDirectory,
-    const Option<string>& user,
-    const Flags& flags)
+    const Option<string>& user)
 {
   // Get out all of the futures we need to wait for so we can wait on
   // them together via 'await'.
@@ -498,8 +467,7 @@ Future<Nothing> FetcherProcess::_fetch(
                      containerId,
                      sandboxDirectory,
                      cacheDirectory,
-                     user,
-                     flags);
+                     user);
     }));
 }
 
@@ -509,8 +477,7 @@ Future<Nothing> FetcherProcess::__fetch(
     const ContainerID& containerId,
     const string& sandboxDirectory,
     const string& cacheDirectory,
-    const Option<string>& user,
-    const Flags& flags)
+    const Option<string>& user)
 {
   // Now construct the FetcherInfo based on which URIs we're using
   // the cache for and which ones we are bypassing the cache.
@@ -553,7 +520,7 @@ Future<Nothing> FetcherProcess::__fetch(
     info.set_frameworks_home(flags.frameworks_home);
   }
 
-  return run(containerId, sandboxDirectory, user, info, flags)
+  return run(containerId, sandboxDirectory, user, info)
     .repair(defer(self(), [=](const Future<Nothing>& future) {
       LOG(ERROR) << "Failed to run mesos-fetcher: " << future.failure();
 
@@ -630,29 +597,20 @@ static off_t delta(
 
 
 // For testing only.
-// TODO(bernd-mesos): After refactoring slave/containerizer,fetcher so
-// that flags and slave ID get injected, replace this with two functions
-// one of which returns a list of cache file paths, the other the number
-// of entries in the cache table.
-Try<list<Path>> FetcherProcess::cacheFiles(
-    const SlaveID& slaveId,
-    const Flags& flags)
+Try<list<Path>> FetcherProcess::cacheFiles()
 {
   list<Path> result;
 
-  const string cacheDirectory =
-    slave::paths::getSlavePath(flags.fetcher_cache_dir, slaveId);
-
-  if (!os::exists(cacheDirectory)) {
+  if (!os::exists(flags.fetcher_cache_dir)) {
     return result;
   }
 
   const Try<list<string>> find =
-    os::find(cacheDirectory, CACHE_FILE_NAME_PREFIX);
+    os::find(flags.fetcher_cache_dir, CACHE_FILE_NAME_PREFIX);
 
   if (find.isError()) {
     return Error("Could not access cache directory '" +
-                 cacheDirectory + "' with error: " + find.error());
+                 flags.fetcher_cache_dir + "' with error: " + find.error());
   }
 
   transform(find.get().begin(),
@@ -724,8 +682,7 @@ Future<Nothing> FetcherProcess::run(
     const ContainerID& containerId,
     const string& sandboxDirectory,
     const Option<string>& user,
-    const FetcherInfo& info,
-    const Flags& flags)
+    const FetcherInfo& info)
 {
   // Before we fetch let's make sure we create 'stdout' and 'stderr'
   // files into which we can redirect the output of the mesos-fetcher

http://git-wip-us.apache.org/repos/asf/mesos/blob/145896bc/src/slave/containerizer/fetcher.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/fetcher.hpp b/src/slave/containerizer/fetcher.hpp
index 9e3018d..efeadbf 100644
--- a/src/slave/containerizer/fetcher.hpp
+++ b/src/slave/containerizer/fetcher.hpp
@@ -25,8 +25,8 @@
 
 #include <mesos/fetcher/fetcher.hpp>
 
-#include <process/id.hpp>
 #include <process/future.hpp>
+#include <process/id.hpp>
 #include <process/process.hpp>
 #include <process/subprocess.hpp>
 
@@ -79,19 +79,13 @@ public:
 
   static bool isNetUri(const std::string& uri);
 
-  Fetcher();
+  Fetcher(const Flags& flags);
 
   // This is only public for tests.
   Fetcher(const process::Owned<FetcherProcess>& process);
 
   virtual ~Fetcher();
 
-  // TODO(bernd-mesos): Inject these parameters at Fetcher creation time.
-  // Then also inject the fetcher into the slave at creation time. Then
-  // it will be possible to make this an instance method instead of a
-  // static one for the slave to call during startup or recovery.
-  static Try<Nothing> recover(const SlaveID& slaveId, const Flags& flags);
-
   // Download the URIs specified in the command info and place the
   // resulting files into the given sandbox directory. Chmod said files
   // to the user if given. Send stdout and stderr output to files
@@ -101,9 +95,7 @@ public:
       const ContainerID& containerId,
       const CommandInfo& commandInfo,
       const std::string& sandboxDirectory,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
-      const Flags& flags);
+      const Option<std::string>& user);
 
   // Best effort to kill the fetcher subprocess associated with the
   // indicated container. Do nothing if no such subprocess exists.
@@ -117,7 +109,9 @@ private:
 class FetcherProcess : public process::Process<FetcherProcess>
 {
 public:
-  FetcherProcess() : ProcessBase(process::ID::generate("fetcher")) {}
+  FetcherProcess(const Flags& _flags)
+    : ProcessBase(process::ID::generate("fetcher")),
+      flags(_flags) {}
 
   virtual ~FetcherProcess();
 
@@ -125,9 +119,7 @@ public:
       const ContainerID& containerId,
       const CommandInfo& commandInfo,
       const std::string& sandboxDirectory,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
-      const Flags& flags);
+      const Option<std::string>& user);
 
   // Runs the mesos-fetcher, creating a "stdout" and "stderr" file
   // in the given directory, using these for trace output.
@@ -135,8 +127,7 @@ public:
       const ContainerID& containerId,
       const std::string& sandboxDirectory,
       const Option<std::string>& user,
-      const mesos::fetcher::FetcherInfo& info,
-      const Flags& flags);
+      const mesos::fetcher::FetcherInfo& info);
 
   // Best effort attempt to kill the external mesos-fetcher process
   // running on behalf of the given container ID, if any.
@@ -297,14 +288,11 @@ public:
       const ContainerID& containerId,
       const std::string& sandboxDirectory,
       const std::string& cacheDirectory,
-      const Option<std::string>& user,
-      const Flags& flags);
+      const Option<std::string>& user);
 
   // Returns a list of cache files on disk for the given slave
   // (for all users combined). For testing.
-  // TODO(bernd-mesos): Remove the parameters after slave/containerizer
-  // refactoring for injection of these.
-  Try<std::list<Path>> cacheFiles(const SlaveID& slaveId, const Flags& flags);
+  Try<std::list<Path>> cacheFiles();
 
   // Returns the number of cache entries for the given slave (for all
   // users combined). For testing.
@@ -321,8 +309,7 @@ private:
       const ContainerID& containerId,
       const std::string& sandboxDirectory,
       const std::string& cacheDirectory,
-      const Option<std::string>& user,
-      const Flags& flags);
+      const Option<std::string>& user);
 
   // Calls Cache::reserve() and returns a ready entry future if successful,
   // else Failure. Claims the space and assigns the entry's size to this
@@ -331,6 +318,8 @@ private:
       const Try<Bytes>& requestedSpace,
       const std::shared_ptr<Cache::Entry>& entry);
 
+  Flags flags;
+
   Cache cache;
 
   hashmap<ContainerID, pid_t> subprocessPids;

http://git-wip-us.apache.org/repos/asf/mesos/blob/145896bc/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index d0dc9c8..0efe67e 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -186,18 +186,15 @@ mesos::internal::slave::Flags::Flags()
       "Size of the fetcher cache in Bytes.",
       DEFAULT_FETCHER_CACHE_SIZE);
 
-  // By default the fetcher cache directory is held inside the work
-  // directory, so everything can be deleted or archived in one swoop,
-  // in particular during testing. However, a typical production
-  // scenario is to use a separate cache volume. First, it is not meant
-  // to be backed up. Second, you want to avoid that sandbox directories
-  // and the cache directory can interfere with each other in
-  // unpredictable ways by occupying shared space. So it is recommended
-  // to set the cache directory explicitly.
   add(&Flags::fetcher_cache_dir,
       "fetcher_cache_dir",
-      "Parent directory for fetcher cache directories\n"
-      "(one subdirectory per agent).",
+      "Directory for the fetcher cache. The agent will clear this directory\n"
+      "on startup. It is recommended to set this value to a separate volume\n"
+      "for several reasons:\n"
+      "  * The cache directories are transient and not meant to be\n"
+      "    backed up. Upon restarting the agent, the cache is always empty.\n"
+      "  * The cache and container sandboxes can potentially interfere with\n"
+      "    each other when occupying a shared space (i.e. disk contention).",
       path::join(os::temp(), "mesos", "fetch"));
 
   add(&Flags::work_dir,

http://git-wip-us.apache.org/repos/asf/mesos/blob/145896bc/src/slave/main.cpp
----------------------------------------------------------------------
diff --git a/src/slave/main.cpp b/src/slave/main.cpp
index a363ae6..80a957b 100644
--- a/src/slave/main.cpp
+++ b/src/slave/main.cpp
@@ -445,7 +445,7 @@ int main(int argc, char** argv)
   }
 #endif // __linux__
 
-  Fetcher* fetcher = new Fetcher();
+  Fetcher* fetcher = new Fetcher(flags);
 
   // Initialize SecretResolver.
   Try<SecretResolver*> secretResolver =

http://git-wip-us.apache.org/repos/asf/mesos/blob/145896bc/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index b7e731b..0f21cf8 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -1125,14 +1125,6 @@ void Slave::registered(
       LOG(INFO) << "Registered with master " << master.get()
                 << "; given agent ID " << slaveId;
 
-      // TODO(bernd-mesos): Make this an instance method call, see comment
-      // in "fetcher.hpp"".
-      Try<Nothing> recovered = Fetcher::recover(slaveId, flags);
-      if (recovered.isError()) {
-        LOG(FATAL) << "Could not initialize fetcher cache: "
-                   << recovered.error();
-      }
-
       state = RUNNING;
 
       // Cancel the pending registration timer to avoid spurious attempts
@@ -5863,13 +5855,6 @@ Future<Nothing> Slave::recover(const Try<state::State>& state)
       metrics.recovery_errors += slaveState->errors;
     }
 
-    // TODO(bernd-mesos): Make this an instance method call, see comment
-    // in "fetcher.hpp"".
-    Try<Nothing> recovered = Fetcher::recover(slaveState->id, flags);
-    if (recovered.isError()) {
-      return Failure(recovered.error());
-    }
-
     // Recover the frameworks.
     foreachvalue (const FrameworkState& frameworkState,
                   slaveState->frameworks) {


[11/16] mesos git commit: Removed extranous sandbox initialization in Docker containerizer.

Posted by jo...@apache.org.
Removed extranous sandbox initialization in Docker containerizer.

The Docker containerizer will create a stdout/stderr file as well
`chown` the sandbox directory to the user specified at container
creation time.  This logic is already performed elsewhere.

The stdout/stderr files will either be created prior to the container
launch via the mesos-fetcher, or during launch via the ContainerLogger.

The sandbox directory will be `chown`d by the Agent when it
starts to launch the executor.

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


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

Branch: refs/heads/master
Commit: 55c86cb4a03746b21b0532aca779f826fc1bc2e1
Parents: 55d7c9f
Author: Joseph Wu <jo...@apache.org>
Authored: Thu Apr 27 18:33:45 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:07 2017 -0700

----------------------------------------------------------------------
 src/slave/containerizer/docker.cpp | 25 -------------------------
 1 file changed, 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/55c86cb4/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index 9418e01..c90750d 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -280,32 +280,7 @@ DockerContainerizerProcess::Container::create(
     bool checkpoint,
     const Flags& flags)
 {
-  // Before we do anything else we first make sure the stdout/stderr
-  // files exist and have the right file ownership.
-  Try<Nothing> touch = os::touch(path::join(directory, "stdout"));
 
-  if (touch.isError()) {
-    return Error("Failed to touch 'stdout': " + touch.error());
-  }
-
-  touch = os::touch(path::join(directory, "stderr"));
-
-  if (touch.isError()) {
-    return Error("Failed to touch 'stderr': " + touch.error());
-  }
-
-  // NOTE: `os::chown` has no meaningful interpretation on Windows. This is
-  // safe to `#ifdef` out because we don't compile the user flag on Windows, so
-  // this should always be `None`.
-#ifndef __WINDOWS__
-  if (user.isSome()) {
-    Try<Nothing> chown = os::chown(user.get(), directory);
-
-    if (chown.isError()) {
-      return Error("Failed to chown: " + chown.error());
-    }
-  }
-#endif // __WINDOWS__
 
   string dockerSymlinkPath = path::join(
       paths::getSlavePath(flags.work_dir, slaveId),


[07/16] mesos git commit: Refactored Docker containerizer launch path per interface changes.

Posted by jo...@apache.org.
Refactored Docker containerizer launch path per interface changes.

This continues the containerizer interface change that replaced
most "Infos" (Task, Executor, Command, Container) with a single
`ContainerConfig` and combined the nested/non-nested container
launch paths.

Notably, this commit also changes the fields stored in the Docker
containerizer to match the new interface.  The somewhat ambiguously
named `directory` field has been renamed to `containerWorkDir`.
And the copy of `Slave::flags` in each container has been removed
because it is not used.

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


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

Branch: refs/heads/master
Commit: 66070ebd35a26273f419deb3ae50df0ae5b7a48a
Parents: 55c86cb
Author: Joseph Wu <jo...@apache.org>
Authored: Mon May 1 12:19:02 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:07 2017 -0700

----------------------------------------------------------------------
 src/slave/containerizer/docker.cpp | 216 ++++++++++++++------------------
 src/slave/containerizer/docker.hpp | 132 +++++++++----------
 2 files changed, 151 insertions(+), 197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/66070ebd/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index c90750d..9f84109 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -72,8 +72,9 @@ using std::set;
 using std::string;
 using std::vector;
 
-using mesos::slave::ContainerLogger;
+using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerIO;
+using mesos::slave::ContainerLogger;
 using mesos::slave::ContainerTermination;
 
 using mesos::internal::slave::state::SlaveState;
@@ -271,19 +272,24 @@ DockerContainerizer::~DockerContainerizer()
 Try<DockerContainerizerProcess::Container*>
 DockerContainerizerProcess::Container::create(
     const ContainerID& id,
-    const Option<TaskInfo>& taskInfo,
-    const ExecutorInfo& executorInfo,
-    const string& directory,
-    const Option<string>& user,
-    const SlaveID& slaveId,
+    const ContainerConfig& containerConfig,
     const map<string, string>& environment,
-    bool checkpoint,
+    const Option<string>& pidCheckpointPath,
     const Flags& flags)
 {
+  // We need to extract a SlaveID based on the sandbox directory,
+  // for the purpose of working around a limitation of the Docker CLI.
+  // If the sandbox directory contains a colon, the sandbox directory
+  // cannot be mounted directly into the container directory. Instead,
+  // we symlink the sandbox directory and mount the symlink.
+  // See MESOS-1833 for more details.
+  Try<paths::ExecutorRunPath> runPath =
+    paths::parseExecutorRunPath(flags.work_dir, containerConfig.directory());
 
+  CHECK_SOME(runPath) << "Unable to determine SlaveID from sandbox directory";
 
   string dockerSymlinkPath = path::join(
-      paths::getSlavePath(flags.work_dir, slaveId),
+      paths::getSlavePath(flags.work_dir, runPath->slaveId),
       DOCKER_SYMLINK_DIRECTORY);
 
   Try<Nothing> mkdir = os::mkdir(dockerSymlinkPath);
@@ -293,17 +299,17 @@ DockerContainerizerProcess::Container::create(
   }
 
   bool symlinked = false;
-  string containerWorkdir = directory;
-  // We need to symlink the sandbox directory if the directory
-  // path has a colon, as Docker CLI uses the colon as a separator.
-  if (strings::contains(directory, ":")) {
+  string containerWorkdir = containerConfig.directory();
+  if (strings::contains(containerConfig.directory(), ":")) {
     containerWorkdir = path::join(dockerSymlinkPath, id.value());
 
-    Try<Nothing> symlink = ::fs::symlink(directory, containerWorkdir);
+    Try<Nothing> symlink =
+      ::fs::symlink(containerConfig.directory(), containerWorkdir);
 
     if (symlink.isError()) {
-      return Error("Failed to symlink directory '" + directory +
-                   "' to '" + containerWorkdir + "': " + symlink.error());
+      return Error(
+          "Failed to symlink directory '" + containerConfig.directory() +
+          "' to '" + containerWorkdir + "': " + symlink.error());
     }
 
     symlinked = true;
@@ -312,7 +318,7 @@ DockerContainerizerProcess::Container::create(
   Option<ContainerInfo> containerInfo = None();
   Option<CommandInfo> commandInfo = None();
   bool launchesExecutorContainer = false;
-  if (taskInfo.isSome() && flags.docker_mesos_image.isSome()) {
+  if (containerConfig.has_task_info() && flags.docker_mesos_image.isSome()) {
     // Override the container and command to launch an executor
     // in a docker container.
     ContainerInfo newContainerInfo;
@@ -381,8 +387,9 @@ DockerContainerizerProcess::Container::create(
       }
     }
 
-    if (taskInfo->has_command()) {
-      newCommandInfo.mutable_uris()->CopyFrom(taskInfo->command().uris());
+    if (containerConfig.task_info().has_command()) {
+      newCommandInfo.mutable_uris()
+        ->CopyFrom(containerConfig.task_info().command().uris());
     }
 
     containerInfo = newContainerInfo;
@@ -392,24 +399,19 @@ DockerContainerizerProcess::Container::create(
 
   return new Container(
       id,
-      taskInfo,
-      executorInfo,
-      containerWorkdir,
-      user,
-      slaveId,
-      checkpoint,
+      containerConfig,
+      environment,
+      pidCheckpointPath,
       symlinked,
-      flags,
+      containerWorkdir,
       commandInfo,
       containerInfo,
-      environment,
       launchesExecutorContainer);
 }
 
 
 Future<Nothing> DockerContainerizerProcess::fetch(
-    const ContainerID& containerId,
-    const SlaveID& slaveId)
+    const ContainerID& containerId)
 {
   CHECK(containers_.contains(containerId));
   Container* container = containers_.at(containerId);
@@ -417,10 +419,8 @@ Future<Nothing> DockerContainerizerProcess::fetch(
   return fetcher->fetch(
       containerId,
       container->command,
-      container->directory,
-      None(),
-      slaveId,
-      flags);
+      container->containerWorkDir,
+      None());
 }
 
 
@@ -437,7 +437,7 @@ Future<Nothing> DockerContainerizerProcess::pull(
   string image = container->image();
 
   Future<Docker::Image> future = docker->pull(
-    container->directory,
+    container->containerWorkDir,
     image,
     container->forcePullImage());
 
@@ -605,7 +605,7 @@ Future<Nothing> DockerContainerizerProcess::mountPersistentVolumes(
   Container* container = containers_.at(containerId);
   container->state = Container::MOUNTING;
 
-  if (container->task.isNone() &&
+  if (!container->containerConfig.has_task_info() &&
       !container->resources.persistentVolumes().empty()) {
     LOG(ERROR) << "Persistent volumes found with container '" << containerId
                << "' but are not supported with custom executors";
@@ -614,7 +614,7 @@ Future<Nothing> DockerContainerizerProcess::mountPersistentVolumes(
 
   Try<Nothing> updateVolumes = updatePersistentVolumes(
       containerId,
-      container->directory,
+      container->containerWorkDir,
       Resources(),
       container->resources);
 
@@ -773,18 +773,12 @@ Try<Nothing> DockerContainerizerProcess::checkpoint(
 
   container->executorPid = pid;
 
-  if (container->checkpoint) {
-    const string& path =
-      slave::paths::getForkedPidPath(
-          slave::paths::getMetaRootDir(flags.work_dir),
-          container->slaveId,
-          container->executor.framework_id(),
-          container->executor.executor_id(),
-          containerId);
-
-    LOG(INFO) << "Checkpointing pid " << pid << " to '" << path << "'";
+  if (container->pidCheckpointPath.isSome()) {
+    LOG(INFO) << "Checkpointing pid " << pid
+              << " to '" << container->pidCheckpointPath.get() << "'";
 
-    return slave::state::checkpoint(path, stringify(pid));
+    return slave::state::checkpoint(
+        container->pidCheckpointPath.get(), stringify(pid));
   }
 
   return Nothing();
@@ -803,25 +797,17 @@ Future<Nothing> DockerContainerizer::recover(
 
 Future<bool> DockerContainerizer::launch(
     const ContainerID& containerId,
-    const Option<TaskInfo>& taskInfo,
-    const ExecutorInfo& executorInfo,
-    const string& directory,
-    const Option<string>& user,
-    const SlaveID& slaveId,
+    const ContainerConfig& containerConfig,
     const map<string, string>& environment,
-    bool checkpoint)
+    const Option<string>& pidCheckpointPath)
 {
   return dispatch(
       process.get(),
       &DockerContainerizerProcess::launch,
       containerId,
-      taskInfo,
-      executorInfo,
-      directory,
-      user,
-      slaveId,
+      containerConfig,
       environment,
-      checkpoint);
+      pidCheckpointPath);
 }
 
 
@@ -1003,7 +989,6 @@ Future<Nothing> DockerContainerizerProcess::_recover(
         // Create and store a container.
         Container* container = new Container(containerId);
         containers_[containerId] = container;
-        container->slaveId = state->id;
         container->state = Container::RUNNING;
         container->launchesExecutorContainer =
           executorContainers.contains(containerId);
@@ -1040,7 +1025,7 @@ Future<Nothing> DockerContainerizerProcess::_recover(
             executor.id,
             containerId);
 
-        container->directory = sandboxDirectory;
+        container->containerWorkDir = sandboxDirectory;
       }
     }
   }
@@ -1106,47 +1091,33 @@ Future<Nothing> DockerContainerizerProcess::__recover(
 
 Future<bool> DockerContainerizerProcess::launch(
     const ContainerID& containerId,
-    const Option<TaskInfo>& taskInfo,
-    const ExecutorInfo& executorInfo,
-    const string& directory,
-    const Option<string>& user,
-    const SlaveID& slaveId,
+    const ContainerConfig& containerConfig,
     const map<string, string>& environment,
-    bool checkpoint)
+    const Option<string>& pidCheckpointPath)
 {
-  CHECK(!containerId.has_parent());
+  if (containerId.has_parent()) {
+    return Failure("Nested containers are not supported");
+  }
 
   if (containers_.contains(containerId)) {
     return Failure("Container already started");
   }
 
-  Option<ContainerInfo> containerInfo;
-
-  if (taskInfo.isSome() && taskInfo.get().has_container()) {
-    containerInfo = taskInfo.get().container();
-  } else if (executorInfo.has_container()) {
-    containerInfo = executorInfo.container();
-  }
-
-  if (containerInfo.isNone()) {
+  if (!containerConfig.has_container_info()) {
     LOG(INFO) << "No container info found, skipping launch";
     return false;
   }
 
-  if (containerInfo.get().type() != ContainerInfo::DOCKER) {
+  if (containerConfig.container_info().type() != ContainerInfo::DOCKER) {
     LOG(INFO) << "Skipping non-docker container";
     return false;
   }
 
   Try<Container*> container = Container::create(
       containerId,
-      taskInfo,
-      executorInfo,
-      directory,
-      user,
-      slaveId,
+      containerConfig,
       environment,
-      checkpoint,
+      pidCheckpointPath,
       flags);
 
   if (container.isError()) {
@@ -1155,28 +1126,27 @@ Future<bool> DockerContainerizerProcess::launch(
 
   containers_[containerId] = container.get();
 
-  if (taskInfo.isSome()) {
-    LOG(INFO) << "Starting container '" << containerId
-              << "' for task '" << taskInfo.get().task_id()
-              << "' (and executor '" << executorInfo.executor_id()
-              << "') of framework " << executorInfo.framework_id();
-  } else {
-    LOG(INFO) << "Starting container '" << containerId
-              << "' for executor '" << executorInfo.executor_id()
-              << "' and framework " << executorInfo.framework_id();
-  }
+  LOG(INFO)
+    << "Starting container '" << containerId
+    << (containerConfig.has_task_info()
+        ? "' for task '" + stringify(containerConfig.task_info().task_id())
+        : "")
+    << "' (and executor '" << containerConfig.executor_info().executor_id()
+    << "') of framework " << containerConfig.executor_info().framework_id();
 
   Future<Nothing> f = Nothing();
 
   if (HookManager::hooksAvailable()) {
     f = HookManager::slavePreLaunchDockerTaskExecutorDecorator(
-        taskInfo,
-        executorInfo,
+        containerConfig.has_task_info()
+          ? containerConfig.task_info()
+          : Option<TaskInfo>::none(),
+        containerConfig.executor_info(),
         container.get()->containerName,
-        container.get()->directory,
+        container.get()->containerWorkDir,
         flags.sandbox_directory,
         container.get()->environment)
-      .then(defer(self(), [this, taskInfo, containerId](
+      .then(defer(self(), [this, containerId, containerConfig](
           const DockerTaskExecutorPrepareInfo& decoratorInfo)
           -> Future<Nothing> {
         if (!containers_.contains(containerId)) {
@@ -1209,7 +1179,7 @@ Future<bool> DockerContainerizerProcess::launch(
           taskEnvironment[variable.name()] = variable.value();
         }
 
-        if (taskInfo.isSome()) {
+        if (containerConfig.has_task_info()) {
           container->taskEnvironment = taskEnvironment;
 
           // For dockerized command executors, the flags have already
@@ -1242,19 +1212,13 @@ Future<bool> DockerContainerizerProcess::launch(
       self(),
       &Self::_launch,
       containerId,
-      taskInfo,
-      executorInfo,
-      directory,
-      slaveId));
+      containerConfig));
 }
 
 
 Future<bool> DockerContainerizerProcess::_launch(
     const ContainerID& containerId,
-    const Option<TaskInfo>& taskInfo,
-    const ExecutorInfo& executorInfo,
-    const string& directory,
-    const SlaveID& slaveId)
+    const ContainerConfig& containerConfig)
 {
   if (!containers_.contains(containerId)) {
     return Failure("Container is already destroyed");
@@ -1262,20 +1226,21 @@ Future<bool> DockerContainerizerProcess::_launch(
 
   Container* container = containers_.at(containerId);
 
-  if (taskInfo.isSome() && flags.docker_mesos_image.isNone()) {
+  if (containerConfig.has_task_info() && flags.docker_mesos_image.isNone()) {
     // Launching task by forking a subprocess to run docker executor.
     // TODO(steveniemitz): We should call 'update' to set CPU/CFS/mem
     // quotas after 'launchExecutorProcess'. However, there is a race
     // where 'update' can be called before mesos-docker-executor
     // creates the Docker container for the task. See more details in
     // the comments of r33174.
-    return container->launch = fetch(containerId, slaveId)
+    return container->launch = fetch(containerId)
       .then(defer(self(), [=]() {
         return pull(containerId);
       }))
       .then(defer(self(), [=]() {
         if (HookManager::hooksAvailable()) {
-          HookManager::slavePostFetchHook(containerId, directory);
+          HookManager::slavePostFetchHook(
+              containerId, containerConfig.directory());
         }
 
         return mountPersistentVolumes(containerId);
@@ -1301,13 +1266,14 @@ Future<bool> DockerContainerizerProcess::_launch(
   // We need to do so for launching a task because as the slave is
   // running in a container (via docker_mesos_image flag) we want the
   // executor to keep running when the slave container dies.
-  return container->launch = fetch(containerId, slaveId)
+  return container->launch = fetch(containerId)
     .then(defer(self(), [=]() {
       return pull(containerId);
     }))
     .then(defer(self(), [=]() {
       if (HookManager::hooksAvailable()) {
-        HookManager::slavePostFetchHook(containerId, directory);
+        HookManager::slavePostFetchHook(
+            containerId, containerConfig.directory());
       }
 
       return mountPersistentVolumes(containerId);
@@ -1321,7 +1287,8 @@ Future<bool> DockerContainerizerProcess::_launch(
       // is >= 1.7 this can be changed to pass --cpu-period and
       // --cpu-quota to the 'docker run' call in
       // launchExecutorContainer.
-      return update(containerId, executorInfo.resources(), true)
+      return update(
+          containerId, containerConfig.executor_info().resources(), true)
         .then([=]() {
           return Future<Docker::Container>(dockerContainer);
         });
@@ -1347,9 +1314,11 @@ Future<Docker::Container> DockerContainerizerProcess::launchExecutorContainer(
   container->state = Container::RUNNING;
 
   return logger->prepare(
-      container->executor,
-      container->directory,
-      container->user)
+      container->containerConfig.executor_info(),
+      container->containerWorkDir,
+      container->containerConfig.has_user()
+        ? container->containerConfig.user()
+        : Option<string>::none())
     .then(defer(
         self(),
         [=](const ContainerIO& containerIO)
@@ -1358,7 +1327,7 @@ Future<Docker::Container> DockerContainerizerProcess::launchExecutorContainer(
         container->container,
         container->command,
         containerName,
-        container->directory,
+        container->containerWorkDir,
         flags.sandbox_directory,
         container->resources,
 #ifdef __linux__
@@ -1436,7 +1405,8 @@ Future<pid_t> DockerContainerizerProcess::launchExecutorProcess(
 
   // Include any environment variables from ExecutorInfo.
   foreach (const Environment::Variable& variable,
-           container->executor.command().environment().variables()) {
+           container->containerConfig.executor_info()
+             .command().environment().variables()) {
     environment[variable.name()] = variable.value();
   }
 
@@ -1486,9 +1456,11 @@ Future<pid_t> DockerContainerizerProcess::launchExecutorProcess(
   return allocateGpus
     .then(defer(self(), [=]() {
       return logger->prepare(
-          container->executor,
-          container->directory,
-          container->user);
+          container->containerConfig.executor_info(),
+          container->containerWorkDir,
+          container->containerConfig.has_user()
+            ? container->containerConfig.user()
+            : Option<string>::none());
     }))
     .then(defer(
         self(),
@@ -1529,7 +1501,7 @@ Future<pid_t> DockerContainerizerProcess::launchExecutorProcess(
     ::mesos::internal::docker::Flags launchFlags = dockerFlags(
         flags,
         container->containerName,
-        container->directory,
+        container->containerWorkDir,
         container->taskEnvironment);
 
     VLOG(1) << "Launching 'mesos-docker-executor' with flags '"
@@ -1549,7 +1521,7 @@ Future<pid_t> DockerContainerizerProcess::launchExecutorProcess(
         None(),
         parentHooks,
         {Subprocess::ChildHook::SETSID(),
-         Subprocess::ChildHook::CHDIR(container->directory)});
+         Subprocess::ChildHook::CHDIR(container->containerWorkDir)});
 
     if (s.isError()) {
       return Failure("Failed to fork executor: " + s.error());

http://git-wip-us.apache.org/repos/asf/mesos/blob/66070ebd/src/slave/containerizer/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.hpp b/src/slave/containerizer/docker.hpp
index 2ed8e1c..b602a56 100644
--- a/src/slave/containerizer/docker.hpp
+++ b/src/slave/containerizer/docker.hpp
@@ -88,13 +88,9 @@ public:
 
   virtual process::Future<bool> launch(
       const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
+      const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,
-      bool checkpoint);
+      const Option<std::string>& pidCheckpointPath);
 
   virtual process::Future<Nothing> update(
       const ContainerID& containerId,
@@ -140,13 +136,9 @@ public:
 
   virtual process::Future<bool> launch(
       const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
+      const mesos::slave::ContainerConfig& containerConfig,
       const std::map<std::string, std::string>& environment,
-      bool checkpoint);
+      const Option<std::string>& pidCheckpointPath);
 
   // force = true causes the containerizer to update the resources
   // for the container, even if they match what it has cached.
@@ -168,9 +160,7 @@ public:
       const ContainerID& containerId,
       bool killed = true); // process is either killed or reaped.
 
-  virtual process::Future<Nothing> fetch(
-      const ContainerID& containerId,
-      const SlaveID& slaveId);
+  virtual process::Future<Nothing> fetch(const ContainerID& containerId);
 
   virtual process::Future<Nothing> pull(const ContainerID& containerId);
 
@@ -188,10 +178,7 @@ private:
 
   process::Future<bool> _launch(
       const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const SlaveID& slaveId);
+      const mesos::slave::ContainerConfig& containerConfig);
 
   process::Future<Nothing> _recover(
       const Option<state::SlaveState>& state,
@@ -307,13 +294,9 @@ private:
   {
     static Try<Container*> create(
         const ContainerID& id,
-        const Option<TaskInfo>& taskInfo,
-        const ExecutorInfo& executorInfo,
-        const std::string& directory,
-        const Option<std::string>& user,
-        const SlaveID& slaveId,
+        const mesos::slave::ContainerConfig& containerConfig,
         const std::map<std::string, std::string>& environment,
-        bool checkpoint,
+        const Option<std::string>& pidCheckpointPath,
         const Flags& flags);
 
     static std::string name(const ContainerID& id)
@@ -324,30 +307,23 @@ private:
     Container(const ContainerID& id)
       : state(FETCHING), id(id) {}
 
-    Container(const ContainerID& id,
-              const Option<TaskInfo>& taskInfo,
-              const ExecutorInfo& executorInfo,
-              const std::string& directory,
-              const Option<std::string>& user,
-              const SlaveID& slaveId,
-              bool checkpoint,
-              bool symlinked,
-              const Flags& flags,
-              const Option<CommandInfo>& _command,
-              const Option<ContainerInfo>& _container,
-              const std::map<std::string, std::string>& _environment,
-              bool launchesExecutorContainer)
+    Container(
+        const ContainerID& _id,
+        const mesos::slave::ContainerConfig& _containerConfig,
+        const std::map<std::string, std::string>& _environment,
+        const Option<std::string>& _pidCheckpointPath,
+        bool symlinked,
+        const std::string& containerWorkDir,
+        const Option<CommandInfo>& _command,
+        const Option<ContainerInfo>& _container,
+        bool launchesExecutorContainer)
       : state(FETCHING),
-        id(id),
-        task(taskInfo),
-        executor(executorInfo),
+        id(_id),
+        containerConfig(_containerConfig),
+        pidCheckpointPath(_pidCheckpointPath),
         environment(_environment),
-        directory(directory),
-        user(user),
-        slaveId(slaveId),
-        checkpoint(checkpoint),
         symlinked(symlinked),
-        flags(flags),
+        containerWorkDir(containerWorkDir),
         containerName(name(id)),
         launchesExecutorContainer(launchesExecutorContainer)
     {
@@ -361,26 +337,24 @@ private:
       // perfect check because an executor might always have a subset
       // of it's resources that match a task, nevertheless, it's
       // better than nothing).
-      resources = executor.resources();
+      resources = containerConfig.resources();
 
-      if (task.isSome()) {
-        CHECK(resources.contains(task.get().resources()));
+      if (containerConfig.has_task_info()) {
+        CHECK(resources.contains(containerConfig.task_info().resources()));
       }
 
       if (_command.isSome()) {
         command = _command.get();
-      } else if (task.isSome()) {
-        command = task.get().command();
       } else {
-        command = executor.command();
+        command = containerConfig.command_info();
       }
 
       if (_container.isSome()) {
         container = _container.get();
-      } else if (task.isSome()) {
-        container = task.get().container();
       } else {
-        container = executor.container();
+        // NOTE: The existence of this field is checked in
+        // DockerContainerizerProcess::launch.
+        container = containerConfig.container_info();
       }
     }
 
@@ -389,7 +363,7 @@ private:
       if (symlinked) {
         // The sandbox directory is a symlink, remove it at container
         // destroy.
-        os::rm(directory);
+        os::rm(containerWorkDir);
       }
     }
 
@@ -404,20 +378,22 @@ private:
 
     std::string image() const
     {
-      if (task.isSome()) {
-        return task.get().container().docker().image();
+      if (containerConfig.has_task_info()) {
+        return containerConfig.task_info().container().docker().image();
       }
 
-      return executor.container().docker().image();
+      return containerConfig.executor_info().container().docker().image();
     }
 
     bool forcePullImage() const
     {
-      if (task.isSome()) {
-        return task.get().container().docker().force_pull_image();
+      if (containerConfig.has_task_info()) {
+        return containerConfig.task_info()
+          .container().docker().force_pull_image();
       }
 
-      return executor.container().docker().force_pull_image();
+      return containerConfig.executor_info()
+        .container().docker().force_pull_image();
     }
 
     // The DockerContainerizer needs to be able to properly clean up
@@ -452,27 +428,33 @@ private:
       DESTROYING = 5
     } state;
 
+    // Copies of the parameters sent to `Container::create`.
     const ContainerID id;
-    const Option<TaskInfo> task;
-    const ExecutorInfo executor;
+    const mesos::slave::ContainerConfig containerConfig;
+    const Option<std::string> pidCheckpointPath;
+
+    // A copy of the parameter sent to `Container::create`.
+    // NOTE: This may be modified further by hooks.
+    std::map<std::string, std::string> environment;
+
+    // The sandbox directory for the container. This holds the
+    // symlinked path if symlinked boolean is true.
+    // TODO(josephw): The symlink path does not persist across failovers,
+    // so we will not delete the symlink if the agent restarts. This results
+    // in gradually leaking hanging symlinks.
+    bool symlinked;
+    std::string containerWorkDir;
+
+    // Copies of the fields in `containerConfig`, except when the
+    // container is a command task and the agent is launched with
+    // the --docker_mesos_image flag.
     ContainerInfo container;
     CommandInfo command;
-    std::map<std::string, std::string> environment;
 
     // Environment variables that the command executor should pass
     // onto a docker-ized task. This is set by a hook.
     Option<std::map<std::string, std::string>> taskEnvironment;
 
-    // The sandbox directory for the container. This holds the
-    // symlinked path if symlinked boolean is true.
-    std::string directory;
-
-    const Option<std::string> user;
-    SlaveID slaveId;
-    bool checkpoint;
-    bool symlinked;
-    const Flags flags;
-
     // The string used to refer to this container via the Docker CLI.
     // This name is either computed by concatenating the DOCKER_NAME_PREFIX
     // and the ContainerID; or during recovery, by taking the recovered


[03/16] mesos git commit: Combined containerizer interface's launch methods.

Posted by jo...@apache.org.
Combined containerizer interface's launch methods.

When nested container support was added, we added a separate `launch`
path in the containerizer because nested containers do not need
an explicit TaskInfo/ExecutorInfo.  Nested containers basically
only need the CommandInfo and ContainerInfo.

This commit combines the two launch methods by replacing most of the
"Infos" (Task, Executor, Command, Container) with a `ContainerConfig`
argument, which may contain multiple combinations of the "Infos".

The goal is to support three launch paths for containers:
  1) When the `ContainerConfig` contains a TaskInfo/ExecutorInfo,
     launch a task or executor.
  2) When the `ContainerID` has a parent, launch a nested container.
  3) (Not implemented yet) When there is no TaskInfo/ExecutorInfo or
     parent container, launch a standalone container.

There are two other notable changes to the interface:
  * The `SlaveID` field has been removed entirely.  The code that
    requires this (in the fetcher and Docker containerizer) will be
    addressed in a separate commit.
  * The `checkpoint` bool has been replaced by an Option<string>,
    which contains the path that should be used for checkpointing.
    This path includes the filename.
    This is also one of the reasons why `SlaveID` was an argument.

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


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

Branch: refs/heads/master
Commit: 0709ce3c7a3c09948e23761281ad9c5059484916
Parents: 7a8f864
Author: Joseph Wu <jo...@apache.org>
Authored: Mon Apr 10 14:06:33 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:06 2017 -0700

----------------------------------------------------------------------
 src/slave/containerizer/containerizer.hpp | 36 ++++++++------------------
 1 file changed, 11 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0709ce3c/src/slave/containerizer/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
index f17e424..0954ed6 100644
--- a/src/slave/containerizer/containerizer.hpp
+++ b/src/slave/containerizer/containerizer.hpp
@@ -81,34 +81,20 @@ public:
   virtual process::Future<Nothing> recover(
       const Option<state::SlaveState>& state) = 0;
 
-  // Launch a containerized task/executor. Returns true if launching
-  // this TaskInfo/ExecutorInfo is supported and it has been launched,
-  // otherwise false or a failure if something went wrong.
-  virtual process::Future<bool> launch(
-      const ContainerID& containerId,
-      const Option<TaskInfo>& taskInfo,
-      const ExecutorInfo& executorInfo,
-      const std::string& directory,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
-      const std::map<std::string, std::string>& environment,
-      bool checkpoint) = 0;
-
-  // Launch a nested container.
-  // TODO(jieyu): Consider combining with the 'launch' above.
+  // Launch a container with the specified ContainerConfig.
+  //
+  // If the ContainerID has a parent, this will attempt to launch
+  // a nested container.
+  // NOTE: For nested containers, the required `directory` field of
+  // the ContainerConfig will be determined by the containerizer.
   //
-  // TODO(gilbert): Remove the 'slaveId' once the fetcher does
-  // not rely on SlaveID.
+  // 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(
       const ContainerID& containerId,
-      const CommandInfo& commandInfo,
-      const Option<ContainerInfo>& containerInfo,
-      const Option<std::string>& user,
-      const SlaveID& slaveId,
-      const Option<mesos::slave::ContainerClass>& containerClass = None())
-  {
-    return process::Failure("Unsupported");
-  }
+      const mesos::slave::ContainerConfig& containerConfig,
+      const std::map<std::string, std::string>& environment,
+      const Option<std::string>& pidCheckpointPath) = 0;
 
   // Create an HTTP connection that can be used to "attach" (i.e.,
   // stream input to or stream output from) a container.


[02/16] mesos git commit: Changed containerizer->launch callsites to new interface.

Posted by jo...@apache.org.
Changed containerizer->launch callsites to new interface.

As part of combining the two `containerizer->launch` code paths,
callers now need to construct the `ContainerConfig` (instead of
passing some fields that are then copied into a `ContainerConfig`).

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


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

Branch: refs/heads/master
Commit: af21bb71655053d223eec1595a0384fe9e26f8c7
Parents: a6a7a38
Author: Joseph Wu <jo...@apache.org>
Authored: Mon Apr 10 16:04:55 2017 -0700
Committer: Joseph Wu <jo...@apache.org>
Committed: Thu May 25 18:37:06 2017 -0700

----------------------------------------------------------------------
 src/slave/http.cpp  |  26 +++++++++---
 src/slave/slave.cpp | 101 +++++++++++++++++++++++++++++------------------
 2 files changed, 83 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/af21bb71/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 4444a38..3160407 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -15,6 +15,7 @@
 // limitations under the License.
 
 #include <list>
+#include <map>
 #include <memory>
 #include <sstream>
 #include <string>
@@ -78,6 +79,7 @@ using mesos::authorization::createSubject;
 using mesos::internal::recordio::Reader;
 
 using mesos::slave::ContainerClass;
+using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerTermination;
 
 using process::AUTHENTICATION;
@@ -116,6 +118,7 @@ using process::metrics::internal::MetricsProcess;
 using ::recordio::Decoder;
 
 using std::list;
+using std::map;
 using std::string;
 using std::tie;
 using std::tuple;
@@ -2327,13 +2330,26 @@ Future<Response> Slave::Http::_launchNestedContainer(
   }
 #endif
 
+  ContainerConfig containerConfig;
+  containerConfig.mutable_command_info()->CopyFrom(commandInfo);
+
+  if (user.isSome()) {
+    containerConfig.set_user(user.get());
+  }
+
+  if (containerInfo.isSome()) {
+    containerConfig.mutable_container_info()->CopyFrom(containerInfo.get());
+  }
+
+  if (containerClass.isSome()) {
+    containerConfig.set_container_class(containerClass.get());
+  }
+
   Future<bool> launched = slave->containerizer->launch(
       containerId,
-      commandInfo,
-      containerInfo,
-      user,
-      slave->info.id(),
-      containerClass);
+      containerConfig,
+      map<string, string>(),
+      None());
 
   // TODO(bmahler): The containerizers currently require that
   // the caller calls destroy if the launch fails. See MESOS-6214.

http://git-wip-us.apache.org/repos/asf/mesos/blob/af21bb71/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 0f21cf8..c578c52 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -117,6 +117,7 @@ using mesos::executor::Call;
 
 using mesos::master::detector::MasterDetector;
 
+using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerTermination;
 using mesos::slave::QoSController;
 using mesos::slave::QoSCorrection;
@@ -2686,6 +2687,8 @@ void Slave::launchExecutor(
   }
 
   // Tell the containerizer to launch the executor.
+  // NOTE: We make a copy of the executor info because we may mutate
+  // it with some default fields and resources.
   ExecutorInfo executorInfo_ = executor->info;
 
   // Populate the command info for default executor. We modify the ExecutorInfo
@@ -2711,6 +2714,41 @@ void Slave::launchExecutor(
 
   executorInfo_.mutable_resources()->CopyFrom(resources);
 
+  // Add the default container info to the executor info.
+  // TODO(jieyu): Rename the flag to be default_mesos_container_info.
+  if (!executorInfo_.has_container() &&
+      flags.default_container_info.isSome()) {
+    executorInfo_.mutable_container()->CopyFrom(
+        flags.default_container_info.get());
+  }
+
+  // Bundle all the container launch fields together.
+  ContainerConfig containerConfig;
+  containerConfig.mutable_executor_info()->CopyFrom(executorInfo_);
+  containerConfig.mutable_command_info()->CopyFrom(executorInfo_.command());
+  containerConfig.mutable_resources()->CopyFrom(executorInfo_.resources());
+  containerConfig.set_directory(executor->directory);
+
+  if (executor->user.isSome()) {
+    containerConfig.set_user(executor->user.get());
+  }
+
+  if (executor->isCommandExecutor()) {
+    if (taskInfo.isSome()) {
+      containerConfig.mutable_task_info()->CopyFrom(taskInfo.get());
+
+      if (taskInfo.get().has_container()) {
+        containerConfig.mutable_container_info()
+          ->CopyFrom(taskInfo.get().container());
+      }
+    }
+  } else {
+    if (executorInfo_.has_container()) {
+      containerConfig.mutable_container_info()
+        ->CopyFrom(executorInfo_.container());
+    }
+  }
+
   // Prepare environment variables for the executor.
   map<string, string> environment = executorEnvironment(
       flags,
@@ -2721,48 +2759,33 @@ void Slave::launchExecutor(
       authenticationToken,
       framework->info.checkpoint());
 
-  // Launch the container.
-  Future<bool> launch;
-  if (!executor->isCommandExecutor()) {
-    // If the executor is _not_ a command executor, this means that
-    // the task will include the executor to run. The actual task to
-    // run will be enqueued and subsequently handled by the executor
-    // when it has registered to the slave.
-    launch = containerizer->launch(
-        executor->containerId,
-        None(),
-        executorInfo_,
-        executor->directory,
-        executor->user,
-        info.id(),
-        environment,
-        framework->info.checkpoint());
-  } else {
-    // An executor has _not_ been provided by the task and will
-    // instead define a command and/or container to run. Right now,
-    // these tasks will require an executor anyway and the slave
-    // creates a command executor. However, it is up to the
-    // containerizer how to execute those tasks and the generated
-    // executor info works as a placeholder.
-    // TODO(nnielsen): Obsolete the requirement for executors to run
-    // one-off tasks.
-    launch = containerizer->launch(
-        executor->containerId,
-        taskInfo,
-        executorInfo_,
-        executor->directory,
-        executor->user,
+  // Prepare the filename of the pidfile, for checkpoint-enabled frameworks.
+  Option<string> pidCheckpointPath = None();
+  if (framework->info.checkpoint()){
+    pidCheckpointPath = slave::paths::getForkedPidPath(
+        slave::paths::getMetaRootDir(flags.work_dir),
         info.id(),
-        environment,
-        framework->info.checkpoint());
+        framework->id(),
+        executor->id,
+        executor->containerId);
   }
 
-  launch.onAny(defer(self(),
-                     &Self::executorLaunched,
-                     frameworkId,
-                     executor->id,
-                     executor->containerId,
-                     lambda::_1));
+  LOG(INFO) << "Launching container " << executor->containerId
+            << " for executor '" << executor->id
+            << "' of framework " << framework->id();
+
+  // Launch the container.
+  containerizer->launch(
+      executor->containerId,
+      containerConfig,
+      environment,
+      pidCheckpointPath)
+    .onAny(defer(self(),
+                 &Self::executorLaunched,
+                 frameworkId,
+                 executor->id,
+                 executor->containerId,
+                 lambda::_1));
 
   // Make sure the executor registers within the given timeout.
   delay(flags.executor_registration_timeout,