You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by bm...@apache.org on 2018/08/16 22:09:00 UTC

[mesos] branch 1.4.x updated (844e405 -> e44ea50)

This is an automated email from the ASF dual-hosted git repository.

bmahler pushed a change to branch 1.4.x
in repository https://gitbox.apache.org/repos/asf/mesos.git.


    from 844e405  Added MESOS-9144 to the 1.4.3 CHANGELOG.
     new fe2639b  Fixed a backoff overflow bug in scheduler authentication retry logic.
     new 01e91c6  Added a scheduler flag for max authentication timeout interval.
     new 40260af  Fixed a backoff overflow bug in agent authentication retry logic.
     new 69cfc10  Added two agent flags for min/max authentication retry interval.
     new 3decd6c  Added MESOS-9146 to the 1.4.3 CHANGELOG.
     new e44ea50  Added MESOS-9147 to the 1.4.3 CHANGELOG.

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 CHANGELOG                          |  2 +
 docs/authentication.md             | 33 ++++++++++++++++
 docs/configuration.md              | 38 ++++++++++++++----
 src/sched/constants.hpp            | 19 ++++-----
 src/sched/flags.hpp                | 42 +++++++++++++-------
 src/sched/sched.cpp                | 80 ++++++++++++++++++--------------------
 src/slave/constants.hpp            | 15 ++++---
 src/slave/flags.cpp                | 29 ++++++++++----
 src/slave/flags.hpp                |  2 +
 src/slave/slave.cpp                | 79 +++++++++++++++++++------------------
 src/slave/slave.hpp                |  8 +---
 src/tests/authentication_tests.cpp |  2 +-
 12 files changed, 220 insertions(+), 129 deletions(-)


[mesos] 03/06: Fixed a backoff overflow bug in agent authentication retry logic.

Posted by bm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bmahler pushed a commit to branch 1.4.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 40260af15b57173aa15f53a6ab2a785b2dbead72
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Tue Aug 14 17:02:45 2018 -0700

    Fixed a backoff overflow bug in agent authentication retry logic.
    
    This patch fixed the backoff time calculation
    overflow bug described in MESOS-9147.
    
    The old approach times out an authentication request after
    5s and then retries after some backoff time. This is not
    optimal because, if the agent is going to backoff some time
    before retry, we might as well wait that long for the
    previous authentication request (instead of timeout early).
    
    This patch combines the authentication timeout and
    authentication retry backoff interval into a single
    wait time interval. Now agent will timeout the previous
    authentication request after the wait time interval and
    then immediately retry.
    
    Review: https://reviews.apache.org/r/68304
---
 src/slave/constants.hpp            | 13 +++++--
 src/slave/flags.cpp                | 13 +++----
 src/slave/slave.cpp                | 79 ++++++++++++++++++++------------------
 src/slave/slave.hpp                |  8 +---
 src/tests/authentication_tests.cpp |  2 +-
 5 files changed, 60 insertions(+), 55 deletions(-)

diff --git a/src/slave/constants.hpp b/src/slave/constants.hpp
index 71837e7..02db43a 100644
--- a/src/slave/constants.hpp
+++ b/src/slave/constants.hpp
@@ -63,10 +63,17 @@ constexpr Duration DEFAULT_REGISTRATION_BACKOFF_FACTOR = Seconds(1);
 // recovery and when it times out slave re-registration.
 constexpr Duration REGISTER_RETRY_INTERVAL_MAX = Minutes(1);
 
-// The maximum interval the slave waits before retrying authentication.
-constexpr Duration AUTHENTICATION_RETRY_INTERVAL_MAX = Minutes(1);
+// The minimum timeout used when authenticating against the master.
+//
+// TODO(mzhu): Make this configurable.
+constexpr Duration AUTHENTICATION_TIMEOUT_MIN = Seconds(5);
+
+// The maximum timeout used when authenticating against the master.
+//
+// TODO(mzhu): Make this configurable.
+constexpr Duration AUTHENTICATION_TIMEOUT_MAX = Minutes(1);
 
