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 21:37:18 UTC

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

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

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

commit ebb4e6ce354700ec38c14e157cf3a8ca0f9e939d
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Thu Aug 16 14:36:20 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 0bd9f37..b52813e 100644
--- a/src/slave/constants.hpp
+++ b/src/slave/constants.hpp
@@ -65,10 +65,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 f1727cd..1331bee 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -353,13 +353,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 78e8666..dd2daca 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -226,7 +226,6 @@ Slave::Slave(const string& id,
     authenticating(None()),
     authenticated(false),
     reauthenticate(false),
-    failedAuthentications(0),
     executorDirectoryMaxAllowedAge(age(0)),
     resourceEstimator(_resourceEstimator),
     qosController(_qosController),
@@ -1286,6 +1285,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);
 
@@ -1293,7 +1294,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."
@@ -1313,7 +1322,7 @@ void Slave::detected(const Future<Option<MasterInfo>>& _master)
 }
 
 
-void Slave::authenticate()
+void Slave::authenticate(Duration minTimeout, Duration maxTimeout)
 {
   authenticated = false;
 
@@ -1361,15 +1370,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;
@@ -1398,24 +1419,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;
   }
 
@@ -1431,25 +1448,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 802d4eb..0420109 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -492,7 +492,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;
@@ -595,8 +595,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);
 
   // Process creation of persistent volumes (for CREATE) and/or deletion
   // of persistent volumes (for DESTROY) as a part of handling
@@ -800,9 +799,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 24c94fe..ecad7e1 100644
--- a/src/tests/authentication_tests.cpp
+++ b/src/tests/authentication_tests.cpp
@@ -385,7 +385,7 @@ TEST_F(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();