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 2018/05/16 15:23:06 UTC

mesos git commit: Updated the container logger interface.

Repository: mesos
Updated Branches:
  refs/heads/master 8c25ef96f -> d30dc724b


Updated the container logger interface.

Container logger's prepare() method now takes `ContainerConfig` to be
consistent with the isolator interfaces. This allows the framework to
control the logger behavior for nested containers, as well as standalone
containers (by using environment variables).

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


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

Branch: refs/heads/master
Commit: d30dc724b512264b1a65f0b5b337fbe24bc4894d
Parents: 8c25ef9
Author: Jie Yu <yu...@gmail.com>
Authored: Tue May 15 17:37:13 2018 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed May 16 08:21:30 2018 -0700

----------------------------------------------------------------------
 CHANGELOG                                       |  9 ++++
 docs/logging.md                                 |  5 +--
 docs/upgrades.md                                | 37 +++++++++++++++++
 include/mesos/slave/container_logger.hpp        | 27 ++++++------
 src/slave/container_loggers/lib_logrotate.cpp   | 43 +++++++++-----------
 src/slave/container_loggers/lib_logrotate.hpp   | 16 ++++----
 src/slave/container_loggers/sandbox.cpp         | 30 ++++++++------
 src/slave/container_loggers/sandbox.hpp         | 19 ++++-----
 src/slave/containerizer/docker.cpp              | 14 +------
 .../containerizer/mesos/io/switchboard.cpp      |  7 +---
 10 files changed, 116 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d30dc724/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 1fb8450..cddc055 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,3 +1,12 @@
+Release Notes - Mesos - Version 1.7.0 (WIP)
+-------------------------------------------
+This release contains the following new features:
+
+Additional API Changes:
+  * Container logger module interface has been changed. The `prepare()` method
+    now takes `ContainerConfig` instead.
+
+
 Release Notes - Mesos - Version 1.6.1 (WIP)
 -------------------------------------------
 * This is a bug fix release.

http://git-wip-us.apache.org/repos/asf/mesos/blob/d30dc724/docs/logging.md
----------------------------------------------------------------------
diff --git a/docs/logging.md b/docs/logging.md
index 2803914..58899a8 100644
--- a/docs/logging.md
+++ b/docs/logging.md
@@ -135,10 +135,9 @@ setting the `--container_logger` Agent flag to
     </td>
     <td>
       Prefix for environment variables meant to modify the behavior of
-      the logrotate logger for the specific executor being launched.
+      the logrotate logger for the specific container being launched.
       The logger will look for four prefixed environment variables in the
-      <code>ExecutorInfo</code>'s <code>CommandInfo</code>'s
-      <code>Environment</code>:
+      container's <code>CommandInfo</code>'s <code>Environment</code>:
       <ul>
         <li><code>MAX_STDOUT_SIZE</code></li>
         <li><code>LOGROTATE_STDOUT_OPTIONS</code></li>

http://git-wip-us.apache.org/repos/asf/mesos/blob/d30dc724/docs/upgrades.md
----------------------------------------------------------------------
diff --git a/docs/upgrades.md b/docs/upgrades.md
index baf4a0f..5813ed7 100644
--- a/docs/upgrades.md
+++ b/docs/upgrades.md
@@ -43,6 +43,37 @@ We categorize the changes as follows:
   </thead>
 <tr>
   <td style="word-wrap: break-word; overflow-wrap: break-word;"><!--Version-->
+  1.7.x
+  </td>
+
+  <td style="word-wrap: break-word; overflow-wrap: break-word;"><!--Mesos Core-->
+    <ul style="padding-left:10px;">
+    </ul>
+  </td>
+
+  <td style="word-wrap: break-word; overflow-wrap: break-word;"><!--Flags-->
+    <ul style="padding-left:10px;">
+    </ul>
+  </td>
+
+  <td style="word-wrap: break-word; overflow-wrap: break-word;"><!--Framework API-->
+    <ul style="padding-left:10px;">
+    </ul>
+  </td>
+
+  <td style="word-wrap: break-word; overflow-wrap: break-word;"><!--Module API-->
+    <ul style="padding-left:10px;">
+      <li>C <a href="#1-7-x-container-logger">ContainerLogger module interface changes</a></li>
+    </ul>
+  </td>
+
+  <td style="word-wrap: break-word; overflow-wrap: break-word;"><!--Endpoints-->
+    <ul style="padding-left:10px;">
+    </ul>
+  </td>
+</tr>
+<tr>
+  <td style="word-wrap: break-word; overflow-wrap: break-word;"><!--Version-->
   1.6.x
   </td>
 
