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/02/08 01:40:47 UTC

[1/6] mesos git commit: Increased granularity of environment handling in command executor.

Repository: mesos
Updated Branches:
  refs/heads/master f1df9b802 -> f3518daa9


Increased granularity of environment handling in command executor.

This adds a new flag to the command executor `--task_environment`
which allows isolators to specify additionaly environment the task
should be given.  This flag is subject to the same priority as other
environment.  User-specified environment is first, isolator's second,
and agent environment last (i.e. `os::environment` from the executor's
context).

This also changes how the task itself receives these environment
variables.  The task is now given the environment explicitly, rather
than inheriting the executor's environment.  This will help prevent
isolators from accidently breaking the command executor or health
checks with task-specific environment variables.

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


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

Branch: refs/heads/master
Commit: 0370ca68ba36f643ac32009d88dee4b9c1400f2f
Parents: 954eaea
Author: Joseph Wu <jo...@apache.org>
Authored: Tue Feb 7 14:06:44 2017 -0800
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Feb 7 17:35:22 2017 -0800

----------------------------------------------------------------------
 src/launcher/executor.cpp       | 40 ++++++++++++++++++++++++++++++++++++
 src/launcher/posix/executor.cpp |  5 +++++
 src/launcher/posix/executor.hpp |  1 +
 3 files changed, 46 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0370ca68/src/launcher/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.cpp b/src/launcher/executor.cpp
index 0c770bb..060b83e 100644
--- a/src/launcher/executor.cpp
+++ b/src/launcher/executor.cpp
@@ -58,6 +58,7 @@
 #include <stout/windows.hpp>
 #endif // __WINDOWS__
 
+#include <stout/os/environment.hpp>
 #include <stout/os/kill.hpp>
 #include <stout/os/killtree.hpp>
 
@@ -95,6 +96,8 @@ using process::Subprocess;
 using process::Time;
 using process::Timer;
 
+using mesos::Environment;
+
 using mesos::executor::Call;
 using mesos::executor::Event;
 
@@ -115,6 +118,7 @@ public:
       const Option<string>& _workingDirectory,
       const Option<string>& _user,
       const Option<string>& _taskCommand,
+      const Option<Environment>& _taskEnvironment,
       const Option<CapabilityInfo>& _capabilities,
       const FrameworkID& _frameworkId,
       const ExecutorID& _executorId,
@@ -135,6 +139,7 @@ public:
       workingDirectory(_workingDirectory),
       user(_user),
       taskCommand(_taskCommand),
+      taskEnvironment(_taskEnvironment),
       capabilities(_capabilities),
       frameworkId(_frameworkId),
       executorId(_executorId),
@@ -388,12 +393,38 @@ protected:
         << "' is not specified!";
     }
 
+    // Determine the environment for the command to be launched.
+    // The priority of the environment should be:
+    //  1) User specified environment in CommandInfo.
+    //  2) Environment returned by isolators (`taskEnvironment`).
+    //  3) Environment passed from agent (`os::environment`).
+    //
+    // TODO(josephw): Windows tasks will inherit the environment
+    // from the executor for now. Change this if a Windows isolator
+    // ever uses the `--task_environment` flag.
+    Environment launchEnvironment;
+
+    foreachpair (const string& name, const string& value, os::environment()) {
+      Environment::Variable* variable = launchEnvironment.add_variables();
+      variable->set_name(name);
+      variable->set_value(value);
+    }
+
+    if (taskEnvironment.isSome()) {
+      launchEnvironment.MergeFrom(taskEnvironment.get());
+    }
+
+    if (command.has_environment()) {
+      launchEnvironment.MergeFrom(command.environment());
+    }
+
     cout << "Starting task " << task->task_id() << endl;
 
 #ifndef __WINDOWS__
     pid = launchTaskPosix(
         command,
         launcherDir,
+        launchEnvironment,
         user,
         rootfs,
         sandboxDirectory,
@@ -791,6 +822,7 @@ private:
   Option<string> workingDirectory;
   Option<string> user;
   Option<string> taskCommand;
+  Option<Environment> taskEnvironment;
   Option<CapabilityInfo> capabilities;
   const FrameworkID frameworkId;
   const ExecutorID executorId;
@@ -833,6 +865,12 @@ public:
         "If specified, this is the overrided command for launching the\n"
         "task (instead of the command from TaskInfo).");
 