-// Default backoff interval used by the slave to wait after failed
+// Default backoff interval factor used by the slave to wait after failed
 // authentication.
 constexpr Duration DEFAULT_AUTHENTICATION_BACKOFF_FACTOR = Seconds(1);
 
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index 3b02f3e..8b61b58 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -307,13 +307,12 @@ mesos::internal::slave::Flags::Flags()
 
   add(&Flags::authentication_backoff_factor,
       "authentication_backoff_factor",
-      "After a failed authentication the agent picks a random amount of time\n"
-      "between `[0, b]`, where `b = authentication_backoff_factor`, to\n"
-      "authenticate with a new master. Subsequent retries are exponentially\n"
-      "backed off based on this interval (e.g., 1st retry uses a random\n"
-      "value between `[0, b * 2^1]`, 2nd retry between `[0, b * 2^2]`, 3rd\n"
-      "retry between `[0, b * 2^3]`, etc up to a maximum of " +
-          stringify(AUTHENTICATION_RETRY_INTERVAL_MAX),
+      "The agent will time out its authentication with the master based on\n"
+      "exponential backoff. The timeout will be randomly chosen within the\n"
+      "range `[min, min + factor*2^n]` where `min` is " +
+      stringify(AUTHENTICATION_TIMEOUT_MIN) + ", `n` is the number of failed\n"
+      "attempts. The max timeout interval is capped at " +
+      stringify(AUTHENTICATION_TIMEOUT_MAX),
       DEFAULT_AUTHENTICATION_BACKOFF_FACTOR);
 
   add(&Flags::executor_environment_variables,
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 4b3d1fc..9cfeb49 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -207,7 +207,6 @@ Slave::Slave(const string& id,
     authenticating(None()),
     authenticated(false),
     reauthenticate(false),
-    failedAuthentications(0),
     executorDirectoryMaxAllowedAge(age(0)),
     resourceEstimator(_resourceEstimator),
     qosController(_qosController),
@@ -982,6 +981,8 @@ void Slave::detected(const Future<Option<MasterInfo>>& _master)
 
     // Wait for a random amount of time before authentication or
     // registration.
+    //
+    // TODO(mzhu): Specialize this for authetication.
     Duration duration =
       flags.registration_backoff_factor * ((double) os::random() / RAND_MAX);
 
@@ -989,7 +990,15 @@ void Slave::detected(const Future<Option<MasterInfo>>& _master)
       // Authenticate with the master.
       // TODO(vinod): Consider adding an "AUTHENTICATED" state to the
       // slave instead of "authenticate" variable.
-      delay(duration, self(), &Slave::authenticate);
+      Duration maxTimeout = AUTHENTICATION_TIMEOUT_MIN +
+                            flags.authentication_backoff_factor * 2;
+
+      delay(
+          duration,
+          self(),
+          &Slave::authenticate,
+          AUTHENTICATION_TIMEOUT_MIN,
+          std::min(maxTimeout, AUTHENTICATION_TIMEOUT_MAX));
     } else {
       // Proceed with registration without authentication.
       LOG(INFO) << "No credentials provided."
@@ -1009,7 +1018,7 @@ void Slave::detected(const Future<Option<MasterInfo>>& _master)
 }
 
 
-void Slave::authenticate()
+void Slave::authenticate(Duration minTimeout, Duration maxTimeout)
 {
   authenticated = false;
 
@@ -1060,15 +1069,27 @@ void Slave::authenticate()
 
   CHECK_SOME(credential);
 
+  // We pick a random duration between `minTimeout` and `maxTimeout`.
+  Duration timeout =
+    minTimeout + (maxTimeout - minTimeout) * ((double)os::random() / RAND_MAX);
+
   authenticating =
     authenticatee->authenticate(master.get(), self(), credential.get())
-      .onAny(defer(self(), &Self::_authenticate));
+      .onAny(defer(self(), &Self::_authenticate, minTimeout, maxTimeout))
+      .after(timeout, [](Future<bool> future) {
+        // NOTE: Discarded future results in a retry in '_authenticate()'.
+        // This is a no-op if the future is already ready.
+        if (future.discard()) {
+          LOG(WARNING) << "Authentication timed out";
+        }
 
-  delay(Seconds(5), self(), &Self::authenticationTimeout, authenticating.get());
+        return future;
+      });
 }
 
 
-void Slave::_authenticate()
+void Slave::_authenticate(
+    Duration currentMinTimeout, Duration currentMaxTimeout)
 {
   delete CHECK_NOTNULL(authenticatee);
   authenticatee = nullptr;
@@ -1097,24 +1118,20 @@ void Slave::_authenticate()
     authenticating = None();
     reauthenticate = false;
 
-    ++failedAuthentications;
-
-    // Backoff.
-    // The backoff is a random duration in the interval [0, b * 2^N)
-    // where `b = authentication_backoff_factor` and `N` the number
-    // of failed authentication attempts. It is capped by
-    // `REGISTER_RETRY_INTERVAL_MAX`.
-    Duration backoff =
-      flags.authentication_backoff_factor * std::pow(2, failedAuthentications);
-    backoff = std::min(backoff, AUTHENTICATION_RETRY_INTERVAL_MAX);
-
-    // Determine the delay for next attempt by picking a random
-    // duration between 0 and 'maxBackoff'.
-    // TODO(vinod): Use random numbers from <random> header.
-    backoff *= double(os::random()) / RAND_MAX;
-
-    // TODO(vinod): Add a limit on number of retries.
-    delay(backoff, self(), &Self::authenticate); // Retry.
+    // Grow the timeout range using exponential backoff:
+    //
+    //   [min, min + factor * 2^0]
+    //   [min, min + factor * 2^1]
+    //   ...
+    //   [min, min + factor * 2^N]
+    //   ...
+    //   [min, max] // Stop at max.
+    Duration maxTimeout =
+      currentMinTimeout + (currentMaxTimeout - currentMinTimeout) * 2;
+
+    authenticate(
+        currentMinTimeout,
+        std::min(maxTimeout, AUTHENTICATION_TIMEOUT_MAX));
     return;
   }
 
@@ -1130,25 +1147,11 @@ void Slave::_authenticate()
   authenticated = true;
   authenticating = None();
 
-  failedAuthentications = 0;
-
   // Proceed with registration.
   doReliableRegistration(flags.registration_backoff_factor * 2);
 }
 
 
