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/08/15 17:53:35 UTC

mesos git commit: Removed slavePid from the containerizer launch API.

Repository: mesos
Updated Branches:
  refs/heads/master 5f06d10b7 -> b0aee8bc5


Removed slavePid from the containerizer launch API.

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


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

Branch: refs/heads/master
Commit: b0aee8bc5bcaa9f900711066c41ebf62da1aca11
Parents: 5f06d10
Author: Gilbert Song <so...@gmail.com>
Authored: Mon Aug 15 10:53:20 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Aug 15 10:53:20 2016 -0700

----------------------------------------------------------------------
 src/slave/containerizer/composing.cpp           |  21 +--
 src/slave/containerizer/composing.hpp           |   2 +-
 src/slave/containerizer/containerizer.cpp       | 127 -----------------
 src/slave/containerizer/containerizer.hpp       |  24 +---
 src/slave/containerizer/docker.cpp              |  19 +--
 src/slave/containerizer/docker.hpp              |   9 +-
 src/slave/containerizer/mesos/containerizer.cpp |  27 ++--
 src/slave/containerizer/mesos/containerizer.hpp |   8 +-
 src/slave/slave.cpp                             | 140 ++++++++++++++++++-
 src/slave/slave.hpp                             |  22 +++
 src/tests/containerizer.cpp                     |  10 +-
 src/tests/containerizer.hpp                     |   4 +-
 .../composing_containerizer_tests.cpp           |   6 +-
 .../containerizer/filesystem_isolator_tests.cpp |  37 +++--
 src/tests/containerizer/isolator_tests.cpp      |   2 +-
 .../containerizer/mesos_containerizer_tests.cpp |  41 ++++--
 src/tests/hook_tests.cpp                        |  20 ++-
 src/tests/mesos.hpp                             |   6 +-
 src/tests/slave_tests.cpp                       |   2 +-
 19 files changed, 273 insertions(+), 254 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/slave/containerizer/composing.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.cpp b/src/slave/containerizer/composing.cpp
index 532b62c..c51cb4b 100644
--- a/src/slave/containerizer/composing.cpp
+++ b/src/slave/containerizer/composing.cpp
@@ -33,6 +33,7 @@
 #include "slave/containerizer/composing.hpp"
 
 using std::list;
+using std::map;
 using std::string;
 using std::vector;
 
@@ -64,7 +65,7 @@ public:
       const string& directory,
       const Option<string>& user,
       const SlaveID& slaveId,
-      const PID<Slave>& slavePid,
+      const map<string, string>& environment,
       bool checkpoint);
 
   Future<Nothing> update(
@@ -99,7 +100,7 @@ private:
       const string& directory,
       const Option<string>& user,
       const SlaveID& slaveId,
-      const PID<Slave>& slavePid,
+      const map<string, string>& environment,
       bool checkpoint,
       vector<Containerizer*>::iterator containerizer,
       bool launched);