+    add(&Flags::task_environment,
+        "task_environment",
+        "If specified, this is a JSON-ified `Environment` protobuf that\n"
+        "should be added to the executor's environment before launching\n"
+        "the task.");
+
     add(&Flags::capabilities,
         "capabilities",
         "Capabilities the command can use.");
@@ -851,6 +889,7 @@ public:
   Option<string> working_directory;
   Option<string> user;
   Option<string> task_command;
+  Option<Environment> task_environment;
   Option<mesos::CapabilityInfo> capabilities;
   string launcher_dir;
 };
@@ -923,6 +962,7 @@ int main(int argc, char** argv)
           flags.working_directory,
           flags.user,
           flags.task_command,
+          flags.task_environment,
           flags.capabilities,
           frameworkId,
           executorId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/0370ca68/src/launcher/posix/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/posix/executor.cpp b/src/launcher/posix/executor.cpp
index 4bc0b0f..59e7c0c 100644
--- a/src/launcher/posix/executor.cpp
+++ b/src/launcher/posix/executor.cpp
@@ -37,6 +37,8 @@ using std::endl;
 using std::string;
 using std::vector;
 
+using mesos::Environment;
+
 using mesos::internal::slave::MESOS_CONTAINERIZER;
 using mesos::internal::slave::MesosContainerizerLaunch;
 