-void Slave::authenticationTimeout(Future<bool> future)
-{
-  // NOTE: Discarded future results in a retry in '_authenticate()'.
-  // Also note that a 'discard' here is safe even if another
-  // authenticator is in progress because this copy of the future
-  // corresponds to the original authenticator that started the timer.
-  if (future.discard()) { // This is a no-op if the future is already ready.
-    LOG(WARNING) << "Authentication timed out";
-  }
-}
-
-
 void Slave::registered(
     const UPID& from,
     const SlaveID& slaveId,
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 5926ef3..ea7f836 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -385,7 +385,7 @@ public:
   // not receive a ping.
   void pingTimeout(process::Future<Option<MasterInfo>> future);
 
-  void authenticate();
+  void authenticate(Duration minTimeout, Duration maxTimeout);
 
   // Helper routines to lookup a framework/executor.
   Framework* getFramework(const FrameworkID& frameworkId) const;
@@ -484,8 +484,7 @@ private:
   Slave(const Slave&) = delete;
   Slave& operator=(const Slave&) = delete;
 
-  void _authenticate();
-  void authenticationTimeout(process::Future<bool> future);
+  void _authenticate(Duration currentMinTimeout, Duration currentMaxTimeout);
 
   // Shut down an executor. This is a two phase process. First, an
   // executor receives a shut down message (shut down phase), then
@@ -628,9 +627,6 @@ private:
   // Indicates if a new authentication attempt should be enforced.
   bool reauthenticate;
 
-  // Indicates the number of failed authentication attempts.
-  uint64_t failedAuthentications;
-
   // Maximum age of executor directories. Will be recomputed
   // periodically every flags.disk_watch_interval.
   Duration executorDirectoryMaxAllowedAge;
diff --git a/src/tests/authentication_tests.cpp b/src/tests/authentication_tests.cpp
index 842badf..70193b5 100644
--- a/src/tests/authentication_tests.cpp
+++ b/src/tests/authentication_tests.cpp
@@ -411,7 +411,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(AuthenticationTest, RetrySlaveAuthentication)
 
   // Advance the clock for the slave to retry.
   Clock::pause();
-  Clock::advance(Seconds(5));
+  Clock::advance(slave::AUTHENTICATION_TIMEOUT_MAX);
   Clock::settle();
   Clock::resume();
 


[mesos] 06/06: Added MESOS-9147 to the 1.4.3 CHANGELOG.

Posted by bm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bmahler pushed a commit to branch 1.4.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit e44ea50c35699630c233dfeab384c1123319c3d4
Author: Benjamin Mahler <bm...@apache.org>
AuthorDate: Thu Aug 16 15:06:42 2018 -0700

    Added MESOS-9147 to the 1.4.3 CHANGELOG.
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index e428316..a044058 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -6,6 +6,7 @@ Release Notes - Mesos - Version 1.4.3 (WIP)
   * [MESOS-9144] - Master authentication handling leads to request amplification.
   * [MESOS-9145] - Master has a fragile burned-in 5s authentication timeout.
   * [MESOS-9146] - Agent has a fragile burn-in 5s authentication timeout.
+  * [MESOS-9147] - Agent and scheduler driver authentication retry backoff time could overflow.
 
 
 Release Notes - Mesos - Version 1.4.2


[mesos] 01/06: Fixed a backoff overflow bug in scheduler authentication retry logic.

Posted by bm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bmahler pushed a commit to branch 1.4.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit fe2639b7d632a0e342a148ea6a24eb115839fd2e
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Mon Aug 13 15:42:09 2018 -0700

    Fixed a backoff overflow bug in scheduler authentication retry logic.
    
    This patch fixed the backoff time calculation
    overflow bug described in MESOS-9147.
    
    The old approach times out an authentication request after
    `flags.authentication_timeout` and then retries after some
    backoff time. This is not optimal because, if the scheduler
    is going to backoff some time before retry, we might as well
    wait that long for the previous authentication request
    (instead of timeout early).
    
    This patch combines the authentication timeout and
    authentication retry backoff interval into a single
    wait time interval. Now scheduler will timeout the previous
    authentication request after the wait time interval and
    then immediately retry.
    
    Review: https://reviews.apache.org/r/68346
---
 src/sched/constants.hpp |  8 +++--
 src/sched/flags.hpp     | 20 +++++++------
 src/sched/sched.cpp     | 80 +++++++++++++++++++++++--------------------------
 3 files changed, 54 insertions(+), 54 deletions(-)

diff --git a/src/sched/constants.hpp b/src/sched/constants.hpp
index 9edb25b..3233ef4 100644
--- a/src/sched/constants.hpp
+++ b/src/sched/constants.hpp
@@ -38,9 +38,11 @@ constexpr Duration DEFAULT_REGISTRATION_BACKOFF_FACTOR = Seconds(2);
 // registration.
 constexpr Duration REGISTRATION_RETRY_INTERVAL_MAX = Minutes(1);
 
-// The maximum interval the scheduler driver waits before retrying
-// authentication.
-constexpr Duration AUTHENTICATION_RETRY_INTERVAL_MAX = Minutes(1);
+// The maximum timeout used when the scheduler driver is authenticating with
+// the master.
+//
+// TODO(mzhu): Make this configurable.
+constexpr Duration AUTHENTICATION_TIMEOUT_MAX = Minutes(1);
 
 // Default backoff interval used by the scheduler to wait after failed
 // authentication.
diff --git a/src/sched/flags.hpp b/src/sched/flags.hpp
index 2492665..c799386 100644
--- a/src/sched/flags.hpp
+++ b/src/sched/flags.hpp
@@ -36,15 +36,6 @@ class Flags : public virtual logging::Flags
 public:
   Flags()
   {
-    add(&Flags::authentication_backoff_factor,
-        "authentication_backoff_factor",
-        "Scheduler driver authentication retries are exponentially backed\n"
-        "off based on 'b', the authentication backoff factor (e.g., 1st retry\n"
-        "uses a random value between `[0, b * 2^1]`, 2nd retry between\n"
-        "`[0, b * 2^2]`, 3rd retry between `[0, b * 2^3]`, etc up to a\n"
-        "maximum of " + stringify(AUTHENTICATION_RETRY_INTERVAL_MAX),
-        DEFAULT_AUTHENTICATION_BACKOFF_FACTOR);
-
     add(&Flags::registration_backoff_factor,
         "registration_backoff_factor",
         "Scheduler driver (re-)registration retries are exponentially backed\n"
@@ -119,6 +110,17 @@ public:
         "or load an alternate authenticatee module using MESOS_MODULES.",
         DEFAULT_AUTHENTICATEE);
 
+    add(&Flags::authentication_backoff_factor,
+        "authentication_backoff_factor",
+        "The scheduler will time out its authentication with the master based\n"
+        "on exponential backoff. The timeout will be randomly chosen within\n"
+        "`[authentication_timeout, authentication_timeout + factor*2^n]`\n"
+        "where `n` is the number of failed attempts. The maximum timeout\n"
+        "internal is capped at " + stringify(AUTHENTICATION_TIMEOUT_MAX) + ".\n"
+        "To tune these parameters, set the `--authentication_timeout` and\n"
+        "`--authentication_backoff_factor` flags.\n",
+        DEFAULT_AUTHENTICATION_BACKOFF_FACTOR);
+
     add(&Flags::authentication_timeout,
         "authentication_timeout",
         "Timeout after which authentication will be retried.",
diff --git a/src/sched/sched.cpp b/src/sched/sched.cpp
index a835ce0..53c19a6 100644
--- a/src/sched/sched.cpp
+++ b/src/sched/sched.cpp
@@ -346,7 +346,12 @@ protected:
         // Authenticate with the master.
         // TODO(adam-mesos): Consider adding an initial delay like we do for
         // slave registration, to combat thundering herds on master failover.
-        authenticate();
+        authenticate(
+            flags.authentication_timeout,
+            std::min(
+                flags.authentication_timeout +
+                  flags.authentication_backoff_factor * 2,
+                scheduler::AUTHENTICATION_TIMEOUT_MAX));
       } else {
         // Proceed with registration without authentication.
         LOG(INFO) << "No credentials provided."
@@ -379,7 +384,7 @@ protected:
   }
 
 #ifdef HAS_AUTHENTICATION
-  void authenticate()
+  void authenticate(Duration minTimeout, Duration maxTimeout)
   {
     if (!running.load()) {
       VLOG(1) << "Ignoring authenticate because the driver is not running!";
@@ -425,6 +430,10 @@ protected:
       authenticatee = module.get();
     }
 
+    // We pick a random duration between `minTimeout` and `maxTimeout`.
+    Duration timeout = minTimeout + (maxTimeout - minTimeout) *
+                                      ((double)os::random() / RAND_MAX);
+
     // NOTE: We do not pass 'Owned<Authenticatee>' here because doing
     // so could make 'AuthenticateeProcess' responsible for deleting
     // 'Authenticatee' causing a deadlock because the destructor of
@@ -439,16 +448,20 @@ protected:
     // --> '~Authenticatee()' is invoked by 'AuthenticateeProcess'.
     // TODO(vinod): Consider using 'Shared' to 'Owned' upgrade.
     authenticating =
-      authenticatee->authenticate(master.get().pid(), self(), credential.get())
-        .onAny(defer(self(), &Self::_authenticate));
+      authenticatee->authenticate(master->pid(), self(), credential.get())
+        .onAny(defer(self(), &Self::_authenticate, minTimeout, maxTimeout))
+        .after(timeout, [](Future<bool> future) {
+          // NOTE: Discarded future results in a retry in '_authenticate()'.
+          // This is a no-op if the future is already ready.
+          if (future.discard()) {
+            LOG(WARNING) << "Authentication timed out";
+          }
 
-    delay(flags.authentication_timeout,
-          self(),
-          &Self::authenticationTimeout,
-          authenticating.get());
+          return future;
+        });
   }
 
-  void _authenticate()
+  void _authenticate(Duration currentMinTimeout, Duration currentMaxTimeout)
   {
     if (!running.load()) {
       VLOG(1) << "Ignoring _authenticate because the driver is not running!";
@@ -482,24 +495,23 @@ protected:
       authenticating = None();
       reauthenticate = false;
 
-      ++failedAuthentications;
-
-      // Backoff.
-      // The backoff is a random duration in the interval [0, b * 2^N)
-      // where `b = authentication_backoff_factor` and `N` the number
-      // of failed authentication attempts. It is capped by
-      // `REGISTER_RETRY_INTERVAL_MAX`.
-      Duration backoff = flags.authentication_backoff_factor *
-                         std::pow(2, failedAuthentications);
-      backoff = std::min(backoff, scheduler::AUTHENTICATION_RETRY_INTERVAL_MAX);
+      // TODO(vinod): Add a limit on number of retries.
 
-      // Determine the delay for next attempt by picking a random
-      // duration between 0 and 'maxBackoff'.
-      // TODO(vinod): Use random numbers from <random> header.
-      backoff *= double(os::random()) / RAND_MAX;
+      // Grow the timeout range using exponential backoff:
+      //
+      //   [min, min + factor * 2^0]
+      //   [min, min + factor * 2^1]
+      //   ...
+      //   [min, min + factor * 2^N]
+      //   ...
+      //   [min, max] // Stop at max.
+      Duration maxTimeout =
+        currentMinTimeout + (currentMaxTimeout - currentMinTimeout) * 2;
+
+      authenticate(
+          currentMinTimeout,
+          std::min(maxTimeout, scheduler::AUTHENTICATION_TIMEOUT_MAX));
 
-      // TODO(vinod): Add a limit on number of retries.
-      delay(backoff, self(), &Self::authenticate);
       return;
     }
 
@@ -520,25 +532,9 @@ protected:
 
     doReliableRegistration(flags.registration_backoff_factor);
   }
-
-  void authenticationTimeout(Future<bool> future)
-  {
-    if (!running.load()) {
-      VLOG(1) << "Ignoring authentication timeout because "
-              << "the driver is not running!";
-      return;
-    }
-
-    // NOTE: Discarded future results in a retry in '_authenticate()'.
-    // Also note that a 'discard' here is safe even if another
-    // authenticator is in progress because this copy of the future
-    // corresponds to the original authenticator that started the timer.
-    if (future.discard()) { // This is a no-op if the future is already ready.
-      LOG(WARNING) << "Authentication timed out";
-    }
-  }
 #endif // HAS_AUTHENTICATION
 
+
   void drop(const Event& event, const string& message)
   {
     // TODO(bmahler): Increment a metric.


[mesos] 02/06: Added a scheduler flag for max authentication timeout interval.

Posted by bm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bmahler pushed a commit to branch 1.4.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 01e91c606c0ed962267111b10b6d4abb3ee69ad2
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Mon Aug 13 16:03:19 2018 -0700

    Added a scheduler flag for max authentication timeout interval.
    
    `flags.authentication_timeout_max` controls the
    maximum amount of time the scheduler waits before retrying
    authenticating with the master after a failed attempt.
    
    Also updated descriptions for related flags
    `flags.authentication_timeout` and
    `flags.authentication_backoff_factor`.
    
    Also deprecated `flags.authentication_timeout` in favor
    of `authentication_timeout_min`.
    
    Also updated documentations regarding configuration and
    authentication.
    
    Review: https://reviews.apache.org/r/68347
---
 docs/authentication.md  | 19 +++++++++++++++++++
 src/sched/constants.hpp | 21 ++++++++++-----------
 src/sched/flags.hpp     | 32 ++++++++++++++++++++++----------
 src/sched/sched.cpp     |  8 ++++----
 4 files changed, 55 insertions(+), 25 deletions(-)

diff --git a/docs/authentication.md b/docs/authentication.md
index 1b00695..c5dd11c 100644
--- a/docs/authentication.md
+++ b/docs/authentication.md
@@ -115,6 +115,25 @@ Mesos master and agent processes. For more information, refer to the
   format) of accepted credentials.  This may be optional depending on the
   authenticator being used.
 
+### Scheduler Driver
+
+* `--authenticatee` - Analog to the master's `--authenticators` option to
+  specify what module to use.  Defaults to `crammd5`.
+
+* `--authentication_backoff_factor` - The scheduler will time out its
+  authentication with the master based on exponential backoff. The timeout will
+  be randomly chosen within the range `[min, min + factor*2^n]` where `n` is
+  the number of failed attempts. To tune these parameters, set the
+  `--authentication_timeout_[min|max|factor]` flags. (default: 1secs)
+
+* `--authentication_timeout_min` - The minimum amount of time the scheduler
+  waits before retrying authenticating with the master. See
+  `--authentication_backoff_factor` for more details. (default: 5secs)
+
+* `--authentication_timeout_max` - The maximum amount of time the scheduler
+  waits before retrying authenticating with the master. See
+  `--authentication_backoff_factor` for more details. (default: 1mins)
+
 ### Multiple HTTP Authenticators
 
 Multiple HTTP authenticators may be loaded into the Mesos master and agent. In
diff --git a/src/sched/constants.hpp b/src/sched/constants.hpp
index 3233ef4..14f577a 100644
--- a/src/sched/constants.hpp
+++ b/src/sched/constants.hpp
@@ -38,21 +38,20 @@ constexpr Duration DEFAULT_REGISTRATION_BACKOFF_FACTOR = Seconds(2);
 // registration.
 constexpr Duration REGISTRATION_RETRY_INTERVAL_MAX = Minutes(1);
 
-// The maximum timeout used when the scheduler driver is authenticating with
-// the master.
-//
-// TODO(mzhu): Make this configurable.
-constexpr Duration AUTHENTICATION_TIMEOUT_MAX = Minutes(1);
+// Name of the default, CRAM-MD5 authenticatee.
+constexpr char DEFAULT_AUTHENTICATEE[] = "crammd5";
 
-// Default backoff interval used by the scheduler to wait after failed
-// authentication.
+// Default value for `--authentication_backoff_factor`. The backoff timeout
+// factor used by the scheduler when authenticating with the master.
 constexpr Duration DEFAULT_AUTHENTICATION_BACKOFF_FACTOR = Seconds(1);
 
-// Name of the default, CRAM-MD5 authenticatee.
-constexpr char DEFAULT_AUTHENTICATEE[] = "crammd5";
+// Default value for `--authentication_timeout_min`. The minimum amount of
+// time the scheduler waits before retrying authenticating with the master.
+constexpr Duration DEFAULT_AUTHENTICATION_TIMEOUT_MIN = Seconds(5);
 
-// Default value for `--authentication_timeout`.
-constexpr Duration DEFAULT_AUTHENTICATION_TIMEOUT = Seconds(5);
+// Default value for `--authentication_timeout_max`. The maximum amount of
+// time the scheduler waits before retrying authenticating with the master.
+constexpr Duration DEFAULT_AUTHENTICATION_TIMEOUT_MAX = Minutes(1);
 
 } // namespace scheduler {
 } // namespace internal {
