You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by al...@apache.org on 2018/06/15 13:41:06 UTC

[1/3] mesos git commit: Made `TestContainerizerProcess::launch()` thread safe.

Repository: mesos
Updated Branches:
  refs/heads/master 3bf6eb7c8 -> cab94e6dc


Made `TestContainerizerProcess::launch()` thread safe.

This patch removes `os::setenv()`, `os::unsetenv()` calls that are
currently used to implicitly configure both MesosExecutorDriver and
v1 Executor via global environment variables. This change prepares a
map containing modified environment and passes it to a constructor of
an executor.

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


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

Branch: refs/heads/master
Commit: cab94e6dc741135abddcddc95c1568b13221a69b
Parents: aea6888
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri Jun 15 15:23:38 2018 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Fri Jun 15 15:40:54 2018 +0200

----------------------------------------------------------------------
 src/tests/containerizer.cpp | 100 +++++++++++++--------------------------
 1 file changed, 33 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/cab94e6d/src/tests/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.cpp b/src/tests/containerizer.cpp
index c4e18b7..b3a09fd 100644
--- a/src/tests/containerizer.cpp
+++ b/src/tests/containerizer.cpp
@@ -126,75 +126,41 @@ public:
     containers_.at(containerId)->frameworkId =
       containerConfig.executor_info().framework_id();
 
-    // We need to synchronize all reads and writes to the environment
-    // as this is global state.
+    // Assemble the environment for the executor.
     //
-    // TODO(jmlvanre): Even this is not sufficient, as other aspects
-    // of the code may read an environment variable while we are
-    // manipulating it. The better solution is to pass the environment
-    // variables into the fork, or to set them on the command line.
-    // See MESOS-3475.
-    static std::mutex mutex;
-
-    synchronized(mutex) {
-      // Since the constructor for `MesosExecutorDriver` reads
-      // environment variables to load flags, even it needs to
-      // be within this synchronization section.
-      //
-      // Prepare additional environment variables for the executor.
-      // TODO(benh): Need to get flags passed into the TestContainerizer
-      // in order to properly use here.
-      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();
-
-      foreachpair (const string& name, const string& variable, environment) {
-        os::setenv(name, variable);
-      }
-
-      // TODO(benh): Can this be removed and done exclusively 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,
-               containerConfig.executor_info()
-                 .command().environment().variables()) {
-        os::setenv(variable.name(), variable.value());
-      }
-
-      os::setenv("MESOS_LOCAL", "1");
-
-      const Owned<ExecutorData>& executorData =
-        executors.at(containerConfig.executor_info().executor_id());
-
-      if (executorData->executor != nullptr) {
-        executorData->driver = Owned<MesosExecutorDriver>(
-            new MesosExecutorDriver(executorData->executor));
-        executorData->driver->start();
-      } else {
-        shared_ptr<v1::MockHTTPExecutor> executor =
-          executorData->v1ExecutorMock;
-        executorData->v1Library = Owned<v1::executor::TestMesos>(
-          new v1::executor::TestMesos(ContentType::PROTOBUF, executor));
-      }
-
-      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);
-      }
+    // NOTE: Since in this case the executor will live in the same OS process,
+    // pass the environment into the executor driver (library) c-tor directly
+    // instead of manipulating `setenv`/`getenv` to avoid concurrent
+    // modification of the environment.
+    map<string, string> fullEnvironment = os::environment();
+
+    fullEnvironment.insert(environment.begin(), environment.end());
+
+    // TODO(benh): Can this be removed and done exclusively 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,
+             containerConfig.executor_info()
+               .command().environment().variables()) {
+      fullEnvironment.emplace(variable.name(), variable.value());
+    }
 