@@ -48,6 +50,7 @@ namespace internal {
 pid_t launchTaskPosix(
     const CommandInfo& command,
     const string& launcherDir,
+    const Environment& environment,
     const Option<string>& user,
     const Option<string>& rootfs,
     const Option<string>& sandboxDirectory,
@@ -101,6 +104,8 @@ pid_t launchTaskPosix(
     }
   }
 
+  launchInfo.mutable_environment()->CopyFrom(environment);
+
   if (user.isSome()) {
     launchInfo.set_user(user.get());
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/0370ca68/src/launcher/posix/executor.hpp
----------------------------------------------------------------------
diff --git a/src/launcher/posix/executor.hpp b/src/launcher/posix/executor.hpp
index d057ff6..2dd9766 100644
--- a/src/launcher/posix/executor.hpp
+++ b/src/launcher/posix/executor.hpp
@@ -29,6 +29,7 @@ namespace internal {
 pid_t launchTaskPosix(
     const CommandInfo& command,
     const std::string& launcherDir,
+    const mesos::Environment& environment,
     const Option<std::string>& user,
     const Option<std::string>& rootfs,
     const Option<std::string>& sandboxDirectory,


[2/6] mesos git commit: Enabled flag parsing of the 'Environment' protobuf.

Posted by jo...@apache.org.
Enabled flag parsing of the 'Environment' protobuf.

This adds 'mesos::Environment' to the template specializations
required for stout flags to parse the protobuf directly.

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


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

Branch: refs/heads/master
Commit: 954eaea71037bbbd19e324332faa33b213b6dcb5
Parents: 90afbfd
Author: Joseph Wu <jo...@apache.org>
Authored: Tue Feb 7 13:54:15 2017 -0800
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Feb 7 17:35:22 2017 -0800

----------------------------------------------------------------------
 include/mesos/type_utils.hpp |  1 +
 src/common/parse.hpp         | 12 ++++++++++++
 src/common/type_utils.cpp    |  6 ++++++
 3 files changed, 19 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/954eaea7/include/mesos/type_utils.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/type_utils.hpp b/include/mesos/type_utils.hpp
index 632a7e5..c7f86ac 100644
--- a/include/mesos/type_utils.hpp
+++ b/include/mesos/type_utils.hpp
@@ -261,6 +261,7 @@ std::ostream& operator<<(
     std::ostream& stream,
     const ContainerInfo& containerInfo);
 
+std::ostream& operator<<(std::ostream& stream, const Environment& environment);
 
 std::ostream& operator<<(std::ostream& stream, const ExecutorID& executorId);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/954eaea7/src/common/parse.hpp
----------------------------------------------------------------------
diff --git a/src/common/parse.hpp b/src/common/parse.hpp
index 1dca9ba..e90738a 100644
--- a/src/common/parse.hpp
+++ b/src/common/parse.hpp
@@ -176,6 +176,18 @@ inline Try<mesos::CapabilityInfo> parse(const std::string& value)
 
 
 template <>
+inline Try<mesos::Environment> parse(const std::string& value)
+{
+  Try<JSON::Object> json = parse<JSON::Object>(value);
+  if (json.isError()) {
+    return Error(json.error());
+  }
+
+  return protobuf::parse<mesos::Environment>(json.get());
+}
+
+
+template <>
 inline Try<mesos::RLimitInfo> parse(const std::string& value)
 {
   Try<JSON::Object> json = parse<JSON::Object>(value);

http://git-wip-us.apache.org/repos/asf/mesos/blob/954eaea7/src/common/type_utils.cpp
----------------------------------------------------------------------
diff --git a/src/common/type_utils.cpp b/src/common/type_utils.cpp
index 516d309..d86d56d 100644
--- a/src/common/type_utils.cpp
+++ b/src/common/type_utils.cpp
@@ -488,6 +488,12 @@ ostream& operator<<(ostream& stream, const ContainerInfo& containerInfo)
 }
 
 
+ostream& operator<<(ostream& stream, const Environment& environment)
+{
+  return stream << JSON::protobuf(environment);
+}
+
+
 ostream& operator<<(ostream& stream, const ExecutorID& executorId)
 {
   return stream << executorId.value();


[5/6] mesos git commit: Disabled ability to launch default executor with ContainerInfo.

Posted by jo...@apache.org.
Disabled ability to launch default executor with ContainerInfo.

The default executor used to launch TaskGroups is generated by the
agent.  The agent generates the executor's `CommandInfo`, hence why
the user may not specify the `CommandInfo` in the `LAUNCH_TASK_GROUP`
call.

This commit adds similar restrictions for `ContainerInfo` plus the
default executor.  The `CommandInfo` constructed by the agent expects
to be run in the same environment as the agent process.  This commit
prevents the user from specifying a `DockerInfo` or a container image
along with the default executor.

If the user explicitly wants to use the default executor, they could
always package the default executor's binary and libraries into a
container and launch it like any other custom executor.

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


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

Branch: refs/heads/master
Commit: 90afbfd93e8bdb71ee0f79fbd0cfff06fd22ae63
Parents: f7a77bf
Author: Joseph Wu <jo...@apache.org>
Authored: Thu Feb 2 18:18:59 2017 -0800
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Feb 7 17:35:22 2017 -0800

----------------------------------------------------------------------
 include/mesos/mesos.proto             |  3 +++
 include/mesos/v1/mesos.proto          |  3 +++
 src/master/validation.cpp             | 14 ++++++++++++
 src/tests/master_validation_tests.cpp | 35 ++++++++++++++++++++++++++++--
 4 files changed, 53 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/90afbfd9/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index 53885cb..34a288b 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -652,6 +652,9 @@ message ExecutorInfo {
     //
     // 2) Default executor only accepts a *single* `LAUNCH` or `LAUNCH_GROUP`
     //    offer operation.
+    //
+    // 3) If `container` is set, `container.type` must be `MESOS`
+    //    and `container.mesos.image` must not be set.
     DEFAULT = 1;
 
     // For frameworks that need custom functionality to run tasks, a `CUSTOM`

http://git-wip-us.apache.org/repos/asf/mesos/blob/90afbfd9/include/mesos/v1/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/mesos.proto b/include/mesos/v1/mesos.proto
index c4ca6de..6638111 100644
--- a/include/mesos/v1/mesos.proto
+++ b/include/mesos/v1/mesos.proto
@@ -652,6 +652,9 @@ message ExecutorInfo {
     //
     // 2) Default executor only accepts a *single* `LAUNCH` or `LAUNCH_GROUP`
     //    offer operation.
+    //
+    // 3) If `container` is set, `container.type` must be `MESOS`
+    //    and `container.mesos.image` must not be set.
     DEFAULT = 1;
 
     // For frameworks that need custom functionality to run tasks, a `CUSTOM`

http://git-wip-us.apache.org/repos/asf/mesos/blob/90afbfd9/src/master/validation.cpp
----------------------------------------------------------------------
diff --git a/src/master/validation.cpp b/src/master/validation.cpp
index 226c526..37d1715 100644
--- a/src/master/validation.cpp
+++ b/src/master/validation.cpp
@@ -629,6 +629,20 @@ Option<Error> validateType(const ExecutorInfo& executor)
         return Error(
             "'ExecutorInfo.command' must not be set for 'DEFAULT' executor");
       }
+
+      if (executor.has_container()) {
+        if (executor.container().type() != ContainerInfo::MESOS) {
+          return Error(
+              "'ExecutorInfo.container.type' must be 'MESOS' for "
+              "'DEFAULT' executor");
+        }
+
+        if (executor.container().mesos().has_image()) {
+          return Error(
+              "'ExecutorInfo.container.mesos.image' must not be set for "
+              "'DEFAULT' executor");
+        }
+      }
       break;
 
     case ExecutorInfo::CUSTOM:

http://git-wip-us.apache.org/repos/asf/mesos/blob/90afbfd9/src/tests/master_validation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_validation_tests.cpp b/src/tests/master_validation_tests.cpp
index 1f833aa..5118503 100644
--- a/src/tests/master_validation_tests.cpp
+++ b/src/tests/master_validation_tests.cpp
@@ -1987,6 +1987,37 @@ TEST_F(ExecutorValidationTest, ExecutorType)
         error->message,
         "'ExecutorInfo.command' must not be set for 'DEFAULT' executor"));
   }
+
+  {
+    // 'DEFAULT' executor with `ContainerInfo` must be a Mesos container.
+    executorInfo.set_type(ExecutorInfo::DEFAULT);
+    executorInfo.clear_command();
+    executorInfo.mutable_container()->set_type(ContainerInfo::DOCKER);
+
+    Option<Error> error = ::executor::internal::validateType(executorInfo);
+
+    EXPECT_SOME(error);
+    EXPECT_TRUE(strings::contains(
+        error->message,
+        "'ExecutorInfo.container.type' must be 'MESOS' for "
+        "'DEFAULT' executor"));
+  }
+
+  {
+    // 'DEFAULT' executor with `ContainerInfo` may not have a container image.
+    executorInfo.set_type(ExecutorInfo::DEFAULT);
+    executorInfo.clear_command();
+    executorInfo.mutable_container()->set_type(ContainerInfo::MESOS);
+    executorInfo.mutable_container()->mutable_mesos()->mutable_image();
+
+    Option<Error> error = ::executor::internal::validateType(executorInfo);
+
+    EXPECT_SOME(error);
+    EXPECT_TRUE(strings::contains(
+        error->message,
+        "'ExecutorInfo.container.mesos.image' must not be set for "
+        "'DEFAULT' executor"));
+  }
 }
 
 
@@ -2265,14 +2296,14 @@ TEST_F(TaskGroupValidationTest, ExecutorUsesDockerContainerInfo)
   EXPECT_EQ(TASK_ERROR, task1Status->state());
   EXPECT_EQ(TaskStatus::REASON_TASK_GROUP_INVALID, task1Status->reason());
   EXPECT_EQ(
-      "Docker ContainerInfo is not supported on the executor",
+      "'ExecutorInfo.container.type' must be 'MESOS' for 'DEFAULT' executor",
       task1Status->message());
 
   AWAIT_READY(task2Status);
   EXPECT_EQ(TASK_ERROR, task2Status->state());
   EXPECT_EQ(TaskStatus::REASON_TASK_GROUP_INVALID, task2Status->reason());
   EXPECT_EQ(
-      "Docker ContainerInfo is not supported on the executor",
+      "'ExecutorInfo.container.type' must be 'MESOS' for 'DEFAULT' executor",
       task2Status->message());
 
   // Make sure the task is not known to master anymore.


[3/6] mesos git commit: Fixed order of arguments in DockerArchive manifest format.

Posted by jo...@apache.org.
Fixed order of arguments in DockerArchive manifest format.

The strings for `entrypoint` and `cmd` were substituted
in the `Cmd` and `Entrypoint` fields respectively.  Precisely
the opposite of what is expected.


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

Branch: refs/heads/master
Commit: 7319e014bbc3756d0b0599e919afb945351f6728
Parents: f1df9b8
Author: Joseph Wu <jo...@apache.org>
Authored: Tue Feb 7 17:20:06 2017 -0800
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Feb 7 17:35:22 2017 -0800

----------------------------------------------------------------------
 src/tests/containerizer/docker_archive.hpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7319e014/src/tests/containerizer/docker_archive.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/docker_archive.hpp b/src/tests/containerizer/docker_archive.hpp
index b36dbdb..9e4895b 100644
--- a/src/tests/containerizer/docker_archive.hpp
+++ b/src/tests/containerizer/docker_archive.hpp
@@ -150,8 +150,8 @@ public:
             "architecture": "amd64",
             "os": "linux"
         })~",
-        entrypoint,
-        cmd).get()).get();
+        cmd,
+        entrypoint).get()).get();
 
     write = os::write(
         path::join(layerPath, "json"),


[4/6] mesos git commit: Changed test DockerArchive to include environment variables.

Posted by jo...@apache.org.
Changed test DockerArchive to include environment variables.

This subtly modifies all tests using the `docker/runtime` isolator
to fail if environment variables from inside the DockerArchive
are passed into the Mesos executor's environment.  This applies
for all executors (default, command, and docker), but mainly
affects the command executor.

The environment variables are `LD_LIBRARY_PATH`, `LIBPROCESS_IP`,
and `LIBPROCESS_PORT`; all of which are set to `invalid`.  This
either causes linking problems or will force libprocess to exit.

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


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

Branch: refs/heads/master
Commit: f7a77bf5306b5e4ef20c9bb1121e60ed22adf19e
Parents: 7319e01
Author: Joseph Wu <jo...@apache.org>
Authored: Wed Feb 1 14:00:19 2017 -0800
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Feb 7 17:35:22 2017 -0800

----------------------------------------------------------------------
 src/tests/containerizer/docker_archive.hpp | 16 ++++++++++++++--
 1 file changed, 14 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f7a77bf5/src/tests/containerizer/docker_archive.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/docker_archive.hpp b/src/tests/containerizer/docker_archive.hpp
index 9e4895b..56211f7 100644
--- a/src/tests/containerizer/docker_archive.hpp
+++ b/src/tests/containerizer/docker_archive.hpp
@@ -23,6 +23,7 @@
 
 #include <stout/error.hpp>
 #include <stout/json.hpp>
+#include <stout/jsonify.hpp>
 #include <stout/nothing.hpp>
 #include <stout/os.hpp>
 #include <stout/path.hpp>
@@ -48,11 +49,21 @@ public:
   // Create a docker test image tarball in docker registry directory.
   // Users can define own entrypoint/cmd as JSON array of JSON string
   // (e.g., `[\"sh\", \"-c\"]`).
+  //
+  // NOTE: The default value for `environment` includes some environment
+  // variables which will cause problems if they are fed into one of Mesos'
+  // built-in executors. This is on purpose, as the environment variables
+  // of the image should not be passed into built-in executors. Tests that
+  // use a custom executor should consider overriding this default.
   static Future<Nothing> create(
       const std::string& directory,
       const std::string& name,
       const std::string& entrypoint = "null",
-      const std::string& cmd = "null")
+      const std::string& cmd = "null",
+      const std::vector<std::string>& environment = {
+        {"LD_LIBRARY_PATH=invalid"},
+        {"LIBPROCESS_IP=invalid"},
+        {"LIBPROCESS_PORT=invalid"}})
   {
     Try<Nothing> mkdir = os::mkdir(directory, true);
     if (mkdir.isError()) {
@@ -138,7 +149,7 @@ public:
                 "Tty": false,
                 "OpenStdin": false,
                 "StdinOnce": false,
-                "Env": null,
+                "Env": %s,
                 "Cmd": %s,
                 "Image": "",
                 "Volumes": null,
@@ -150,6 +161,7 @@ public:
             "architecture": "amd64",
             "os": "linux"
         })~",
