You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by vi...@apache.org on 2016/10/18 20:28:47 UTC

[2/2] mesos git commit: Revert "Refactored the agent 'launcher' flag to always have a value."

Revert "Refactored the agent 'launcher' flag to always have a value."

This reverts commit 4a529e92209497d7561db8ec2516667b97c9bc59.

This was incorrectly cherry picked onto 1.0.x to fix a build issue.
While it fixed the build it broke `mesos-local` that was fixed as
part of MESOS-5613.

The real issue is that when the patch corresponding to
https://reviews.apache.org/r/50003/ was cherry-picked it's dependent
review https://reviews.apache.org/r/50002/ was accidentally not
cherry-picked.


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

Branch: refs/heads/1.0.x
Commit: 041249f6912ca8920461d17f3730bb12fd0f6349
Parents: 839537a
Author: Vinod Kone <vi...@gmail.com>
Authored: Mon Oct 17 14:42:44 2016 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Mon Oct 17 14:42:44 2016 -0700

----------------------------------------------------------------------
 src/local/local.cpp                             | 84 +++++++++-----------
 src/slave/containerizer/mesos/containerizer.cpp | 30 ++++---
 src/slave/flags.cpp                             | 14 +---
 src/slave/flags.hpp                             |  2 +-
 4 files changed, 58 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/041249f6/src/local/local.cpp
----------------------------------------------------------------------
diff --git a/src/local/local.cpp b/src/local/local.cpp
index 2be5bcf..1e1d246 100644
--- a/src/local/local.cpp
+++ b/src/local/local.cpp
@@ -161,8 +161,8 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
     // Save the instance for deleting later.
     allocator = defaultAllocator.get();
   } else {
-    // TODO(benh): Figure out the behavior of allocator pointer and
-    // remove the else block.
+    // TODO(benh): Figure out the behavior of allocator pointer and remove the
+    // else block.
     allocator = nullptr;
   }
 
@@ -176,13 +176,12 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
       << flags.work_dir << "': " << mkdir.error();
   }
 