-      foreachkey (const string& name, environment) {
-        if (!original.contains(name)) {
-          os::unsetenv(name);
-        }
-      }
+    fullEnvironment.emplace("MESOS_LOCAL", "1");
+
+    const Owned<ExecutorData>& executorData =
+      executors.at(containerConfig.executor_info().executor_id());
+
+    if (executorData->executor != nullptr) {
+      executorData->driver = Owned<MesosExecutorDriver>(
+          new MesosExecutorDriver(executorData->executor, fullEnvironment));
+      executorData->driver->start();
+    } else {
+      shared_ptr<v1::MockHTTPExecutor> executor =
+        executorData->v1ExecutorMock;
+      executorData->v1Library = Owned<v1::executor::TestMesos>(
+          new v1::executor::TestMesos(
+              ContentType::PROTOBUF, executor, fullEnvironment));
     }
 
     return slave::Containerizer::LaunchResult::SUCCESS;


[3/3] mesos git commit: Removed `os::getenv()` calls from `MesosExecutorDriver`.

Posted by al...@apache.org.
Removed `os::getenv()` calls from `MesosExecutorDriver`.

This patch adds overloaded constructor for `MesosExecutorDriver` that
accepts `environment` parameter and stores it in the class variable.
This new constructor is needed to get rid of `os::getenv()` calls,
so that `MesosExecutorDriver` can be used in tests that require
thread safety.

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


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

Branch: refs/heads/master
Commit: 20edb8caa394b41521bc70f39664bb97fe2e1791
Parents: 3bf6eb7
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri Jun 15 15:23:27 2018 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Fri Jun 15 15:40:54 2018 +0200

----------------------------------------------------------------------
 include/mesos/executor.hpp | 15 +++++++++++++++
 src/exec/exec.cpp          | 42 ++++++++++++++++++++++++++++++-----------
 2 files changed, 46 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/20edb8ca/include/mesos/executor.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/executor.hpp b/include/mesos/executor.hpp
index d14c036..6a9e6fc 100644
--- a/include/mesos/executor.hpp
+++ b/include/mesos/executor.hpp
@@ -17,6 +17,7 @@
 #ifndef __MESOS_EXECUTOR_HPP__
 #define __MESOS_EXECUTOR_HPP__
 
+#include <map>
 #include <mutex>
 #include <string>
 
@@ -213,8 +214,20 @@ class MesosExecutorDriver : public ExecutorDriver
 public:
   // Creates a new driver that uses the specified Executor. Note, the
   // executor pointer must outlive the driver.
+  //
+  // Note that the other constructor overload that accepts `environment`
+  // argument is preferable to this one in a multithreaded environment,
+  // because the implementation of this one accesses global environment
+  // which is unsafe due to a potential concurrent modification of the
+  // environment by another thread.
   explicit MesosExecutorDriver(Executor* executor);
 
+  // Creates a new driver that uses the specified `Executor` and environment
+  // variables. Note, the executor pointer must outlive the driver.
+  explicit MesosExecutorDriver(
+      Executor* executor,
+      const std::map<std::string, std::string>& environment);
+
   // This destructor will block indefinitely if
   // MesosExecutorDriver::start was invoked successfully (possibly via
   // MesosExecutorDriver::run) and MesosExecutorDriver::stop has not
@@ -246,6 +259,8 @@ private:
 
   // Current status of the driver.
   Status status;
+
+  std::map<std::string, std::string> environment;
 };
 
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/20edb8ca/src/exec/exec.cpp
----------------------------------------------------------------------
diff --git a/src/exec/exec.cpp b/src/exec/exec.cpp
index 65a671d..ca4f065 100644
--- a/src/exec/exec.cpp
+++ b/src/exec/exec.cpp
@@ -633,17 +633,36 @@ private:
 
 
 MesosExecutorDriver::MesosExecutorDriver(mesos::Executor* _executor)
+  : MesosExecutorDriver(_executor, os::environment())
+{}
+
+
+MesosExecutorDriver::MesosExecutorDriver(
+    mesos::Executor* _executor,
+    const std::map<std::string, std::string>& _environment)
   : executor(_executor),
     process(nullptr),
     latch(nullptr),