@@ -162,7 +163,7 @@ Future<bool> ComposingContainerizer::launch(
     const string& directory,
     const Option<string>& user,
     const SlaveID& slaveId,
-    const PID<Slave>& slavePid,
+    const map<string, string>& environment,
     bool checkpoint)
 {
   return dispatch(process,
@@ -173,7 +174,7 @@ Future<bool> ComposingContainerizer::launch(
                   directory,
                   user,
                   slaveId,
-                  slavePid,
+                  environment,
                   checkpoint);
 }
 
@@ -293,7 +294,7 @@ Future<bool> ComposingContainerizerProcess::_launch(
     const string& directory,
     const Option<string>& user,
     const SlaveID& slaveId,
-    const PID<Slave>& slavePid,
+    const map<string, string>& environment,
     bool checkpoint,
     vector<Containerizer*>::iterator containerizer,
     bool launched)
@@ -332,7 +333,7 @@ Future<bool> ComposingContainerizerProcess::_launch(
       directory,
       user,
       slaveId,
-      slavePid,
+      environment,
       checkpoint)
     .then(defer(
         self(),
@@ -343,7 +344,7 @@ Future<bool> ComposingContainerizerProcess::_launch(
         directory,
         user,
         slaveId,
-        slavePid,
+        environment,
         checkpoint,
         containerizer,
         lambda::_1));
@@ -357,7 +358,7 @@ Future<bool> ComposingContainerizerProcess::launch(
     const string& directory,
     const Option<string>& user,
     const SlaveID& slaveId,
-    const PID<Slave>& slavePid,
+    const map<string, string>& environment,
     bool checkpoint)
 {
   if (containers_.contains(containerId)) {
@@ -381,7 +382,7 @@ Future<bool> ComposingContainerizerProcess::launch(
       directory,
       user,
       slaveId,
-      slavePid,
+      environment,
       checkpoint)
     .then(defer(self(),
                 &Self::_launch,
@@ -391,7 +392,7 @@ Future<bool> ComposingContainerizerProcess::launch(
                 directory,
                 user,
                 slaveId,
-                slavePid,
+                environment,
                 checkpoint,
                 containerizer,
                 lambda::_1));

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/slave/containerizer/composing.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/composing.hpp b/src/slave/containerizer/composing.hpp
index 492f1c1..406015a 100644
--- a/src/slave/containerizer/composing.hpp
+++ b/src/slave/containerizer/composing.hpp
@@ -61,7 +61,7 @@ public:
       const std::string& directory,
       const Option<std::string>& user,
       const SlaveID& slaveId,
-      const process::PID<Slave>& slavePid,
+      const std::map<std::string, std::string>& environment,
       bool checkpoint);
 
   virtual process::Future<Nothing> update(

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/slave/containerizer/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.cpp b/src/slave/containerizer/containerizer.cpp
index 2d59caf..d46882b 100644
--- a/src/slave/containerizer/containerizer.cpp
+++ b/src/slave/containerizer/containerizer.cpp
@@ -290,133 +290,6 @@ Try<Containerizer*> Containerizer::create(
   return containerizer.get();
 }
 
-
-map<string, string> executorEnvironment(
-    const ExecutorInfo& executorInfo,
-    const string& directory,
-    const SlaveID& slaveId,
-    const PID<Slave>& slavePid,
-    bool checkpoint,
-    const Flags& flags)
-{
-  map<string, string> environment;
-
-  // In cases where DNS is not available on the slave, the absence of
-  // LIBPROCESS_IP in the executor's environment will cause an error when the
-  // new executor process attempts a hostname lookup. Thus, we pass the slave's
-  // LIBPROCESS_IP through here, even if the executor environment is specified
-  // explicitly. Note that a LIBPROCESS_IP present in the provided flags will
-  // override this value.
-  Option<string> libprocessIP = os::getenv("LIBPROCESS_IP");
-  if (libprocessIP.isSome()) {
-    environment["LIBPROCESS_IP"] = libprocessIP.get();
-  }
-
-  if (flags.executor_environment_variables.isSome()) {
-    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;
-    }
-  }
-
-  // Include a default $PATH if there isn't.
-  if (environment.count("PATH") == 0) {
-    environment["PATH"] =
-      "/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin";
-  }
-
-  // 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.
-  environment["LIBPROCESS_PORT"] = "0";
-
-  // Also add MESOS_NATIVE_JAVA_LIBRARY if it's not already present (and
-  // like above, we do this before the environment variables below in
-  // case the framework wants to override).
-  // TODO(tillt): Adapt library towards JNI specific name once libmesos
-  // has been split.
-  if (environment.count("MESOS_NATIVE_JAVA_LIBRARY") == 0) {
-    string path =
-#ifdef __APPLE__
-      LIBDIR "/libmesos-" VERSION ".dylib";
-#else
-      LIBDIR "/libmesos-" VERSION ".so";
-#endif
-    if (os::exists(path)) {
-      environment["MESOS_NATIVE_JAVA_LIBRARY"] = path;
-    }
-  }
-
-  // Also add MESOS_NATIVE_LIBRARY if it's not already present.
-  // This environment variable is kept for offering non JVM-based
-  // frameworks a more compact and JNI independent library.
-  if (environment.count("MESOS_NATIVE_LIBRARY") == 0) {
-    string path =
-#ifdef __APPLE__
-      LIBDIR "/libmesos-" VERSION ".dylib";
-#else
-      LIBDIR "/libmesos-" VERSION ".so";
-#endif
-    if (os::exists(path)) {
-      environment["MESOS_NATIVE_LIBRARY"] = path;
-    }
-  }
-
-  environment["MESOS_FRAMEWORK_ID"] = executorInfo.framework_id().value();
-  environment["MESOS_EXECUTOR_ID"] = executorInfo.executor_id().value();
-  environment["MESOS_DIRECTORY"] = directory;
-  environment["MESOS_SLAVE_ID"] = slaveId.value();
-  environment["MESOS_SLAVE_PID"] = stringify(slavePid);
-  environment["MESOS_AGENT_ENDPOINT"] = stringify(slavePid.address);
-  environment["MESOS_CHECKPOINT"] = checkpoint ? "1" : "0";
-  environment["MESOS_HTTP_COMMAND_EXECUTOR"] =
-    flags.http_command_executor ? "1" : "0";
-
-  // Set executor's shutdown grace period. If set, the customized value
-  // from `ExecutorInfo` overrides the default from agent flags.
-  Duration executorShutdownGracePeriod = flags.executor_shutdown_grace_period;
-  if (executorInfo.has_shutdown_grace_period()) {
-    executorShutdownGracePeriod =
-      Nanoseconds(executorInfo.shutdown_grace_period().nanoseconds());
-  }
-
-  environment["MESOS_EXECUTOR_SHUTDOWN_GRACE_PERIOD"] =
-    stringify(executorShutdownGracePeriod);
-
-  if (checkpoint) {
-    environment["MESOS_RECOVERY_TIMEOUT"] = stringify(flags.recovery_timeout);
-
-    // The maximum backoff duration to be used by an executor between two
-    // retries when disconnected.
-    environment["MESOS_SUBSCRIPTION_BACKOFF_MAX"] =
-      stringify(EXECUTOR_REREGISTER_TIMEOUT);
-  }
-
-  if (HookManager::hooksAvailable()) {
-    // Include any environment variables from Hooks.
-    // TODO(karya): Call environment decorator hook _after_ putting all
-    // variables from executorInfo into 'env'. This would prevent the
-    // ones provided by hooks from being overwritten by the ones in
-    // executorInfo in case of a conflict. The overwriting takes places
-    // at the callsites of executorEnvironment (e.g., ___launch function
-    // in src/slave/containerizer/docker.cpp)
-    // TODO(karya): Provide a mechanism to pass the new environment
-    // variables created above (MESOS_*) on to the hook modules.
-    const Environment& hooksEnvironment =
-      HookManager::slaveExecutorEnvironmentDecorator(executorInfo);
-
-    foreach (const Environment::Variable& variable,
-             hooksEnvironment.variables()) {
-      environment[variable.name()] = variable.value();
-    }
-  }
-
-  return environment;
-}
-
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/slave/containerizer/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
index d005375..14f298e 100644
--- a/src/slave/containerizer/containerizer.hpp
+++ b/src/slave/containerizer/containerizer.hpp
@@ -87,7 +87,7 @@ public:
       const std::string& directory,
       const Option<std::string>& user,
       const SlaveID& slaveId,
-      const process::PID<Slave>& slavePid,
+      const std::map<std::string, std::string>& environment,
       bool checkpoint) = 0;
 
   // Update the resources for a container.
@@ -126,28 +126,6 @@ public:
   virtual process::Future<hashset<ContainerID>> containers() = 0;
 };
 
-
-/**
- * Returns a map of environment variables necessary in order to launch
- * an executor.
- *
- * @param executorInfo ExecutorInfo being launched.
- * @param directory Path to the sandbox directory.
- * @param slaveId SlaveID where this executor is being launched.
- * @param slavePid PID of the slave launching the executor.
- * @param checkpoint Whether or not the framework is checkpointing.
- * @param flags Flags used to launch the slave.
- *
- * @return Map of environment variables (name, value).
- */
-std::map<std::string, std::string> executorEnvironment(
-    const ExecutorInfo& executorInfo,
-    const std::string& directory,
-    const SlaveID& slaveId,
-    const process::PID<Slave>& slavePid,
-    bool checkpoint,
-    const Flags& flags);
-
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index 4c2def3..03668d2 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -242,7 +242,7 @@ DockerContainerizerProcess::Container::create(
     const string& directory,
     const Option<string>& user,
     const SlaveID& slaveId,
-    const PID<Slave>& slavePid,
+    const map<string, string>& environment,
     bool checkpoint,
     const Flags& flags)
 {
@@ -363,14 +363,6 @@ DockerContainerizerProcess::Container::create(
     launchesExecutorContainer = true;
   }
 
-  map<string, string> environment = executorEnvironment(
-        executorInfo,
-        containerWorkdir,
-        slaveId,
-        slavePid,
-        checkpoint,
-        flags);
-
   return new Container(
       id,
       taskInfo,
@@ -378,7 +370,6 @@ DockerContainerizerProcess::Container::create(
       containerWorkdir,
       user,
       slaveId,
-      slavePid,
       checkpoint,
       symlinked,
       flags,
@@ -681,7 +672,7 @@ Future<bool> DockerContainerizer::launch(
     const string& directory,
     const Option<string>& user,
     const SlaveID& slaveId,
-    const PID<Slave>& slavePid,
+    const map<string, string>& environment,
     bool checkpoint)
 {
   return dispatch(
@@ -693,7 +684,7 @@ Future<bool> DockerContainerizer::launch(
       directory,
       user,
       slaveId,
-      slavePid,
+      environment,
       checkpoint);
 }
 
@@ -973,7 +964,7 @@ Future<bool> DockerContainerizerProcess::launch(
     const string& directory,
     const Option<string>& user,
     const SlaveID& slaveId,
-    const PID<Slave>& slavePid,
+    const map<string, string>& environment,
     bool checkpoint)
 {
   if (containers_.contains(containerId)) {
@@ -1005,7 +996,7 @@ Future<bool> DockerContainerizerProcess::launch(
       directory,
       user,
       slaveId,
-      slavePid,
+      environment,
       checkpoint,
       flags);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/slave/containerizer/docker.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.hpp b/src/slave/containerizer/docker.hpp
index 0ba4ed2..6597d86 100644
--- a/src/slave/containerizer/docker.hpp
+++ b/src/slave/containerizer/docker.hpp
@@ -86,7 +86,7 @@ public:
       const std::string& directory,
       const Option<std::string>& user,
       const SlaveID& slaveId,
-      const process::PID<Slave>& slavePid,
+      const std::map<std::string, std::string>& environment,
       bool checkpoint);
 
   virtual process::Future<Nothing> update(
@@ -133,7 +133,7 @@ public:
       const std::string& directory,
       const Option<std::string>& user,
       const SlaveID& slaveId,
-      const process::PID<Slave>& slavePid,
+      const std::map<std::string, std::string>& environment,
       bool checkpoint);
 
   // force = true causes the containerizer to update the resources
@@ -271,7 +271,7 @@ private:
         const std::string& directory,
         const Option<std::string>& user,
         const SlaveID& slaveId,
-        const process::PID<Slave>& slavePid,
+        const std::map<std::string, std::string>& environment,
         bool checkpoint,
         const Flags& flags);
 
@@ -290,7 +290,6 @@ private:
               const std::string& directory,
               const Option<std::string>& user,
               const SlaveID& slaveId,
-              const process::PID<Slave>& slavePid,
               bool checkpoint,
               bool symlinked,
               const Flags& flags,
@@ -306,7 +305,6 @@ private:
         directory(directory),
         user(user),
         slaveId(slaveId),
-        slavePid(slavePid),
         checkpoint(checkpoint),
         symlinked(symlinked),
         flags(flags),
@@ -435,7 +433,6 @@ private:
 
     const Option<std::string> user;
     SlaveID slaveId;
-    const process::PID<Slave> slavePid;
     bool checkpoint;
     bool symlinked;
     const Flags flags;

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 5ed894d..8b525ce 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -430,7 +430,7 @@ Future<bool> MesosContainerizer::launch(
     const string& directory,
     const Option<string>& user,
     const SlaveID& slaveId,
-    const PID<Slave>& slavePid,
+    const map<string, string>& environment,
     bool checkpoint)
 {
   return dispatch(process.get(),
@@ -441,7 +441,7 @@ Future<bool> MesosContainerizer::launch(
                   directory,
                   user,
                   slaveId,
-                  slavePid,
+                  environment,
                   checkpoint);
 }
 
@@ -731,7 +731,7 @@ Future<bool> MesosContainerizerProcess::launch(
     const string& directory,
     const Option<string>& user,
     const SlaveID& slaveId,
-    const PID<Slave>& slavePid,
+    const map<string, string>& environment,
     bool checkpoint)
 {
   if (containers_.contains(containerId)) {
@@ -795,7 +795,7 @@ Future<bool> MesosContainerizerProcess::launch(
                   directory,
                   user,
                   slaveId,
-                  slavePid,
+                  environment,
                   checkpoint,
                   None(),
                   lambda::_1));
@@ -822,7 +822,7 @@ Future<bool> MesosContainerizerProcess::launch(
                    directory,
                    user,
                    slaveId,
-                   slavePid,
+                   environment,
                    checkpoint,
                    None());
   }
@@ -842,7 +842,7 @@ Future<bool> MesosContainerizerProcess::launch(
                 directory,
                 user,
                 slaveId,
-                slavePid,
+                environment,
                 checkpoint,
                 lambda::_1));
 }
@@ -855,7 +855,7 @@ Future<bool> MesosContainerizerProcess::_launch(
     const string& directory,
     const Option<string>& user,
     const SlaveID& slaveId,
-    const PID<Slave>& slavePid,
+    const map<string, string>& environment,
     bool checkpoint,
     const Option<ProvisionInfo>& provisionInfo)
 {
@@ -927,7 +927,7 @@ Future<bool> MesosContainerizerProcess::_launch(
                     directory,
                     user,
                     slaveId,
-                    slavePid,
+                    environment,
                     checkpoint,
                     provisionInfo,
                     lambda::_1));
@@ -1080,7 +1080,7 @@ Future<bool> MesosContainerizerProcess::__launch(
     const string& directory,
     const Option<string>& user,
     const SlaveID& slaveId,
-    const PID<Slave>& slavePid,
+    const map<string, string>& _environment,
     bool checkpoint,
     const Option<ProvisionInfo>& provisionInfo,
     const list<Option<ContainerLaunchInfo>>& launchInfos)
@@ -1095,14 +1095,7 @@ Future<bool> MesosContainerizerProcess::__launch(
 
   CHECK_EQ(containers_[containerId]->state, PREPARING);
 
-  // Prepare environment variables for the executor.
-  map<string, string> environment = executorEnvironment(
-      executorInfo,
-      directory,
-      slaveId,
-      slavePid,
-      checkpoint,
-      flags);
+  map<string, string> environment = _environment;
 
   // Determine the root filesystem for the executor.
   Option<string> executorRootfs;

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 5a43dac..c3e3066 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -81,7 +81,7 @@ public:
       const std::string& directory,
       const Option<std::string>& user,
       const SlaveID& slaveId,
-      const process::PID<Slave>& slavePid,
+      const std::map<std::string, std::string>& environment,
       bool checkpoint);
 
   virtual process::Future<Nothing> update(
@@ -139,7 +139,7 @@ public:
       const std::string& directory,
       const Option<std::string>& user,
       const SlaveID& slaveId,
-      const process::PID<Slave>& slavePid,
+      const std::map<std::string, std::string>& environment,
       bool checkpoint);
 
   virtual process::Future<Nothing> update(
@@ -207,7 +207,7 @@ private:
       const std::string& directory,
       const Option<std::string>& user,
       const SlaveID& slaveId,
-      const process::PID<Slave>& slavePid,
+      const std::map<std::string, std::string>& environment,
       bool checkpoint,
       const Option<ProvisionInfo>& provisionInfo);
 
@@ -218,7 +218,7 @@ private:
       const std::string& directory,
       const Option<std::string>& user,
       const SlaveID& slaveId,
-      const process::PID<Slave>& slavePid,
+      const std::map<std::string, std::string>& _environment,
       bool checkpoint,
       const Option<ProvisionInfo>& provisionInfo,
       const std::list<Option<mesos::slave::ContainerLaunchInfo>>& launchInfos);

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 25493cb..0feb5c5 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -5794,6 +5794,15 @@ Executor* Framework::launchExecutor(
   resources += taskInfo.resources();
   executorInfo_.mutable_resources()->CopyFrom(resources);
 
+  // Prepare environment variables for the executor.
+  map<string, string> environment = executorEnvironment(
+      slave->flags,
+      executorInfo_,
+      executor->directory,
+      slave->info.id(),
+      slave->self(),
+      info.checkpoint());
+
   // Launch the container.
   Future<bool> launch;
   if (!executor->isCommandExecutor()) {
@@ -5808,7 +5817,7 @@ Executor* Framework::launchExecutor(
         executor->directory,
         user,
         slave->info.id(),
-        slave->self(),
+        environment,
         info.checkpoint());
   } else {
     // An executor has _not_ been provided by the task and will
@@ -5826,7 +5835,7 @@ Executor* Framework::launchExecutor(
         executor->directory,
         user,
         slave->info.id(),
-        slave->self(),
+        environment,
         info.checkpoint());
   }
 
@@ -6275,6 +6284,133 @@ void Executor::closeHttpConnection()
 }
 
 
+map<string, string> executorEnvironment(
+    const Flags& flags,
+    const ExecutorInfo& executorInfo,
+    const string& directory,
+    const SlaveID& slaveId,
+    const PID<Slave>& slavePid,
+    bool checkpoint)
+{
+  map<string, string> environment;
+
+  // In cases where DNS is not available on the slave, the absence of
+  // LIBPROCESS_IP in the executor's environment will cause an error when the
+  // new executor process attempts a hostname lookup. Thus, we pass the slave's
+  // LIBPROCESS_IP through here, even if the executor environment is specified
+  // explicitly. Note that a LIBPROCESS_IP present in the provided flags will
+  // override this value.
+  Option<string> libprocessIP = os::getenv("LIBPROCESS_IP");
+  if (libprocessIP.isSome()) {
+    environment["LIBPROCESS_IP"] = libprocessIP.get();
+  }
+
+  if (flags.executor_environment_variables.isSome()) {
+    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;
+    }
+  }
+
+  // Include a default $PATH if there isn't.
+  if (environment.count("PATH") == 0) {
+    environment["PATH"] =
+      "/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin";
+  }
+
+  // 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.
+  environment["LIBPROCESS_PORT"] = "0";
+
+  // Also add MESOS_NATIVE_JAVA_LIBRARY if it's not already present (and
+  // like above, we do this before the environment variables below in
+  // case the framework wants to override).
+  // TODO(tillt): Adapt library towards JNI specific name once libmesos
+  // has been split.
+  if (environment.count("MESOS_NATIVE_JAVA_LIBRARY") == 0) {
+    string path =
+#ifdef __APPLE__
+      LIBDIR "/libmesos-" VERSION ".dylib";
+#else
+      LIBDIR "/libmesos-" VERSION ".so";
+#endif
+    if (os::exists(path)) {
+      environment["MESOS_NATIVE_JAVA_LIBRARY"] = path;
+    }
+  }
+
+  // Also add MESOS_NATIVE_LIBRARY if it's not already present.
+  // This environment variable is kept for offering non JVM-based
+  // frameworks a more compact and JNI independent library.
+  if (environment.count("MESOS_NATIVE_LIBRARY") == 0) {
+    string path =
+#ifdef __APPLE__
+      LIBDIR "/libmesos-" VERSION ".dylib";
+#else
+      LIBDIR "/libmesos-" VERSION ".so";
+#endif
+    if (os::exists(path)) {
+      environment["MESOS_NATIVE_LIBRARY"] = path;
+    }
+  }
+
+  environment["MESOS_FRAMEWORK_ID"] = executorInfo.framework_id().value();
+  environment["MESOS_EXECUTOR_ID"] = executorInfo.executor_id().value();
+  environment["MESOS_DIRECTORY"] = directory;
+  environment["MESOS_SLAVE_ID"] = slaveId.value();
+  environment["MESOS_SLAVE_PID"] = stringify(slavePid);
+  environment["MESOS_AGENT_ENDPOINT"] = stringify(slavePid.address);
+  environment["MESOS_CHECKPOINT"] = checkpoint ? "1" : "0";
+  environment["MESOS_HTTP_COMMAND_EXECUTOR"] =
+    flags.http_command_executor ? "1" : "0";
+
+  // Set executor's shutdown grace period. If set, the customized value
+  // from `ExecutorInfo` overrides the default from agent flags.
+  Duration executorShutdownGracePeriod = flags.executor_shutdown_grace_period;
+  if (executorInfo.has_shutdown_grace_period()) {
+    executorShutdownGracePeriod =
+      Nanoseconds(executorInfo.shutdown_grace_period().nanoseconds());
+  }
+
+  environment["MESOS_EXECUTOR_SHUTDOWN_GRACE_PERIOD"] =
+    stringify(executorShutdownGracePeriod);
+
+  if (checkpoint) {
+    environment["MESOS_RECOVERY_TIMEOUT"] = stringify(flags.recovery_timeout);
+
+    // The maximum backoff duration to be used by an executor between two
+    // retries when disconnected.
+    environment["MESOS_SUBSCRIPTION_BACKOFF_MAX"] =
+      stringify(EXECUTOR_REREGISTER_TIMEOUT);
+  }
+
+  if (HookManager::hooksAvailable()) {
+    // Include any environment variables from Hooks.
+    // TODO(karya): Call environment decorator hook _after_ putting all
+    // variables from executorInfo into 'env'. This would prevent the
+    // ones provided by hooks from being overwritten by the ones in
+    // executorInfo in case of a conflict. The overwriting takes places
+    // at the callsites of executorEnvironment (e.g., ___launch function
+    // in src/slave/containerizer/docker.cpp)
+    // TODO(karya): Provide a mechanism to pass the new environment
+    // variables created above (MESOS_*) on to the hook modules.
+    const Environment& hooksEnvironment =
+      HookManager::slaveExecutorEnvironmentDecorator(executorInfo);
+
+    foreach (const Environment::Variable& variable,
+             hooksEnvironment.variables()) {
+      environment[variable.name()] = variable.value();
+    }
+  }
+
+  return environment;
+}
+
+
 std::ostream& operator<<(std::ostream& stream, const Executor& executor)
 {
   stream << "'" << executor.id << "' of framework " << executor.frameworkId;

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index ffe4220..9f29d80 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -968,6 +968,28 @@ private:
 };
 
 
+/**
+ * Returns a map of environment variables necessary in order to launch
+ * an executor.
+ *
+ * @param executorInfo ExecutorInfo being launched.
+ * @param directory Path to the sandbox directory.
+ * @param slaveId SlaveID where this executor is being launched.
+ * @param slavePid PID of the slave launching the executor.
+ * @param checkpoint Whether or not the framework is checkpointing.
+ * @param flags Flags used to launch the slave.
+ *
+ * @return Map of environment variables (name, value).
+ */
+std::map<std::string, std::string> executorEnvironment(
+    const Flags& flags,
+    const ExecutorInfo& executorInfo,
+    const std::string& directory,
+    const SlaveID& slaveId,
+    const process::PID<Slave>& slavePid,
+    bool checkpoint);
+
+
 std::ostream& operator<<(std::ostream& stream, Slave::State state);
 std::ostream& operator<<(std::ostream& stream, Framework::State state);
 std::ostream& operator<<(std::ostream& stream, Executor::State state);

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/tests/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.cpp b/src/tests/containerizer.cpp
index 28087cd..761b1ed 100644
--- a/src/tests/containerizer.cpp
+++ b/src/tests/containerizer.cpp
@@ -95,7 +95,7 @@ Future<bool> TestContainerizer::_launch(
     const string& directory,
     const Option<string>& user,
     const SlaveID& slaveId,
-    const PID<slave::Slave>& slavePid,
+    const map<string, string>& environment,
     bool checkpoint)
 {
   CHECK(!drivers.contains(containerId))
@@ -137,14 +137,6 @@ Future<bool> TestContainerizer::_launch(
     // can reset the environment after calling 'driver->start()' below.
     hashmap<string, string> original = os::environment();
 
-    const map<string, string> environment = executorEnvironment(
-        executorInfo,
-        directory,
-        slaveId,
-        slavePid,
-        checkpoint,
-        flags);
-
     foreachpair (const string& name, const string variable, environment) {
       os::setenv(name, variable);
     }

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/tests/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.hpp b/src/tests/containerizer.hpp
index 6436b52..cd7eeb0 100644
--- a/src/tests/containerizer.hpp
+++ b/src/tests/containerizer.hpp
@@ -86,7 +86,7 @@ public:
           const std::string&,
           const Option<std::string>&,
           const SlaveID&,
-          const process::PID<slave::Slave>&,
+          const std::map<std::string, std::string>&,
           bool checkpoint));
 
   MOCK_METHOD2(
@@ -124,7 +124,7 @@ private:
       const std::string& directory,
       const Option<std::string>& user,
       const SlaveID& slaveId,
-      const process::PID<slave::Slave>& slavePid,
+      const std::map<std::string, std::string>& environment,
       bool checkpoint);
 
   process::Future<containerizer::Termination> _wait(

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/tests/containerizer/composing_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/composing_containerizer_tests.cpp b/src/tests/containerizer/composing_containerizer_tests.cpp
index 24d458d..5a9d5e4 100644
--- a/src/tests/containerizer/composing_containerizer_tests.cpp
+++ b/src/tests/containerizer/composing_containerizer_tests.cpp
@@ -65,7 +65,7 @@ public:
           const std::string&,
           const Option<std::string>&,
           const SlaveID&,
-          const process::PID<Slave>&,
+          const std::map<std::string, std::string>&,
           bool));
 
   MOCK_METHOD2(
@@ -114,7 +114,7 @@ TEST_F(ComposingContainerizerTest, DestroyWhileLaunching)
   TaskInfo taskInfo;
   ExecutorInfo executorInfo;
   SlaveID slaveId;
-  PID<Slave> slavePid;
+  std::map<std::string, std::string> environment;
 
   Promise<bool> launchPromise;
 
@@ -133,7 +133,7 @@ TEST_F(ComposingContainerizerTest, DestroyWhileLaunching)
       "dir",
       "user",
       slaveId,
-      slavePid,
+      environment,
       false);
 
   Resources resources = Resources::parse("cpus:1;mem:256").get();

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/tests/containerizer/filesystem_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/filesystem_isolator_tests.cpp b/src/tests/containerizer/filesystem_isolator_tests.cpp
index 3e812ec..2b216c3 100644
--- a/src/tests/containerizer/filesystem_isolator_tests.cpp
+++ b/src/tests/containerizer/filesystem_isolator_tests.cpp
@@ -56,6 +56,7 @@ using process::Owned;
 using process::PID;
 using process::Shared;
 
+using std::map;
 using std::string;
 using std::vector;
 
@@ -262,7 +263,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ChangeRootFilesystem)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait for the launch to complete.
@@ -774,7 +775,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_Metrics)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait for the launch to complete.
@@ -836,7 +837,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromSandbox)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait for the launch to complete.
@@ -891,7 +892,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromHost)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait for the launch to complete.
@@ -947,7 +948,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_FileVolumeFromHost)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   AWAIT_READY_FOR(launch, Seconds(60));
@@ -997,7 +998,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_VolumeFromHostSandboxMountPoint)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait for the launch to complete.
@@ -1053,7 +1054,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_FileVolumeFromHostSandboxMountPoint)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   AWAIT_READY_FOR(launch, Seconds(60));
@@ -1118,7 +1119,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithRootFilesystem)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait for the launch to complete.
@@ -1192,7 +1193,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_PersistentVolumeWithoutRootFilesystem)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait for the launch to complete.
@@ -1247,7 +1248,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ImageInVolumeWithoutRootFilesystem)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait for the launch to complete.
@@ -1302,7 +1303,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_ImageInVolumeWithRootFilesystem)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait for the launch to complete.
@@ -1387,7 +1388,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_MultipleContainers)
       directory1,
       None(),
       slaveId,
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait for the launch to complete.
@@ -1416,7 +1417,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_MultipleContainers)
       directory2,
       None(),
       slaveId,
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Need to wait for Rootfs copy.
@@ -1475,6 +1476,14 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_SandboxEnvironmentVariable)
 
   executor.mutable_container()->CopyFrom(createContainerInfo("test_image"));
 