@@ -397,6 +428,12 @@ We categorize the changes as follows:
 </tr>
 </table>
 
+## Upgrading from 1.6.x to 1.7.x ##
+
+<a name="1-7-x-container-logger"></a>
+
+* `ContainerLogger` module interface has been changed. The `prepare()` method now takes `ContainerConfig` instead.
+
 ## Upgrading from 1.5.x to 1.6.x ##
 
 <a name="1-6-x-grpc-requirement"></a>

http://git-wip-us.apache.org/repos/asf/mesos/blob/d30dc724/include/mesos/slave/container_logger.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/slave/container_logger.hpp b/include/mesos/slave/container_logger.hpp
index e8f8838..5fe2369 100644
--- a/include/mesos/slave/container_logger.hpp
+++ b/include/mesos/slave/container_logger.hpp
@@ -41,9 +41,7 @@ namespace slave {
  * A containerizer component used to manage container logs.
  *
  * The `ContainerLogger` is responsible for handling the stdout/stderr of
- * containers.  The stdout/stderr of tasks launched without can executor
- * (that implicitly use the command executor) will also be handled by the
- * container logger.
+ * containers.
  *
  * The container logger is also responsible for providing a public interface
  * for retrieving the logs.
@@ -86,21 +84,20 @@ public:
    * within the `ContainerIO` as much as necessary, with some exceptions;
    * see the struct `ContainerIO` above.
    *
-   * NOTE: The container logger should not lose stdout/stderr if the agent
-   * fails over.  Additionally, if the container logger is stateful, the logger
-   * should be capable of recovering managed executors during the agent recovery
-   * process.  See `ContainerLogger::recover`.
+   * NOTE: The container logger should not lose stdout/stderr if the agent fails
+   * over. Additionally, if the container logger is stateful, the logger should
+   * be capable of recovering managed containers during the agent recovery
+   * process. See `ContainerLogger::recover`.
    *
-   * @param executorInfo Provided for the container logger to track logs.
-   * @param sandboxDirectory An absolute path to the executor's sandbox. This
-   *     is provided in case the container logger needs to store files in the
-   *     executor's sandbox, such as persistent state between agent failovers.
-   *     NOTE: All files in the sandbox are exposed via the `/files` endpoint.
+   * @param containerConfig The configurations of the container, including the
+   *     ExecutorInfo (if applied), as well as the absolute path to the sandbox
+   *     directory of the container. The sandbox path is provided in case
+   *     the container logger needs to store files in the container's sandbox,
+   *     such as persistent state between agent failovers.  NOTE: All files in
+   *     the sandbox are exposed via the `/files` endpoint.
    */
   virtual process::Future<ContainerIO> prepare(
-      const ExecutorInfo& executorInfo,
-      const std::string& sandboxDirectory,
-      const Option<std::string>& user) = 0;
+      const ContainerConfig& containerConfig) = 0;
 };
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/d30dc724/src/slave/container_loggers/lib_logrotate.cpp
----------------------------------------------------------------------
diff --git a/src/slave/container_loggers/lib_logrotate.cpp b/src/slave/container_loggers/lib_logrotate.cpp
index 575ed6e..f0a59d0 100644
--- a/src/slave/container_loggers/lib_logrotate.cpp
+++ b/src/slave/container_loggers/lib_logrotate.cpp
@@ -58,6 +58,7 @@ using namespace process;
 using std::map;
 using std::string;
 
+using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLogger;
 using mesos::slave::ContainerIO;
 
@@ -74,10 +75,7 @@ public:
   // Spawns two subprocesses that read from their stdin and write to
   // "stdout" and "stderr" files in the sandbox.  The subprocesses will rotate
   // the files according to the configured maximum size and number of files.