-    status(DRIVER_NOT_STARTED)
+    status(DRIVER_NOT_STARTED),
+    environment(_environment)
 {
   GOOGLE_PROTOBUF_VERIFY_VERSION;
 
   // Load any logging flags from the environment.
   logging::Flags flags;
 
-  Try<flags::Warnings> load = flags.load("MESOS_");
+  // Filter out environment variables whose keys don't start with "MESOS_".
+  //
+  // TODO(alexr): This should be supported by `FlagsBase`, see MESOS-9001.
+  std::map<std::string, std::string> env;
+
+  foreachpair (const string& key, const string& value, environment) {
+    if (strings::startsWith(key, "MESOS_")) {
+      env.emplace(key, value);
+    }
+  }
+
+  Try<flags::Warnings> load = flags.load(env, true);
 
   if (load.isError()) {
     status = DRIVER_ABORTED;
@@ -719,12 +738,13 @@ Status MesosExecutorDriver::start()
 
     Option<string> value;
     std::istringstream iss;
+    hashmap<string, string> env(environment);
 
     // Check if this is local (for example, for testing).
-    local = os::getenv("MESOS_LOCAL").isSome();
+    local = env.get("MESOS_LOCAL").isSome();
 
     // Get slave PID from environment.
-    value = os::getenv("MESOS_SLAVE_PID");
+    value = env.get("MESOS_SLAVE_PID");
     if (value.isNone()) {
       EXIT(EXIT_FAILURE)
         << "Expecting 'MESOS_SLAVE_PID' to be set in the environment";
@@ -734,7 +754,7 @@ Status MesosExecutorDriver::start()
     CHECK(slave) << "Cannot parse MESOS_SLAVE_PID '" << value.get() << "'";
 
     // Get slave ID from environment.
-    value = os::getenv("MESOS_SLAVE_ID");
+    value = env.get("MESOS_SLAVE_ID");
     if (value.isNone()) {
       EXIT(EXIT_FAILURE)
         << "Expecting 'MESOS_SLAVE_ID' to be set in the environment";
@@ -742,7 +762,7 @@ Status MesosExecutorDriver::start()
     slaveId.set_value(value.get());
 
     // Get framework ID from environment.
-    value = os::getenv("MESOS_FRAMEWORK_ID");
+    value = env.get("MESOS_FRAMEWORK_ID");
     if (value.isNone()) {
       EXIT(EXIT_FAILURE)
         << "Expecting 'MESOS_FRAMEWORK_ID' to be set in the environment";
@@ -750,7 +770,7 @@ Status MesosExecutorDriver::start()
     frameworkId.set_value(value.get());
 
     // Get executor ID from environment.
-    value = os::getenv("MESOS_EXECUTOR_ID");
+    value = env.get("MESOS_EXECUTOR_ID");
     if (value.isNone()) {
       EXIT(EXIT_FAILURE)
         << "Expecting 'MESOS_EXECUTOR_ID' to be set in the environment";
@@ -758,7 +778,7 @@ Status MesosExecutorDriver::start()
     executorId.set_value(value.get());
 
     // Get working directory from environment.
-    value = os::getenv("MESOS_DIRECTORY");
+    value = env.get("MESOS_DIRECTORY");
     if (value.isNone()) {
       EXIT(EXIT_FAILURE)
         << "Expecting 'MESOS_DIRECTORY' to be set in the environment";
@@ -771,7 +791,7 @@ Status MesosExecutorDriver::start()
     // (in contrast to the others above) for backwards
     // compatibility: agents < 0.28.0 do not set it.
     Duration shutdownGracePeriod = DEFAULT_EXECUTOR_SHUTDOWN_GRACE_PERIOD;
-    value = os::getenv("MESOS_EXECUTOR_SHUTDOWN_GRACE_PERIOD");
+    value = env.get("MESOS_EXECUTOR_SHUTDOWN_GRACE_PERIOD");
     if (value.isSome()) {
       Try<Duration> parse = Duration::parse(value.get());
       if (parse.isError()) {
@@ -784,14 +804,14 @@ Status MesosExecutorDriver::start()
     }
 
     // Get checkpointing status from environment.
-    value = os::getenv("MESOS_CHECKPOINT");
+    value = env.get("MESOS_CHECKPOINT");
     checkpoint = value.isSome() && value.get() == "1";
 
     Duration recoveryTimeout = RECOVERY_TIMEOUT;
 
     // Get the recovery timeout if checkpointing is enabled.
     if (checkpoint) {
-      value = os::getenv("MESOS_RECOVERY_TIMEOUT");
+      value = env.get("MESOS_RECOVERY_TIMEOUT");
 
       if (value.isSome()) {
         Try<Duration> parse = Duration::parse(value.get());


[2/3] mesos git commit: Removed `os::getenv()` calls from `MesosProcess`.

Posted by al...@apache.org.
Removed `os::getenv()` calls from `MesosProcess`.

This patch adds overloaded constructor for `v1::executor::Mesos` class
that accepts `environment` parameter that is passed over to
`MesosProcess` constructor. This change is needed to get rid of
`os::getenv()` calls, so that v1 Executor can be used in tests that
require thread safety.

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


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

Branch: refs/heads/master
Commit: aea688832d8f2ee9ff6b7fff702ffc75e71dfdb1
Parents: 20edb8c
Author: Andrei Budnik <ab...@mesosphere.com>
Authored: Fri Jun 15 15:23:32 2018 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Fri Jun 15 15:40:54 2018 +0200

----------------------------------------------------------------------
 include/mesos/v1/executor.hpp | 12 ++++++++++
 src/executor/executor.cpp     | 48 ++++++++++++++++++++++++++++++--------
 src/tests/mesos.hpp           |  6 +++--
 3 files changed, 54 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/aea68883/include/mesos/v1/executor.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/v1/executor.hpp b/include/mesos/v1/executor.hpp
index ca48f29..9a2eb45 100644
--- a/include/mesos/v1/executor.hpp
+++ b/include/mesos/v1/executor.hpp
@@ -19,6 +19,7 @@
 
 #include <functional>
 #include <queue>
+#include <map>
 #include <string>
 
 #include <mesos/http.hpp>
@@ -54,11 +55,22 @@ public:
 class Mesos : public MesosBase
 {
 public:
+  // The other constructor overload that accepts `environment`
+  // argument is preferable to this one in a multithreaded environment,
+  // because the implementation of this one accesses global environment
+  // which is unsafe due to a potential concurrent modification of the
+  // environment by another thread.
   Mesos(ContentType contentType,
         const std::function<void(void)>& connected,
         const std::function<void(void)>& disconnected,
         const std::function<void(const std::queue<Event>&)>& received);
 
+  Mesos(ContentType contentType,
+        const std::function<void(void)>& connected,
+        const std::function<void(void)>& disconnected,
+        const std::function<void(const std::queue<Event>&)>& received,
+        const std::map<std::string, std::string>& environment);
+
   // Delete copy constructor.
   Mesos(const Mesos& other) = delete;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/aea68883/src/executor/executor.cpp
----------------------------------------------------------------------
diff --git a/src/executor/executor.cpp b/src/executor/executor.cpp
index 5e95f99..ab67cae 100644
--- a/src/executor/executor.cpp
+++ b/src/executor/executor.cpp
@@ -157,7 +157,8 @@ public:
       ContentType _contentType,
       const lambda::function<void(void)>& connected,
       const lambda::function<void(void)>& disconnected,
-      const lambda::function<void(const queue<Event>&)>& received)
+      const lambda::function<void(const queue<Event>&)>& received,
+      const std::map<std::string, std::string>& environment)
     : ProcessBase(generate("executor")),
       state(DISCONNECTED),
       contentType(_contentType),
@@ -168,7 +169,18 @@ public:
     // Load any logging flags from the environment.
     logging::Flags flags;
 
-    Try<flags::Warnings> load = flags.load("MESOS_");
+    // Filter out environment variables whose keys don't start with "MESOS_".
+    //
+    // TODO(alexr): This should be supported by `FlagsBase`, see MESOS-9001.
+    std::map<std::string, std::string> mesosEnvironment;
+
+    foreachpair (const string& key, const string& value, environment) {
+      if (strings::startsWith(key, "MESOS_")) {
+        mesosEnvironment.emplace(key, value);
+      }
+    }
+
+    Try<flags::Warnings> load = flags.load(mesosEnvironment, true);
 
     if (load.isError()) {
       EXIT(EXIT_FAILURE) << "Failed to load flags: " << load.error();
@@ -193,13 +205,15 @@ public:
 
     spawn(new VersionProcess(), true);
 
+    hashmap<string, string> env(mesosEnvironment);
+
     // Check if this is local (for example, for testing).
-    local = os::getenv("MESOS_LOCAL").isSome();
+    local = env.get("MESOS_LOCAL").isSome();
 
     Option<string> value;
 
     // Get agent PID from environment.
-    value = os::getenv("MESOS_SLAVE_PID");
+    value = env.get("MESOS_SLAVE_PID");
     if (value.isNone()) {
       EXIT(EXIT_FAILURE)
         << "Expecting 'MESOS_SLAVE_PID' to be set in the environment";
@@ -223,7 +237,7 @@ public:
         upid.id +
         "/api/v1/executor");
 
-    value = os::getenv("MESOS_EXECUTOR_AUTHENTICATION_TOKEN");
+    value = env.get("MESOS_EXECUTOR_AUTHENTICATION_TOKEN");
     if (value.isSome()) {
       authenticationToken = value.get();
     }
@@ -233,12 +247,12 @@ public:
     os::eraseenv("MESOS_EXECUTOR_AUTHENTICATION_TOKEN");
 
     // Get checkpointing status from environment.
-    value = os::getenv("MESOS_CHECKPOINT");
+    value = env.get("MESOS_CHECKPOINT");
     checkpoint = value.isSome() && value.get() == "1";
 
     if (checkpoint) {
       // Get recovery timeout from environment.
-      value = os::getenv("MESOS_RECOVERY_TIMEOUT");
+      value = env.get("MESOS_RECOVERY_TIMEOUT");
       if (value.isSome()) {
         Try<Duration> _recoveryTimeout = Duration::parse(value.get());
 
@@ -253,7 +267,7 @@ public:
       }
 
       // Get maximum backoff factor from environment.
-      value = os::getenv("MESOS_SUBSCRIPTION_BACKOFF_MAX");
+      value = env.get("MESOS_SUBSCRIPTION_BACKOFF_MAX");
       if (value.isSome()) {
         Try<Duration> _maxBackoff = Duration::parse(value.get());
 
@@ -270,7 +284,7 @@ public:
     }
 
     // Get executor shutdown grace period from the environment.
-    value = os::getenv("MESOS_EXECUTOR_SHUTDOWN_GRACE_PERIOD");
+    value = env.get("MESOS_EXECUTOR_SHUTDOWN_GRACE_PERIOD");
     if (value.isSome()) {
       Try<Duration> _shutdownGracePeriod = Duration::parse(value.get());
 
@@ -833,7 +847,21 @@ Mesos::Mesos(
     const lambda::function<void(void)>& connected,
     const lambda::function<void(void)>& disconnected,
     const lambda::function<void(const queue<Event>&)>& received)
-  : process(new MesosProcess(contentType, connected, disconnected, received))
+  : process(new MesosProcess(
+      contentType, connected, disconnected, received, os::environment()))
+{
+  spawn(process.get());
+}
+
+
+Mesos::Mesos(
+    ContentType contentType,
+    const lambda::function<void(void)>& connected,
+    const lambda::function<void(void)>& disconnected,
+    const lambda::function<void(const queue<Event>&)>& received,
+    const std::map<std::string, std::string>& environment)
+  : process(new MesosProcess(
+      contentType, connected, disconnected, received, environment))
 {
   spawn(process.get());
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/aea68883/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 8f529fa..a23b02c 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -2836,7 +2836,8 @@ class TestMesos : public Mesos
 public:
   TestMesos(
       ContentType contentType,
-      const std::shared_ptr<MockHTTPExecutor<Mesos, Event>>& executor)
+      const std::shared_ptr<MockHTTPExecutor<Mesos, Event>>& executor,
+      const std::map<std::string, std::string>& environment)
     : Mesos(
           contentType,
           lambda::bind(&MockHTTPExecutor<Mesos, Event>::connected,
@@ -2848,7 +2849,8 @@ public:
           lambda::bind(&MockHTTPExecutor<Mesos, Event>::events,
                        executor,
                        this,
-                       lambda::_1)) {}
+                       lambda::_1),
+          environment) {}
 };
 
 } // namespace executor {