+  map<string, string> environment = executorEnvironment(
+      flags,
+      executor,
+      directory,
+      SlaveID(),
+      PID<Slave>(),
+      false);
+
   Future<bool> launch = containerizer.get()->launch(
       containerId,
       None(),
@@ -1482,7 +1491,7 @@ TEST_F(LinuxFilesystemIsolatorTest, ROOT_SandboxEnvironmentVariable)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      environment,
       false);
 
   // Wait for the launch to complete.

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/tests/containerizer/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index f917db7..05620d2 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -1528,7 +1528,7 @@ TEST_F(NamespacesPidIsolatorTest, ROOT_PidNamespace)
       directory,
       None(),
       SlaveID(),
-      process::PID<Slave>(),
+      std::map<string, string>(),
       false);
   AWAIT_READY(launch);
   ASSERT_TRUE(launch.get());

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/tests/containerizer/mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/mesos_containerizer_tests.cpp b/src/tests/containerizer/mesos_containerizer_tests.cpp
index a01492d..b47e569 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -184,7 +184,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptSucceeds)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait until the launch completes.
@@ -236,7 +236,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ScriptFails)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait until the launch completes.
@@ -299,7 +299,7 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait until the launch completes.
@@ -361,14 +361,27 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, ExecutorEnvironmentVariable)
     "if [ -n \"$LIBPROCESS_IP\" ]; "
     "then touch $TEST_ENVIRONMENT; fi";
 
+  ExecutorInfo executorInfo = CREATE_EXECUTOR_INFO("executor", executorCmd);
+  SlaveID slaveId = SlaveID();
+
+  slave::Flags flags;
+
+  map<string, string> environment = executorEnvironment(
+      flags,
+      executorInfo,
+      directory,
+      slaveId,
+      PID<Slave>(),
+      false);
+
   Future<bool> launch = containerizer.get()->launch(
       containerId,
       None(),
-      CREATE_EXECUTOR_INFO("executor", executorCmd),
+      executorInfo,
       directory,
       None(),
-      SlaveID(),
-      PID<Slave>(),
+      slaveId,
+      environment,
       false);
 
   // Wait until the launch completes.
@@ -438,7 +451,7 @@ TEST_F(MesosContainerizerExecuteTest, IoRedirection)
       directory,
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   // Wait for the launch to complete.
@@ -625,7 +638,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhileFetching)
       os::getcwd(),
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   Future<containerizer::Termination> wait = containerizer.wait(containerId);
@@ -693,7 +706,7 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
       os::getcwd(),
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   Future<containerizer::Termination> wait = containerizer.wait(containerId);
@@ -814,7 +827,7 @@ TEST_F(MesosContainerizerProvisionerTest, ProvisionFailed)
       os::getcwd(),
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   AWAIT_READY(provision);
@@ -909,7 +922,7 @@ TEST_F(MesosContainerizerProvisionerTest, DestroyWhileProvisioning)
       os::getcwd(),
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   Future<containerizer::Termination> wait = containerizer.wait(containerId);
@@ -1010,7 +1023,7 @@ TEST_F(MesosContainerizerProvisionerTest, DestroyWhileProvisioningVolumeImage)
       os::getcwd(),
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   Future<containerizer::Termination> wait = containerizer.wait(containerId);
@@ -1112,7 +1125,7 @@ TEST_F(MesosContainerizerProvisionerTest, IsolatorCleanupBeforePrepare)
       os::getcwd(),
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   Future<containerizer::Termination> wait = containerizer.wait(containerId);
@@ -1199,7 +1212,7 @@ TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
       os::getcwd(),
       None(),
       SlaveID(),