+        std::string(jsonify(environment)),
         cmd,
         entrypoint).get()).get();
 


[6/6] mesos git commit: Updated AppC & Docker runtime isolators' handling of Environment.

Posted by jo...@apache.org.
Updated AppC & Docker runtime isolators' handling of Environment.

This commit builds upon the command executor's new `--task_environment`
flag, which allows isolators to specify environment variables meant
for the task, without affecting the executor's environment.

This is important as the command executor is both an executor and
a task.  Some environment variables from isolators are intended
for the executor, while others are intended for the task (such as
from the two runtime isolators).

For example, a container image may provide an environment variable
like `LD_LIBRARY_PATH=/image/specific/location`, whereas the default
executor expects to find libraries in the host's environment.  If the
image's environment end up in the command executor at launch time,
the command executor may simply fail to launch.

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


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

Branch: refs/heads/master
Commit: f3518daa9dca03dbdb59b092d23ebe1356f352d4
Parents: 0370ca6
Author: Joseph Wu <jo...@apache.org>
Authored: Tue Feb 7 14:10:48 2017 -0800
Committer: Joseph Wu <jo...@apache.org>
Committed: Tue Feb 7 17:35:23 2017 -0800

----------------------------------------------------------------------
 .../containerizer/mesos/isolators/appc/runtime.cpp    | 14 ++++++++++----
 .../containerizer/mesos/isolators/docker/runtime.cpp  | 14 ++++++++++----
 2 files changed, 20 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f3518daa/src/slave/containerizer/mesos/isolators/appc/runtime.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/appc/runtime.cpp b/src/slave/containerizer/mesos/isolators/appc/runtime.cpp
