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 2014/11/04 00:16:00 UTC

[4/5] git commit: Integrated CRAM-MD5 Authenticator module.

Integrated CRAM-MD5 Authenticator module.

Enables selecting a module based authenticator via the new
--authenticators flag for mesos master.

Additionally, all "> >" have been fixed towards ">>" in master.hpp and
master.cpp.

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


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

Branch: refs/heads/master
Commit: 9ca757fc5abf24d9eaa371abcdab2a4ad5449cc3
Parents: 880ea7d
Author: Till Toenshoff <to...@me.com>
Authored: Mon Nov 3 14:55:01 2014 -0800
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Nov 3 14:55:28 2014 -0800

----------------------------------------------------------------------
 src/master/constants.cpp                    |   2 +
 src/master/constants.hpp                    |   3 +
 src/master/flags.hpp                        |   8 ++
 src/master/master.cpp                       |  90 ++++++++++++++------
 src/master/master.hpp                       |  47 ++++++-----
 src/tests/cram_md5_authentication_tests.cpp | 100 +++++++++++++----------
 6 files changed, 155 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9ca757fc/src/master/constants.cpp
----------------------------------------------------------------------
diff --git a/src/master/constants.cpp b/src/master/constants.cpp
index 3ebd246..9ee17e9 100644
--- a/src/master/constants.cpp
+++ b/src/master/constants.cpp
@@ -42,6 +42,8 @@ const Duration WHITELIST_WATCH_INTERVAL = Seconds(5);
 const uint32_t TASK_LIMIT = 100;
 const std::string MASTER_INFO_LABEL = "info";
 const Duration ZOOKEEPER_SESSION_TIMEOUT = Seconds(10);
+const std::string DEFAULT_AUTHENTICATOR = "crammd5";
+
 } // namespace master {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9ca757fc/src/master/constants.hpp
----------------------------------------------------------------------
diff --git a/src/master/constants.hpp b/src/master/constants.hpp
index a8298bc..c386eab 100644
--- a/src/master/constants.hpp
+++ b/src/master/constants.hpp
@@ -105,6 +105,9 @@ extern const std::string MASTER_INFO_LABEL;
 // TODO(vinod): Master detector/contender should use this timeout.
 extern const Duration ZOOKEEPER_SESSION_TIMEOUT;
 
+// Name of the default, CRAM-MD5 authenticator.
+extern const std::string DEFAULT_AUTHENTICATOR;
+
 } // namespace master {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9ca757fc/src/master/flags.hpp
----------------------------------------------------------------------
diff --git a/src/master/flags.hpp b/src/master/flags.hpp
index b60306f..a144db0 100644
--- a/src/master/flags.hpp
+++ b/src/master/flags.hpp
@@ -347,6 +347,13 @@ public:
         "    }\n"
         "  ]\n"
         "}");
+
+    add(&Flags::authenticators,
+        "authenticators",
+        "Authenticator implementation to use when authenticating frameworks\n"
+        "and/or slaves. Use the default '" + DEFAULT_AUTHENTICATOR + "', or\n"
+        "load an alternate authenticator module using --modules.",
+        DEFAULT_AUTHENTICATOR);
   }
 
   bool version;
@@ -377,6 +384,7 @@ public:
   Option<RateLimits> rate_limits;
   Option<Duration> offer_timeout;
   Option<Modules> modules;
+  std::string authenticators;
 
 #ifdef WITH_NETWORK_ISOLATOR
   Option<size_t> max_executors_per_slave;

http://git-wip-us.apache.org/repos/asf/mesos/blob/9ca757fc/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 39e27c5..d914786 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -25,6 +25,8 @@
 #include <list>
 #include <sstream>
 
+#include <mesos/module.hpp>
+
 #include <process/check.hpp>
 #include <process/collect.hpp>
 #include <process/defer.hpp>
@@ -51,6 +53,7 @@
 #include <stout/utils.hpp>
 #include <stout/uuid.hpp>
 
+#include "authentication/authenticator.hpp"
 #include "authentication/cram_md5/authenticator.hpp"
 
 #include "authorizer/authorizer.hpp"
@@ -69,6 +72,9 @@
 #include "master/flags.hpp"
 #include "master/master.hpp"
 
+#include "module/authenticator.hpp"
+#include "module/manager.hpp"
+
 using std::list;
 using std::string;
 using std::vector;