-  Future<ContainerIO> prepare(
-      const ExecutorInfo& executorInfo,
-      const string& sandboxDirectory,
-      const Option<string>& user)
+  Future<ContainerIO> prepare(const ContainerConfig& containerConfig)
   {
     // Prepare the environment for the container logger subprocess.
     // We inherit agent environment variables except for those
@@ -104,7 +102,7 @@ public:
       stringify(flags.libprocess_num_worker_threads);
 
     // Copy the global rotation flags.
-    // These will act as the defaults in case the executor environment
+    // These will act as the defaults in case the container's environment
     // overrides a subset of them.
     LoggerFlags overriddenFlags;
     overriddenFlags.max_stdout_size = flags.max_stdout_size;
@@ -113,30 +111,29 @@ public:
     overriddenFlags.logrotate_stderr_options = flags.logrotate_stderr_options;
 
     // Check for overrides of the rotation settings in the
-    // `ExecutorInfo`s environment variables.
-    if (executorInfo.has_command() &&
-        executorInfo.command().has_environment()) {
+    // `CommandInfo`s environment variables.
+    if (containerConfig.command_info().has_environment()) {
       // Search the environment for prefixed environment variables.
       // We un-prefix those variables before parsing the flag values.
-      map<string, string> executorEnvironment;
+      map<string, string> containerEnvironment;
       foreach (const Environment::Variable variable,
-               executorInfo.command().environment().variables()) {
+               containerConfig.command_info().environment().variables()) {
         if (strings::startsWith(
               variable.name(), flags.environment_variable_prefix)) {
           string unprefixed = strings::lower(strings::remove(
               variable.name(),
               flags.environment_variable_prefix,
               strings::PREFIX));
-          executorEnvironment[unprefixed] = variable.value();
+          containerEnvironment[unprefixed] = variable.value();
         }
       }
 
       // We will error out if there are unknown flags with the same prefix.
-      Try<flags::Warnings> load = overriddenFlags.load(executorEnvironment);
+      Try<flags::Warnings> load = overriddenFlags.load(containerEnvironment);
 
       if (load.isError()) {
         return Failure(
-            "Failed to load executor logger settings: " + load.error());
+            "Failed to load container logger settings: " + load.error());
       }
 
       // Log any flag warnings.
@@ -174,9 +171,11 @@ public:
     mesos::internal::logger::rotate::Flags outFlags;
     outFlags.max_size = overriddenFlags.max_stdout_size;
     outFlags.logrotate_options = overriddenFlags.logrotate_stdout_options;
-    outFlags.log_filename = path::join(sandboxDirectory, "stdout");
+    outFlags.log_filename = path::join(containerConfig.directory(), "stdout");
     outFlags.logrotate_path = flags.logrotate_path;
-    outFlags.user = user;
+    outFlags.user = containerConfig.has_user()
+      ? Option<string>(containerConfig.user())
+      : Option<string>::none();
 
     // If we are on systemd, then extend the life of the process as we
     // do with the executor. Any grandchildren's lives will also be
@@ -232,9 +231,11 @@ public:
     mesos::internal::logger::rotate::Flags errFlags;
     errFlags.max_size = overriddenFlags.max_stderr_size;
     errFlags.logrotate_options = overriddenFlags.logrotate_stderr_options;
-    errFlags.log_filename = path::join(sandboxDirectory, "stderr");
+    errFlags.log_filename = path::join(containerConfig.directory(), "stderr");
     errFlags.logrotate_path = flags.logrotate_path;
-    errFlags.user = user;
+    errFlags.user = containerConfig.has_user()
+      ? Option<string>(containerConfig.user())
+      : Option<string>::none();
 
     Try<Subprocess> errProcess = subprocess(
         path::join(flags.launcher_dir, mesos::internal::logger::rotate::NAME),
@@ -289,16 +290,12 @@ Try<Nothing> LogrotateContainerLogger::initialize()
 
 
 Future<ContainerIO> LogrotateContainerLogger::prepare(
-    const ExecutorInfo& executorInfo,
-    const string& sandboxDirectory,
-    const Option<string>& user)
+    const ContainerConfig& containerConfig)
 {
   return dispatch(
       process.get(),
       &LogrotateContainerLoggerProcess::prepare,
-      executorInfo,
-      sandboxDirectory,
-      user);
+      containerConfig);
 }
 
 } // namespace logger {