-      PID<Slave>(),
+      map<string, string>(),
       false);
 
   AWAIT_READY(launch);

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/tests/hook_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hook_tests.cpp b/src/tests/hook_tests.cpp
index 5e0c487..ac1188e 100644
--- a/src/tests/hook_tests.cpp
+++ b/src/tests/hook_tests.cpp
@@ -292,18 +292,32 @@ TEST_F(HookTest, VerifySlaveExecutorEnvironmentDecorator)
   ContainerID containerId;
   containerId.set_value("test_container");
 
+  ExecutorInfo executorInfo =
+    CREATE_EXECUTOR_INFO("executor", "test $FOO = 'bar'");
+
+  SlaveID slaveId = SlaveID();
+
+  std::map<string, string> environment = executorEnvironment(
+      CreateSlaveFlags(),
+      executorInfo,
+      directory,
+      slaveId,
+      PID<Slave>(),
+      false);
+
   // Test hook adds a new environment variable "FOO" to the executor
   // with a value "bar". A '0' (success) exit status for the following
   // command validates the hook.
   process::Future<bool> launch = containerizer->launch(
       containerId,
       None(),
-      CREATE_EXECUTOR_INFO("executor", "test $FOO = 'bar'"),
+      executorInfo,
       directory,
       None(),
-      SlaveID(),
-      process::PID<Slave>(),
+      slaveId,
+      environment,
       false);
