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

[2/5] mesos git commit: Checkpoint and recover `ContainerConfig` in Mesos containerizer.

Checkpoint and recover `ContainerConfig` in Mesos containerizer.

This patch includes the following change:
- Checkpointed `ContainerConfig` used to launch a container;
- Added helper function to read checkpointed `ContainerConfig`;
- Recovered `ContainerConfig`.
- Make `ContainerConfig` Option in `Container` struct to indicate
  whether recovered.

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


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

Branch: refs/heads/master
Commit: 03a2a4dfa47b1d47c5eb23e81f5ef8213e46d545
Parents: b433262
Author: Zhitao Li <zh...@gmail.com>
Authored: Fri Nov 17 16:36:24 2017 -0800
Committer: Gilbert Song <so...@gmail.com>
Committed: Mon Nov 20 12:28:11 2017 -0800

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 144 +++++++++++++------
 src/slave/containerizer/mesos/containerizer.hpp |   9 +-
 src/slave/containerizer/mesos/paths.cpp         |  37 ++++-
 src/slave/containerizer/mesos/paths.hpp         |   8 ++
 4 files changed, 152 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/03a2a4df/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index db5f044..bf71db1 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -759,6 +759,24 @@ Future<Nothing> MesosContainerizerProcess::recover(
     container->state = RUNNING;
     container->pid = state.pid();
     container->directory = state.directory();
+
+    // Attempt to read the launch config of the container.
+    Result<ContainerConfig> config =
+      containerizer::paths::getContainerConfig(flags.runtime_dir, containerId);
+
+    if (config.isError()) {
+      return Failure(
+        "Failed to get config for container " + stringify(containerId) +
+        ": " + config.error());
+    }
+
+    if (config.isSome()) {
+      container->config = config.get();
+    } else {
+      VLOG(1) << "No config is recovered for container " << containerId
+              << ", this means image pruning will be disabled.";
+    }
+
     containers_[containerId] = container;
   }
 
@@ -812,6 +830,14 @@ Future<Nothing> MesosContainerizerProcess::recover(
       return Failure("Failed to get container pid: " + pid.error());
     }
 