http://git-wip-us.apache.org/repos/asf/mesos/blob/d30dc724/src/slave/container_loggers/lib_logrotate.hpp
----------------------------------------------------------------------
diff --git a/src/slave/container_loggers/lib_logrotate.hpp b/src/slave/container_loggers/lib_logrotate.hpp
index 5c6d3a9..303511c 100644
--- a/src/slave/container_loggers/lib_logrotate.hpp
+++ b/src/slave/container_loggers/lib_logrotate.hpp
@@ -42,9 +42,9 @@ class LogrotateContainerLoggerProcess;
 
 
 // These flags are loaded twice: once when the `ContainerLogger` module
-// is created and each time before launching executors. The flags loaded
+// is created and each time before launching containers. The flags loaded
 // at module creation act as global default values, whereas flags loaded
-// prior to executors can override the global values.
+// prior to containers can override the global values.
 struct LoggerFlags : public virtual flags::FlagsBase
 {
   LoggerFlags()
@@ -112,9 +112,9 @@ struct Flags : public virtual LoggerFlags
     add(&Flags::environment_variable_prefix,
         "environment_variable_prefix",
         "Prefix for environment variables meant to modify the behavior of\n"
-        "the logrotate logger for the specific executor being launched.\n"
+        "the logrotate logger for the specific container being launched.\n"
         "The logger will look for four prefixed environment variables in the\n"
-        "'ExecutorInfo's 'CommandInfo's 'Environment':\n"
+        "container's 'CommandInfo's 'Environment':\n"
         "  * MAX_STDOUT_SIZE\n"
         "  * LOGROTATE_STDOUT_OPTIONS\n"
         "  * MAX_STDERR_SIZE\n"
@@ -194,13 +194,11 @@ public:
 
   virtual ~LogrotateContainerLogger();
 
-  // This is a noop.  The logrotate container logger has nothing to initialize.
-  virtual Try<Nothing> initialize();
+  // This is a noop. The logrotate container logger has nothing to initialize.
+  virtual Try<Nothing> initialize() override;
 
   virtual process::Future<mesos::slave::ContainerIO> prepare(
-      const ExecutorInfo& executorInfo,
-      const std::string& sandboxDirectory,
-      const Option<std::string>& user);
+      const mesos::slave::ContainerConfig& containerConfig) override;
 
 protected:
   Flags flags;

http://git-wip-us.apache.org/repos/asf/mesos/blob/d30dc724/src/slave/container_loggers/sandbox.cpp
----------------------------------------------------------------------
diff --git a/src/slave/container_loggers/sandbox.cpp b/src/slave/container_loggers/sandbox.cpp
index b01e359..ce4faec 100644
--- a/src/slave/container_loggers/sandbox.cpp
+++ b/src/slave/container_loggers/sandbox.cpp
@@ -38,8 +38,16 @@
 
 #include "slave/container_loggers/sandbox.hpp"
 
-using namespace process;
+using process::Future;
+using process::Process;
+using process::ProcessBase;
 
+using process::dispatch;
+using process::spawn;
+using process::terminate;
+using process::wait;
+
+using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLogger;
 using mesos::slave::ContainerIO;
 
@@ -54,15 +62,15 @@ public:
   SandboxContainerLoggerProcess()
     : ProcessBase(process::ID::generate("sandbox-logger")) {}
 
-  process::Future<ContainerIO> prepare(
-      const ExecutorInfo& executorInfo,
-      const std::string& sandboxDirectory,
-      const Option<std::string>& user)
+  Future<ContainerIO> prepare(const ContainerConfig& containerConfig)
   {
     ContainerIO io;
 
-    io.out = ContainerIO::IO::PATH(path::join(sandboxDirectory, "stdout"));
-    io.err = ContainerIO::IO::PATH(path::join(sandboxDirectory, "stderr"));
+    io.out = ContainerIO::IO::PATH(
+        path::join(containerConfig.directory(), "stdout"));
+
+    io.err = ContainerIO::IO::PATH(
+        path::join(containerConfig.directory(), "stderr"));
 
     return io;
   }
@@ -90,16 +98,12 @@ Try<Nothing> SandboxContainerLogger::initialize()
 
 
 Future<ContainerIO> SandboxContainerLogger::prepare(
-    const ExecutorInfo& executorInfo,
-    const std::string& sandboxDirectory,
-    const Option<std::string>& user)
+    const ContainerConfig& containerConfig)
 {
   return dispatch(
       process.get(),
       &SandboxContainerLoggerProcess::prepare,
-      executorInfo,
-      sandboxDirectory,
-      user);
+      containerConfig);
 }
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/d30dc724/src/slave/container_loggers/sandbox.hpp
----------------------------------------------------------------------
diff --git a/src/slave/container_loggers/sandbox.hpp b/src/slave/container_loggers/sandbox.hpp
index fb06a70..39e0eeb 100644
--- a/src/slave/container_loggers/sandbox.hpp
+++ b/src/slave/container_loggers/sandbox.hpp
@@ -44,25 +44,24 @@ class SandboxContainerLoggerProcess;
 
 // The default container logger.
 //
