You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2015/06/25 02:28:25 UTC

[9/9] mesos git commit: Added 'executor_environment_variables' flag to slave.

Added 'executor_environment_variables' flag to slave.

This new flag, 'executor_environment_variables', let's an operator
specify the environment variables that should be passed to an
executor, and thus, any of it's tasks. By default, an executor will
inherit the environment variables of the slave.

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


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

Branch: refs/heads/master
Commit: 46dc9979e9cc38d36dc7300db13af39bdfbfd52e
Parents: 0914a01
Author: Benjamin Hindman <be...@gmail.com>
Authored: Mon Jun 15 00:29:20 2015 -0700
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Wed Jun 24 17:27:25 2015 -0700

----------------------------------------------------------------------
 src/slave/containerizer/containerizer.cpp | 11 ++++
 src/slave/flags.cpp                       | 35 +++++++++++-
 src/slave/flags.hpp                       |  3 +
 src/tests/containerizer.cpp               | 26 +++++++--
 src/tests/slave_tests.cpp                 | 76 ++++++++++++++++++++++++++
 5 files changed, 142 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/46dc9979/src/slave/containerizer/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.cpp b/src/slave/containerizer/containerizer.cpp
index ca1c3c1..b9ac94c 100644
--- a/src/slave/containerizer/containerizer.cpp
+++ b/src/slave/containerizer/containerizer.cpp
@@ -244,6 +244,17 @@ map<string, string> executorEnvironment(
 {
   map<string, string> environment = os::environment();
 
+  if (flags.executor_environment_variables.isSome()) {
+    environment.clear();
+    foreachpair (const string& key,
+                 const JSON::Value& value,
+                 flags.executor_environment_variables.get().values) {
+      // See slave/flags.cpp where we validate each value is a string.
+      CHECK(value.is<JSON::String>());
+      environment[key] = value.as<JSON::String>().value;
+    }
+  }
+
   // Set LIBPROCESS_PORT so that we bind to a random free port (since
   // this might have been set via --port option). We do this before
   // the environment variables below in case it is included.

http://git-wip-us.apache.org/repos/asf/mesos/blob/46dc9979/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index cbf431e..6ba5a1b 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -16,13 +16,18 @@
  * limitations under the License.
  */
 
+#include "slave/flags.hpp"
+
+#include <stout/error.hpp>
 #include <stout/flags.hpp>
+#include <stout/json.hpp>
+#include <stout/option.hpp>
 
 #include <mesos/type_utils.hpp>
 
 #include "common/parse.hpp"
+
 #include "slave/constants.hpp"
-#include "slave/flags.hpp"
 
 
 mesos::internal::slave::Flags::Flags()
@@ -124,6 +129,30 @@ mesos::internal::slave::Flags::Flags()
         stringify(REGISTER_RETRY_INTERVAL_MAX),
       REGISTRATION_BACKOFF_FACTOR);
 
+  add(&Flags::executor_environment_variables,
+      "executor_environment_variables",
+      "JSON object representing the environment\n"
+      "variables that should be passed to the\n"
+      "executor, and thus subsequently task(s).\n"
+      "By default the executor will inherit the\n"
+      "slave's environment variables.\n"
+      "Example:\n"
+      "{\n"
+      "  \"PATH\": \"/bin:/usr/bin\",\n"
+      "  \"LD_LIBRARY_PATH\": \"/usr/local/lib\"\n"
+      "}",
+      [](const Option<JSON::Object>& object) -> Option<Error> {
+        if (object.isSome()) {
+          foreachvalue (const JSON::Value& value, object.get().values) {
+            if (!value.is<JSON::String>()) {
+              return Error("'executor_environment_variables' must "
+                           "only contain string values");
+            }
+          }
+        }
+        return None();
+      });
+
   add(&Flags::executor_registration_timeout,
       "executor_registration_timeout",
       "Amount of time to wait for an executor\n"
@@ -274,8 +303,8 @@ mesos::internal::slave::Flags::Flags()
       "\n"
       "Example:\n"
       "{\n"
-      "  \"disabled_endpoints\" : {\n"
-      "    \"paths\" : [\n"
+      "  \"disabled_endpoints\": {\n"
+      "    \"paths\": [\n"
       "      \"/files/browse.json\",\n"
       "      \"/slave(0)/stats.json\",\n"
       "    ]\n"

http://git-wip-us.apache.org/repos/asf/mesos/blob/46dc9979/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 7634e36..26c778d 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -23,6 +23,7 @@
 
 #include <stout/bytes.hpp>
 #include <stout/duration.hpp>
+#include <stout/json.hpp>
 #include <stout/option.hpp>
 #include <stout/path.hpp>
 
@@ -40,6 +41,7 @@ class Flags : public logging::Flags
 {
 public:
   Flags();
+
   bool version;
   Option<std::string> hostname;
   Option<std::string> resources;
@@ -54,6 +56,7 @@ public:
   bool switch_user;
   std::string frameworks_home;  // TODO(benh): Make an Option.
   Duration registration_backoff_factor;
+  Option<JSON::Object> executor_environment_variables;
   Duration executor_registration_timeout;
   Duration executor_shutdown_grace_period;
   Duration gc_delay;

http://git-wip-us.apache.org/repos/asf/mesos/blob/46dc9979/src/tests/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.cpp b/src/tests/containerizer.cpp
index 355fb90..5134e63 100644
--- a/src/tests/containerizer.cpp
+++ b/src/tests/containerizer.cpp
@@ -108,6 +108,10 @@ Future<bool> TestContainerizer::_launch(
   slave::Flags flags;
   flags.recovery_timeout = Duration::zero();
 
+  // We need to save the original set of environment variables so we
+  // can reset the environment after calling 'driver->start()' below.
+  hashmap<string, string> original = os::environment();
+
   const map<string, string> environment = executorEnvironment(
       executorInfo,
       directory,
@@ -120,23 +124,33 @@ Future<bool> TestContainerizer::_launch(
     os::setenv(name, variable);
   }
 
+  // TODO(benh): Can this be removed and done exlusively in the
+  // 'executorEnvironment()' function? There are other places in the
+  // code where we do this as well and it's likely we can do this once
+  // in 'executorEnvironment()'.
   foreach (const Environment::Variable& variable,
            executorInfo.command().environment().variables()) {
     os::setenv(variable.name(), variable.value());
   }
+
   os::setenv("MESOS_LOCAL", "1");
 
   driver->start();
 
-  foreachkey (const string& name, environment) {
-    os::unsetenv(name);
+  os::unsetenv("MESOS_LOCAL");
+
+  // Unset the environment variables we set by resetting them to their
+  // original values and also removing any that were not part of the
+  // original environment.
+  foreachpair (const string& name, const string& value, original) {
+    os::setenv(name, value);
   }
 
-  foreach (const Environment::Variable& variable,
-           executorInfo.command().environment().variables()) {
-    os::unsetenv(variable.name());
+  foreachkey (const string& name, environment) {
+    if (!original.contains(name)) {
+      os::unsetenv(name);
+    }
   }
-  os::unsetenv("MESOS_LOCAL");
 
   promises[containerId] =
     Owned<Promise<containerizer::Termination>>(

http://git-wip-us.apache.org/repos/asf/mesos/blob/46dc9979/src/tests/slave_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_tests.cpp b/src/tests/slave_tests.cpp
index 8bb5a97..e9002e8 100644
--- a/src/tests/slave_tests.cpp
+++ b/src/tests/slave_tests.cpp
@@ -2105,6 +2105,82 @@ TEST_F(SlaveTest, TaskLabels)
   Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }
 
+
+// Test that we can set the executors environment variables and it
+// won't inhert the slaves.
+TEST_F(SlaveTest, ExecutorEnvironmentVariables)
+{
+  Try<PID<Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  // Need flags for 'executor_environment_variables'.
+  slave::Flags flags = CreateSlaveFlags();
+
+  Try<JSON::Object> parse = JSON::parse<JSON::Object>("{\"PATH\": \"/bin\"}");
+
+  ASSERT_SOME(parse);
+
+  flags.executor_environment_variables = parse.get();
+
+  Try<PID<Slave>> slave = StartSlave(flags);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .Times(1);
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  // Launch a task with the command executor.
+  TaskInfo task;
+  task.set_name("");
+  task.mutable_task_id()->set_value("1");
+  task.mutable_slave_id()->MergeFrom(offers.get()[0].slave_id());
+  task.mutable_resources()->MergeFrom(offers.get()[0].resources());
+
+  // Command executor will run as user running test.
+  CommandInfo command;
+  command.set_shell(true);
+  command.set_value("test $PATH = /bin");
+
+  task.mutable_command()->MergeFrom(command);
+
+  vector<TaskInfo> tasks;
+  tasks.push_back(task);
+
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusFinished;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&statusFinished));
+
+  driver.launchTasks(offers.get()[0].id(), tasks);
+
+  // Scheduler should first receive TASK_RUNNING followed by the
+  // TASK_FINISHED from the executor.
+  AWAIT_READY(statusRunning);
+  EXPECT_EQ(TASK_RUNNING, statusRunning.get().state());
+
+  AWAIT_READY(statusFinished);
+  EXPECT_EQ(TASK_FINISHED, statusFinished.get().state());
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {