You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2016/08/22 20:46:33 UTC

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

Refactored MesosContainerizer to prepare for nested container support.

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

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


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

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

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


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

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

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

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