diff --git a/src/sched/flags.hpp b/src/sched/flags.hpp
index c799386..811c729 100644
--- a/src/sched/flags.hpp
+++ b/src/sched/flags.hpp
@@ -114,17 +114,28 @@ public:
         "authentication_backoff_factor",
         "The scheduler will time out its authentication with the master based\n"
         "on exponential backoff. The timeout will be randomly chosen within\n"
-        "`[authentication_timeout, authentication_timeout + factor*2^n]`\n"
-        "where `n` is the number of failed attempts. The maximum timeout\n"
-        "internal is capped at " + stringify(AUTHENTICATION_TIMEOUT_MAX) + ".\n"
-        "To tune these parameters, set the `--authentication_timeout` and\n"
-        "`--authentication_backoff_factor` flags.\n",
+        "the range `[min, min + factor*2^n]` where `n` is the number of\n"
+        "failed attempts. To tune these parameters, set the\n"
+        "`--authentication_timeout_[min|max|factor]` flags.\n",
         DEFAULT_AUTHENTICATION_BACKOFF_FACTOR);
 
-    add(&Flags::authentication_timeout,
-        "authentication_timeout",
-        "Timeout after which authentication will be retried.",
-        DEFAULT_AUTHENTICATION_TIMEOUT);
+    add(&Flags::authentication_timeout_min,
+        "authentication_timeout_min",
+        flags::DeprecatedName("authentication_timeout"),
+        "The minimum amount of time the scheduler waits before retrying\n"
+        "authenticating with the master. See `authentication_backoff_factor`\n"
+        "for more details. NOTE: since authentication retry cancels the\n"
+        "previous authentication request, one should consider what is the\n"
+        "normal authentication delay when setting this flag to prevent\n"
+        "premature retry",
+        DEFAULT_AUTHENTICATION_TIMEOUT_MIN);
+
+    add(&Flags::authentication_timeout_max,
+      "authentication_timeout_max",
+      "The maximum amount of time the scheduler waits before retrying\n"
+      "authenticating with the master. See `authentication_backoff_factor`\n"
+      "for more details",
+      DEFAULT_AUTHENTICATION_TIMEOUT_MAX);
   }
 
   Duration authentication_backoff_factor;
