You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by me...@apache.org on 2015/03/30 20:48:00 UTC

mesos git commit: Revised authenticator interface to allow for two fold implementations.

Repository: mesos
Updated Branches:
  refs/heads/master 1876ac2fc -> b5d6dfdb6


Revised authenticator interface to allow for two fold implementations.

The initial design and implementation of the authenticator module
interface caused issues and was not optimal for heavy lifting setup of
external dependencies. By introducing a two fold design, this has been
decoupled from the authentication message processing. The new design
also gets us back on track to the goal of makeing SASL a soft dependency
of mesos.

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


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

Branch: refs/heads/master
Commit: b5d6dfdb680b98ed640e4d34dc6c140d70cc19c8
Parents: 1876ac2
Author: Till Toenshoff <to...@me.com>
Authored: Mon Mar 30 11:20:13 2015 -0700
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Mar 30 11:20:13 2015 -0700

----------------------------------------------------------------------
 include/mesos/authentication/authenticator.hpp |   9 +-
 src/authentication/cram_md5/authenticator.cpp  | 216 +++++++++++++-------
 src/authentication/cram_md5/authenticator.hpp  |  25 +--
 src/authentication/cram_md5/auxprop.hpp        |   2 +-
 src/master/master.cpp                          | 134 +++++++-----
 src/master/master.hpp                          |   9 +-
 src/tests/cram_md5_authentication_tests.cpp    |  42 ++--
 7 files changed, 265 insertions(+), 172 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b5d6dfdb/include/mesos/authentication/authenticator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/authentication/authenticator.hpp b/include/mesos/authentication/authenticator.hpp
index f66217a..aa3818c 100644
--- a/include/mesos/authentication/authenticator.hpp
+++ b/include/mesos/authentication/authenticator.hpp
@@ -28,19 +28,19 @@
 #include <process/future.hpp>
 #include <process/pid.hpp>
 