+
   AWAIT_READY(launch);
   ASSERT_TRUE(launch.get());
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index ad31276..3a1860d 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -1622,7 +1622,7 @@ public:
           const std::string&,
           const Option<std::string>&,
           const SlaveID&,
-          const process::PID<slave::Slave>&,
+          const std::map<std::string, std::string>&,
           bool checkpoint));
 
   MOCK_METHOD2(
@@ -1640,7 +1640,7 @@ public:
       const std::string& directory,
       const Option<std::string>& user,
       const SlaveID& slaveId,
-      const slave::PID<slave::Slave>& slavePid,
+      const std::map<std::string, std::string>& environment,
       bool checkpoint)
   {
     return slave::DockerContainerizer::launch(
@@ -1650,7 +1650,7 @@ public:
         directory,
         user,
         slaveId,
-        slavePid,
+        environment,
         checkpoint);
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0aee8bc/src/tests/slave_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_tests.cpp b/src/tests/slave_tests.cpp
index 04fab1e..30ca3da 100644
--- a/src/tests/slave_tests.cpp
+++ b/src/tests/slave_tests.cpp
@@ -682,7 +682,7 @@ TEST_F(SlaveTest, LaunchTaskInfoWithContainerInfo)
       sandbox.get(),
       "test",
       slaveID,
-      slave.self(),
+      map<string, string>(),
       false);
   AWAIT_READY(launch);