@@ -132,7 +143,8 @@ public:
   Option<Modules> modules;
   Option<std::string> modulesDir;
   std::string authenticatee;
-  Duration authentication_timeout;
+  Duration authentication_timeout_min;
+  Duration authentication_timeout_max;
 };
 
 } // namespace scheduler {
diff --git a/src/sched/sched.cpp b/src/sched/sched.cpp
index 53c19a6..aa8459d 100644
--- a/src/sched/sched.cpp
+++ b/src/sched/sched.cpp
@@ -347,11 +347,11 @@ protected:
         // TODO(adam-mesos): Consider adding an initial delay like we do for
         // slave registration, to combat thundering herds on master failover.
         authenticate(
-            flags.authentication_timeout,
+            flags.authentication_timeout_min,
             std::min(
-                flags.authentication_timeout +
+                flags.authentication_timeout_min +
                   flags.authentication_backoff_factor * 2,
-                scheduler::AUTHENTICATION_TIMEOUT_MAX));
+                flags.authentication_timeout_max));
       } else {
         // Proceed with registration without authentication.
         LOG(INFO) << "No credentials provided."
@@ -510,7 +510,7 @@ protected:
 
       authenticate(
           currentMinTimeout,
-          std::min(maxTimeout, scheduler::AUTHENTICATION_TIMEOUT_MAX));
+          std::min(maxTimeout, flags.authentication_timeout_max));
 
       return;
     }