-// Executors and tasks launched through this container logger will have their
-// stdout and stderr piped to the files "stdout" and "stderr", respectively, in
-// the sandbox.  These logs are accessible via the agent's `/files` endpoint.
+// Containers launched through this container logger will have their
+// stdout and stderr piped to the files "stdout" and "stderr",
+// respectively, in the sandbox. These logs are accessible via the
+// agent's `/files` endpoint.
 class SandboxContainerLogger : public mesos::slave::ContainerLogger
 {
 public:
   SandboxContainerLogger();
   virtual ~SandboxContainerLogger();
 
-  // This is a noop.  The sandbox container logger has nothing to initialize.
-  virtual Try<Nothing> initialize();
+  // This is a noop. The sandbox container logger has nothing to initialize.
+  virtual Try<Nothing> initialize() override;
 
-  // Tells the subprocess to redirect the executor/task's stdout and stderr
-  // to separate "stdout" and "stderr" files in the sandbox.
+  // Tells the subprocess to redirect the container's stdout and
+  // stderr to separate "stdout" and "stderr" files in the sandbox.
   // The `path`, `argv`, and `environment` are not changed.
   virtual process::Future<mesos::slave::ContainerIO> prepare(
-      const ExecutorInfo& executorInfo,
-      const std::string& sandboxDirectory,
-      const Option<std::string>& user);
+      const mesos::slave::ContainerConfig& containerConfig) override;
 
 protected:
   process::Owned<SandboxContainerLoggerProcess> process;

http://git-wip-us.apache.org/repos/asf/mesos/blob/d30dc724/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index 7171cb5..725cdb5 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -1383,12 +1383,7 @@ Future<Docker::Container> DockerContainerizerProcess::launchExecutorContainer(
   Container* container = containers_.at(containerId);
   container->state = Container::RUNNING;
 
-  return logger->prepare(
-      container->containerConfig.executor_info(),
-      container->containerWorkDir,
-      container->containerConfig.has_user()
-        ? container->containerConfig.user()
-        : Option<string>::none())
+  return logger->prepare(container->containerConfig)
     .then(defer(
         self(),
         [=](const ContainerIO& containerIO)
@@ -1548,12 +1543,7 @@ Future<pid_t> DockerContainerizerProcess::launchExecutorProcess(
 
   return allocateGpus
     .then(defer(self(), [=]() {
-      return logger->prepare(
-          container->containerConfig.executor_info(),
-          container->containerWorkDir,
-          container->containerConfig.has_user()
-            ? container->containerConfig.user()
-            : Option<string>::none());
+      return logger->prepare(container->containerConfig);
     }))
     .then(defer(
         self(),

http://git-wip-us.apache.org/repos/asf/mesos/blob/d30dc724/src/slave/containerizer/mesos/io/switchboard.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/io/switchboard.cpp b/src/slave/containerizer/mesos/io/switchboard.cpp
index 517d459..d0f98ad 100644
--- a/src/slave/containerizer/mesos/io/switchboard.cpp
+++ b/src/slave/containerizer/mesos/io/switchboard.cpp
@@ -287,12 +287,7 @@ Future<Option<ContainerLaunchInfo>> IOSwitchboard::prepare(
   // launched, the nested containers launched later might not have
   // access to the root parent container's ExecutorInfo (i.e.,
   // 'containerConfig.executor_info()' will be empty).
-  return logger->prepare(
-      containerConfig.executor_info(),
-      containerConfig.directory(),
-      containerConfig.has_user()
-        ? Option<string>(containerConfig.user())
-        : None())
+  return logger->prepare(containerConfig)
     .then(defer(
         PID<IOSwitchboard>(this),
         &IOSwitchboard::_prepare,