@@ -115,7 +121,7 @@ protected:
   void watch()
   {
     // Get the list of white listed slaves.
-    Option<hashset<string> > whitelist;
+    Option<hashset<string>> whitelist;
     if (path == "*") { // Accept all slaves.
       VLOG(1) << "No whitelist given. Advertising offers for all slaves";
     } else {
@@ -156,7 +162,7 @@ protected:
 private:
   const string path;
   Allocator* allocator;
-  Option<hashset<string> > lastWhitelist;
+  Option<hashset<string>> lastWhitelist;
 };
 
 
@@ -365,6 +371,23 @@ void Master::initialize()
     LOG(INFO) << "Master allowing unauthenticated slaves to register";
   }
 
+  // Extract authenticator names and validate them.
+  authenticatorNames = strings::split(flags.authenticators, ",");
+  if (authenticatorNames.empty()) {
+    EXIT(1) << "No authenticator specified";
+  }
+  if (authenticatorNames.size() > 1) {
+    EXIT(1) << "Multiple authenticators not supported";
+  }
+  if (authenticatorNames[0] != DEFAULT_AUTHENTICATOR &&
+      !modules::ModuleManager::contains<Authenticator>(
+          authenticatorNames[0])) {
+    EXIT(1) << "Authenticator '" << authenticatorNames[0] << "' not found. "
+            << "Check the spelling (compare to '" << DEFAULT_AUTHENTICATOR
+            << "'') or verify that the authenticator was loaded successfully "
+            << "(see --modules)";
+  }
+
   // Load credentials.
   if (flags.credentials.isSome()) {
     const string& path =
@@ -379,13 +402,6 @@ void Master::initialize()
     }
     // Store credentials in master to use them in routes.
     credentials = _credentials.get();
-
-    // Load "registration" credentials into CRAM-MD5 Authenticator.
-    cram_md5::secrets::load(_credentials.get());
-
-  } else if (flags.authenticate_frameworks || flags.authenticate_slaves) {
-    EXIT(1) << "Authentication requires a credentials file"
-            << " (see --credentials flag)";
   }
 
   if (authorizer.isSome()) {
@@ -1195,7 +1211,7 @@ void Master::submitScheduler(const string& name)
 }
 
 
-void Master::contended(const Future<Future<Nothing> >& candidacy)
+void Master::contended(const Future<Future<Nothing>>& candidacy)
 {
   CHECK(!candidacy.isDiscarded());
 
@@ -1227,7 +1243,7 @@ void Master::lostCandidacy(const Future<Nothing>& lost)
 }
 
 