[mesos] 05/06: Added MESOS-9146 to the 1.4.3 CHANGELOG.

Posted by bm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bmahler pushed a commit to branch 1.4.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 3decd6c94fd2be9e8dd88642133ae3c23a238f9e
Author: Benjamin Mahler <bm...@apache.org>
AuthorDate: Thu Aug 16 15:06:31 2018 -0700

    Added MESOS-9146 to the 1.4.3 CHANGELOG.
---
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG b/CHANGELOG
index acb4145..e428316 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -5,6 +5,7 @@ Release Notes - Mesos - Version 1.4.3 (WIP)
 ** Bug
   * [MESOS-9144] - Master authentication handling leads to request amplification.
   * [MESOS-9145] - Master has a fragile burned-in 5s authentication timeout.
+  * [MESOS-9146] - Agent has a fragile burn-in 5s authentication timeout.
 
 
 Release Notes - Mesos - Version 1.4.2


[mesos] 04/06: Added two agent flags for min/max authentication retry interval.

Posted by bm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

bmahler pushed a commit to branch 1.4.x
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 69cfc10272d60269a48c506aa266a1d97037bf49
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Tue Aug 14 17:03:25 2018 -0700

    Added two agent flags for min/max authentication retry interval.
    
    flags.authentication_timeout_min controls the
    minimum amount of time the agent waits before retrying
    authenticating with the master after a failed attempt.
    
    flags.authentication_timeout_max controls the
    maximum amount of time the agent waits before retrying
    authenticating with the master after a failed attempt.
    
    Review: https://reviews.apache.org/r/68324