+#include <stout/nothing.hpp>
 #include <stout/option.hpp>
 
 namespace mesos {
 
-// Note that this interface definition is not hardened yet and will
-// slightly change within the next release. See MESOS-2050.
 class Authenticator
 {
 public:
   Authenticator() {}
+
   virtual ~Authenticator() {}
 
-  virtual void initialize(const process::UPID& clientPid) = 0;
+  virtual Try<Nothing> initialize(const Option<Credentials>& credentials) = 0;
 
   // Returns the principal of the Authenticatee if successfully
   // authenticated otherwise None or an error. Note that we
@@ -50,7 +50,8 @@ public:
   // will cause the future to fail if it hasn't already completed
   // since we have already started the authentication procedure and
   // can't reliably cancel.
-  virtual process::Future<Option<std::string>> authenticate(void) = 0;
+  virtual process::Future<Option<std::string>> authenticate(
+      const process::UPID& pid) = 0;
 };
 
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/b5d6dfdb/src/authentication/cram_md5/authenticator.cpp
----------------------------------------------------------------------
diff --git a/src/authentication/cram_md5/authenticator.cpp b/src/authentication/cram_md5/authenticator.cpp
index 2125b45..6a84e91 100644
--- a/src/authentication/cram_md5/authenticator.cpp
+++ b/src/authentication/cram_md5/authenticator.cpp
@@ -28,14 +28,17 @@
 
 #include <process/defer.hpp>
 #include <process/once.hpp>
+#include <process/owned.hpp>
 #include <process/protobuf.hpp>
 
 #include <stout/check.hpp>
-
-#include "authentication/cram_md5/auxprop.hpp"
+#include <stout/hashmap.hpp>
+#include <stout/lambda.hpp>
 
 #include "authenticator.hpp"
 
+#include "authentication/cram_md5/auxprop.hpp"
+
 #include "messages/messages.hpp"
 
 namespace mesos {
@@ -45,17 +48,17 @@ namespace cram_md5 {
 using namespace process;
 using std::string;
 
-class CRAMMD5AuthenticatorProcess
-  : public ProtobufProcess<CRAMMD5AuthenticatorProcess>
+class CRAMMD5AuthenticatorSessionProcess :
+  public ProtobufProcess<CRAMMD5AuthenticatorSessionProcess>
 {
 public:
-  explicit CRAMMD5AuthenticatorProcess(const UPID& _pid)
-    : ProcessBase(ID::generate("crammd5_authenticator")),
+  explicit CRAMMD5AuthenticatorSessionProcess(const UPID& _pid)
+    : ProcessBase(ID::generate("crammd5_authenticator_session")),
       status(READY),
       pid(_pid),
       connection(NULL) {}
 
-  virtual ~CRAMMD5AuthenticatorProcess()
+  virtual ~CRAMMD5AuthenticatorSessionProcess()
   {
     if (connection != NULL) {
       sasl_dispose(&connection);
@@ -69,55 +72,6 @@ public:
 
   Future<Option<string>> authenticate()
   {
-    static Once* initialize = new Once();
-    static bool initialized = false;
-
-    if (!initialize->once()) {
-      LOG(INFO) << "Initializing server SASL";
-
-      int result = sasl_server_init(NULL, "mesos");
-
-      if (result != SASL_OK) {
-        string error = "Failed to initialize SASL: ";
-        error += sasl_errstring(result, NULL, NULL);
-        LOG(ERROR) << error;
-        AuthenticationErrorMessage message;
-        message.set_error(error);
-        send(pid, message);
-        status = ERROR;
-        promise.fail(error);
-        initialize->done();
-        return promise.future();
-      }
-
-      result = sasl_auxprop_add_plugin(
-          InMemoryAuxiliaryPropertyPlugin::name(),
-          &InMemoryAuxiliaryPropertyPlugin::initialize);
-
-      if (result != SASL_OK) {
-        string error =
-          "Failed to add \"in-memory\" auxiliary property plugin: ";
-        error += sasl_errstring(result, NULL, NULL);
-        LOG(ERROR) << error;
-        AuthenticationErrorMessage message;
-        message.set_error(error);
-        send(pid, message);
-        status = ERROR;
-        promise.fail(error);
-        initialize->done();
-        return promise.future();
-      }
-
-      initialized = true;
-
-      initialize->done();
-    }
-
-    if (!initialized) {
-      promise.fail("Failed to initialize SASL");
-      return promise.future();
-    }
-
     if (status != READY) {
       return promise.future();
     }
@@ -206,19 +160,18 @@ public:
     return promise.future();
   }
 
-protected:
   virtual void initialize()
   {
     link(pid); // Don't bother waiting for a lost authenticatee.
 
     // Anticipate start and steps messages from the client.
     install<AuthenticationStartMessage>(
-        &CRAMMD5AuthenticatorProcess::start,
+        &CRAMMD5AuthenticatorSessionProcess::start,
         &AuthenticationStartMessage::mechanism,
         &AuthenticationStartMessage::data);
 
     install<AuthenticationStepMessage>(
-        &CRAMMD5AuthenticatorProcess::step,
+        &CRAMMD5AuthenticatorSessionProcess::step,
         &AuthenticationStepMessage::data);
   }
 
@@ -408,6 +361,76 @@ private:
 };
 
 
+class CRAMMD5AuthenticatorSession
+{
+public:
+  explicit CRAMMD5AuthenticatorSession(const UPID& pid)
+  {
+    process = new CRAMMD5AuthenticatorSessionProcess(pid);
+    spawn(process);
+  }
+
+  virtual ~CRAMMD5AuthenticatorSession()
+  {
+    // TODO(vinod): As a short term fix for the race condition #1 in
+    // MESOS-1866, we inject the 'terminate' event at the end of the
+    // CRAMMD5AuthenticatorSessionProcess queue instead of at the front.
+    // The long term fix for this is https://reviews.apache.org/r/25945/.
+    terminate(process, false);
+    wait(process);
+    delete process;
+  }
+
+  virtual Future<Option<string>> authenticate()
+  {
+    return dispatch(
+        process, &CRAMMD5AuthenticatorSessionProcess::authenticate);
+  }
+
+private:
+  CRAMMD5AuthenticatorSessionProcess* process;
+};
+
+
+class CRAMMD5AuthenticatorProcess :
+  public Process<CRAMMD5AuthenticatorProcess>
+{
+public:
+  CRAMMD5AuthenticatorProcess() :
+    ProcessBase(ID::generate("crammd5_authenticator")) {}
+
+  virtual ~CRAMMD5AuthenticatorProcess() {}
+
+  Future<Option<string>> authenticate(const UPID& pid)
+  {
+    VLOG(1) << "Starting authentication session for " << pid;
+
+    if (sessions.contains(pid)) {
+      return Failure("Authentication session already active");
+    }
+
+    Owned<CRAMMD5AuthenticatorSession> session(
+        new CRAMMD5AuthenticatorSession(pid));
+
+    sessions.put(pid, session);
+
+    return session->authenticate()
+      .onAny(defer(self(), &Self::_authenticate, pid));
+  }
+
+  virtual void _authenticate(const UPID& pid)
+  {
+    if (sessions.contains(pid)){
+      VLOG(1) << "Authentication session cleanup for " << pid;
+      sessions.erase(pid);
+  }
+}
+
+private:
+  hashmap <UPID, Owned<CRAMMD5AuthenticatorSession>> sessions;
+};
+
+
 namespace secrets {
 
 // Loads secrets (principal -> secret) into the in-memory auxiliary
@@ -451,31 +474,82 @@ CRAMMD5Authenticator::CRAMMD5Authenticator() : process(NULL) {}
 CRAMMD5Authenticator::~CRAMMD5Authenticator()
 {
   if (process != NULL) {
-    // TODO(vinod): As a short term fix for the race condition #1 in
-    // MESOS-1866, we inject the 'terminate' event at the end of the
-    // CRAMMD5AuthenticatorProcess queue instead of at the front.
-    // The long term fix for this is https://reviews.apache.org/r/25945/.
-    terminate(process, false);
-
+    terminate(process);
     wait(process);
     delete process;
   }
 }
 
 
-void CRAMMD5Authenticator::initialize(const UPID& pid)
+Try<Nothing> CRAMMD5Authenticator::initialize(
+    const Option<Credentials>& credentials)
 {
-  CHECK(process == NULL) << "Authenticator has already been initialized";
-  process = new CRAMMD5AuthenticatorProcess(pid);
+  static Once* initialize = new Once();
+
+  // The 'error' is set atmost once per os process.
+  // To allow subsequent calls to return the possibly set Error
+  // object, we make this a static pointer.
+  static Option<Error>* error = new Option<Error>();
+
+  if (process != NULL) {
+    return Error("Authenticator initialized already");
+  }
+
+  if (credentials.isSome()) {
+    // Load the credentials into the auxiliary memory plugin's storage.
+    // It is necessary for this to be re-entrant as our tests may
+    // re-load credentials.
+    secrets::load(credentials.get());
+  } else {
+    LOG(WARNING) << "No credentials provided, authentication requests will be "
+                 << "refused.";
+  }
+
+  // Initialize SASL and add the auxiliary memory plugin. We must
+  // not do this more than once per os-process.
+  if (!initialize->once()) {
+    LOG(INFO) << "Initializing server SASL";
+
+    int result = sasl_server_init(NULL, "mesos");
+
+    if (result != SASL_OK) {
+      *error = Error(
+          string("Failed to initialize SASL: ") +
+          sasl_errstring(result, NULL, NULL));
+    } else {
+      result = sasl_auxprop_add_plugin(
+          InMemoryAuxiliaryPropertyPlugin::name(),
+          &InMemoryAuxiliaryPropertyPlugin::initialize);
+
+      if (result != SASL_OK) {
+        *error = Error(
+            string("Failed to add in-memory auxiliary property plugin: ") +
+            sasl_errstring(result, NULL, NULL));
+      }
+    }
+
+    initialize->done();
+  }
+
+  if (error->isSome()) {
+    return error->get();
+  }
+
+  process = new CRAMMD5AuthenticatorProcess();
   spawn(process);
+
+  return Nothing();
 }
 
 
-Future<Option<string>> CRAMMD5Authenticator::authenticate(void)
+Future<Option<string>> CRAMMD5Authenticator::authenticate(
+    const UPID& pid)
 {
-  CHECK(process != NULL) << "Authenticator has not been initialized";
+  if (process == NULL) {
+    return Failure("Authenticator not initialized");
+  }
   return dispatch(
-      process, &CRAMMD5AuthenticatorProcess::authenticate);
+      process, &CRAMMD5AuthenticatorProcess::authenticate, pid);
 }
 
 } // namespace cram_md5 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/b5d6dfdb/src/authentication/cram_md5/authenticator.hpp
----------------------------------------------------------------------
diff --git a/src/authentication/cram_md5/authenticator.hpp b/src/authentication/cram_md5/authenticator.hpp
index 3463d7c..e75469a 100644
--- a/src/authentication/cram_md5/authenticator.hpp
+++ b/src/authentication/cram_md5/authenticator.hpp
@@ -36,7 +36,6 @@ namespace cram_md5 {
 // Forward declaration.
 class CRAMMD5AuthenticatorProcess;
 
-
 class CRAMMD5Authenticator : public Authenticator
 {
 public:
@@ -47,33 +46,15 @@ public:
 
   virtual ~CRAMMD5Authenticator();
 
-  virtual void initialize(const process::UPID& clientPid);
+  virtual Try<Nothing> initialize(const Option<Credentials>& credentials);
 
-  // Returns the principal of the Authenticatee if successfully
-  // authenticated otherwise None or an error. Note that we
-  // distinguish authentication failure (None) from a failed future
-  // in the event the future failed due to a transient error and
-  // authentication can (should) be retried. Discarding the future
-  // will cause the future to fail if it hasn't already completed
-  // since we have already started the authentication procedure and
-  // can't reliably cancel.
-  virtual process::Future<Option<std::string>> authenticate();
+  virtual process::Future<Option<std::string>> authenticate(
+      const process::UPID& pid);
 
 private:
   CRAMMD5AuthenticatorProcess* process;
 };
 
-
-namespace secrets {
-
-// Loads secrets (principal -> secret) into the in-memory auxiliary
-// property plugin that is used by the authenticators.
-void load(const std::map<std::string, std::string>& secrets);
-
-void load(const Credentials& credentials);
-
-} // namespace secrets {
-
 } // namespace cram_md5 {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/b5d6dfdb/src/authentication/cram_md5/auxprop.hpp
----------------------------------------------------------------------
diff --git a/src/authentication/cram_md5/auxprop.hpp b/src/authentication/cram_md5/auxprop.hpp
index 6aa3ce6..1a054ba 100644
--- a/src/authentication/cram_md5/auxprop.hpp
+++ b/src/authentication/cram_md5/auxprop.hpp
@@ -55,7 +55,7 @@ public:
     properties = _properties;
   }
 
-  static Option<std::list<std::string> > lookup(
+  static Option<std::list<std::string>> lookup(
       const std::string& user,
       const std::string& name)
   {

http://git-wip-us.apache.org/repos/asf/mesos/blob/b5d6dfdb/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 1d30c87..618db68 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -284,6 +284,7 @@ Master::Master(
     contender(_contender),
     detector(_detector),
     authorizer(_authorizer),
+    authenticator(None()),
     metrics(new Metrics(*this)),
     electedTime(None())
 {
@@ -412,6 +413,20 @@ void Master::initialize()
     LOG(INFO) << "Master allowing unauthenticated slaves to register";
   }
 
+  // Load credentials.
+  if (flags.credentials.isSome()) {
+    Result<Credentials> _credentials =
+      credentials::read(flags.credentials.get());
+    if (_credentials.isError()) {
+      EXIT(1) << _credentials.error() << " (see --credentials flag)";
+    } else if (_credentials.isNone()) {
+      EXIT(1) << "Credentials file must contain at least one credential"
+              << " (see --credentials flag)";
+    }
+    // Store credentials in master to use them in routes.
+    credentials = _credentials.get();
+  }
+
   // Extract authenticator names and validate them.
   authenticatorNames = strings::split(flags.authenticators, ",");
   if (authenticatorNames.empty()) {
@@ -429,23 +444,43 @@ void Master::initialize()
             << "(see --modules)";
   }
 
-  // Load credentials.
-  if (flags.credentials.isSome()) {
-    Result<Credentials> _credentials =
-      credentials::read(flags.credentials.get());
-    if (_credentials.isError()) {
-      EXIT(1) << _credentials.error() << " (see --credentials flag)";
-    } else if (_credentials.isNone()) {
-      EXIT(1) << "Credentials file must contain at least one credential"
-              << " (see --credentials flag)";
+  // TODO(tillt): Allow multiple authenticators to be loaded and enable
+  // the authenticatee to select the appropriate one. See MESOS-1939.
+  if (authenticatorNames[0] == DEFAULT_AUTHENTICATOR) {
+    LOG(INFO) << "Using default '" << DEFAULT_AUTHENTICATOR
+              << "' authenticator";
+    authenticator = new cram_md5::CRAMMD5Authenticator();
+  } else {
+    Try<Authenticator*> module =
+      modules::ModuleManager::create<Authenticator>(authenticatorNames[0]);
+    if (module.isError()) {
+      EXIT(1) << "Could not create authenticator module '"
+              << authenticatorNames[0] << "': " << module.error();
     }
-    // Store credentials in master to use them in routes.
-    credentials = _credentials.get();
+    LOG(INFO) << "Using '" << authenticatorNames[0] << "' authenticator";
+    authenticator = module.get();
+  }
 
-    // Give Authenticator access to credentials.
-    // TODO(tillt): Move this into a mechanism (module) specific
-    // Authenticator factory. See MESOS-2050.
-    cram_md5::secrets::load(credentials.get());
+  // Give Authenticator access to credentials when needed.
+  CHECK_SOME(authenticator);
+  Try<Nothing> initialize = authenticator.get()->initialize(credentials);
+  if (initialize.isError()) {
+    const string error =
+      "Failed to initialize authenticator '" + authenticatorNames[0] +
+      "': " + initialize.error();
+    if (flags.authenticate_frameworks || flags.authenticate_slaves) {
+      EXIT(1) << "Failed to start master with authentication enabled: "
+              << error;
+    } else {
+      // A failure to initialize the authenticator does lead to
+      // unusable authentication but still allows non authenticating
+      // frameworks and slaves to connect.
+      LOG(WARNING) << "Only non-authenticating frameworks and slaves are "
+                   << "allowed to connect. "
+                   << "Authentication is disabled: " << error;
+      delete authenticator.get();
+      authenticator = None();
+    }
   }
 
   if (authorizer.isSome()) {
@@ -847,6 +882,10 @@ void Master::finalize()
   terminate(whitelistWatcher);
   wait(whitelistWatcher);
   delete whitelistWatcher;
+
+  if (authenticator.isSome()) {
+    delete authenticator.get();
+  }
 }
 
 
@@ -3772,6 +3811,8 @@ void Master::offer(const FrameworkID& frameworkId,
 // authenticate with master they would be stepping on each other's
 // toes. Currently it is tricky to detect this case because the
 // 'authenticate' message doesn't contain the 'FrameworkID'.
+// 'from' is the authenticatee process with which to communicate.
+// 'pid' is the framework/slave process being authenticated.
 void Master::authenticate(const UPID& from, const UPID& pid)
 {
   ++metrics->messages_authenticate;
@@ -3803,15 +3844,36 @@ void Master::authenticate(const UPID& from, const UPID& pid)
 
   authenticated.erase(pid);
 
+  if (authenticator.isNone()) {
+    // The default authenticator is CRAM-MD5 rather than none.
+    // Since the default parameters specify CRAM-MD5 authenticator, no
+    // required authentication, and no credentials, we must support
+    // this for starting successfully.
+    // In this case, we must allow non-authenticating frameworks /
+    // slaves to register without authentication, but we will return
+    // an AuthenticationError if they actually try to authenticate.
+
+    // TODO(tillt): We need to make sure this does not cause retries.
+    // See MESOS-2379.
+    LOG(ERROR) << "Received authentication request from " << pid
+               << " but authenticator is not loaded";
+
+    AuthenticationErrorMessage message;
+    message.set_error("No authenticator loaded");
+    send(from, message);
+
+    return;
+  }
+
   if (authenticating.contains(pid)) {
     LOG(INFO) << "Queuing up authentication request from " << pid
               << " because authentication is still in progress";
 
-    // Try to cancel the in progress authentication by deleting
-    // the authenticator.
-    authenticators.erase(pid);
+    // Try to cancel the in progress authentication by discarding the
+    // future.
+    authenticating[pid].discard();
 
-    // Retry after the current authenticator finishes.
+    // Retry after the current authenticator session finishes.
     authenticating[pid]
       .onAny(defer(self(), &Self::authenticate, from, pid));
 
@@ -3820,40 +3882,19 @@ void Master::authenticate(const UPID& from, const UPID& pid)
 
   LOG(INFO) << "Authenticating " << pid;
 
-  // Create and initialize the authenticator.
-  Authenticator* authenticator;
-  // TODO(tillt): Allow multiple authenticators to be loaded and enable
-  // the authenticatee to select the appropriate one. See MESOS-1939.
-  if (authenticatorNames[0] == DEFAULT_AUTHENTICATOR) {
-    LOG(INFO) << "Using default CRAM-MD5 authenticator";
-    authenticator = new cram_md5::CRAMMD5Authenticator();
-  } else {
-    Try<Authenticator*> module =
-      modules::ModuleManager::create<Authenticator>(authenticatorNames[0]);
-    if (module.isError()) {
-      EXIT(1) << "Could not create authenticator module '"
-              << authenticatorNames[0] << "': " << module.error();
-    }
-    LOG(INFO) << "Using '" << authenticatorNames[0] << "' authenticator";
-    authenticator = module.get();
-  }
-  Owned<Authenticator> authenticator_ = Owned<Authenticator>(authenticator);
+  // Start authentication.
+  const Future<Option<string>> future = authenticator.get()->authenticate(from);
 
-  authenticator_->initialize(from);
+  // Save our state.
+  authenticating[pid] = future;
 
-  // Start authentication.
-  const Future<Option<string>>& future = authenticator_->authenticate()
-     .onAny(defer(self(), &Self::_authenticate, pid, lambda::_1));
+  future.onAny(defer(self(), &Self::_authenticate, pid, lambda::_1));
 
-  // Don't wait for authentication to happen for ever.
+  // Don't wait for authentication to complete forever.
   delay(Seconds(5),
         self(),
         &Self::authenticationTimeout,
         future);
-
-  // Save our state.
-  authenticating[pid] = future;
-  authenticators.put(pid, authenticator_);
 }
 
 
@@ -3875,7 +3916,6 @@ void Master::_authenticate(
     authenticated.put(pid, future.get().get());
   }
 
-  authenticators.erase(pid);
   authenticating.erase(pid);
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/b5d6dfdb/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 744886e..05be911 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -294,11 +294,10 @@ protected:
       Slave* slave,
       const std::vector<Task>& tasks);
 
-  // 'promise' is used to signal finish of authentication.
-  // 'future' is the future returned by the authenticator.
+  // 'authenticate' is the future returned by the authenticator.
   void _authenticate(
       const process::UPID& pid,
-      const process::Future<Option<std::string>>& future);
+      const process::Future<Option<std::string>>& authenticate);
 
   void authenticationTimeout(process::Future<Option<std::string>> future);
 
@@ -650,14 +649,14 @@ private:
   // Authenticator names as supplied via flags.
   std::vector<std::string> authenticatorNames;
 
+  Option<Authenticator*> authenticator;
+
   // Frameworks/slaves that are currently in the process of authentication.
   // 'authenticating' future is completed when authenticator
   // completes authentication.
   // The future is removed from the map when master completes authentication.
   hashmap<process::UPID, process::Future<Option<std::string>>> authenticating;
 
-  hashmap<process::UPID, process::Owned<Authenticator>> authenticators;
-
   // Principals of authenticated frameworks/slaves keyed by PID.
   hashmap<process::UPID, std::string> authenticated;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/b5d6dfdb/src/tests/cram_md5_authentication_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cram_md5_authentication_tests.cpp b/src/tests/cram_md5_authentication_tests.cpp
index 92a89c5..9923023 100644
--- a/src/tests/cram_md5_authentication_tests.cpp
+++ b/src/tests/cram_md5_authentication_tests.cpp
@@ -92,8 +92,6 @@ TYPED_TEST(CRAMMD5Authentication, success)
   credential2->set_principal(credential1.principal());
   credential2->set_secret(credential1.secret());
 
-  secrets::load(credentials);
-
   Future<Message> message =
     FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
 
@@ -108,15 +106,17 @@ TYPED_TEST(CRAMMD5Authentication, success)
   Try<Authenticator*> authenticator = TypeParam::TypeAuthenticator::create();
   CHECK_SOME(authenticator);
 
-  authenticator.get()->initialize(message.get().from);
+  EXPECT_SOME(authenticator.get()->initialize(credentials));
 
-  Future<Option<string>> principal = authenticator.get()->authenticate();
+  Future<Option<string>> principal =
+    authenticator.get()->authenticate(message.get().from);
 
   AWAIT_EQ(true, client);
   AWAIT_READY(principal);
   EXPECT_SOME_EQ("benh", principal.get());
 
   terminate(pid);
+
   delete authenticator.get();
   delete authenticatee.get();
 }
@@ -137,8 +137,6 @@ TYPED_TEST(CRAMMD5Authentication, failed1)
   credential2->set_principal(credential1.principal());
   credential2->set_secret("secret2");
 
-  secrets::load(credentials);
-
   Future<Message> message =
     FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
 
@@ -153,15 +151,17 @@ TYPED_TEST(CRAMMD5Authentication, failed1)
   Try<Authenticator*> authenticator = TypeParam::TypeAuthenticator::create();
   CHECK_SOME(authenticator);
 
-  authenticator.get()->initialize(message.get().from);
+  EXPECT_SOME(authenticator.get()->initialize(credentials));
 
-  Future<Option<string>> server = authenticator.get()->authenticate();
+  Future<Option<string>> server =
+    authenticator.get()->authenticate(message.get().from);
 
   AWAIT_EQ(false, client);
   AWAIT_READY(server);
   EXPECT_NONE(server.get());
 
   terminate(pid);
+
   delete authenticator.get();
   delete authenticatee.get();
 }
@@ -182,8 +182,6 @@ TYPED_TEST(CRAMMD5Authentication, failed2)
   credential2->set_principal("vinod");
   credential2->set_secret(credential1.secret());
 
-  secrets::load(credentials);
-
   Future<Message> message =
     FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
 
@@ -198,23 +196,25 @@ TYPED_TEST(CRAMMD5Authentication, failed2)
   Try<Authenticator*> authenticator = TypeParam::TypeAuthenticator::create();
   CHECK_SOME(authenticator);
 
-  authenticator.get()->initialize(message.get().from);
+  EXPECT_SOME(authenticator.get()->initialize(credentials));
 
-  Future<Option<string>> server = authenticator.get()->authenticate();
+  Future<Option<string>> server =
+    authenticator.get()->authenticate(message.get().from);
 
   AWAIT_EQ(false, client);
   AWAIT_READY(server);
   EXPECT_NONE(server.get());
 
   terminate(pid);
+
   delete authenticator.get();
   delete authenticatee.get();
 }
 
 
 // This test verifies that the pending future returned by
-// 'Authenticator::authenticate()' is properly failed when the Authenticator is
-// destructed in the middle of authentication.
+// 'Authenticator::authenticate()' is properly failed when the
+// Authenticator Session is destroyed in the middle of authentication.
 TYPED_TEST(CRAMMD5Authentication, AuthenticatorDestructionRace)
 {
   // Launch a dummy process (somebody to send the AuthenticateMessage).
@@ -229,8 +229,6 @@ TYPED_TEST(CRAMMD5Authentication, AuthenticatorDestructionRace)
   credential2->set_principal(credential1.principal());
   credential2->set_secret(credential1.secret());
 
-  secrets::load(credentials);
-
   Future<Message> message =
     FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
 
@@ -245,20 +243,20 @@ TYPED_TEST(CRAMMD5Authentication, AuthenticatorDestructionRace)
   Try<Authenticator*> authenticator = TypeParam::TypeAuthenticator::create();
   CHECK_SOME(authenticator);
 
-  authenticator.get()->initialize(message.get().from);
+  EXPECT_SOME(authenticator.get()->initialize(credentials));
 
-  // Drop the AuthenticationStepMessage from authenticator to keep
-  // the authentication from getting completed.
+  // Drop the AuthenticationStepMessage from authenticator session to
+  // keep the authentication from getting completed.
   Future<AuthenticationStepMessage> authenticationStepMessage =
     DROP_PROTOBUF(AuthenticationStepMessage(), _, _);
 
-  Future<Option<string>> principal = authenticator.get()->authenticate();
+  Future<Option<string>> principal =
+    authenticator.get()->authenticate(message.get().from);
 
   AWAIT_READY(authenticationStepMessage);
 
   // At this point 'AuthenticatorProcess::authenticate()' has been
-  // executed and its promise associated with the promise returned
-  // by 'Authenticator::authenticate()'.
+  // executed.
   // Authentication should be pending.
   ASSERT_TRUE(principal.isPending());