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/10/12 00:00:02 UTC

[1/4] mesos git commit: Made mesos-containerizer launch helper inherit agent env variables.

Repository: mesos
Updated Branches:
  refs/heads/master fa526f271 -> af2d40628


Made mesos-containerizer launch helper inherit agent env variables.

This patch addressed MESOS-6323. The idea is that we pass the
environment variables that the command needs to the
mesos-containerizer launch helper, and let the helper inherit agent
environment variables.

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


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

Branch: refs/heads/master
Commit: af2d406282d8da0ed93eacf997bf8f94aa77b492
Parents: 7d60be7
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Oct 10 22:26:12 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Oct 11 16:59:31 2016 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 24 ++++++++++++--------
 src/slave/containerizer/mesos/containerizer.hpp |  2 +-
 2 files changed, 16 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/af2d4062/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 32058c3..5c4db56 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1189,7 +1189,7 @@ Future<Nothing> MesosContainerizerProcess::fetch(
 
 Future<bool> MesosContainerizerProcess::_launch(
     const ContainerID& containerId,
-    map<string, string> environment,
+    const map<string, string>& _environment,
     const SlaveID& slaveId,
     bool checkpoint)
 {
@@ -1205,6 +1205,11 @@ Future<bool> MesosContainerizerProcess::_launch(
 
   CHECK_EQ(container->state, PREPARING);
 
+  JSON::Object environment;
+  foreachpair (const string& key, const string& value, _environment) {
+    environment.values[key] = value;
+  }
+
   // TODO(jieyu): Consider moving this to 'executorEnvironment' and
   // consolidating with docker containerizer.
   //
@@ -1212,7 +1217,7 @@ Future<bool> MesosContainerizerProcess::_launch(
   // 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()
+  environment.values["MESOS_SANDBOX"] = container->config.has_rootfs()
     ? flags.sandbox_directory
     : container->config.directory();
 
@@ -1248,15 +1253,15 @@ Future<bool> MesosContainerizerProcess::_launch(
         const string& name = variable.name();
         const string& value = variable.value();
 
-        if (environment.count(name) > 0) {
+        if (environment.values.count(name) > 0) {
           VLOG(1) << "Overwriting environment variable '"
                   << name << "', original: '"
-                  << environment[name] << "', new: '"
+                  << environment.values[name] << "', new: '"
                   << value << "', for container "
                   << containerId;
         }
 
-        environment[name] = value;
+        environment.values[name] = value;
       }
     }
 
@@ -1331,15 +1336,15 @@ Future<bool> MesosContainerizerProcess::_launch(
     const string& name = variable.name();
     const string& value = variable.value();
 
-    if (environment.count(name) > 0) {
+    if (environment.values.count(name) > 0) {
       VLOG(1) << "Overwriting environment variable '"
               << name << "', original: '"
-              << environment[name] << "', new: '"
+              << environment.values[name] << "', new: '"
               << value << "', for container "
               << containerId;
     }
 
-    environment[name] = value;
+    environment.values[name] = value;
   }
 
   return logger->prepare(
@@ -1371,6 +1376,7 @@ Future<bool> MesosContainerizerProcess::_launch(
     MesosContainerizerLaunch::Flags launchFlags;
 
     launchFlags.command = JSON::protobuf(launchCommand.get());
+    launchFlags.environment = environment;
 
     if (rootfs.isNone()) {
       // NOTE: If the executor shares the host filesystem, we should
@@ -1458,7 +1464,7 @@ Future<bool> MesosContainerizerProcess::_launch(
         (local ? Subprocess::FD(STDERR_FILENO)
                : Subprocess::IO(subprocessInfo.err)),
         &launchFlags,
-        environment,
+        None(),
         namespaces); // 'namespaces' will be ignored by PosixLauncher.
 
     if (forked.isError()) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/af2d4062/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index eef2cbb..c4fea8b 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -227,7 +227,7 @@ private:
 
   process::Future<bool> _launch(
       const ContainerID& containerId,
-      std::map<std::string, std::string> environment,
+      const std::map<std::string, std::string>& environment,
       const SlaveID& slaveId,
       bool checkpoint);
 


[3/4] mesos git commit: Added 'environment' flag to mesos-containerizer launch helper.

Posted by ji...@apache.org.
Added 'environment' flag to mesos-containerizer launch helper.

The flag is optional. If set, the command to be launched will use the
specified environment variables. If not set, the command will inherit
the environment variables of the calling process.

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


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

Branch: refs/heads/master
Commit: 7d60be7c17387673045f9486cd2b3b23a1ebae6e
Parents: c43c54a
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Oct 10 22:25:28 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Oct 11 16:59:31 2016 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/launch.cpp | 34 +++++++++++++++++----------
 src/slave/containerizer/mesos/launch.hpp |  3 +++
 2 files changed, 25 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7d60be7c/src/slave/containerizer/mesos/launch.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index c6b669a..8a30ff8 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -71,6 +71,10 @@ MesosContainerizerLaunch::Flags::Flags()
       "command",
       "The command to execute.");
 
+  add(&environment,
+      "environment",
+      "The environment variables for the command.");
+
   add(&working_directory,
       "working_directory",
       "The working directory for the command. It has to be an absolute path \n"
@@ -605,8 +609,21 @@ int MesosContainerizerLaunch::execute()
     }
   }
 
-  // Relay the environment variables.
-  // TODO(jieyu): Consider using a clean environment.
+  // Prepare the executable and the argument list for the child.
+  string executable(command->shell()
+    ? os::Shell::name
+    : command->value().c_str());
+
+  os::raw::Argv argv(command->shell()
+    ? vector<string>({os::Shell::arg0, os::Shell::arg1, command->value()})
+    : vector<string>(command->arguments().begin(), command->arguments().end()));
+
+  // Prepare the environment for the child. If 'environment' is not
+  // specified, inherit the environment of the current process.
+  Option<os::raw::Envp> envp;
+  if (flags.environment.isSome()) {
+    envp = os::raw::Envp(flags.environment.get());
+  }
 
 #ifndef __WINDOWS__
   // If we have `containerStatusFd` set, then we need to fork-exec the
@@ -682,17 +699,10 @@ int MesosContainerizerLaunch::execute()
   }
 #endif // __WINDOWS__
 
-  if (command->shell()) {
-    // Execute the command using shell.
-    os::execlp(os::Shell::name,
-               os::Shell::arg0,
-               os::Shell::arg1,
-               command->value().c_str(),
-               (char*) nullptr);
+  if (envp.isSome()) {
+    os::execvpe(executable.c_str(), argv, envp.get());
   } else {
-    // Use execvp to launch the command.
-    os::execvp(command->value().c_str(),
-               os::raw::Argv(command->arguments()));
+    os::execvp(executable.c_str(), argv);
   }
 
   // If we get here, the execle call failed.

http://git-wip-us.apache.org/repos/asf/mesos/blob/7d60be7c/src/slave/containerizer/mesos/launch.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.hpp b/src/slave/containerizer/mesos/launch.hpp
index 208ad2b..cfbf790 100644
--- a/src/slave/containerizer/mesos/launch.hpp
+++ b/src/slave/containerizer/mesos/launch.hpp
@@ -17,6 +17,8 @@
 #ifndef __MESOS_CONTAINERIZER_LAUNCH_HPP__
 #define __MESOS_CONTAINERIZER_LAUNCH_HPP__
 
+#include <string>
+
 #include <stout/json.hpp>
 #include <stout/option.hpp>
 #include <stout/subcommand.hpp>
@@ -37,6 +39,7 @@ public:
     Flags();
 
     Option<JSON::Object> command;
+    Option<JSON::Object> environment;
     Option<std::string> working_directory;
 #ifndef __WINDOWS__
     Option<std::string> runtime_directory;


[2/4] mesos git commit: Added an abstraction for Envp pointer expected by exec routines.

Posted by ji...@apache.org.
Added an abstraction for Envp pointer expected by exec routines.

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


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

Branch: refs/heads/master
Commit: c43c54aa97ebad486f04094ae79a59a4c95c10ca
Parents: 90ad84b
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Oct 10 22:23:11 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Oct 11 16:59:31 2016 -0700

----------------------------------------------------------------------
 .../stout/include/stout/os/raw/environment.hpp  | 96 ++++++++++++++++++++
 1 file changed, 96 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c43c54aa/3rdparty/stout/include/stout/os/raw/environment.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/raw/environment.hpp b/3rdparty/stout/include/stout/os/raw/environment.hpp
index 80cc45b..b3e82ac 100644
--- a/3rdparty/stout/include/stout/os/raw/environment.hpp
+++ b/3rdparty/stout/include/stout/os/raw/environment.hpp
@@ -13,6 +13,14 @@
 #ifndef __STOUT_OS_RAW_ENVIRONMENT_HPP__
 #define __STOUT_OS_RAW_ENVIRONMENT_HPP__
 
+#include <string.h>
+
+#include <string>
+
+#include <stout/foreach.hpp>
+#include <stout/json.hpp>
+#include <stout/stringify.hpp>
+
 #ifdef __APPLE__
 #include <crt_externs.h> // For _NSGetEnviron().
 #elif !defined(__WINDOWS__)
@@ -73,6 +81,94 @@ inline char*** environmentp()
 #endif
 }
 
+
+// Represents the environment variable list expected by 'exec'
+// routines. The environment variable list is an array of pointers
+// that point to null-terminated strings. The array of pointers must
+// be terminated by a nullptr. To use this abstraction, see the
+// following example:
+//
+//   map<string, string> environment = {
+//     {"key1", "value1"},
+//     {"key2", "value2"}
+//   };
+//   os::raw::Envp envp(environment);
+//   execle("/bin/sh", "sh", "-c", "echo hello", envp);
+class Envp
+{
+public:
+  Envp(Envp&& that)
+    : envp(that.envp),
+      size(that.size)
+  {
+    that.envp = nullptr;
+    that.size = 0;
+  }
+
+  template <typename Map>
+  explicit Envp(const Map& map)
+  {
+    size = map.size();
+
+    // NOTE: We add 1 to the size for a `nullptr` terminator.
+    envp = new char*[size + 1];
+    size_t index = 0;
+
+    for (auto it = map.begin(); it != map.end(); ++it) {
+      std::string entry = stringify(it->first) + "=" + stringify(it->second);
+      envp[index] = new char[entry.size() + 1];
+      ::memcpy(envp[index], entry.c_str(), entry.size() + 1);
+      ++index;
+    }
+
+    envp[index] = nullptr;
+  }
+
+  explicit Envp(const JSON::Object& object)
+  {
+    size = object.values.size();
+
+    // NOTE: We add 1 to the size for a `nullptr` terminator.
+    envp = new char*[size + 1];
+    size_t index = 0;
+
+    foreachpair (const std::string& key,
+                 const JSON::Value& value,
+                 object.values) {
+      std::string entry = key + "=" + value.as<JSON::String>().value;
+      envp[index] = new char[entry.size() + 1];
+      ::memcpy(envp[index], entry.c_str(), entry.size() + 1);
+      ++index;
+    }
+
+    envp[index] = nullptr;
+  }
+
+  ~Envp()
+  {
+    if (envp == nullptr) {
+      return;
+    }
+
+    for (size_t i = 0; i < size; i++) {
+      delete[] envp[i];
+    }
+    delete[] envp;
+  }
+
+  operator char**()
+  {
+    return envp;
+  }
+
+private:
+  Envp(const Envp&) = delete;
+  Envp& operator=(const Envp&) = delete;
+
+  char **envp;
+  size_t size;
+};
+
 } // namespace raw {
 } // namespace os {
 


[4/4] mesos git commit: Made execvp explicit in posix/shell.hpp.

Posted by ji...@apache.org.
Made execvp explicit in posix/shell.hpp.

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


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

Branch: refs/heads/master
Commit: 90ad84b78f53c18cf0e7fa479c21eea4f5c587a8
Parents: fa526f2
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Oct 10 22:22:42 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Oct 11 16:59:31 2016 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/os/posix/shell.hpp | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/90ad84b7/3rdparty/stout/include/stout/os/posix/shell.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/os/posix/shell.hpp b/3rdparty/stout/include/stout/os/posix/shell.hpp
index e7047ac..70a9184 100644
--- a/3rdparty/stout/include/stout/os/posix/shell.hpp
+++ b/3rdparty/stout/include/stout/os/posix/shell.hpp
@@ -31,9 +31,6 @@
 
 namespace os {
 
-// Import `::execvp` into `os::` namespace.
-using ::execvp;
-
 namespace Shell {
 
 // Canonical constants used as platform-dependent args to `exec`
@@ -185,6 +182,12 @@ inline int execlp(const char* file, T... t)
   return ::execlp(file, t...);
 }
 
+
+inline int execvp(const char* file, char* const argv[])
+{
+  return ::execvp(file, argv);
+}
+
 } // namespace os {
 
 #endif // __STOUT_OS_POSIX_SHELL_HPP__