+    // Attempt to read the launch config of the container.
+    Result<ContainerConfig> config =
+      containerizer::paths::getContainerConfig(flags.runtime_dir, containerId);
+
+    if (config.isError()) {
+      return Failure("Failed to get container config: " + config.error());
+    }
+
     // Determine the sandbox if this is a nested or standalone container.
     const bool isStandaloneContainer =
       containerizer::paths::isStandaloneContainer(
@@ -848,6 +874,15 @@ Future<Nothing> MesosContainerizerProcess::recover(
       container->status = Future<Option<int>>(None());
     }
 
+    if (config.isSome()) {
+      container->config = ContainerConfig();
+      container->config->CopyFrom(config.get());
+    } else {
+      VLOG(1) << "No checkpointed config recovered for container "
+              << containerId << ", this means image pruning will "
+              << "be disabled.";
+    }
+
     containers_[containerId] = container;
 
     // TODO(klueska): The final check in the if statement makes sure
@@ -989,8 +1024,8 @@ Future<Nothing> MesosContainerizerProcess::__recover(
 
     if (containerLaunchInfo.isError()) {
       return Failure(
-          "Failed to recover launch information of container '" +
-          stringify(containerId) + "': " + containerLaunchInfo.error());
+          "Failed to recover launch information of container " +
+          stringify(containerId) + ": " + containerLaunchInfo.error());
     }
 
     if (containerLaunchInfo.isSome()) {
@@ -1140,9 +1175,14 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::launch(
     // 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());
+    if (containers_[rootContainerId]->config.isSome()) {
+      if (containers_[rootContainerId]->config->has_executor_info()) {
+        containerConfig.mutable_executor_info()->CopyFrom(
+          containers_[rootContainerId]->config->executor_info());
+      }
+    } else {
+      LOG(WARNING) << "Cannot determine executor_info for root container '"
+                   << rootContainerId << "' which has no config recovered.";
     }
   }
 
@@ -1279,11 +1319,10 @@ Future<Nothing> MesosContainerizerProcess::prepare(
   }
 
   CHECK_EQ(container->state, PROVISIONING);
-
-  transition(containerId, PREPARING);
+  CHECK_SOME(container->config);
 
   if (provisionInfo.isSome()) {
-    container->config.set_rootfs(provisionInfo->rootfs);
+    container->config->set_rootfs(provisionInfo->rootfs);
 
     if (provisionInfo->dockerManifest.isSome() &&
         provisionInfo->appcManifest.isSome()) {
@@ -1291,18 +1330,37 @@ Future<Nothing> MesosContainerizerProcess::prepare(
     }
 
     if (provisionInfo->dockerManifest.isSome()) {
-      ContainerConfig::Docker* docker = container->config.mutable_docker();
+      ContainerConfig::Docker* docker = container->config->mutable_docker();
       docker->mutable_manifest()->CopyFrom(provisionInfo->dockerManifest.get());
     }
 
     if (provisionInfo->appcManifest.isSome()) {
-      ContainerConfig::Appc* appc = container->config.mutable_appc();
+      ContainerConfig::Appc* appc = container->config->mutable_appc();
       appc->mutable_manifest()->CopyFrom(provisionInfo->appcManifest.get());
     }
   }
 
   // Captured for lambdas below.
-  ContainerConfig containerConfig = container->config;
+  ContainerConfig containerConfig = container->config.get();
+
+  // Checkpoint the `ContainerConfig` which includes all information to launch a
+  // container. Critical information (e.g., `ContainerInfo`) can be used for
+  // tracking container image usage.
+  const string configPath = path::join(
+      containerizer::paths::getRuntimePath(flags.runtime_dir, containerId),
+      containerizer::paths::CONTAINER_CONFIG_FILE);
+
+  Try<Nothing> configCheckpointed =
+    slave::state::checkpoint(configPath, containerConfig);
+
+  if (configCheckpointed.isError()) {
+    return Failure("Failed to checkpoint the container config to '" +
+                   configPath + "': " + configCheckpointed.error());
+  }
+
+  VLOG(1) << "Checkpointed ContainerConfig at '" << configPath << "'";
+
+  transition(containerId, PREPARING);
 
   // We prepare the isolators sequentially according to their ordering
   // to permit basic dependency specification, e.g., preparing a
@@ -1350,14 +1408,16 @@ Future<Nothing> MesosContainerizerProcess::fetch(
 
   transition(containerId, FETCHING);
 
-  const string directory = container->config.directory();
+  CHECK_SOME(container->config);
+
+  const string directory = container->config->directory();
 
   return fetcher->fetch(
       containerId,
-      container->config.command_info(),
+      container->config->command_info(),
       directory,
-      container->config.has_user()
-        ? container->config.user()
+      container->config->has_user()
+        ? container->config->user()
         : Option<string>::none())
     .then([=]() -> Future<Nothing> {
       if (HookManager::hooksAvailable()) {
@@ -1387,6 +1447,7 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
   CHECK(containerIO.isSome());
   CHECK_EQ(container->state, PREPARING);
   CHECK_READY(container->launchInfos);
+  CHECK_SOME(container->config);
 
   ContainerLaunchInfo launchInfo;
 
@@ -1461,11 +1522,11 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
 
   // Determine the launch command for the container.
   if (!launchInfo.has_command()) {
-    launchInfo.mutable_command()->CopyFrom(container->config.command_info());
+    launchInfo.mutable_command()->CopyFrom(container->config->command_info());
   } else {
     // For command tasks, merge the launch commands with the executor
     // launch command.
-    if (container->config.has_task_info()) {
+    if (container->config->has_task_info()) {
       // Isolators are not supposed to set any other fields in the
       // command except the arguments for the command executor.
       CHECK(launchInfo.command().uris().empty())
@@ -1482,7 +1543,7 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
       // NOTE: The ordering here is important because we want the
       // command executor arguments to be in front of the arguments
       // set by isolators. See details in MESOS-7909.
-      CommandInfo launchCommand = container->config.command_info();
+      CommandInfo launchCommand = container->config->command_info();
       launchCommand.MergeFrom(launchInfo.command());
       launchInfo.mutable_command()->CopyFrom(launchCommand);
     }
@@ -1492,7 +1553,7 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
   // flag to the launch command of the command executor.
   // TODO(tillt): Remove this once we no longer support the old style
   // command task (i.e., that uses mesos-execute).
-  if (container->config.has_task_info() && launchInfo.has_task_environment()) {
+  if (container->config->has_task_info() && launchInfo.has_task_environment()) {
     hashmap<string, string> commandTaskEnvironment;
 
     foreach (const Environment::Variable& variable,
@@ -1526,10 +1587,10 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
   // TODO(jieyu): Remove this once we no longer support the old style
   // command task (i.e., that uses mesos-execute).
   // TODO(jieyu): Consider move this to filesystem isolator.
-  if (container->config.has_task_info() &&
-      container->config.has_rootfs()) {
+  if (container->config->has_task_info() &&
+      container->config->has_rootfs()) {
     launchInfo.mutable_command()->add_arguments(
-        "--rootfs=" + container->config.rootfs());
+        "--rootfs=" + container->config->rootfs());
   }
 
   // TODO(jieyu): 'uris', 'environment' and 'user' in the launch
@@ -1561,8 +1622,8 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
   Environment containerEnvironment;
 
   // Inherit environment from the parent container for DEBUG containers.
-  if (container->config.has_container_class() &&
-      container->config.container_class() == ContainerClass::DEBUG) {
+  if (container->config->has_container_class() &&
+      container->config->container_class() == ContainerClass::DEBUG) {
     // DEBUG containers must have a parent.
     CHECK(containerId.has_parent());
     if (containers_[containerId.parent()]->launchInfo.isSome()) {
@@ -1579,8 +1640,8 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
   }
 
   // DEBUG containers inherit MESOS_SANDBOX from their parent.
-  if (!container->config.has_container_class() ||
-      container->config.container_class() != ContainerClass::DEBUG) {
+  if (!container->config->has_container_class() ||
+      container->config->container_class() != ContainerClass::DEBUG) {
     // TODO(jieyu): Consider moving this to filesystem isolator.
     //
     // NOTE: For the command executor case, although it uses the host
@@ -1589,9 +1650,9 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
     // itself does not use this environment variable.
     Environment::Variable* variable = containerEnvironment.add_variables();
     variable->set_name("MESOS_SANDBOX");
-    variable->set_value(container->config.has_rootfs()
+    variable->set_value(container->config->has_rootfs()
       ? flags.sandbox_directory
-      : container->config.directory());
+      : container->config->directory());
   }
 
   // `launchInfo.environment` contains the environment returned by
@@ -1603,9 +1664,9 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
   // Include user specified environment.
   // Skip over any secrets as they should have been resolved by the
   // environment_secret isolator.
-  if (container->config.command_info().has_environment()) {
+  if (container->config->command_info().has_environment()) {
     foreach (const Environment::Variable& variable,
-             container->config.command_info().environment().variables()) {
+             container->config->command_info().environment().variables()) {
       if (variable.type() != Environment::Variable::SECRET) {
         containerEnvironment.add_variables()->CopyFrom(variable);
       }
@@ -1620,8 +1681,9 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
   // NOTE: Command task is a special case. Even if the container
   // config has a root filesystem, the executor container still uses
   // the host filesystem.
-  if (!container->config.has_task_info() && container->config.has_rootfs()) {
-    launchInfo.set_rootfs(container->config.rootfs());
+  if (!container->config->has_task_info() &&
+      container->config->has_rootfs()) {
+    launchInfo.set_rootfs(container->config->rootfs());
   }
 
   // For a non-DEBUG container, working directory is set to container sandbox,
@@ -1631,22 +1693,22 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
   //
   // TODO(alexr): Determining working directory is a convoluted process. We
   // should either simplify the logic or extract it into a helper routine.
-  if (container->config.has_container_class() &&
-      container->config.container_class() == ContainerClass::DEBUG) {
+  if (container->config->has_container_class() &&
+      container->config->container_class() == ContainerClass::DEBUG) {
     // DEBUG containers must have a parent.
     CHECK(containerId.has_parent());
 
     if (containers_[containerId.parent()]->launchInfo.isSome()) {
       // TODO(alexr): Remove this once we no longer support executorless
       // command tasks in favor of default executor.
-      if (containers_[containerId.parent()]->config.has_task_info()) {
+      if (containers_[containerId.parent()]->config->has_task_info()) {
         // For the command executor case, even if the task itself has a root
         // filesystem, the executor container still uses the host filesystem,
         // hence `ContainerLaunchInfo.working_directory`, which points to the
         // executor working directory in the host filesystem, may be different
         // from the task working directory when task defines an image. Fall back
         // to the sandbox directory if task working directory is not present.
-        if (containers_[containerId.parent()]->config.has_rootfs()) {
+        if (containers_[containerId.parent()]->config->has_rootfs()) {
           // We can extract the task working directory from the flag being
           // passed to the command executor.
           foreach (
@@ -1699,18 +1761,18 @@ Future<Containerizer::LaunchResult> MesosContainerizerProcess::_launch(
                    << "host filesystem";
     }
 
-    launchInfo.set_working_directory(container->config.directory());
+    launchInfo.set_working_directory(container->config->directory());
   }
 
   // Determine the user to launch the container as.
-  if (container->config.has_user()) {
-    launchInfo.set_user(container->config.user());
+  if (container->config->has_user()) {
+    launchInfo.set_user(container->config->user());
   }
 
   // TODO(gilbert): Remove this once we no longer support command
   // task in favor of default executor.
-  if (container->config.has_task_info() &&
-      container->config.has_rootfs()) {
+  if (container->config->has_task_info() &&
+      container->config->has_rootfs()) {
     // We need to set the executor user as root as it needs to
     // perform chroot (even when switch_user is set to false).
     launchInfo.set_user("root");

http://git-wip-us.apache.org/repos/asf/mesos/blob/03a2a4df/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index f5d5146..e859b5b 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -342,9 +342,12 @@ private:
     // the ResourceStatistics limits in usage().
     Resources resources;
 
-    // The configuration for the container to be launched. This field
-    // is only used during the launch of a container.
-    mesos::slave::ContainerConfig config;
+    // The configuration for the container to be launched.
+    // This can only be None if the underlying container is launched
+    // before we checkpiont `ContainerConfig` in MESOS-6894.
+    // TODO(zhitao): Drop the `Option` part at the end of deprecation
+    // cycle.
+    Option<mesos::slave::ContainerConfig> config;
 
     // Container's information at the moment it was launched. For example,
     // used to bootstrap the launch information of future child DEBUG

http://git-wip-us.apache.org/repos/asf/mesos/blob/03a2a4df/src/slave/containerizer/mesos/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/paths.cpp b/src/slave/containerizer/mesos/paths.cpp
index 23f1fee..8a188a9 100644
--- a/src/slave/containerizer/mesos/paths.cpp
+++ b/src/slave/containerizer/mesos/paths.cpp
@@ -19,12 +19,15 @@
 #include <stout/path.hpp>
 #include <stout/protobuf.hpp>
 
+#include "common/protobuf_utils.hpp"
+
 #include "slave/containerizer/mesos/paths.hpp"
 
 #ifndef __WINDOWS__
 namespace unix = process::network::unix;
 #endif // __WINDOWS__
 
+using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
 using mesos::slave::ContainerTermination;
 
@@ -276,8 +279,8 @@ Result<ContainerTermination> getContainerTermination(
     ::protobuf::read<ContainerTermination>(path);
 
   if (termination.isError()) {
-    return Error("Failed to read termination state of container:"
-                 " " + termination.error());
+    return Error("Failed to read termination state of container: " +
+                 termination.error());
   }
 
   return termination;
@@ -304,6 +307,34 @@ bool isStandaloneContainer(
 }
 
 
+Result<ContainerConfig> getContainerConfig(
+    const string& runtimeDir,
+    const ContainerID& containerId)
+{
+  const string path = path::join(
+      getRuntimePath(runtimeDir, containerId),
+      CONTAINER_CONFIG_FILE);
+
+  if (!os::exists(path)) {
+    // This is possible if we recovered a container launched before we
+    // started to checkpoint `ContainerConfig`.
+    VLOG(1) << "Config path '" << path << "' is missing for container' "
+            << containerId << "'";
+    return None();
+  }
+
+  const Result<ContainerConfig>& containerConfig =
+    ::protobuf::read<ContainerConfig>(path);
+
+  if (containerConfig.isError()) {
+    return Error("Failed to read launch config of container: " +
+                 containerConfig.error());
+  }
+
+  return containerConfig;
+}
+
+
 Try<vector<ContainerID>> getContainerIds(const string& runtimeDir)
 {
   lambda::function<Try<vector<ContainerID>>(const Option<ContainerID>&)> helper;
@@ -453,6 +484,8 @@ Try<ContainerID> parseSandboxPath(
   return currentContainerId;
 }
 
+
+
 } // namespace paths {
 } // namespace containerizer {
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/03a2a4df/src/slave/containerizer/mesos/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/paths.hpp b/src/slave/containerizer/mesos/paths.hpp
index 7b67ccf..65830a1 100644
--- a/src/slave/containerizer/mesos/paths.hpp
+++ b/src/slave/containerizer/mesos/paths.hpp
@@ -48,6 +48,7 @@ namespace paths {
 //   root ('--runtime_dir' flag)
 //   |-- containers
 //       |-- <container_id>
+//           |-- config
 //           |-- containers
 //           |   |-- <container_id>
 //           |   |   |-- <more nesting of containers>
@@ -65,6 +66,7 @@ namespace paths {
 
 
 constexpr char PID_FILE[] = "pid";
+constexpr char CONTAINER_CONFIG_FILE[] = "config";
 constexpr char STATUS_FILE[] = "status";
 constexpr char TERMINATION_FILE[] = "termination";
 constexpr char SOCKET_FILE[] = "socket";
@@ -182,6 +184,12 @@ bool isStandaloneContainer(
     const ContainerID& containerId);
 
 
+// The helper method to read the launch config of the contaienr.
+Result<mesos::slave::ContainerConfig> getContainerConfig(
+    const std::string& runtimeDir,
+    const ContainerID& containerId);
+
+
 // The helper method to list all container IDs (including nested
 // containers) from the container runtime directory. The order of
 // returned vector is a result of pre-ordering walk (i.e., parent