-  {
-    master::Flags masterFlags;
-
-    Try<flags::Warnings> load = masterFlags.load("MESOS_");
-
-    masterFlags.work_dir = flags.work_dir;
+  std::map<std::string, std::string> propagated_flags;
+  propagated_flags["work_dir"] = flags.work_dir;
 
+  {
+    master::Flags flags;
+    Try<flags::Warnings> load = flags.load(propagated_flags, false, "MESOS_");
     if (load.isError()) {
       EXIT(EXIT_FAILURE)
         << "Failed to start a local cluster while loading"
@@ -196,42 +195,40 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
 
     // Load modules. Note that this covers both, master and slave
     // specific modules as both use the same flag (--modules).
-    if (masterFlags.modules.isSome()) {
-      Try<Nothing> result = ModuleManager::load(masterFlags.modules.get());
+    if (flags.modules.isSome()) {
+      Try<Nothing> result = ModuleManager::load(flags.modules.get());
       if (result.isError()) {
         EXIT(EXIT_FAILURE) << "Error loading modules: " << result.error();
       }
     }
 
-    if (masterFlags.registry == "in_memory") {
+    if (flags.registry == "in_memory") {
       storage = new mesos::state::InMemoryStorage();
-    } else if (masterFlags.registry == "replicated_log") {
+    } else if (flags.registry == "replicated_log") {
       // TODO(vinod): Add support for replicated log with ZooKeeper.
       log = new Log(
           1,
-          path::join(masterFlags.work_dir.get(), "replicated_log"),
+          path::join(flags.work_dir.get(), "replicated_log"),
           set<UPID>(),
-          masterFlags.log_auto_initialize,
+          flags.log_auto_initialize,
           "registrar/");
       storage = new mesos::state::LogStorage(log);
     } else {
       EXIT(EXIT_FAILURE)
-        << "'" << masterFlags.registry << "' is not a supported"
+        << "'" << flags.registry << "' is not a supported"
         << " option for registry persistence";
     }
 
     CHECK_NOTNULL(storage);
 
     state = new mesos::state::protobuf::State(storage);
-    registrar = new Registrar(
-        masterFlags,
-        state,
-        master::READONLY_HTTP_AUTHENTICATION_REALM);
+    registrar =
+      new Registrar(flags, state, master::READONLY_HTTP_AUTHENTICATION_REALM);
 
     contender = new StandaloneMasterContender();
     detector = new StandaloneMasterDetector();
 
-    auto authorizerNames = strings::split(masterFlags.authorizers, ",");
+    auto authorizerNames = strings::split(flags.authorizers, ",");
     if (authorizerNames.empty()) {
       EXIT(EXIT_FAILURE) << "No authorizer specified";
     }
@@ -251,10 +248,10 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
       authorizer = Authorizer::create(authorizerName);
     } else {
       // `authorizerName` is `DEFAULT_AUTHORIZER` at this point.
-      if (masterFlags.acls.isSome()) {
+      if (flags.acls.isSome()) {
         LOG(INFO) << "Creating default '" << authorizerName << "' authorizer";
 
-        authorizer = Authorizer::create(masterFlags.acls.get());
+        authorizer = Authorizer::create(flags.acls.get());
       }
     }
 
@@ -266,17 +263,17 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
     }
 
     Option<shared_ptr<RateLimiter>> slaveRemovalLimiter = None();
-    if (masterFlags.agent_removal_rate_limit.isSome()) {
+    if (flags.agent_removal_rate_limit.isSome()) {
       // Parse the flag value.
       // TODO(vinod): Move this parsing logic to flags once we have a
       // 'Rate' abstraction in stout.
       vector<string> tokens =
-        strings::tokenize(masterFlags.agent_removal_rate_limit.get(), "/");
+        strings::tokenize(flags.agent_removal_rate_limit.get(), "/");
 
       if (tokens.size() != 2) {
         EXIT(EXIT_FAILURE)
           << "Invalid agent_removal_rate_limit: "
-          << masterFlags.agent_removal_rate_limit.get()
+          << flags.agent_removal_rate_limit.get()
           << ". Format is <Number of agents>/<Duration>";
       }
 
@@ -284,7 +281,7 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
       if (permits.isError()) {
         EXIT(EXIT_FAILURE)
           << "Invalid agent_removal_rate_limit: "
-          << masterFlags.agent_removal_rate_limit.get()
+          << flags.agent_removal_rate_limit.get()
           << ". Format is <Number of agents>/<Duration>"
           << ": " << permits.error();
       }
@@ -293,7 +290,7 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
       if (duration.isError()) {
         EXIT(EXIT_FAILURE)
           << "Invalid agent_removal_rate_limit: "
-          << masterFlags.agent_removal_rate_limit.get()
+          << flags.agent_removal_rate_limit.get()
           << ". Format is <Number of agents>/<Duration>"
           << ": " << duration.error();
       }
@@ -326,7 +323,7 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
         detector,
         authorizer_,
         slaveRemovalLimiter,
-        masterFlags);
+        flags);
 
     detector->appoint(master->info());
   }
@@ -342,11 +339,9 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
   vector<UPID> pids;
 
   for (int i = 0; i < flags.num_slaves; i++) {
-    slave::Flags slaveFlags;
-
-    Try<flags::Warnings> load = slaveFlags.load("MESOS_");
+    slave::Flags flags;
 
-    slaveFlags.work_dir = flags.work_dir;
+    Try<flags::Warnings> load = flags.load(propagated_flags, false, "MESOS_");
 
     if (load.isError()) {
       EXIT(EXIT_FAILURE)
@@ -360,14 +355,14 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
     }
 
     // Use a different work directory for each slave.
-    slaveFlags.work_dir = path::join(slaveFlags.work_dir, stringify(i));
+    flags.work_dir = path::join(flags.work_dir, stringify(i));
 
     garbageCollectors->push_back(new GarbageCollector());
-    statusUpdateManagers->push_back(new StatusUpdateManager(slaveFlags));
+    statusUpdateManagers->push_back(new StatusUpdateManager(flags));
     fetchers->push_back(new Fetcher());
 
     Try<ResourceEstimator*> resourceEstimator =
-      ResourceEstimator::create(slaveFlags.resource_estimator);
+      ResourceEstimator::create(flags.resource_estimator);
 
     if (resourceEstimator.isError()) {
       EXIT(EXIT_FAILURE)
@@ -377,7 +372,7 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
     resourceEstimators->push_back(resourceEstimator.get());
 
     Try<QoSController*> qosController =
-      QoSController::create(slaveFlags.qos_controller);
+      QoSController::create(flags.qos_controller);
 
     if (qosController.isError()) {
       EXIT(EXIT_FAILURE)
@@ -386,20 +381,13 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
 
     qosControllers->push_back(qosController.get());
 
-    // Override the default launcher that gets created per agent to
-    // 'posix' if we're creating multiple agents because the
-    // LinuxLauncher does not support multiple agents on the same host
-    // (see MESOS-3793).
-    if (flags.num_slaves > 1 && slaveFlags.launcher != "posix") {
-      // TODO(benh): This log line will print out for each slave!
-      LOG(WARNING) << "Using the 'posix' launcher instead of '"
-                   << slaveFlags.launcher << "' since currently only the "
-                   << "'posix' launcher supports multiple agents per host";
-      slaveFlags.launcher = "posix";
+    // Set default launcher to 'posix'(see MESOS-3793).
+    if (flags.launcher.isNone()) {
+      flags.launcher = "posix";
     }
 
     Try<Containerizer*> containerizer =
-      Containerizer::create(slaveFlags, true, fetchers->back());
+      Containerizer::create(flags, true, fetchers->back());
 
     if (containerizer.isError()) {
       EXIT(EXIT_FAILURE)
@@ -410,7 +398,7 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
     // Master.
     Slave* slave = new Slave(
         process::ID::generate("slave"),
-        slaveFlags,
+        flags,
         detector,
         containerizer.get(),
         files,

http://git-wip-us.apache.org/repos/asf/mesos/blob/041249f6/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index d57b6fa..e900d75 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -206,24 +206,34 @@ Try<MesosContainerizer*> MesosContainerizer::create(
   // Create the launcher for the MesosContainerizer.
   Try<Launcher*> launcher = [&flags_]() -> Try<Launcher*> {
 #ifdef __linux__
-    if (flags_.launcher == "linux") {
-      return LinuxLauncher::create(flags_);
-    } else if (flags_.launcher == "posix") {
-      return PosixLauncher::create(flags_);
-    } else {
-      return Error("Unknown or unsupported launcher: " + flags_.launcher);
+    if (flags_.launcher.isSome()) {
+      // If the user has specified the launcher, use it.
+      if (flags_.launcher.get() == "linux") {
+        return LinuxLauncher::create(flags_);
+      } else if (flags_.launcher.get() == "posix") {
+        return PosixLauncher::create(flags_);
+      } else {
+        return Error(
+            "Unknown or unsupported launcher: " + flags_.launcher.get());
+      }
     }
+
+    // Use Linux launcher if it is available, POSIX otherwise.
+    return LinuxLauncher::available()
+      ? LinuxLauncher::create(flags_)
+      : PosixLauncher::create(flags_);
 #elif __WINDOWS__
     // NOTE: Because the most basic launcher historically has been "posix", we
     // accept this flag on Windows, but map it to the `WindowsLauncher`.
-    if (flags_.launcher != "posix" && flags_.launcher != "windows") {
-      return Error("Unsupported launcher: " + flags_.launcher);
+    if (flags_.launcher.isSome() && !(flags_.launcher.get() == "posix" ||
+        flags_.launcher.get() == "windows")) {
+      return Error("Unsupported launcher: " + flags_.launcher.get());
     }
 
     return WindowsLauncher::create(flags_);
 #else
-    if (flags_.launcher != "posix") {
-      return Error("Unsupported launcher: " + flags_.launcher);
+    if (flags_.launcher.isSome() && flags_.launcher.get() != "posix") {
+      return Error("Unsupported launcher: " + flags_.launcher.get());
     }
 
     return PosixLauncher::create(flags_);

http://git-wip-us.apache.org/repos/asf/mesos/blob/041249f6/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index 236b55a..b0c02ae 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -30,10 +30,6 @@
 
 #include "slave/constants.hpp"
 
-#ifdef __linux__
-#include "slave/containerizer/mesos/linux_launcher.hpp"
-#endif // __linux__
-
 using std::string;
 
 mesos::internal::slave::Flags::Flags()
@@ -109,15 +105,7 @@ mesos::internal::slave::Flags::Flags()
       "`linux` or `posix`. The Linux launcher is required for cgroups\n"
       "isolation and for any isolators that require Linux namespaces such as\n"
       "network, pid, etc. If unspecified, the agent will choose the Linux\n"
-      "launcher if it's running as root on Linux.",
-#ifdef __linux__
-      LinuxLauncher::available() ? "linux" : "posix"
-#elif __WINDOWS__
-      "windows"
-#else
-      "posix"
-#endif // __linux__
-      );
+      "launcher if it's running as root on Linux.");
 
   add(&Flags::image_providers,
       "image_providers",

http://git-wip-us.apache.org/repos/asf/mesos/blob/041249f6/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index a3b752e..58fba4a 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -45,7 +45,7 @@ public:
   bool hostname_lookup;
   Option<std::string> resources;
   std::string isolation;
-  std::string launcher;
+  Option<std::string> launcher;
 
   Option<std::string> image_providers;
   std::string image_provisioner_backend;