---
 docs/authentication.md             | 14 ++++++++++++++
 docs/configuration.md              | 38 +++++++++++++++++++++++++++++++-------
 src/slave/constants.hpp            | 20 +++++++++-----------
 src/slave/flags.cpp                | 24 ++++++++++++++++++++----
 src/slave/flags.hpp                |  2 ++
 src/slave/slave.cpp                |  8 ++++----
 src/tests/authentication_tests.cpp |  2 +-
 7 files changed, 81 insertions(+), 27 deletions(-)

diff --git a/docs/authentication.md b/docs/authentication.md
index c5dd11c..dbb64e0 100644
--- a/docs/authentication.md
+++ b/docs/authentication.md
@@ -115,6 +115,20 @@ Mesos master and agent processes. For more information, refer to the
   format) of accepted credentials.  This may be optional depending on the
   authenticator being used.
 
+* `--authentication_backoff_factor` - The agent will time out its authentication
+  with the master based on exponential backoff. The timeout will be randomly
+  chosen within the range `[min, min + factor*2^n]` where `n` is the number of
+  failed attempts. To tune these parameters, set the
+  `--authentication_timeout_[min|max|factor]` flags. (default: 1secs)
+
+* `--authentication_timeout_min` - The minimum amount of time the agent waits
+  before retrying authenticating with the master. See
+  `--authentication_backoff_factor` for more details. (default: 5secs)
+
+* `--authentication_timeout_max` - The maximum amount of time the agent waits
+  before retrying authenticating with the master. See
+  `--authentication_backoff_factor` for more details. (default: 1mins)
+
 ### Scheduler Driver
 
 * `--authenticatee` - Analog to the master's `--authenticators` option to