-void Master::detected(const Future<Option<MasterInfo> >& _leader)
+void Master::detected(const Future<Option<MasterInfo>>& _leader)
 {
   CHECK(!_leader.isDiscarded());
 
@@ -1272,7 +1288,7 @@ void Master::detected(const Future<Option<MasterInfo> >& _leader)
 
 
 // Helper to convert authorization result to Future<Option<Error> >.
-static Future<Option<Error> > _authorize(const string& message, bool authorized)
+static Future<Option<Error>> _authorize(const string& message, bool authorized)
 {
   if (authorized) {
     return None();
@@ -1282,7 +1298,7 @@ static Future<Option<Error> > _authorize(const string& message, bool authorized)
 }
 
 
-Future<Option<Error> > Master::validate(
+Future<Option<Error>> Master::validate(
     const FrameworkInfo& frameworkInfo,
     const UPID& from)
 {
@@ -1374,7 +1390,7 @@ void Master::registerFramework(
 void Master::_registerFramework(
     const UPID& from,
     const FrameworkInfo& frameworkInfo,
-    const Future<Option<Error> >& validationError)
+    const Future<Option<Error>>& validationError)
 {
   CHECK_READY(validationError);
   if (validationError.get().isSome()) {
@@ -1514,7 +1530,7 @@ void Master::_reregisterFramework(
     const UPID& from,
     const FrameworkInfo& frameworkInfo,
     bool failover,
-    const Future<Option<Error> >& validationError)
+    const Future<Option<Error>>& validationError)
 {
   CHECK_READY(validationError);
   if (validationError.get().isSome()) {
@@ -2259,7 +2275,7 @@ void Master::launchTasks(
   if (offerIds.empty()) {
     error = Error("No offers specified");
   } else {
-    list<Owned<OfferVisitor> > offerVisitors;
+    list<Owned<OfferVisitor>> offerVisitors;
     offerVisitors.push_back(Owned<OfferVisitor>(new ValidOfferChecker()));
     offerVisitors.push_back(Owned<OfferVisitor>(new FrameworkChecker()));
     offerVisitors.push_back(Owned<OfferVisitor>(new SlaveChecker()));
@@ -2368,7 +2384,7 @@ Option<Error> Master::validateTask(
   // Create task visitors.
   // TODO(vinod): Create the visitors on the stack and make the visit
   // operation const.
-  list<Owned<TaskInfoVisitor> > taskVisitors;
+  list<Owned<TaskInfoVisitor>> taskVisitors;
   taskVisitors.push_back(Owned<TaskInfoVisitor>(new TaskIDChecker()));
   taskVisitors.push_back(Owned<TaskInfoVisitor>(new SlaveIDChecker()));
   taskVisitors.push_back(Owned<TaskInfoVisitor>(new UniqueTaskIDChecker()));
@@ -3854,15 +3870,35 @@ void Master::authenticate(const UPID& from, const UPID& pid)
 
   // Create a promise to capture the entire "authenticating"
   // procedure. We'll set this _after_ we finish _authenticate.
-  Owned<Promise<Nothing> > promise(new Promise<Nothing>());
+  Owned<Promise<Nothing>> promise(new Promise<Nothing>());
+
+  // 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);
 
-  // Create the authenticator.
-  Owned<cram_md5::Authenticator> authenticator(
-    new cram_md5::Authenticator(from));
+  Try<Nothing> initialize = authenticator_->initialize(from, credentials);
+  if (initialize.isError()) {
+    EXIT(1) << "Failed to initialize authenticator: " << initialize.error();
+  }
 
   // Start authentication.
-  const Future<Option<string> >& future = authenticator->authenticate()
-    .onAny(defer(self(), &Self::_authenticate, pid, promise, lambda::_1));
+  const Future<Option<string>>& future = authenticator_->authenticate()
+     .onAny(defer(self(), &Self::_authenticate, pid, promise, lambda::_1));
 
   // Don't wait for authentication to happen for ever.
   delay(Seconds(5),
@@ -3872,14 +3908,14 @@ void Master::authenticate(const UPID& from, const UPID& pid)
 
   // Save our state.
   authenticating[pid] = promise->future();
-  authenticators.put(pid, authenticator);
+  authenticators.put(pid, authenticator_);
 }
 
 
 void Master::_authenticate(
     const UPID& pid,
-    const Owned<Promise<Nothing> >& promise,
-    const Future<Option<string> >& future)
+    const Owned<Promise<Nothing>>& promise,
+    const Future<Option<string>>& future)
 {
   if (!future.isReady() || future.get().isNone()) {
     const string& error = future.isReady()
@@ -3903,7 +3939,7 @@ void Master::_authenticate(
 }
 
 
-void Master::authenticationTimeout(Future<Option<string> > future)
+void Master::authenticationTimeout(Future<Option<string>> future)
 {
   // Note that a 'discard' here is safe even if another
   // authenticator is in progress because this copy of the future

http://git-wip-us.apache.org/repos/asf/mesos/blob/9ca757fc/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 468b6e1..687f178 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -72,10 +72,7 @@ namespace registry {
 class Slaves;
 }
 
-namespace cram_md5 {
 class Authenticator;
-}
-
 class Authorizer;
 
 namespace master {
@@ -208,7 +205,7 @@ public:
 
   // Invoked when there is a newly elected leading master.
   // Made public for testing purposes.
-  void detected(const process::Future<Option<MasterInfo> >& pid);
+  void detected(const process::Future<Option<MasterInfo>>& pid);
 
   // Invoked when the contender has lost the candidacy.
   // Made public for testing purposes.
@@ -281,16 +278,16 @@ protected:
   // 'future' is the future returned by the authenticator.
   void _authenticate(
       const process::UPID& pid,
-      const process::Owned<process::Promise<Nothing> >& promise,
-      const process::Future<Option<std::string> >& future);
+      const process::Owned<process::Promise<Nothing>>& promise,
+      const process::Future<Option<std::string>>& future);
 
-  void authenticationTimeout(process::Future<Option<std::string> > future);
+  void authenticationTimeout(process::Future<Option<std::string>> future);
 
   void fileAttached(const process::Future<Nothing>& result,
                     const std::string& path);
 
   // Invoked when the contender has entered the contest.
-  void contended(const process::Future<process::Future<Nothing> >& candidacy);
+  void contended(const process::Future<process::Future<Nothing>>& candidacy);
 
   // Handles a known re-registering slave by reconciling the master's
   // view of the slave's tasks and executors.
@@ -303,14 +300,14 @@ protected:
   void _registerFramework(
       const process::UPID& from,
       const FrameworkInfo& frameworkInfo,
-      const process::Future<Option<Error> >& validationError);
+      const process::Future<Option<Error>>& validationError);
 
   // 'reregisterFramework()' continuation.
   void _reregisterFramework(
       const process::UPID& from,
       const FrameworkInfo& frameworkInfo,
       bool failover,
-      const process::Future<Option<Error> >& validationError);
+      const process::Future<Option<Error>>& validationError);
 
   // Add a framework.
   void addFramework(Framework* framework);
@@ -501,7 +498,7 @@ private:
 
   // Indicates when recovery is complete. Recovery begins once the
   // master is elected as a leader.
-  Option<process::Future<Nothing> > recovered;
+  Option<process::Future<Nothing>> recovered;
 
   struct Slaves
   {
@@ -560,7 +557,7 @@ private:
     Frameworks() : completed(MAX_COMPLETED_FRAMEWORKS) {}
 
     hashmap<FrameworkID, Framework*> registered;
-    boost::circular_buffer<memory::shared_ptr<Framework> > completed;
+    boost::circular_buffer<memory::shared_ptr<Framework>> completed;
 
     // Principals of frameworks keyed by PID.
     // NOTE: Multiple PIDs can map to the same principal. The
@@ -571,7 +568,7 @@ private:
     // 2) This map includes unauthenticated frameworks (when Master
     //    allows them) if they have principals specified in
     //    FrameworkInfo.
-    hashmap<process::UPID, Option<std::string> > principals;
+    hashmap<process::UPID, Option<std::string>> principals;
   } frameworks;
 
   hashmap<OfferID, Offer*> offers;
@@ -579,13 +576,15 @@ private:
 
   hashmap<std::string, Role*> roles;
 
+  // Authenticator names as supplied via flags.
+  std::vector<std::string> authenticatorNames;
+
   // Frameworks/slaves that are currently in the process of authentication.
   // 'authenticating' future for an authenticatee is ready when it is
   // authenticated.
-  hashmap<process::UPID, process::Future<Nothing> > authenticating;
+  hashmap<process::UPID, process::Future<Nothing>> authenticating;
 
-  hashmap<process::UPID, process::Owned<cram_md5::Authenticator> >
-    authenticators;
+  hashmap<process::UPID, process::Owned<Authenticator>> authenticators;
 
   // Principals of authenticated frameworks/slaves keyed by PID.
   hashmap<process::UPID, std::string> authenticated;
@@ -678,7 +677,7 @@ private:
 
     // Per-framework-principal metrics keyed by the framework
     // principal.
-    hashmap<std::string, process::Owned<Frameworks> > frameworks;
+    hashmap<std::string, process::Owned<Frameworks>> frameworks;
 
     // Messages from schedulers.
     process::metrics::Counter messages_register_framework;
@@ -789,7 +788,7 @@ private:
   // Returns None if the framework is valid.
   // Returns Error if the framework is invalid.
   // Returns Failure if authorization returns 'Failure'.
-  process::Future<Option<Error> > validate(
+  process::Future<Option<Error>> validate(
       const FrameworkInfo& frameworkInfo,
       const process::UPID& from);
 
@@ -812,11 +811,11 @@ private:
   // BoundedRateLimiters keyed by the framework principal.
   // Like Metrics::Frameworks, all frameworks of the same principal
   // are throttled together at a common rate limit.
-  hashmap<std::string, Option<process::Owned<BoundedRateLimiter> > > limiters;
+  hashmap<std::string, Option<process::Owned<BoundedRateLimiter>>> limiters;
 
   // The default limiter is for frameworks not specified in
   // 'flags.rate_limits'.
-  Option<process::Owned<BoundedRateLimiter> > defaultLimiter;
+  Option<process::Owned<BoundedRateLimiter>> defaultLimiter;
 };
 
 
@@ -995,13 +994,13 @@ struct Slave
   bool active;
 
   // Executors running on this slave.
-  hashmap<FrameworkID, hashmap<ExecutorID, ExecutorInfo> > executors;
+  hashmap<FrameworkID, hashmap<ExecutorID, ExecutorInfo>> executors;
 
   // Tasks present on this slave.
   // TODO(bmahler): The task pointer ownership complexity arises from the fact
   // that we own the pointer here, but it's shared with the Framework struct.
   // We should find a way to eliminate this.
-  hashmap<FrameworkID, hashmap<TaskID, Task*> > tasks;
+  hashmap<FrameworkID, hashmap<TaskID, Task*>> tasks;
 
   // Tasks that were asked to kill by frameworks.
   // This is used for reconciliation when the slave re-registers.
@@ -1181,11 +1180,11 @@ struct Framework
   // NOTE: We use a shared pointer for Task because clang doesn't like
   // Boost's implementation of circular_buffer with Task (Boost
   // attempts to do some memset's which are unsafe).
-  boost::circular_buffer<memory::shared_ptr<Task> > completedTasks;
+  boost::circular_buffer<memory::shared_ptr<Task>> completedTasks;
 
   hashset<Offer*> offers; // Active offers for framework.
 
-  hashmap<SlaveID, hashmap<ExecutorID, ExecutorInfo> > executors;
+  hashmap<SlaveID, hashmap<ExecutorID, ExecutorInfo>> executors;
 
   // TODO(bmahler): Summing set and ranges resources across slaves
   // does not yield meaningful totals.

http://git-wip-us.apache.org/repos/asf/mesos/blob/9ca757fc/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 d27c905..74ea2ad 100644
--- a/src/tests/cram_md5_authentication_tests.cpp
+++ b/src/tests/cram_md5_authentication_tests.cpp
@@ -47,19 +47,14 @@ namespace cram_md5 {
 
 TEST(CRAMMD5Authentication, success)
 {
-  // Set up secrets.
-  map<string, string> secrets;
-  secrets["benh"] = "secret";
-  cram_md5::secrets::load(secrets);
-
   // Launch a dummy process (somebody to send the AuthenticateMessage).
   UPID pid = spawn(new ProcessBase(), true);
 
-  Credential credential;
-  credential.set_principal("benh");
-  credential.set_secret("secret");
+  Credential credential1;
+  credential1.set_principal("benh");
+  credential1.set_secret("secret");
 
-  Authenticatee authenticatee(credential, UPID());
+  Authenticatee authenticatee(credential1, UPID());
 
   Future<Message> message =
     FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
@@ -68,9 +63,15 @@ TEST(CRAMMD5Authentication, success)
 
   AWAIT_READY(message);
 
-  Authenticator authenticator(message.get().from);
+  Credentials credentials;
+  Credential* credential2 = credentials.add_credentials();
+  credential2->set_principal(credential1.principal());
+  credential2->set_secret(credential1.secret());
+
+  CRAMMD5Authenticator authenticator;
+  EXPECT_SOME(authenticator.initialize(message.get().from, credentials));
 
-  Future<Option<string> > principal = authenticator.authenticate();
+  Future<Option<string>> principal = authenticator.authenticate();
 
   AWAIT_EQ(true, client);
   AWAIT_READY(principal);
@@ -83,19 +84,14 @@ TEST(CRAMMD5Authentication, success)
 // Bad password should return an authentication failure.
 TEST(CRAMMD5Authentication, failed1)
 {
-  // Set up secrets.
-  map<string, string> secrets;
-  secrets["benh"] = "secret1";
-  cram_md5::secrets::load(secrets);
-
   // Launch a dummy process (somebody to send the AuthenticateMessage).
   UPID pid = spawn(new ProcessBase(), true);
 
-  Credential credential;
-  credential.set_principal("benh");
-  credential.set_secret("secret");
+  Credential credential1;
+  credential1.set_principal("benh");
+  credential1.set_secret("secret");
 
-  Authenticatee authenticatee(credential, UPID());
+  Authenticatee authenticatee(credential1, UPID());
 
   Future<Message> message =
     FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
@@ -104,9 +100,15 @@ TEST(CRAMMD5Authentication, failed1)
 
   AWAIT_READY(message);
 
-  Authenticator authenticator(message.get().from);
+  Credentials credentials;
+  Credential* credential2 = credentials.add_credentials();
+  credential2->set_principal(credential1.principal());
+  credential2->set_secret("secret2");
 
-  Future<Option<string> > server = authenticator.authenticate();
+  CRAMMD5Authenticator authenticator;
+  EXPECT_SOME(authenticator.initialize(message.get().from, credentials));
+
+  Future<Option<string>> server = authenticator.authenticate();
 
   AWAIT_EQ(false, client);
   AWAIT_READY(server);
@@ -119,19 +121,14 @@ TEST(CRAMMD5Authentication, failed1)
 // No user should return an authentication failure.
 TEST(CRAMMD5Authentication, failed2)
 {
-  // Set up secrets.
-  map<string, string> secrets;
-  secrets["vinod"] = "secret";
-  cram_md5::secrets::load(secrets);
-
   // Launch a dummy process (somebody to send the AuthenticateMessage).
   UPID pid = spawn(new ProcessBase(), true);
 
-  Credential credential;
-  credential.set_principal("benh");
-  credential.set_secret("secret");
+  Credential credential1;
+  credential1.set_principal("benh");
+  credential1.set_secret("secret");
 
-  Authenticatee authenticatee(credential, UPID());
+  Authenticatee authenticatee(credential1, UPID());
 
   Future<Message> message =
     FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
@@ -140,9 +137,15 @@ TEST(CRAMMD5Authentication, failed2)
 
   AWAIT_READY(message);
 
-  Authenticator authenticator(message.get().from);
+  Credentials credentials;
+  Credential* credential2 = credentials.add_credentials();
+  credential2->set_principal("vinod");
+  credential2->set_secret(credential1.secret());
+
+  CRAMMD5Authenticator authenticator;
+  EXPECT_SOME(authenticator.initialize(message.get().from, credentials));
 
-  Future<Option<string> > server = authenticator.authenticate();
+  Future<Option<string>> server = authenticator.authenticate();
 
   AWAIT_EQ(false, client);
   AWAIT_READY(server);
@@ -157,19 +160,14 @@ TEST(CRAMMD5Authentication, failed2)
 // destructed in the middle of authentication.
 TEST(CRAMMD5Authentication, AuthenticatorDestructionRace)
 {
-  // Set up secrets.
-  map<string, string> secrets;
-  secrets["benh"] = "secret";
-  cram_md5::secrets::load(secrets);
-
   // Launch a dummy process (somebody to send the AuthenticateMessage).
   UPID pid = spawn(new ProcessBase(), true);
 
-  Credential credential;
-  credential.set_principal("benh");
-  credential.set_secret("secret");
+  Credential credential1;
+  credential1.set_principal("benh");
+  credential1.set_secret("secret");
 
-  Authenticatee authenticatee(credential, UPID());
+  Authenticatee authenticatee(credential1, UPID());
 
   Future<Message> message =
     FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
@@ -178,14 +176,20 @@ TEST(CRAMMD5Authentication, AuthenticatorDestructionRace)
 
   AWAIT_READY(message);
 
-  Authenticator* authenticator = new Authenticator(message.get().from);
+  Credentials credentials;
+  Credential* credential2 = credentials.add_credentials();
+  credential2->set_principal(credential1.principal());
+  credential2->set_secret(credential1.secret());
+
+  CRAMMD5Authenticator* authenticator = new CRAMMD5Authenticator();
+  EXPECT_SOME(authenticator->initialize(message.get().from, credentials));
 
   // Drop the AuthenticationStepMessage from authenticator to keep
   // the authentication from getting completed.
   Future<AuthenticationStepMessage> authenticationStepMessage =
     DROP_PROTOBUF(AuthenticationStepMessage(), _, _);
 
-  Future<Option<string> > principal = authenticator->authenticate();
+  Future<Option<string>> principal = authenticator->authenticate();
 
   AWAIT_READY(authenticationStepMessage);
 
@@ -204,6 +208,14 @@ TEST(CRAMMD5Authentication, AuthenticatorDestructionRace)
   terminate(pid);
 }
 
+
+// Missing credentials should fail the initializing.
+TEST(CRAMMD5Authentication, AuthenticatorCredentialsMissing)
+{
+  CRAMMD5Authenticator authenticator;
+  EXPECT_ERROR(authenticator.initialize(UPID(), None()));
+}
+
 } // namespace cram_md5 {
 } // namespace internal {
 } // namespace mesos {