index 9bc3fd8..ffaec5a 100644
--- a/src/slave/containerizer/mesos/isolators/appc/runtime.cpp
+++ b/src/slave/containerizer/mesos/isolators/appc/runtime.cpp
@@ -103,10 +103,6 @@ Future<Option<ContainerLaunchInfo>> AppcRuntimeIsolatorProcess::prepare(
   // Set 'launchInfo'.
   ContainerLaunchInfo launchInfo;
 
-  if (environment.isSome()) {
-    launchInfo.mutable_environment()->CopyFrom(environment.get());
-  }
-
   // If working directory or command exists, operation has to be
   // handled specially for the command task. For the command task,
   // the working directory and task command will be passed to
@@ -119,6 +115,12 @@ Future<Option<ContainerLaunchInfo>> AppcRuntimeIsolatorProcess::prepare(
     // command with value as 'mesos-executor'.
     CommandInfo executorCommand = containerConfig.executor_info().command();
 
+    if (environment.isSome()) {
+      executorCommand.add_arguments(
+          "--task_environment=" +
+          stringify(JSON::protobuf(environment.get())));
+    }
+
     // Pass working directory to command executor as a flag.
     if (workingDirectory.isSome()) {
       executorCommand.add_arguments(
@@ -136,6 +138,10 @@ Future<Option<ContainerLaunchInfo>> AppcRuntimeIsolatorProcess::prepare(
     launchInfo.mutable_command()->CopyFrom(executorCommand);
   } else {
     // The custom executor, default executor and nested container cases.
+    if (environment.isSome()) {
+      launchInfo.mutable_environment()->CopyFrom(environment.get());
+    }
+
     if (workingDirectory.isSome()) {
       launchInfo.set_working_directory(workingDirectory.get());
     }

http://git-wip-us.apache.org/repos/asf/mesos/blob/f3518daa/src/slave/containerizer/mesos/isolators/docker/runtime.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/docker/runtime.cpp b/src/slave/containerizer/mesos/isolators/docker/runtime.cpp
index 2d816e5..08350e6 100644
--- a/src/slave/containerizer/mesos/isolators/docker/runtime.cpp
+++ b/src/slave/containerizer/mesos/isolators/docker/runtime.cpp
@@ -123,10 +123,6 @@ Future<Option<ContainerLaunchInfo>> DockerRuntimeIsolatorProcess::prepare(
   // Set 'launchInfo'.
   ContainerLaunchInfo launchInfo;
 
-  if (environment.isSome()) {
-    launchInfo.mutable_environment()->CopyFrom(environment.get());
-  }
-
   // If working directory or command exists, operation has to be
   // handled specially for the command task. For the command task,
   // the working directory and task command will be passed to
@@ -139,6 +135,12 @@ Future<Option<ContainerLaunchInfo>> DockerRuntimeIsolatorProcess::prepare(
     // command with value as 'mesos-executor'.
     CommandInfo executorCommand = containerConfig.executor_info().command();
 
+    if (environment.isSome()) {
+      executorCommand.add_arguments(
+          "--task_environment=" +
+          stringify(JSON::protobuf(environment.get())));
+    }
+
     // Pass working directory to command executor as a flag.
     if (workingDirectory.isSome()) {
       executorCommand.add_arguments(
@@ -156,6 +158,10 @@ Future<Option<ContainerLaunchInfo>> DockerRuntimeIsolatorProcess::prepare(
     launchInfo.mutable_command()->CopyFrom(executorCommand);
   } else {
     // The custom executor, default executor and nested container cases.
+    if (environment.isSome()) {
+      launchInfo.mutable_environment()->CopyFrom(environment.get());
+    }
+
     if (workingDirectory.isSome()) {
       launchInfo.set_working_directory(workingDirectory.get());
     }