diff --git a/docs/configuration.md b/docs/configuration.md
index 173b8a7..5d90acf 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -1194,15 +1194,39 @@ load an alternate authenticatee module using <code>--modules</code>. (default: c
     --authentication_backoff_factor=VALUE
   </td>
   <td>
-After a failed authentication the agent picks a random amount of time between
-<code>[0, b]</code>, where <code>b = authentication_backoff_factor</code>, to
-authenticate with a new master. Subsequent retries are exponentially backed
-off based on this interval (e.g., 1st retry uses a random value between
-<code>[0, b * 2^1]</code>, 2nd retry between <code>[0, b * 2^2]</code>, 3rd
-retry between <code>[0, b * 2^3]</code>, etc up to a maximum of 1mins
-(default: 1secs)
+The agent will time out its authentication with the master based on
+exponential backoff. The timeout will be randomly chosen within the
+range <code>[min, min + factor*2^n]</code> where <code>n</code> is the number
+of failed attempts. To tune these parameters, set the
+<code>--authentication_timeout_[min|max|factor]</code> flags. (default: 1secs)
   </td>
 </tr>
+
+<tr id="authentication_timeout_min">
+  <td>
+    --authentication_timeout_min=VALUE
+  </td>
+  <td>
+The minimum amount of time the agent waits before retrying authenticating
+with the master. See <code>--authentication_backoff_factor</code> for more
+details. (default: 5secs)
+<p/>NOTE that since authentication retry cancels the previous authentication
+request, one should consider what is the normal authentication delay when
+setting this flag to prevent premature retry.</p>
+  </td>
+</tr>
+
+<tr id="authentication_timeout_max">
+  <td>
+    --authentication_timeout_max=VALUE
+  </td>
+  <td>
+The maximum amount of time the agent waits before retrying authenticating
+with the master. See <code>--authentication_backoff_factor</code> for more
+details. (default: 1mins)
+  </td>
+</tr>
+
 <tr>
   <td>
     --authorizer=VALUE
diff --git a/src/slave/constants.hpp b/src/slave/constants.hpp
index 02db43a..31ec95c 100644
--- a/src/slave/constants.hpp
+++ b/src/slave/constants.hpp
@@ -63,19 +63,17 @@ constexpr Duration DEFAULT_REGISTRATION_BACKOFF_FACTOR = Seconds(1);
 // recovery and when it times out slave re-registration.
 constexpr Duration REGISTER_RETRY_INTERVAL_MAX = Minutes(1);
 
-// The minimum timeout used when authenticating against the master.
-//
-// TODO(mzhu): Make this configurable.
-constexpr Duration AUTHENTICATION_TIMEOUT_MIN = Seconds(5);
+// Default value for `--authentication_backoff_factor`. The backoff interval
+// factor affects the agent timeout interval after failed authentications.
+constexpr Duration DEFAULT_AUTHENTICATION_BACKOFF_FACTOR = Seconds(1);
 
-// The maximum timeout used when authenticating against the master.
-//
-// TODO(mzhu): Make this configurable.
-constexpr Duration AUTHENTICATION_TIMEOUT_MAX = Minutes(1);
+// Default value for `--authentication_timeout_min`. The minimum timeout
+// interval the agent waits before retrying authentication.
+constexpr Duration DEFAULT_AUTHENTICATION_TIMEOUT_MIN = Seconds(5);
 
-// Default backoff interval factor used by the slave to wait after failed
-// authentication.
-constexpr Duration DEFAULT_AUTHENTICATION_BACKOFF_FACTOR = Seconds(1);
+// Default value for `--authentication_timeout_max`. The maximum timeout
+// interval the agent waits before retrying authentication.
+constexpr Duration DEFAULT_AUTHENTICATION_TIMEOUT_MAX = Minutes(1);
 
 constexpr Duration GC_DELAY = Weeks(1);
 constexpr Duration DISK_WATCH_INTERVAL = Minutes(1);
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index 8b61b58..96ce83a 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -309,12 +309,28 @@ mesos::internal::slave::Flags::Flags()
       "authentication_backoff_factor",
       "The agent will time out its authentication with the master based on\n"
       "exponential backoff. The timeout will be randomly chosen within the\n"
-      "range `[min, min + factor*2^n]` where `min` is " +
-      stringify(AUTHENTICATION_TIMEOUT_MIN) + ", `n` is the number of failed\n"
-      "attempts. The max timeout interval is capped at " +
-      stringify(AUTHENTICATION_TIMEOUT_MAX),
+      "range `[min, min + factor*2^n]` where `n` is the number of failed\n"
+      "attempts. To tune these parameters, set the\n"
+      "`--authentication_timeout_[min|max|factor]` flags.\n",
       DEFAULT_AUTHENTICATION_BACKOFF_FACTOR);
 
+  add(&Flags::authentication_timeout_min,
+      "authentication_timeout_min",
+      "The minimum amount of time the agent waits before retrying\n"
+      "authenticating with the master. See `authentication_backoff_factor`\n"
+      "for more details. NOTE that since authentication retry cancels the\n"
+      "previous authentication request, one should consider what is the\n"
+      "normal authentication delay when setting this flag to prevent\n"
+      "premature retry.",
+      DEFAULT_AUTHENTICATION_TIMEOUT_MIN);
+
+  add(&Flags::authentication_timeout_max,
+      "authentication_timeout_max",
+      "The maximum amount of time the agent waits before retrying\n"
+      "authenticating with the master. See `authentication_backoff_factor`\n"
+      "for more details.",
+      DEFAULT_AUTHENTICATION_TIMEOUT_MAX);
+
   add(&Flags::executor_environment_variables,
       "executor_environment_variables",
       "JSON object representing the environment variables that should be\n"
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 2970fea..1458d7d 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -76,6 +76,8 @@ public:
   std::string frameworks_home;  // TODO(benh): Make an Option.
   Duration registration_backoff_factor;
   Duration authentication_backoff_factor;
+  Duration authentication_timeout_min;
+  Duration authentication_timeout_max;
   Option<JSON::Object> executor_environment_variables;
   Duration executor_registration_timeout;
   Duration executor_reregistration_timeout;
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 9cfeb49..739fbf9 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -990,15 +990,15 @@ void Slave::detected(const Future<Option<MasterInfo>>& _master)
       // Authenticate with the master.
       // TODO(vinod): Consider adding an "AUTHENTICATED" state to the
       // slave instead of "authenticate" variable.
-      Duration maxTimeout = AUTHENTICATION_TIMEOUT_MIN +
+      Duration maxTimeout = flags.authentication_timeout_min +
                             flags.authentication_backoff_factor * 2;
 
       delay(
           duration,
           self(),
           &Slave::authenticate,
-          AUTHENTICATION_TIMEOUT_MIN,
-          std::min(maxTimeout, AUTHENTICATION_TIMEOUT_MAX));
+          flags.authentication_timeout_min,
+          std::min(maxTimeout, flags.authentication_timeout_max));
     } else {
       // Proceed with registration without authentication.
       LOG(INFO) << "No credentials provided."
@@ -1131,7 +1131,7 @@ void Slave::_authenticate(
 
     authenticate(
         currentMinTimeout,
-        std::min(maxTimeout, AUTHENTICATION_TIMEOUT_MAX));
+        std::min(maxTimeout, flags.authentication_timeout_max));
     return;
   }
 
diff --git a/src/tests/authentication_tests.cpp b/src/tests/authentication_tests.cpp
index 70193b5..6d21ff1 100644
--- a/src/tests/authentication_tests.cpp
+++ b/src/tests/authentication_tests.cpp
@@ -411,7 +411,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(AuthenticationTest, RetrySlaveAuthentication)
 
   // Advance the clock for the slave to retry.
   Clock::pause();
-  Clock::advance(slave::AUTHENTICATION_TIMEOUT_MAX);
+  Clock::advance(slave::DEFAULT_AUTHENTICATION_TIMEOUT_MAX);
   Clock::settle();
   Clock::resume();