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:15:57 UTC

[1/5] Reorganized authentication implementation locations.

Repository: mesos
Updated Branches:
  refs/heads/master 64dcf8e6e -> 0756d185a


http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/tests/sasl_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/sasl_tests.cpp b/src/tests/sasl_tests.cpp
deleted file mode 100644
index 30556fa..0000000
--- a/src/tests/sasl_tests.cpp
+++ /dev/null
@@ -1,209 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include <map>
-#include <string>
-
-#include <process/gmock.hpp>
-#include <process/gtest.hpp>
-#include <process/pid.hpp>
-#include <process/process.hpp>
-
-#include <stout/gtest.hpp>
-
-#include "sasl/authenticatee.hpp"
-#include "sasl/authenticator.hpp"
-
-#include "tests/mesos.hpp"
-
-using namespace mesos::internal::tests;
-
-using namespace process;
-
-using std::map;
-using std::string;
-
-using testing::_;
-using testing::Eq;
-
-namespace mesos {
-namespace internal {
-namespace sasl {
-
-TEST(SASL, success)
-{
-  // Set up secrets.
-  map<string, string> secrets;
-  secrets["benh"] = "secret";
-  sasl::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");
-
-  Authenticatee authenticatee(credential, UPID());
-
-  Future<Message> message =
-    FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
-
-  Future<bool> client = authenticatee.authenticate(pid);
-
-  AWAIT_READY(message);
-
-  Authenticator authenticator(message.get().from);
-
-  Future<Option<string> > principal = authenticator.authenticate();
-
-  AWAIT_EQ(true, client);
-  AWAIT_READY(principal);
-  EXPECT_SOME_EQ("benh", principal.get());
-
-  terminate(pid);
-}
-
-
-// Bad password should return an authentication failure.
-TEST(SASL, failed1)
-{
-  // Set up secrets.
-  map<string, string> secrets;
-  secrets["benh"] = "secret1";
-  sasl::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");
-
-  Authenticatee authenticatee(credential, UPID());
-
-  Future<Message> message =
-    FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
-
-  Future<bool> client = authenticatee.authenticate(pid);
-
-  AWAIT_READY(message);
-
-  Authenticator authenticator(message.get().from);
-
-  Future<Option<string> > server = authenticator.authenticate();
-
-  AWAIT_EQ(false, client);
-  AWAIT_READY(server);
-  EXPECT_NONE(server.get());
-
-  terminate(pid);
-}
-
-
-// No user should return an authentication failure.
-TEST(SASL, failed2)
-{
-  // Set up secrets.
-  map<string, string> secrets;
-  secrets["vinod"] = "secret";
-  sasl::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");
-
-  Authenticatee authenticatee(credential, UPID());
-
-  Future<Message> message =
-    FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
-
-  Future<bool> client = authenticatee.authenticate(pid);
-
-  AWAIT_READY(message);
-
-  Authenticator authenticator(message.get().from);
-
-  Future<Option<string> > server = authenticator.authenticate();
-
-  AWAIT_EQ(false, client);
-  AWAIT_READY(server);
-  EXPECT_NONE(server.get());
-
-  terminate(pid);
-}
-
-
-// This test verifies that the pending future returned by
-// 'Authenticator::authenticate()' is properly failed when the Authenticator is
-// destructed in the middle of authentication.
-TEST(SASL, AuthenticatorDestructionRace)
-{
-  // Set up secrets.
-  map<string, string> secrets;
-  secrets["benh"] = "secret";
-  sasl::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");
-
-  Authenticatee authenticatee(credential, UPID());
-
-  Future<Message> message =
-    FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
-
-  Future<bool> client = authenticatee.authenticate(pid);
-
-  AWAIT_READY(message);
-
-  Authenticator* authenticator = new Authenticator(message.get().from);
-
-  // Drop the AuthenticationStepMessage from authenticator to keep
-  // the authentication from getting completed.
-  Future<AuthenticationStepMessage> authenticationStepMessage =
-    DROP_PROTOBUF(AuthenticationStepMessage(), _, _);
-
-  Future<Option<string> > principal = authenticator->authenticate();
-
-  AWAIT_READY(authenticationStepMessage);
-
-  // At this point 'AuthenticatorProcess::authenticate()' has been
-  // executed and its promise associated with the promise returned
-  // by 'Authenticator::authenticate()'.
-  // Authentication should be pending.
-  ASSERT_TRUE(principal.isPending());
-
-  // Now delete the authenticator.
-  delete authenticator;
-
-  // The future should be failed at this point.
-  AWAIT_FAILED(principal);
-
-  terminate(pid);
-}
-
-} // namespace sasl {
-} // namespace internal {
-} // namespace mesos {


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

Posted by me...@apache.org.
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 {


[2/5] git commit: Reorganized authentication implementation locations.

Posted by me...@apache.org.
Reorganized authentication implementation locations.

For paving the way into further authentication mechanism
implementations, the existing is reorganized.

Move src/sasl to src/authentication/cram_md5.
Rename src/tests/sasl_tests.cpp to src/tests/cram_md5_authentication_tests.cpp.
Adapt depending include paths.
Rename specific test implementations from SASL to CRAMMD5Authentication.
Rename sasl namespace to cram_md5 throughout all uses.
Add missing license blob to auxprop.cpp.

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


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

Branch: refs/heads/master
Commit: b51f55500b809b452123c011012227d821e02c04
Parents: 64dcf8e
Author: Till Toenshoff <to...@me.com>
Authored: Mon Nov 3 14:39:42 2014 -0800
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Nov 3 14:39:42 2014 -0800

----------------------------------------------------------------------
 src/Makefile.am                               |  10 +-
 src/authentication/cram_md5/authenticatee.hpp | 412 +++++++++++++++++
 src/authentication/cram_md5/authenticator.hpp | 494 +++++++++++++++++++++
 src/authentication/cram_md5/auxprop.cpp       | 204 +++++++++
 src/authentication/cram_md5/auxprop.hpp       |  96 ++++
 src/master/master.cpp                         |  11 +-
 src/master/master.hpp                         |   5 +-
 src/sasl/authenticatee.hpp                    | 412 -----------------
 src/sasl/authenticator.hpp                    | 494 ---------------------
 src/sasl/auxprop.cpp                          | 186 --------
 src/sasl/auxprop.hpp                          |  96 ----
 src/sched/sched.cpp                           |   6 +-
 src/scheduler/scheduler.cpp                   |   6 +-
 src/slave/slave.cpp                           |   6 +-
 src/slave/slave.hpp                           |   6 +-
 src/tests/cram_md5_authentication_tests.cpp   | 209 +++++++++
 src/tests/sasl_tests.cpp                      | 209 ---------
 17 files changed, 1441 insertions(+), 1421 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index e6a0715..21e1e20 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -249,6 +249,10 @@ nodist_libmesos_no_3rdparty_la_SOURCES =				\
   $(REGISTRY_PROTOS)
 
 libmesos_no_3rdparty_la_SOURCES =					\
+	authentication/cram_md5/authenticatee.hpp			\
+	authentication/cram_md5/authenticator.hpp			\
+	authentication/cram_md5/auxprop.hpp				\
+	authentication/cram_md5/auxprop.cpp				\
 	authorizer/authorizer.cpp					\
 	common/attributes.cpp						\
 	common/date_utils.cpp						\
@@ -276,10 +280,6 @@ libmesos_no_3rdparty_la_SOURCES =					\
 	master/registrar.cpp						\
 	master/repairer.cpp						\
 	module/manager.cpp						\
-	sasl/authenticatee.hpp						\
-	sasl/authenticator.hpp						\
-	sasl/auxprop.hpp						\
-	sasl/auxprop.cpp						\
 	sched/sched.cpp							\
 	scheduler/scheduler.cpp						\
 	slave/constants.cpp						\
@@ -1183,6 +1183,7 @@ mesos_tests_SOURCES =				\
   tests/composing_containerizer_tests.cpp       \
   tests/containerizer.cpp			\
   tests/containerizer_tests.cpp			\
+  tests/cram_md5_authentication_tests.cpp	\
   tests/credentials_tests.cpp			\
   tests/docker_containerizer_tests.cpp          \
   tests/docker_tests.cpp			\
@@ -1217,7 +1218,6 @@ mesos_tests_SOURCES =				\
   tests/repair_tests.cpp			\
   tests/resource_offers_tests.cpp		\
   tests/resources_tests.cpp			\
-  tests/sasl_tests.cpp				\
   tests/scheduler_tests.cpp			\
   tests/script.cpp				\
   tests/shutdown_tests.cpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/authentication/cram_md5/authenticatee.hpp
----------------------------------------------------------------------
diff --git a/src/authentication/cram_md5/authenticatee.hpp b/src/authentication/cram_md5/authenticatee.hpp
new file mode 100644
index 0000000..3088a77
--- /dev/null
+++ b/src/authentication/cram_md5/authenticatee.hpp
@@ -0,0 +1,412 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __AUTHENTICATION_CRAM_MD5_AUTHENTICATEE_HPP__
+#define __AUTHENTICATION_CRAM_MD5_AUTHENTICATEE_HPP__
+
+#include <sasl/sasl.h>
+
+#include <mesos/mesos.hpp>
+
+#include <process/defer.hpp>
+#include <process/future.hpp>
+#include <process/id.hpp>
+#include <process/once.hpp>
+#include <process/process.hpp>
+#include <process/protobuf.hpp>
+
+#include <stout/strings.hpp>
+
+#include "messages/messages.hpp"
+
+namespace mesos {
+namespace internal {
+namespace cram_md5 {
+
+// Forward declaration.
+class AuthenticateeProcess;
+
+
+class Authenticatee
+{
+public:
+  // 'credential' is used to authenticate the 'client'.
+  Authenticatee(const Credential& credential, const process::UPID& client);
+  ~Authenticatee();
+
+  // Returns true if successfully authenticated otherwise false or an
+  // error. Note that we distinguish authentication failure (false)
+  // 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.
+  process::Future<bool> authenticate(const process::UPID& pid);
+
+private:
+  AuthenticateeProcess* process;
+};
+
+
+class AuthenticateeProcess : public ProtobufProcess<AuthenticateeProcess>
+{
+public:
+  AuthenticateeProcess(const Credential& _credential,
+                       const process::UPID& _client)
+    : ProcessBase(process::ID::generate("authenticatee")),
+      credential(_credential),
+      client(_client),
+      status(READY),
+      connection(NULL)
+  {
+    const char* data = credential.secret().data();
+    size_t length = credential.secret().length();
+
+    // Need to allocate the secret via 'malloc' because SASL is
+    // expecting the data appended to the end of the struct. *sigh*
+    secret = (sasl_secret_t*) malloc(sizeof(sasl_secret_t) + length);
+
+    CHECK(secret != NULL) << "Failed to allocate memory for secret";
+
+    memcpy(secret->data, data, length);
+    secret->len = length;
+  }
+
+  virtual ~AuthenticateeProcess()
+  {
+    if (connection != NULL) {
+      sasl_dispose(&connection);
+    }
+    free(secret);
+  }
+
+  virtual void finalize()
+  {
+    discarded(); // Fail the promise.
+  }
+
+  process::Future<bool> authenticate(const process::UPID& pid)
+  {
+    static process::Once* initialize = new process::Once();
+    static bool initialized = false;
+
+    if (!initialize->once()) {
+      LOG(INFO) << "Initializing client SASL";
+      int result = sasl_client_init(NULL);
+      if (result != SASL_OK) {
+        status = ERROR;
+        std::string error(sasl_errstring(result, NULL, NULL));
+        promise.fail("Failed to initialize SASL: " + 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();
+    }
+
+    LOG(INFO) << "Creating new client SASL connection";
+
+    callbacks[0].id = SASL_CB_GETREALM;
+    callbacks[0].proc = NULL;
+    callbacks[0].context = NULL;
+
+    callbacks[1].id = SASL_CB_USER;
+    callbacks[1].proc = (int(*)()) &user;
+    callbacks[1].context = (void*) credential.principal().c_str();
+
+    // NOTE: Some SASL mechanisms do not allow/enable "proxying",
+    // i.e., authorization. Therefore, some mechanisms send _only_ the
+    // authoriation name rather than both the user (authentication
+    // name) and authorization name. Thus, for now, we assume
+    // authorization is handled out of band. Consider the
+    // SASL_NEED_PROXY flag if we want to reconsider this in the
+    // future.
+    callbacks[2].id = SASL_CB_AUTHNAME;
+    callbacks[2].proc = (int(*)()) &user;
+    callbacks[2].context = (void*) credential.principal().c_str();
+
+    callbacks[3].id = SASL_CB_PASS;
+    callbacks[3].proc = (int(*)()) &pass;
+    callbacks[3].context = (void*) secret;
+
+    callbacks[4].id = SASL_CB_LIST_END;
+    callbacks[4].proc = NULL;
+    callbacks[4].context = NULL;
+
+    int result = sasl_client_new(
+        "mesos",    // Registered name of service.
+        NULL,       // Server's FQDN.
+        NULL, NULL, // IP Address information strings.
+        callbacks,  // Callbacks supported only for this connection.
+        0,          // Security flags (security layers are enabled
+                    // using security properties, separately).
+        &connection);
+
+    if (result != SASL_OK) {
+      status = ERROR;
+      std::string error(sasl_errstring(result, NULL, NULL));
+      promise.fail("Failed to create client SASL connection: " + error);
+      return promise.future();
+    }
+
+    AuthenticateMessage message;
+    message.set_pid(client);
+    send(pid, message);
+
+    status = STARTING;
+
+    // Stop authenticating if nobody cares.
+    promise.future().onDiscard(defer(self(), &Self::discarded));
+
+    return promise.future();
+  }
+
+protected:
+  virtual void initialize()
+  {
+    // Anticipate mechanisms and steps from the server.
+    install<AuthenticationMechanismsMessage>(
+        &AuthenticateeProcess::mechanisms,
+        &AuthenticationMechanismsMessage::mechanisms);
+
+    install<AuthenticationStepMessage>(
+        &AuthenticateeProcess::step,
+        &AuthenticationStepMessage::data);
+
+    install<AuthenticationCompletedMessage>(
+        &AuthenticateeProcess::completed);
+
+    install<AuthenticationFailedMessage>(
+        &AuthenticateeProcess::failed);
+
+    install<AuthenticationErrorMessage>(
+        &AuthenticateeProcess::error,
+        &AuthenticationErrorMessage::error);
+  }
+
+  void mechanisms(const std::vector<std::string>& mechanisms)
+  {
+    if (status != STARTING) {
+      status = ERROR;
+      promise.fail("Unexpected authentication 'mechanisms' received");
+      return;
+    }
+
+    // TODO(benh): Store 'from' in order to ensure we only communicate
+    // with the same Authenticator.
+
+    LOG(INFO) << "Received SASL authentication mechanisms: "
+              << strings::join(",", mechanisms);
+
+    sasl_interact_t* interact = NULL;
+    const char* output = NULL;
+    unsigned length = 0;
+    const char* mechanism = NULL;
+
+    int result = sasl_client_start(
+        connection,
+        strings::join(" ", mechanisms).c_str(),
+        &interact,     // Set if an interaction is needed.
+        &output,       // The output string (to send to server).
+        &length,       // The length of the output string.
+        &mechanism);   // The chosen mechanism.
+
+    CHECK_NE(SASL_INTERACT, result)
+      << "Not expecting an interaction (ID: " << interact->id << ")";
+
+    if (result != SASL_OK && result != SASL_CONTINUE) {
+      std::string error(sasl_errdetail(connection));
+      status = ERROR;
+      promise.fail("Failed to start the SASL client: " + error);
+      return;
+    }
+
+    LOG(INFO) << "Attempting to authenticate with mechanism '"
+              << mechanism << "'";
+
+    AuthenticationStartMessage message;
+    message.set_mechanism(mechanism);
+    message.set_data(output, length);
+
+    reply(message);
+
+    status = STEPPING;
+  }
+
+  void step(const std::string& data)
+  {
+    if (status != STEPPING) {
+      status = ERROR;
+      promise.fail("Unexpected authentication 'step' received");
+      return;
+    }
+
+    LOG(INFO) << "Received SASL authentication step";
+
+    sasl_interact_t* interact = NULL;
+    const char* output = NULL;
+    unsigned length = 0;
+
+    int result = sasl_client_step(
+        connection,
+        data.length() == 0 ? NULL : data.data(),
+        data.length(),
+        &interact,
+        &output,
+        &length);
+
+    CHECK_NE(SASL_INTERACT, result)
+      << "Not expecting an interaction (ID: " << interact->id << ")";
+
+    if (result == SASL_OK || result == SASL_CONTINUE) {
+      // We don't start the client with SASL_SUCCESS_DATA so we may
+      // need to send one more "empty" message to the server.
+      AuthenticationStepMessage message;
+      if (output != NULL && length > 0) {
+        message.set_data(output, length);
+      }
+      reply(message);
+    } else {
+      status = ERROR;
+      std::string error(sasl_errdetail(connection));
+      promise.fail("Failed to perform authentication step: " + error);
+    }
+  }
+
+  void completed()
+  {
+    if (status != STEPPING) {
+      status = ERROR;
+      promise.fail("Unexpected authentication 'completed' received");
+      return;
+    }
+
+    LOG(INFO) << "Authentication success";
+
+    status = COMPLETED;
+    promise.set(true);
+  }
+
+  void failed()
+  {
+    status = FAILED;
+    promise.set(false);
+  }
+
+  void error(const std::string& error)
+  {
+    status = ERROR;
+    promise.fail("Authentication error: " + error);
+  }
+
+  void discarded()
+  {
+    status = DISCARDED;
+    promise.fail("Authentication discarded");
+  }
+
+private:
+  static int user(
+      void* context,
+      int id,
+      const char** result,
+      unsigned* length)
+  {
+    CHECK(SASL_CB_USER == id || SASL_CB_AUTHNAME == id);
+    *result = static_cast<const char*>(context);
+    if (length != NULL) {
+      *length = strlen(*result);
+    }
+    return SASL_OK;
+  }
+
+  static int pass(
+      sasl_conn_t* connection,
+      void* context,
+      int id,
+      sasl_secret_t** secret)
+  {
+    CHECK_EQ(SASL_CB_PASS, id);
+    *secret = static_cast<sasl_secret_t*>(context);
+    return SASL_OK;
+  }
+
+  const Credential credential;
+
+  // PID of the client that needs to be authenticated.
+  const process::UPID client;
+
+  sasl_secret_t* secret;
+
+  sasl_callback_t callbacks[5];
+
+  enum {
+    READY,
+    STARTING,
+    STEPPING,
+    COMPLETED,
+    FAILED,
+    ERROR,
+    DISCARDED
+  } status;
+
+  sasl_conn_t* connection;
+
+  process::Promise<bool> promise;
+};
+
+
+inline Authenticatee::Authenticatee(
+    const Credential& credential,
+    const process::UPID& client)
+{
+  process = new AuthenticateeProcess(credential, client);
+  process::spawn(process);
+}
+
+
+inline Authenticatee::~Authenticatee()
+{
+  process::terminate(process);
+  process::wait(process);
+  delete process;
+}
+
+
+inline process::Future<bool> Authenticatee::authenticate(
+    const process::UPID& pid)
+{
+  return process::dispatch(process, &AuthenticateeProcess::authenticate, pid);
+}
+
+} // namespace cram_md5 {
+} // namespace internal {
+} // namespace mesos {
+
+#endif //__AUTHENTICATION_CRAM_MD5_AUTHENTICATEE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/authentication/cram_md5/authenticator.hpp
----------------------------------------------------------------------
diff --git a/src/authentication/cram_md5/authenticator.hpp b/src/authentication/cram_md5/authenticator.hpp
new file mode 100644
index 0000000..7953418
--- /dev/null
+++ b/src/authentication/cram_md5/authenticator.hpp
@@ -0,0 +1,494 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __AUTHENTICATION_CRAM_MD5_AUTHENTICATOR_HPP__
+#define __AUTHENTICATION_CRAM_MD5_AUTHENTICATOR_HPP__
+
+#include <sasl/sasl.h>
+#include <sasl/saslplug.h>
+
+#include <string>
+#include <vector>
+
+#include <mesos/mesos.hpp>
+
+#include <process/defer.hpp>
+#include <process/future.hpp>
+#include <process/id.hpp>
+#include <process/once.hpp>
+#include <process/process.hpp>
+#include <process/protobuf.hpp>
+
+#include <stout/check.hpp>
+
+#include "authentication/cram_md5/auxprop.hpp"
+
+#include "messages/messages.hpp"
+
+namespace mesos {
+namespace internal {
+namespace cram_md5 {
+
+// Forward declaration.
+class AuthenticatorProcess;
+
+
+class Authenticator
+{
+public:
+  explicit Authenticator(const process::UPID& pid);
+  ~Authenticator();
+
+  // 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.
+  process::Future<Option<std::string> > authenticate();
+
+private:
+  AuthenticatorProcess* process;
+};
+
+
+class AuthenticatorProcess : public ProtobufProcess<AuthenticatorProcess>
+{
+public:
+  explicit AuthenticatorProcess(const process::UPID& _pid)
+    : ProcessBase(process::ID::generate("authenticator")),
+      status(READY),
+      pid(_pid),
+      connection(NULL) {}
+
+  virtual ~AuthenticatorProcess()
+  {
+    if (connection != NULL) {
+      sasl_dispose(&connection);
+    }
+  }
+
+  virtual void finalize()
+  {
+    discarded(); // Fail the promise.
+  }
+
+  process::Future<Option<std::string> > authenticate()
+  {
+    static process::Once* initialize = new process::Once();
+    static bool initialized = false;
+
+    if (!initialize->once()) {
+      LOG(INFO) << "Initializing server SASL";
+
+      int result = sasl_server_init(NULL, "mesos");
+
+      if (result != SASL_OK) {
+        std::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) {
+        std::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();
+    }
+
+    callbacks[0].id = SASL_CB_GETOPT;
+    callbacks[0].proc = (int(*)()) &getopt;
+    callbacks[0].context = NULL;
+
+    callbacks[1].id = SASL_CB_CANON_USER;
+    callbacks[1].proc = (int(*)()) &canonicalize;
+    // Pass in the principal so we can set it in canon_user().
+    callbacks[1].context = &principal;
+
+    callbacks[2].id = SASL_CB_LIST_END;
+    callbacks[2].proc = NULL;
+    callbacks[2].context = NULL;
+
+    LOG(INFO) << "Creating new server SASL connection";
+
+    int result = sasl_server_new(
+        "mesos",    // Registered name of service.
+        NULL,       // Server's FQDN; NULL uses gethostname().
+        NULL,       // The user realm used for password lookups;
+                    // NULL means default to FQDN.
+                    // NOTE: This does not affect Kerberos.
+        NULL, NULL, // IP address information strings.
+        callbacks,  // Callbacks supported only for this connection.
+        0,          // Security flags (security layers are enabled
+                    // using security properties, separately).
+        &connection);
+
+    if (result != SASL_OK) {
+      std::string error = "Failed to create server SASL connection: ";
+      error += sasl_errstring(result, NULL, NULL);
+      LOG(ERROR) << error;
+      AuthenticationErrorMessage message;
+      message.set_error(error);
+      send(pid, message);
+      status = ERROR;
+      promise.fail(error);
+      return promise.future();
+    }
+
+    // Get the list of mechanisms.
+    const char* output = NULL;
+    unsigned length = 0;
+    int count = 0;
+
+    result = sasl_listmech(
+        connection,  // The context for this connection.
+        NULL,        // Not supported.
+        "",          // What to prepend to the output string.
+        ",",         // What to separate mechanisms with.
+        "",          // What to append to the output string.
+        &output,     // The output string.
+        &length,     // The length of the output string.
+        &count);     // The count of the mechanisms in output.
+
+    if (result != SASL_OK) {
+      std::string error = "Failed to get list of mechanisms: ";
+      LOG(WARNING) << error << sasl_errstring(result, NULL, NULL);
+      AuthenticationErrorMessage message;
+      error += sasl_errdetail(connection);
+      message.set_error(error);
+      send(pid, message);
+      status = ERROR;
+      promise.fail(error);
+      return promise.future();
+    }
+
+    std::vector<std::string> mechanisms = strings::tokenize(output, ",");
+
+    // Send authentication mechanisms.
+    AuthenticationMechanismsMessage message;
+    foreach (const std::string& mechanism, mechanisms) {
+      message.add_mechanisms(mechanism);
+    }
+
+    send(pid, message);
+
+    status = STARTING;
+
+    // Stop authenticating if nobody cares.
+    promise.future().onDiscard(defer(self(), &Self::discarded));
+
+    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>(
+        &AuthenticatorProcess::start,
+        &AuthenticationStartMessage::mechanism,
+        &AuthenticationStartMessage::data);
+
+    install<AuthenticationStepMessage>(
+        &AuthenticatorProcess::step,
+        &AuthenticationStepMessage::data);
+  }
+
+  virtual void exited(const process::UPID& _pid)
+  {
+    if (pid == _pid) {
+      status = ERROR;
+      promise.fail("Failed to communicate with authenticatee");
+    }
+  }
+
+  void start(const std::string& mechanism, const std::string& data)
+  {
+    if (status != STARTING) {
+      AuthenticationErrorMessage message;
+      message.set_error("Unexpected authentication 'start' received");
+      send(pid, message);
+      status = ERROR;
+      promise.fail(message.error());
+      return;
+    }
+
+    LOG(INFO) << "Received SASL authentication start";
+
+    // Start the server.
+    const char* output = NULL;
+    unsigned length = 0;
+
+    int result = sasl_server_start(
+        connection,
+        mechanism.c_str(),
+        data.length() == 0 ? NULL : data.data(),
+        data.length(),
+        &output,
+        &length);
+
+    handle(result, output, length);
+  }
+
+  void step(const std::string& data)
+  {
+    if (status != STEPPING) {
+      AuthenticationErrorMessage message;
+      message.set_error("Unexpected authentication 'step' received");
+      send(pid, message);
+      status = ERROR;
+      promise.fail(message.error());
+      return;
+    }
+
+    LOG(INFO) << "Received SASL authentication step";
+
+    const char* output = NULL;
+    unsigned length = 0;
+
+    int result = sasl_server_step(
+        connection,
+        data.length() == 0 ? NULL : data.data(),
+        data.length(),
+        &output,
+        &length);
+
+    handle(result, output, length);
+  }
+
+  void discarded()
+  {
+    status = DISCARDED;
+    promise.fail("Authentication discarded");
+  }
+
+private:
+  static int getopt(
+      void* context,
+      const char* plugin,
+      const char* option,
+      const char** result,
+      unsigned* length)
+  {
+    bool found = false;
+    if (std::string(option) == "auxprop_plugin") {
+      *result = "in-memory-auxprop";
+      found = true;
+    } else if (std::string(option) == "mech_list") {
+      *result = "CRAM-MD5";
+      found = true;
+    } else if (std::string(option) == "pwcheck_method") {
+      *result = "auxprop";
+      found = true;
+    }
+
+    if (found && length != NULL) {
+      *length = strlen(*result);
+    }
+
+    return SASL_OK;
+  }
+
+  // Callback for canonicalizing the username (principal). We use it
+  // to record the principal in Authenticator.
+  static int canonicalize(
+      sasl_conn_t* connection,
+      void* context,
+      const char* input,
+      unsigned inputLength,
+      unsigned flags,
+      const char* userRealm,
+      char* output,
+      unsigned outputMaxLength,
+      unsigned* outputLength)
+  {
+    CHECK_NOTNULL(input);
+    CHECK_NOTNULL(context);
+    CHECK_NOTNULL(output);
+
+    // Save the input.
+    Option<std::string>* principal =
+      static_cast<Option<std::string>*>(context);
+    CHECK(principal->isNone());
+    *principal = std::string(input, inputLength);
+
+    // Tell SASL that the canonical username is the same as the
+    // client-supplied username.
+    memcpy(output, input, inputLength);
+    *outputLength = inputLength;
+
+    return SASL_OK;
+  }
+
+  // Helper for handling result of server start and step.
+  void handle(int result, const char* output, unsigned length)
+  {
+    if (result == SASL_OK) {
+      // Principal must have been set if authentication succeeded.
+      CHECK_SOME(principal);
+
+      LOG(INFO) << "Authentication success";
+      // Note that we're not using SASL_SUCCESS_DATA which means that
+      // we should not have any data to send when we get a SASL_OK.
+      CHECK(output == NULL);
+      send(pid, AuthenticationCompletedMessage());
+      status = COMPLETED;
+      promise.set(principal);
+    } else if (result == SASL_CONTINUE) {
+      LOG(INFO) << "Authentication requires more steps";
+      AuthenticationStepMessage message;
+      message.set_data(CHECK_NOTNULL(output), length);
+      send(pid, message);
+      status = STEPPING;
+    } else if (result == SASL_NOUSER || result == SASL_BADAUTH) {
+      LOG(WARNING) << "Authentication failure: "
+                   << sasl_errstring(result, NULL, NULL);
+      send(pid, AuthenticationFailedMessage());
+      status = FAILED;
+      promise.set(Option<std::string>::none());
+    } else {
+      LOG(ERROR) << "Authentication error: "
+                 << sasl_errstring(result, NULL, NULL);
+      AuthenticationErrorMessage message;
+      std::string error(sasl_errdetail(connection));
+      message.set_error(error);
+      send(pid, message);
+      status = ERROR;
+      promise.fail(message.error());
+    }
+  }
+
+  enum {
+    READY,
+    STARTING,
+    STEPPING,
+    COMPLETED,
+    FAILED,
+    ERROR,
+    DISCARDED
+  } status;
+
+  sasl_callback_t callbacks[3];
+
+  const process::UPID pid;
+
+  sasl_conn_t* connection;
+
+  process::Promise<Option<std::string> > promise;
+
+  Option<std::string> principal;
+};
+
+
+Authenticator::Authenticator(const process::UPID& pid)
+{
+  process = new AuthenticatorProcess(pid);
+  process::spawn(process);
+}
+
+
+Authenticator::~Authenticator()
+{
+  // 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
+  // AuthenticatorProcess queue instead of at the front.
+  // The long term fix for this https://reviews.apache.org/r/25945/.
+  process::terminate(process, false);
+
+  process::wait(process);
+  delete process;
+}
+
+
+process::Future<Option<std::string> > Authenticator::authenticate()
+{
+  return process::dispatch(process, &AuthenticatorProcess::authenticate);
+}
+
+
+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)
+{
+  Multimap<std::string, Property> properties;
+
+  foreachpair (const std::string& principal,
+               const std::string& secret, secrets) {
+    Property property;
+    property.name = SASL_AUX_PASSWORD_PROP;
+    property.values.push_back(secret);
+    properties.put(principal, property);
+  }
+
+  InMemoryAuxiliaryPropertyPlugin::load(properties);
+}
+
+void load(const Credentials& credentials)
+{
+  std::map<std::string, std::string> secrets;
+  foreach(const Credential& credential, credentials.credentials()) {
+    secrets[credential.principal()] = credential.secret();
+  }
+  load(secrets);
+}
+
+} // namespace secrets {
+
+} // namespace cram_md5 {
+} // namespace internal {
+} // namespace mesos {
+
+#endif //__AUTHENTICATION_CRAM_MD5_AUTHENTICATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/authentication/cram_md5/auxprop.cpp
----------------------------------------------------------------------
diff --git a/src/authentication/cram_md5/auxprop.cpp b/src/authentication/cram_md5/auxprop.cpp
new file mode 100644
index 0000000..cf503a2
--- /dev/null
+++ b/src/authentication/cram_md5/auxprop.cpp
@@ -0,0 +1,204 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "authentication/cram_md5/auxprop.hpp"
+
+#include "logging/logging.hpp"
+
+using std::list;
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace cram_md5 {
+
+// Storage for the static members.
+Multimap<string, Property> InMemoryAuxiliaryPropertyPlugin::properties;
+sasl_auxprop_plug_t InMemoryAuxiliaryPropertyPlugin::plugin;
+
+
+int InMemoryAuxiliaryPropertyPlugin::initialize(
+    const sasl_utils_t* utils,
+    int api,
+    int* version,
+    sasl_auxprop_plug_t** plug,
+    const char* name)
+{
+  if (version == NULL || plug == NULL) {
+    return SASL_BADPARAM;
+  }
+
+  // Check if SASL API is older than the one we were compiled against.
+  if (api < SASL_AUXPROP_PLUG_VERSION) {
+    return SASL_BADVERS;
+  }
+
+  *version = SASL_AUXPROP_PLUG_VERSION;
+
+  plugin.features = 0;
+  plugin.spare_int1 = 0;
+  plugin.glob_context = NULL;
+  plugin.auxprop_free = NULL;
+  plugin.auxprop_lookup = &InMemoryAuxiliaryPropertyPlugin::lookup;
+  plugin.name = const_cast<char*>(InMemoryAuxiliaryPropertyPlugin::name());
+  plugin.auxprop_store = NULL;
+
+  *plug = &plugin;
+
+  VLOG(1) << "Initialized in-memory auxiliary property plugin";
+
+  return SASL_OK;
+}
+
+
+#if SASL_AUXPROP_PLUG_VERSION <= 4
+  void InMemoryAuxiliaryPropertyPlugin::lookup(
+#else
+  int InMemoryAuxiliaryPropertyPlugin::lookup(
+#endif
+    void* context,
+    sasl_server_params_t* sparams,
+    unsigned flags,
+    const char* user,
+    unsigned length)
+{
+  // Pull out the utils.
+  const sasl_utils_t* utils = sparams->utils;
+
+  // We determine the properties we should be looking up by doing a
+  // 'prop_get' on the property context. Note that some of the
+  // properties we get might might need to be skipped depending on the
+  // flags (see below).
+  const propval* properties = utils->prop_get(sparams->propctx);
+
+  CHECK(properties != NULL)
+    << "Invalid auxiliary properties requested for lookup";
+
+  // TODO(benh): Consider "parsing" 'user' if it has an '@' separating
+  // the actual user and a realm.
+
+  string realm = sparams->user_realm != NULL
+    ? sparams->user_realm
+    : sparams->serverFQDN;
+
+  VLOG(1)
+    << "Request to lookup properties for "
+    << "user: '" << user << "' "
+    << "realm: '" << realm << "' "
+    << "server FQDN: '" << sparams->serverFQDN << "' "
+#ifdef SASL_AUXPROP_VERIFY_AGAINST_HASH
+    << "SASL_AUXPROP_VERIFY_AGAINST_HASH: "
+    << (flags & SASL_AUXPROP_VERIFY_AGAINST_HASH ? "true ": "false ")
+#endif
+    << "SASL_AUXPROP_OVERRIDE: "
+    << (flags & SASL_AUXPROP_OVERRIDE ? "true ": "false ")
+    << "SASL_AUXPROP_AUTHZID: "
+    << (flags & SASL_AUXPROP_AUTHZID ? "true ": "false ");
+
+  // Now iterate through each property requested.
+  const propval* property = properties;
+  for (; property->name != NULL; property++) {
+    const char* name = property->name;
+
+    // Skip properties that don't apply to this lookup given the flags.
+    if (flags & SASL_AUXPROP_AUTHZID) {
+      if (name[0] == '*') {
+        VLOG(1) << "Skipping auxiliary property '" << name
+                << "' since SASL_AUXPROP_AUTHZID == true";
+        continue;
+      }
+    } else {
+      // Only consider properties that start with '*' if
+      // SASL_AUXPROP_AUTHZID is not set but don't include the '*'
+      // when looking up the property name.
+      if (name[0] != '*') {
+        VLOG(1) << "Skipping auxiliary property '" << name
+                << "' since SASL_AUXPROP_AUTHZID == false "
+                << "but property name starts with '*'";
+        continue;
+      } else {
+        name = name + 1;
+      }
+    }
+
+    // Don't override already set values unless instructed otherwise.
+    if (property->values != NULL && !(flags & SASL_AUXPROP_OVERRIDE)) {
+#ifdef SASL_AUXPROP_VERIFY_AGAINST_HASH
+      // Regardless of SASL_AUXPROP_OVERRIDE we're expected to
+      // override property 'userPassword' when the
+      // SASL_AUXPROP_VERIFY_AGAINST_HASH flag is set, so we erase it
+      // here.
+      if (flags & SASL_AUXPROP_VERIFY_AGAINST_HASH &&
+          string(name) == string(SASL_AUX_PASSWORD_PROP)) {
+        VLOG(1) << "Erasing auxiliary property '" << name
+                << "' even though SASL_AUXPROP_OVERRIDE == true "
+                << "since SASL_AUXPROP_VERIFY_AGAINST_HASH == true";
+        utils->prop_erase(sparams->propctx, property->name);
+      } else {
+        VLOG(1) << "Skipping auxiliary property '" << name
+                << "' since SASL_AUXPROP_OVERRIDE == false "
+                << "and value(s) already set";
+        continue;
+      }
+#else
+      VLOG(1) << "Skipping auxiliary property '" << name
+              << "' since SASL_AUXPROP_OVERRIDE == false "
+              << "and value(s) already set";
+      continue;
+#endif
+    } else if (property->values != NULL) {
+      CHECK(flags & SASL_AUXPROP_OVERRIDE);
+      VLOG(1) << "Erasing auxiliary property '" << name
+              << "' since SASL_AUXPROP_OVERRIDE == true";
+      utils->prop_erase(sparams->propctx, property->name);
+    }
+
+    VLOG(1) << "Looking up auxiliary property '" << property->name << "'";
+
+    Option<list<string> > values = lookup(user, name);
+
+    if (values.isSome()) {
+      if (values.get().empty()) {
+        // Add the 'NULL' value to indicate there were no values.
+        utils->prop_set(sparams->propctx, property->name, NULL, 0);
+      } else {
+        // Add all the values. Note that passing NULL as the property
+        // name for 'prop_set' will append values to the same name as
+        // the previous 'prop_set' calls which is the behavior we want
+        // after adding the first value.
+        bool append = false;
+        foreach (const string& value, values.get()) {
+          sparams->utils->prop_set(
+              sparams->propctx,
+              append ? NULL : property->name,
+              value.c_str(),
+              -1); // Let 'prop_set' use strlen.
+          append = true;
+        }
+      }
+    }
+  }
+
+#if SASL_AUXPROP_PLUG_VERSION > 4
+  return SASL_OK;
+#endif
+}
+
+} // namespace cram_md5 {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/authentication/cram_md5/auxprop.hpp
----------------------------------------------------------------------
diff --git a/src/authentication/cram_md5/auxprop.hpp b/src/authentication/cram_md5/auxprop.hpp
new file mode 100644
index 0000000..b894386
--- /dev/null
+++ b/src/authentication/cram_md5/auxprop.hpp
@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __AUTHENTICATION_CRAM_MD5_AUXPROP_HPP__
+#define __AUTHENTICATION_CRAM_MD5_AUXPROP_HPP__
+
+#include <string>
+
+#include <sasl/sasl.h>
+#include <sasl/saslplug.h>
+
+#include <stout/foreach.hpp>
+#include <stout/multimap.hpp>
+#include <stout/none.hpp>
+#include <stout/option.hpp>
+
+namespace mesos {
+namespace internal {
+namespace cram_md5 {
+
+struct Property
+{
+  std::string name;
+  std::list<std::string> values;
+};
+
+
+class InMemoryAuxiliaryPropertyPlugin
+{
+public:
+  static const char* name() { return "in-memory-auxprop"; }
+
+  static void load(const Multimap<std::string, Property>& _properties)
+  {
+    properties = _properties;
+  }
+
+  static Option<std::list<std::string> > lookup(
+      const std::string& user,
+      const std::string& name)
+  {
+    if (properties.contains(user)) {
+      foreach (const Property& property, properties.get(user)) {
+        if (property.name == name) {
+          return property.values;
+        }
+      }
+    }
+    return None();
+  }
+
+  // SASL plugin initialize entry.
+  static int initialize(
+      const sasl_utils_t* utils,
+      int api,
+      int* version,
+      sasl_auxprop_plug_t** plug,
+      const char* name);
+
+private:
+#if SASL_AUXPROP_PLUG_VERSION <= 4
+  static void lookup(
+#else
+  static int lookup(
+#endif
+      void* context,
+      sasl_server_params_t* sparams,
+      unsigned flags,
+      const char* user,
+      unsigned length);
+
+  static Multimap<std::string, Property> properties;
+
+  static sasl_auxprop_plug_t plugin;
+};
+
+} // namespace cram_md5 {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __AUTHENTICATION_CRAM_MD5_AUXPROP_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 762d2ff..39e27c5 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -51,9 +51,9 @@
 #include <stout/utils.hpp>
 #include <stout/uuid.hpp>
 
-#include "authorizer/authorizer.hpp"
+#include "authentication/cram_md5/authenticator.hpp"
 
-#include "sasl/authenticator.hpp"
+#include "authorizer/authorizer.hpp"
 
 #include "common/build.hpp"
 #include "common/date_utils.hpp"
@@ -380,8 +380,8 @@ void Master::initialize()
     // Store credentials in master to use them in routes.
     credentials = _credentials.get();
 
-    // Load "registration" credentials into SASL based Authenticator.
-    sasl::secrets::load(_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"
@@ -3857,7 +3857,8 @@ void Master::authenticate(const UPID& from, const UPID& pid)
   Owned<Promise<Nothing> > promise(new Promise<Nothing>());
 
   // Create the authenticator.
-  Owned<sasl::Authenticator> authenticator(new sasl::Authenticator(from));
+  Owned<cram_md5::Authenticator> authenticator(
+    new cram_md5::Authenticator(from));
 
   // Start authentication.
   const Future<Option<string> >& future = authenticator->authenticate()

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index b1a2cd0..468b6e1 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -72,7 +72,7 @@ namespace registry {
 class Slaves;
 }
 
-namespace sasl {
+namespace cram_md5 {
 class Authenticator;
 }
 
@@ -584,7 +584,8 @@ private:
   // authenticated.
   hashmap<process::UPID, process::Future<Nothing> > authenticating;
 
-  hashmap<process::UPID, process::Owned<sasl::Authenticator> > authenticators;
+  hashmap<process::UPID, process::Owned<cram_md5::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/b51f5550/src/sasl/authenticatee.hpp
----------------------------------------------------------------------
diff --git a/src/sasl/authenticatee.hpp b/src/sasl/authenticatee.hpp
deleted file mode 100644
index ec2c841..0000000
--- a/src/sasl/authenticatee.hpp
+++ /dev/null
@@ -1,412 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#ifndef __SASL_AUTHENTICATEE_HPP__
-#define __SASL_AUTHENTICATEE_HPP__
-
-#include <sasl/sasl.h>
-
-#include <mesos/mesos.hpp>
-
-#include <process/defer.hpp>
-#include <process/future.hpp>
-#include <process/id.hpp>
-#include <process/once.hpp>
-#include <process/process.hpp>
-#include <process/protobuf.hpp>
-
-#include <stout/strings.hpp>
-
-#include "messages/messages.hpp"
-
-namespace mesos {
-namespace internal {
-namespace sasl {
-
-// Forward declaration.
-class AuthenticateeProcess;
-
-
-class Authenticatee
-{
-public:
-  // 'credential' is used to authenticate the 'client'.
-  Authenticatee(const Credential& credential, const process::UPID& client);
-  ~Authenticatee();
-
-  // Returns true if successfully authenticated otherwise false or an
-  // error. Note that we distinguish authentication failure (false)
-  // 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.
-  process::Future<bool> authenticate(const process::UPID& pid);
-
-private:
-  AuthenticateeProcess* process;
-};
-
-
-class AuthenticateeProcess : public ProtobufProcess<AuthenticateeProcess>
-{
-public:
-  AuthenticateeProcess(const Credential& _credential,
-                       const process::UPID& _client)
-    : ProcessBase(process::ID::generate("authenticatee")),
-      credential(_credential),
-      client(_client),
-      status(READY),
-      connection(NULL)
-  {
-    const char* data = credential.secret().data();
-    size_t length = credential.secret().length();
-
-    // Need to allocate the secret via 'malloc' because SASL is
-    // expecting the data appended to the end of the struct. *sigh*
-    secret = (sasl_secret_t*) malloc(sizeof(sasl_secret_t) + length);
-
-    CHECK(secret != NULL) << "Failed to allocate memory for secret";
-
-    memcpy(secret->data, data, length);
-    secret->len = length;
-  }
-
-  virtual ~AuthenticateeProcess()
-  {
-    if (connection != NULL) {
-      sasl_dispose(&connection);
-    }
-    free(secret);
-  }
-
-  virtual void finalize()
-  {
-    discarded(); // Fail the promise.
-  }
-
-  process::Future<bool> authenticate(const process::UPID& pid)
-  {
-    static process::Once* initialize = new process::Once();
-    static bool initialized = false;
-
-    if (!initialize->once()) {
-      LOG(INFO) << "Initializing client SASL";
-      int result = sasl_client_init(NULL);
-      if (result != SASL_OK) {
-        status = ERROR;
-        std::string error(sasl_errstring(result, NULL, NULL));
-        promise.fail("Failed to initialize SASL: " + 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();
-    }
-
-    LOG(INFO) << "Creating new client SASL connection";
-
-    callbacks[0].id = SASL_CB_GETREALM;
-    callbacks[0].proc = NULL;
-    callbacks[0].context = NULL;
-
-    callbacks[1].id = SASL_CB_USER;
-    callbacks[1].proc = (int(*)()) &user;
-    callbacks[2].context = (void*) credential.principal().c_str();
-
-    // NOTE: Some SASL mechanisms do not allow/enable "proxying",
-    // i.e., authorization. Therefore, some mechanisms send _only_ the
-    // authoriation name rather than both the user (authentication
-    // name) and authorization name. Thus, for now, we assume
-    // authorization is handled out of band. Consider the
-    // SASL_NEED_PROXY flag if we want to reconsider this in the
-    // future.
-    callbacks[2].id = SASL_CB_AUTHNAME;
-    callbacks[2].proc = (int(*)()) &user;
-    callbacks[2].context = (void*) credential.principal().c_str();
-
-    callbacks[3].id = SASL_CB_PASS;
-    callbacks[3].proc = (int(*)()) &pass;
-    callbacks[3].context = (void*) secret;
-
-    callbacks[4].id = SASL_CB_LIST_END;
-    callbacks[4].proc = NULL;
-    callbacks[4].context = NULL;
-
-    int result = sasl_client_new(
-        "mesos",    // Registered name of service.
-        NULL,       // Server's FQDN.
-        NULL, NULL, // IP Address information strings.
-        callbacks,  // Callbacks supported only for this connection.
-        0,          // Security flags (security layers are enabled
-                    // using security properties, separately).
-        &connection);
-
-    if (result != SASL_OK) {
-      status = ERROR;
-      std::string error(sasl_errstring(result, NULL, NULL));
-      promise.fail("Failed to create client SASL connection: " + error);
-      return promise.future();
-    }
-
-    AuthenticateMessage message;
-    message.set_pid(client);
-    send(pid, message);
-
-    status = STARTING;
-
-    // Stop authenticating if nobody cares.
-    promise.future().onDiscard(defer(self(), &Self::discarded));
-
-    return promise.future();
-  }
-
-protected:
-  virtual void initialize()
-  {
-    // Anticipate mechanisms and steps from the server.
-    install<AuthenticationMechanismsMessage>(
-        &AuthenticateeProcess::mechanisms,
-        &AuthenticationMechanismsMessage::mechanisms);
-
-    install<AuthenticationStepMessage>(
-        &AuthenticateeProcess::step,
-        &AuthenticationStepMessage::data);
-
-    install<AuthenticationCompletedMessage>(
-        &AuthenticateeProcess::completed);
-
-    install<AuthenticationFailedMessage>(
-        &AuthenticateeProcess::failed);
-
-    install<AuthenticationErrorMessage>(
-        &AuthenticateeProcess::error,
-        &AuthenticationErrorMessage::error);
-  }
-
-  void mechanisms(const std::vector<std::string>& mechanisms)
-  {
-    if (status != STARTING) {
-      status = ERROR;
-      promise.fail("Unexpected authentication 'mechanisms' received");
-      return;
-    }
-
-    // TODO(benh): Store 'from' in order to ensure we only communicate
-    // with the same Authenticator.
-
-    LOG(INFO) << "Received SASL authentication mechanisms: "
-              << strings::join(",", mechanisms);
-
-    sasl_interact_t* interact = NULL;
-    const char* output = NULL;
-    unsigned length = 0;
-    const char* mechanism = NULL;
-
-    int result = sasl_client_start(
-        connection,
-        strings::join(" ", mechanisms).c_str(),
-        &interact,     // Set if an interaction is needed.
-        &output,       // The output string (to send to server).
-        &length,       // The length of the output string.
-        &mechanism);   // The chosen mechanism.
-
-    CHECK_NE(SASL_INTERACT, result)
-      << "Not expecting an interaction (ID: " << interact->id << ")";
-
-    if (result != SASL_OK && result != SASL_CONTINUE) {
-      std::string error(sasl_errdetail(connection));
-      status = ERROR;
-      promise.fail("Failed to start the SASL client: " + error);
-      return;
-    }
-
-    LOG(INFO) << "Attempting to authenticate with mechanism '"
-              << mechanism << "'";
-
-    AuthenticationStartMessage message;
-    message.set_mechanism(mechanism);
-    message.set_data(output, length);
-
-    reply(message);
-
-    status = STEPPING;
-  }
-
-  void step(const std::string& data)
-  {
-    if (status != STEPPING) {
-      status = ERROR;
-      promise.fail("Unexpected authentication 'step' received");
-      return;
-    }
-
-    LOG(INFO) << "Received SASL authentication step";
-
-    sasl_interact_t* interact = NULL;
-    const char* output = NULL;
-    unsigned length = 0;
-
-    int result = sasl_client_step(
-        connection,
-        data.length() == 0 ? NULL : data.data(),
-        data.length(),
-        &interact,
-        &output,
-        &length);
-
-    CHECK_NE(SASL_INTERACT, result)
-      << "Not expecting an interaction (ID: " << interact->id << ")";
-
-    if (result == SASL_OK || result == SASL_CONTINUE) {
-      // We don't start the client with SASL_SUCCESS_DATA so we may
-      // need to send one more "empty" message to the server.
-      AuthenticationStepMessage message;
-      if (output != NULL && length > 0) {
-        message.set_data(output, length);
-      }
-      reply(message);
-    } else {
-      status = ERROR;
-      std::string error(sasl_errdetail(connection));
-      promise.fail("Failed to perform authentication step: " + error);
-    }
-  }
-
-  void completed()
-  {
-    if (status != STEPPING) {
-      status = ERROR;
-      promise.fail("Unexpected authentication 'completed' received");
-      return;
-    }
-
-    LOG(INFO) << "Authentication success";
-
-    status = COMPLETED;
-    promise.set(true);
-  }
-
-  void failed()
-  {
-    status = FAILED;
-    promise.set(false);
-  }
-
-  void error(const std::string& error)
-  {
-    status = ERROR;
-    promise.fail("Authentication error: " + error);
-  }
-
-  void discarded()
-  {
-    status = DISCARDED;
-    promise.fail("Authentication discarded");
-  }
-
-private:
-  static int user(
-      void* context,
-      int id,
-      const char** result,
-      unsigned* length)
-  {
-    CHECK(SASL_CB_USER == id || SASL_CB_AUTHNAME == id);
-    *result = static_cast<const char*>(context);
-    if (length != NULL) {
-      *length = strlen(*result);
-    }
-    return SASL_OK;
-  }
-
-  static int pass(
-      sasl_conn_t* connection,
-      void* context,
-      int id,
-      sasl_secret_t** secret)
-  {
-    CHECK_EQ(SASL_CB_PASS, id);
-    *secret = static_cast<sasl_secret_t*>(context);
-    return SASL_OK;
-  }
-
-  const Credential credential;
-
-  // PID of the client that needs to be authenticated.
-  const process::UPID client;
-
-  sasl_secret_t* secret;
-
-  sasl_callback_t callbacks[5];
-
-  enum {
-    READY,
-    STARTING,
-    STEPPING,
-    COMPLETED,
-    FAILED,
-    ERROR,
-    DISCARDED
-  } status;
-
-  sasl_conn_t* connection;
-
-  process::Promise<bool> promise;
-};
-
-
-inline Authenticatee::Authenticatee(
-    const Credential& credential,
-    const process::UPID& client)
-{
-  process = new AuthenticateeProcess(credential, client);
-  process::spawn(process);
-}
-
-
-inline Authenticatee::~Authenticatee()
-{
-  process::terminate(process);
-  process::wait(process);
-  delete process;
-}
-
-
-inline process::Future<bool> Authenticatee::authenticate(
-    const process::UPID& pid)
-{
-  return process::dispatch(process, &AuthenticateeProcess::authenticate, pid);
-}
-
-} // namespace sasl {
-} // namespace internal {
-} // namespace mesos {
-
-#endif //__SASL_AUTHENTICATEE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/sasl/authenticator.hpp
----------------------------------------------------------------------
diff --git a/src/sasl/authenticator.hpp b/src/sasl/authenticator.hpp
deleted file mode 100644
index 6f4d3db..0000000
--- a/src/sasl/authenticator.hpp
+++ /dev/null
@@ -1,494 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#ifndef __SASL_AUTHENTICATOR_HPP__
-#define __SASL_AUTHENTICATOR_HPP__
-
-#include <sasl/sasl.h>
-#include <sasl/saslplug.h>
-
-#include <string>
-#include <vector>
-
-#include <mesos/mesos.hpp>
-
-#include <process/defer.hpp>
-#include <process/future.hpp>
-#include <process/id.hpp>
-#include <process/once.hpp>
-#include <process/process.hpp>
-#include <process/protobuf.hpp>
-
-#include <stout/check.hpp>
-
-#include "messages/messages.hpp"
-
-#include "sasl/auxprop.hpp"
-
-namespace mesos {
-namespace internal {
-namespace sasl {
-
-// Forward declaration.
-class AuthenticatorProcess;
-
-
-class Authenticator
-{
-public:
-  explicit Authenticator(const process::UPID& pid);
-  ~Authenticator();
-
-  // 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.
-  process::Future<Option<std::string> > authenticate();
-
-private:
-  AuthenticatorProcess* process;
-};
-
-
-class AuthenticatorProcess : public ProtobufProcess<AuthenticatorProcess>
-{
-public:
-  explicit AuthenticatorProcess(const process::UPID& _pid)
-    : ProcessBase(process::ID::generate("authenticator")),
-      status(READY),
-      pid(_pid),
-      connection(NULL) {}
-
-  virtual ~AuthenticatorProcess()
-  {
-    if (connection != NULL) {
-      sasl_dispose(&connection);
-    }
-  }
-
-  virtual void finalize()
-  {
-    discarded(); // Fail the promise.
-  }
-
-  process::Future<Option<std::string> > authenticate()
-  {
-    static process::Once* initialize = new process::Once();
-    static bool initialized = false;
-
-    if (!initialize->once()) {
-      LOG(INFO) << "Initializing server SASL";
-
-      int result = sasl_server_init(NULL, "mesos");
-
-      if (result != SASL_OK) {
-        std::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) {
-        std::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();
-    }
-
-    callbacks[0].id = SASL_CB_GETOPT;
-    callbacks[0].proc = (int(*)()) &getopt;
-    callbacks[0].context = NULL;
-
-    callbacks[1].id = SASL_CB_CANON_USER;
-    callbacks[1].proc = (int(*)()) &canonicalize;
-    // Pass in the principal so we can set it in canon_user().
-    callbacks[1].context = &principal;
-
-    callbacks[2].id = SASL_CB_LIST_END;
-    callbacks[2].proc = NULL;
-    callbacks[2].context = NULL;
-
-    LOG(INFO) << "Creating new server SASL connection";
-
-    int result = sasl_server_new(
-        "mesos",    // Registered name of service.
-        NULL,       // Server's FQDN; NULL uses gethostname().
-        NULL,       // The user realm used for password lookups;
-                    // NULL means default to FQDN.
-                    // NOTE: This does not affect Kerberos.
-        NULL, NULL, // IP address information strings.
-        callbacks,  // Callbacks supported only for this connection.
-        0,          // Security flags (security layers are enabled
-                    // using security properties, separately).
-        &connection);
-
-    if (result != SASL_OK) {
-      std::string error = "Failed to create server SASL connection: ";
-      error += sasl_errstring(result, NULL, NULL);
-      LOG(ERROR) << error;
-      AuthenticationErrorMessage message;
-      message.set_error(error);
-      send(pid, message);
-      status = ERROR;
-      promise.fail(error);
-      return promise.future();
-    }
-
-    // Get the list of mechanisms.
-    const char* output = NULL;
-    unsigned length = 0;
-    int count = 0;
-
-    result = sasl_listmech(
-        connection,  // The context for this connection.
-        NULL,        // Not supported.
-        "",          // What to prepend to the output string.
-        ",",         // What to separate mechanisms with.
-        "",          // What to append to the output string.
-        &output,     // The output string.
-        &length,     // The length of the output string.
-        &count);     // The count of the mechanisms in output.
-
-    if (result != SASL_OK) {
-      std::string error = "Failed to get list of mechanisms: ";
-      LOG(WARNING) << error << sasl_errstring(result, NULL, NULL);
-      AuthenticationErrorMessage message;
-      error += sasl_errdetail(connection);
-      message.set_error(error);
-      send(pid, message);
-      status = ERROR;
-      promise.fail(error);
-      return promise.future();
-    }
-
-    std::vector<std::string> mechanisms = strings::tokenize(output, ",");
-
-    // Send authentication mechanisms.
-    AuthenticationMechanismsMessage message;
-    foreach (const std::string& mechanism, mechanisms) {
-      message.add_mechanisms(mechanism);
-    }
-
-    send(pid, message);
-
-    status = STARTING;
-
-    // Stop authenticating if nobody cares.
-    promise.future().onDiscard(defer(self(), &Self::discarded));
-
-    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>(
-        &AuthenticatorProcess::start,
-        &AuthenticationStartMessage::mechanism,
-        &AuthenticationStartMessage::data);
-
-    install<AuthenticationStepMessage>(
-        &AuthenticatorProcess::step,
-        &AuthenticationStepMessage::data);
-  }
-
-  virtual void exited(const process::UPID& _pid)
-  {
-    if (pid == _pid) {
-      status = ERROR;
-      promise.fail("Failed to communicate with authenticatee");
-    }
-  }
-
-  void start(const std::string& mechanism, const std::string& data)
-  {
-    if (status != STARTING) {
-      AuthenticationErrorMessage message;
-      message.set_error("Unexpected authentication 'start' received");
-      send(pid, message);
-      status = ERROR;
-      promise.fail(message.error());
-      return;
-    }
-
-    LOG(INFO) << "Received SASL authentication start";
-
-    // Start the server.
-    const char* output = NULL;
-    unsigned length = 0;
-
-    int result = sasl_server_start(
-        connection,
-        mechanism.c_str(),
-        data.length() == 0 ? NULL : data.data(),
-        data.length(),
-        &output,
-        &length);
-
-    handle(result, output, length);
-  }
-
-  void step(const std::string& data)
-  {
-    if (status != STEPPING) {
-      AuthenticationErrorMessage message;
-      message.set_error("Unexpected authentication 'step' received");
-      send(pid, message);
-      status = ERROR;
-      promise.fail(message.error());
-      return;
-    }
-
-    LOG(INFO) << "Received SASL authentication step";
-
-    const char* output = NULL;
-    unsigned length = 0;
-
-    int result = sasl_server_step(
-        connection,
-        data.length() == 0 ? NULL : data.data(),
-        data.length(),
-        &output,
-        &length);
-
-    handle(result, output, length);
-  }
-
-  void discarded()
-  {
-    status = DISCARDED;
-    promise.fail("Authentication discarded");
-  }
-
-private:
-  static int getopt(
-      void* context,
-      const char* plugin,
-      const char* option,
-      const char** result,
-      unsigned* length)
-  {
-    bool found = false;
-    if (std::string(option) == "auxprop_plugin") {
-      *result = "in-memory-auxprop";
-      found = true;
-    } else if (std::string(option) == "mech_list") {
-      *result = "CRAM-MD5";
-      found = true;
-    } else if (std::string(option) == "pwcheck_method") {
-      *result = "auxprop";
-      found = true;
-    }
-
-    if (found && length != NULL) {
-      *length = strlen(*result);
-    }
-
-    return SASL_OK;
-  }
-
-  // Callback for canonicalizing the username (principal). We use it
-  // to record the principal in Authenticator.
-  static int canonicalize(
-      sasl_conn_t* connection,
-      void* context,
-      const char* input,
-      unsigned inputLength,
-      unsigned flags,
-      const char* userRealm,
-      char* output,
-      unsigned outputMaxLength,
-      unsigned* outputLength)
-  {
-    CHECK_NOTNULL(input);
-    CHECK_NOTNULL(context);
-    CHECK_NOTNULL(output);
-
-    // Save the input.
-    Option<std::string>* principal =
-      static_cast<Option<std::string>*>(context);
-    CHECK(principal->isNone());
-    *principal = std::string(input, inputLength);
-
-    // Tell SASL that the canonical username is the same as the
-    // client-supplied username.
-    memcpy(output, input, inputLength);
-    *outputLength = inputLength;
-
-    return SASL_OK;
-  }
-
-  // Helper for handling result of server start and step.
-  void handle(int result, const char* output, unsigned length)
-  {
-    if (result == SASL_OK) {
-      // Principal must have been set if authentication succeeded.
-      CHECK_SOME(principal);
-
-      LOG(INFO) << "Authentication success";
-      // Note that we're not using SASL_SUCCESS_DATA which means that
-      // we should not have any data to send when we get a SASL_OK.
-      CHECK(output == NULL);
-      send(pid, AuthenticationCompletedMessage());
-      status = COMPLETED;
-      promise.set(principal);
-    } else if (result == SASL_CONTINUE) {
-      LOG(INFO) << "Authentication requires more steps";
-      AuthenticationStepMessage message;
-      message.set_data(CHECK_NOTNULL(output), length);
-      send(pid, message);
-      status = STEPPING;
-    } else if (result == SASL_NOUSER || result == SASL_BADAUTH) {
-      LOG(WARNING) << "Authentication failure: "
-                   << sasl_errstring(result, NULL, NULL);
-      send(pid, AuthenticationFailedMessage());
-      status = FAILED;
-      promise.set(Option<std::string>::none());
-    } else {
-      LOG(ERROR) << "Authentication error: "
-                 << sasl_errstring(result, NULL, NULL);
-      AuthenticationErrorMessage message;
-      std::string error(sasl_errdetail(connection));
-      message.set_error(error);
-      send(pid, message);
-      status = ERROR;
-      promise.fail(message.error());
-    }
-  }
-
-  enum {
-    READY,
-    STARTING,
-    STEPPING,
-    COMPLETED,
-    FAILED,
-    ERROR,
-    DISCARDED
-  } status;
-
-  sasl_callback_t callbacks[3];
-
-  const process::UPID pid;
-
-  sasl_conn_t* connection;
-
-  process::Promise<Option<std::string> > promise;
-
-  Option<std::string> principal;
-};
-
-
-Authenticator::Authenticator(const process::UPID& pid)
-{
-  process = new AuthenticatorProcess(pid);
-  process::spawn(process);
-}
-
-
-Authenticator::~Authenticator()
-{
-  // 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
-  // AuthenticatorProcess queue instead of at the front.
-  // The long term fix for this https://reviews.apache.org/r/25945/.
-  process::terminate(process, false);
-
-  process::wait(process);
-  delete process;
-}
-
-
-process::Future<Option<std::string> > Authenticator::authenticate()
-{
-  return process::dispatch(process, &AuthenticatorProcess::authenticate);
-}
-
-
-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)
-{
-  Multimap<std::string, Property> properties;
-
-  foreachpair (const std::string& principal,
-               const std::string& secret, secrets) {
-    Property property;
-    property.name = SASL_AUX_PASSWORD_PROP;
-    property.values.push_back(secret);
-    properties.put(principal, property);
-  }
-
-  InMemoryAuxiliaryPropertyPlugin::load(properties);
-}
-
-void load(const Credentials& credentials)
-{
-  std::map<std::string, std::string> secrets;
-  foreach(const Credential& credential, credentials.credentials()) {
-    secrets[credential.principal()] = credential.secret();
-  }
-  load(secrets);
-}
-
-} // namespace secrets {
-
-} // namespace sasl {
-} // namespace internal {
-} // namespace mesos {
-
-#endif //__SASL_AUTHENTICATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/sasl/auxprop.cpp
----------------------------------------------------------------------
diff --git a/src/sasl/auxprop.cpp b/src/sasl/auxprop.cpp
deleted file mode 100644
index 6de1222..0000000
--- a/src/sasl/auxprop.cpp
+++ /dev/null
@@ -1,186 +0,0 @@
-#include "logging/logging.hpp"
-
-#include "sasl/auxprop.hpp"
-
-using std::list;
-using std::string;
-
-namespace mesos {
-namespace internal {
-namespace sasl {
-
-// Storage for the static members.
-Multimap<string, Property> InMemoryAuxiliaryPropertyPlugin::properties;
-sasl_auxprop_plug_t InMemoryAuxiliaryPropertyPlugin::plugin;
-
-
-int InMemoryAuxiliaryPropertyPlugin::initialize(
-    const sasl_utils_t* utils,
-    int api,
-    int* version,
-    sasl_auxprop_plug_t** plug,
-    const char* name)
-{
-  if (version == NULL || plug == NULL) {
-    return SASL_BADPARAM;
-  }
-
-  // Check if SASL API is older than the one we were compiled against.
-  if (api < SASL_AUXPROP_PLUG_VERSION) {
-    return SASL_BADVERS;
-  }
-
-  *version = SASL_AUXPROP_PLUG_VERSION;
-
-  plugin.features = 0;
-  plugin.spare_int1 = 0;
-  plugin.glob_context = NULL;
-  plugin.auxprop_free = NULL;
-  plugin.auxprop_lookup = &InMemoryAuxiliaryPropertyPlugin::lookup;
-  plugin.name = const_cast<char*>(InMemoryAuxiliaryPropertyPlugin::name());
-  plugin.auxprop_store = NULL;
-
-  *plug = &plugin;
-
-  VLOG(1) << "Initialized in-memory auxiliary property plugin";
-
-  return SASL_OK;
-}
-
-
-#if SASL_AUXPROP_PLUG_VERSION <= 4
-  void InMemoryAuxiliaryPropertyPlugin::lookup(
-#else
-  int InMemoryAuxiliaryPropertyPlugin::lookup(
-#endif
-    void* context,
-    sasl_server_params_t* sparams,
-    unsigned flags,
-    const char* user,
-    unsigned length)
-{
-  // Pull out the utils.
-  const sasl_utils_t* utils = sparams->utils;
-
-  // We determine the properties we should be looking up by doing a
-  // 'prop_get' on the property context. Note that some of the
-  // properties we get might might need to be skipped depending on the
-  // flags (see below).
-  const propval* properties = utils->prop_get(sparams->propctx);
-
-  CHECK(properties != NULL)
-    << "Invalid auxiliary properties requested for lookup";
-
-  // TODO(benh): Consider "parsing" 'user' if it has an '@' separating
-  // the actual user and a realm.
-
-  string realm = sparams->user_realm != NULL
-    ? sparams->user_realm
-    : sparams->serverFQDN;
-
-  VLOG(1)
-    << "Request to lookup properties for "
-    << "user: '" << user << "' "
-    << "realm: '" << realm << "' "
-    << "server FQDN: '" << sparams->serverFQDN << "' "
-#ifdef SASL_AUXPROP_VERIFY_AGAINST_HASH
-    << "SASL_AUXPROP_VERIFY_AGAINST_HASH: "
-    << (flags & SASL_AUXPROP_VERIFY_AGAINST_HASH ? "true ": "false ")
-#endif
-    << "SASL_AUXPROP_OVERRIDE: "
-    << (flags & SASL_AUXPROP_OVERRIDE ? "true ": "false ")
-    << "SASL_AUXPROP_AUTHZID: "
-    << (flags & SASL_AUXPROP_AUTHZID ? "true ": "false ");
-
-  // Now iterate through each property requested.
-  const propval* property = properties;
-  for (; property->name != NULL; property++) {
-    const char* name = property->name;
-
-    // Skip properties that don't apply to this lookup given the flags.
-    if (flags & SASL_AUXPROP_AUTHZID) {
-      if (name[0] == '*') {
-        VLOG(1) << "Skipping auxiliary property '" << name
-                << "' since SASL_AUXPROP_AUTHZID == true";
-        continue;
-      }
-    } else {
-      // Only consider properties that start with '*' if
-      // SASL_AUXPROP_AUTHZID is not set but don't include the '*'
-      // when looking up the property name.
-      if (name[0] != '*') {
-        VLOG(1) << "Skipping auxiliary property '" << name
-                << "' since SASL_AUXPROP_AUTHZID == false "
-                << "but property name starts with '*'";
-        continue;
-      } else {
-        name = name + 1;
-      }
-    }
-
-    // Don't override already set values unless instructed otherwise.
-    if (property->values != NULL && !(flags & SASL_AUXPROP_OVERRIDE)) {
-#ifdef SASL_AUXPROP_VERIFY_AGAINST_HASH
-      // Regardless of SASL_AUXPROP_OVERRIDE we're expected to
-      // override property 'userPassword' when the
-      // SASL_AUXPROP_VERIFY_AGAINST_HASH flag is set, so we erase it
-      // here.
-      if (flags & SASL_AUXPROP_VERIFY_AGAINST_HASH &&
-          string(name) == string(SASL_AUX_PASSWORD_PROP)) {
-        VLOG(1) << "Erasing auxiliary property '" << name
-                << "' even though SASL_AUXPROP_OVERRIDE == true "
-                << "since SASL_AUXPROP_VERIFY_AGAINST_HASH == true";
-        utils->prop_erase(sparams->propctx, property->name);
-      } else {
-        VLOG(1) << "Skipping auxiliary property '" << name
-                << "' since SASL_AUXPROP_OVERRIDE == false "
-                << "and value(s) already set";
-        continue;
-      }
-#else
-      VLOG(1) << "Skipping auxiliary property '" << name
-              << "' since SASL_AUXPROP_OVERRIDE == false "
-              << "and value(s) already set";
-      continue;
-#endif
-    } else if (property->values != NULL) {
-      CHECK(flags & SASL_AUXPROP_OVERRIDE);
-      VLOG(1) << "Erasing auxiliary property '" << name
-              << "' since SASL_AUXPROP_OVERRIDE == true";
-      utils->prop_erase(sparams->propctx, property->name);
-    }
-
-    VLOG(1) << "Looking up auxiliary property '" << property->name << "'";
-
-    Option<list<string> > values = lookup(user, name);
-
-    if (values.isSome()) {
-      if (values.get().empty()) {
-        // Add the 'NULL' value to indicate there were no values.
-        utils->prop_set(sparams->propctx, property->name, NULL, 0);
-      } else {
-        // Add all the values. Note that passing NULL as the property
-        // name for 'prop_set' will append values to the same name as
-        // the previous 'prop_set' calls which is the behavior we want
-        // after adding the first value.
-        bool append = false;
-        foreach (const string& value, values.get()) {
-          sparams->utils->prop_set(
-              sparams->propctx,
-              append ? NULL : property->name,
-              value.c_str(),
-              -1); // Let 'prop_set' use strlen.
-          append = true;
-        }
-      }
-    }
-  }
-
-#if SASL_AUXPROP_PLUG_VERSION > 4
-  return SASL_OK;
-#endif
-}
-
-} // namespace sasl {
-} // namespace internal {
-} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/sasl/auxprop.hpp
----------------------------------------------------------------------
diff --git a/src/sasl/auxprop.hpp b/src/sasl/auxprop.hpp
deleted file mode 100644
index 44e201b..0000000
--- a/src/sasl/auxprop.hpp
+++ /dev/null
@@ -1,96 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#ifndef __SASL_AUXPROP_HPP__
-#define __SASL_AUXPROP_HPP__
-
-#include <sasl/sasl.h>
-#include <sasl/saslplug.h>
-
-#include <string>
-
-#include <stout/foreach.hpp>
-#include <stout/multimap.hpp>
-#include <stout/none.hpp>
-#include <stout/option.hpp>
-
-namespace mesos {
-namespace internal {
-namespace sasl {
-
-struct Property
-{
-  std::string name;
-  std::list<std::string> values;
-};
-
-
-class InMemoryAuxiliaryPropertyPlugin
-{
-public:
-  static const char* name() { return "in-memory-auxprop"; }
-
-  static void load(const Multimap<std::string, Property>& _properties)
-  {
-    properties = _properties;
-  }
-
-  static Option<std::list<std::string> > lookup(
-      const std::string& user,
-      const std::string& name)
-  {
-    if (properties.contains(user)) {
-      foreach (const Property& property, properties.get(user)) {
-        if (property.name == name) {
-          return property.values;
-        }
-      }
-    }
-    return None();
-  }
-
-  // SASL plugin initialize entry.
-  static int initialize(
-      const sasl_utils_t* utils,
-      int api,
-      int* version,
-      sasl_auxprop_plug_t** plug,
-      const char* name);
-
-private:
-#if SASL_AUXPROP_PLUG_VERSION <= 4
-  static void lookup(
-#else
-  static int lookup(
-#endif
-      void* context,
-      sasl_server_params_t* sparams,
-      unsigned flags,
-      const char* user,
-      unsigned length);
-
-  static Multimap<std::string, Property> properties;
-
-  static sasl_auxprop_plug_t plugin;
-};
-
-} // namespace sasl {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __SASL_AUXPROP_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/sched/sched.cpp
----------------------------------------------------------------------
diff --git a/src/sched/sched.cpp b/src/sched/sched.cpp
index 0fb8c7b..d84465c 100644
--- a/src/sched/sched.cpp
+++ b/src/sched/sched.cpp
@@ -59,7 +59,7 @@
 #include <stout/utils.hpp>
 #include <stout/uuid.hpp>
 
-#include "sasl/authenticatee.hpp"
+#include "authentication/cram_md5/authenticatee.hpp"
 
 #include "common/lock.hpp"
 #include "common/type_utils.hpp"
@@ -285,7 +285,7 @@ protected:
     CHECK_SOME(credential);
 
     CHECK(authenticatee == NULL);
-    authenticatee = new sasl::Authenticatee(credential.get(), self());
+    authenticatee = new cram_md5::Authenticatee(credential.get(), self());
 
     // NOTE: We do not pass 'Owned<Authenticatee>' here because doing
     // so could make 'AuthenticateeProcess' responsible for deleting
@@ -1024,7 +1024,7 @@ private:
 
   const Option<Credential> credential;
 
-  sasl::Authenticatee* authenticatee;
+  cram_md5::Authenticatee* authenticatee;
 
   // Indicates if an authentication attempt is in progress.
   Option<Future<bool> > authenticating;

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/scheduler/scheduler.cpp
----------------------------------------------------------------------
diff --git a/src/scheduler/scheduler.cpp b/src/scheduler/scheduler.cpp
index fb88a3e..c74187c 100644
--- a/src/scheduler/scheduler.cpp
+++ b/src/scheduler/scheduler.cpp
@@ -53,7 +53,7 @@
 #include <stout/os.hpp>
 #include <stout/uuid.hpp>
 
-#include "sasl/authenticatee.hpp"
+#include "authentication/cram_md5/authenticatee.hpp"
 
 #include "common/type_utils.hpp"
 
@@ -465,7 +465,7 @@ protected:
     CHECK_SOME(credential);
 
     CHECK(authenticatee == NULL);
-    authenticatee = new sasl::Authenticatee(credential.get(), self());
+    authenticatee = new cram_md5::Authenticatee(credential.get(), self());
 
     // NOTE: We do not pass 'Owned<Authenticatee>' here because doing
     // so could make 'AuthenticateeProcess' responsible for deleting
@@ -808,7 +808,7 @@ private:
 
   Option<UPID> master;
 
-  sasl::Authenticatee* authenticatee;
+  cram_md5::Authenticatee* authenticatee;
 
   // Indicates if an authentication attempt is in progress.
   Option<Future<bool> > authenticating;

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 5e9b0e4..b893517 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -62,6 +62,8 @@
 #include "linux/cgroups.hpp"
 #endif // __linux__
 
+#include "authentication/cram_md5/authenticatee.hpp"
+
 #include "common/build.hpp"
 #include "common/protobuf_utils.hpp"
 #include "common/type_utils.hpp"
@@ -71,8 +73,6 @@
 
 #include "logging/logging.hpp"
 
-#include "sasl/authenticatee.hpp"
-
 #include "slave/constants.hpp"
 #include "slave/flags.hpp"
 #include "slave/paths.hpp"
@@ -667,7 +667,7 @@ void Slave::authenticate()
   CHECK_SOME(credential);
 
   CHECK(authenticatee == NULL);
-  authenticatee = new sasl::Authenticatee(credential.get(), self());
+  authenticatee = new cram_md5::Authenticatee(credential.get(), self());
 
   authenticating = authenticatee->authenticate(master.get())
     .onAny(defer(self(), &Self::_authenticate));

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index eb5de73..6c183f8 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -70,9 +70,9 @@ namespace internal {
 
 class MasterDetector; // Forward declaration.
 
-namespace sasl {
+namespace cram_md5 {
 class Authenticatee;
-} // namespace sasl {
+} // namespace cram_md5 {
 
 namespace slave {
 
@@ -481,7 +481,7 @@ private:
 
   Option<Credential> credential;
 
-  sasl::Authenticatee* authenticatee;
+  cram_md5::Authenticatee* authenticatee;
 
   // Indicates if an authentication attempt is in progress.
   Option<Future<bool> > authenticating;

http://git-wip-us.apache.org/repos/asf/mesos/blob/b51f5550/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
new file mode 100644
index 0000000..d27c905
--- /dev/null
+++ b/src/tests/cram_md5_authentication_tests.cpp
@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <map>
+#include <string>
+
+#include <process/gmock.hpp>
+#include <process/gtest.hpp>
+#include <process/pid.hpp>
+#include <process/process.hpp>
+
+#include <stout/gtest.hpp>
+
+#include "authentication/cram_md5/authenticatee.hpp"
+#include "authentication/cram_md5/authenticator.hpp"
+
+#include "tests/mesos.hpp"
+
+using namespace mesos::internal::tests;
+
+using namespace process;
+
+using std::map;
+using std::string;
+
+using testing::_;
+using testing::Eq;
+
+namespace mesos {
+namespace internal {
+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");
+
+  Authenticatee authenticatee(credential, UPID());
+
+  Future<Message> message =
+    FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
+
+  Future<bool> client = authenticatee.authenticate(pid);
+
+  AWAIT_READY(message);
+
+  Authenticator authenticator(message.get().from);
+
+  Future<Option<string> > principal = authenticator.authenticate();
+
+  AWAIT_EQ(true, client);
+  AWAIT_READY(principal);
+  EXPECT_SOME_EQ("benh", principal.get());
+
+  terminate(pid);
+}
+
+
+// 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");
+
+  Authenticatee authenticatee(credential, UPID());
+
+  Future<Message> message =
+    FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
+
+  Future<bool> client = authenticatee.authenticate(pid);
+
+  AWAIT_READY(message);
+
+  Authenticator authenticator(message.get().from);
+
+  Future<Option<string> > server = authenticator.authenticate();
+
+  AWAIT_EQ(false, client);
+  AWAIT_READY(server);
+  EXPECT_NONE(server.get());
+
+  terminate(pid);
+}
+
+
+// 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");
+
+  Authenticatee authenticatee(credential, UPID());
+
+  Future<Message> message =
+    FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
+
+  Future<bool> client = authenticatee.authenticate(pid);
+
+  AWAIT_READY(message);
+
+  Authenticator authenticator(message.get().from);
+
+  Future<Option<string> > server = authenticator.authenticate();
+
+  AWAIT_EQ(false, client);
+  AWAIT_READY(server);
+  EXPECT_NONE(server.get());
+
+  terminate(pid);
+}
+
+
+// This test verifies that the pending future returned by
+// 'Authenticator::authenticate()' is properly failed when the Authenticator is
+// 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");
+
+  Authenticatee authenticatee(credential, UPID());
+
+  Future<Message> message =
+    FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
+
+  Future<bool> client = authenticatee.authenticate(pid);
+
+  AWAIT_READY(message);
+
+  Authenticator* authenticator = new Authenticator(message.get().from);
+
+  // Drop the AuthenticationStepMessage from authenticator to keep
+  // the authentication from getting completed.
+  Future<AuthenticationStepMessage> authenticationStepMessage =
+    DROP_PROTOBUF(AuthenticationStepMessage(), _, _);
+
+  Future<Option<string> > principal = authenticator->authenticate();
+
+  AWAIT_READY(authenticationStepMessage);
+
+  // At this point 'AuthenticatorProcess::authenticate()' has been
+  // executed and its promise associated with the promise returned
+  // by 'Authenticator::authenticate()'.
+  // Authentication should be pending.
+  ASSERT_TRUE(principal.isPending());
+
+  // Now delete the authenticator.
+  delete authenticator;
+
+  // The future should be failed at this point.
+  AWAIT_FAILED(principal);
+
+  terminate(pid);
+}
+
+} // namespace cram_md5 {
+} // namespace internal {
+} // namespace mesos {


[3/5] git commit: Added CRAM-MD5 Test Authenticator module.

Posted by me...@apache.org.
Added CRAM-MD5 Test Authenticator module.

Introducing the CRAM-MD5 SASL test authenticator module based on the
former sasl/authenticator.

The former sasl/authenticator.hpp still remains as part of libmesos but
now is located at authentication/cram_md5.

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


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

Branch: refs/heads/master
Commit: 880ea7da6b5c4c1ac1fcc05f2ad5f5dfc875b94e
Parents: b51f555
Author: Till Toenshoff <to...@me.com>
Authored: Mon Nov 3 14:52:24 2014 -0800
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Nov 3 14:52:50 2014 -0800

----------------------------------------------------------------------
 src/Makefile.am                               |  20 +++-
 src/authentication/authenticator.hpp          |  57 +++++++++++
 src/authentication/cram_md5/authenticator.hpp | 109 +++++++++++++--------
 src/examples/test_authenticator_module.cpp    |  51 ++++++++++
 src/module/authenticator.hpp                  |  64 ++++++++++++
 src/module/manager.cpp                        |   3 +-
 6 files changed, 258 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/880ea7da/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 21e1e20..9ab3b9c 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -248,10 +248,9 @@ nodist_libmesos_no_3rdparty_la_SOURCES =				\
   $(MESSAGES_PROTOS)							\
   $(REGISTRY_PROTOS)
 
+# TODO(tillt): Remove authentication/cram_md5/* which will enable us to
+# lose the immediate cyrus-sasl2 dependency.
 libmesos_no_3rdparty_la_SOURCES =					\
-	authentication/cram_md5/authenticatee.hpp			\
-	authentication/cram_md5/authenticator.hpp			\
-	authentication/cram_md5/auxprop.hpp				\
 	authentication/cram_md5/auxprop.cpp				\
 	authorizer/authorizer.cpp					\
 	common/attributes.cpp						\
@@ -388,6 +387,10 @@ if WITH_NETWORK_ISOLATOR
 endif
 
 libmesos_no_3rdparty_la_SOURCES +=					\
+	authentication/authenticator.hpp				\
+	authentication/cram_md5/authenticatee.hpp			\
+	authentication/cram_md5/authenticator.hpp			\
+	authentication/cram_md5/auxprop.hpp				\
 	authorizer/authorizer.hpp					\
 	common/attributes.hpp						\
 	common/build.hpp						\
@@ -425,6 +428,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	master/registrar.hpp						\
 	master/sorter.hpp						\
 	messages/messages.hpp						\
+	module/authenticator.hpp					\
 	module/isolator.hpp						\
 	module/manager.hpp						\
 	slave/constants.hpp						\
@@ -1175,6 +1179,16 @@ libtestisolator_la_SOURCES = examples/test_isolator_module.cpp
 libtestisolator_la_CPPFLAGS = $(MESOS_CPPFLAGS)
 libtestisolator_la_LDFLAGS = -release $(PACKAGE_VERSION) -shared
 
+# Library containing the test CRAM-MD5 authenticator module.
+# TODO(tillt): Add authenticatee module and rename to libauthentication.
+# TODO(tillt): Add cyrus-sasl2 dependency while removing it from libmesos.
+# TODO(tillt): Enable optional building of this module library.
+# TODO(tillt): Make this module library installable. See MESOS-1940.
+lib_LTLIBRARIES += libtestauthenticator.la
+libtestauthenticator_la_SOURCES = examples/test_authenticator_module.cpp
+libtestauthenticator_la_CPPFLAGS = $(MESOS_CPPFLAGS)
+libtestauthenticator_la_LDFLAGS = -release $(PACKAGE_VERSION) -shared
+
 mesos_tests_SOURCES =				\
   tests/allocator_tests.cpp			\
   tests/attributes_tests.cpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/880ea7da/src/authentication/authenticator.hpp
----------------------------------------------------------------------
diff --git a/src/authentication/authenticator.hpp b/src/authentication/authenticator.hpp
new file mode 100644
index 0000000..2f95db1
--- /dev/null
+++ b/src/authentication/authenticator.hpp
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __AUTHENTICATOR_HPP__
+#define __AUTHENTICATOR_HPP__
+
+#include <mesos/mesos.hpp>
+
+#include <process/future.hpp>
+#include <process/pid.hpp>
+
+#include <stout/nothing.hpp>
+#include <stout/try.hpp>
+
+namespace mesos {
+namespace internal {
+
+class Authenticator
+{
+public:
+  Authenticator() {}
+  virtual ~Authenticator() {}
+
+  virtual Try<Nothing> initialize(
+      const process::UPID& clientPid,
+      const Option<mesos::Credentials>& credentials) = 0;
+
+  // 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(void) = 0;
+};
+
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __AUTHENTICATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/880ea7da/src/authentication/cram_md5/authenticator.hpp
----------------------------------------------------------------------
diff --git a/src/authentication/cram_md5/authenticator.hpp b/src/authentication/cram_md5/authenticator.hpp
index 7953418..601248d 100644
--- a/src/authentication/cram_md5/authenticator.hpp
+++ b/src/authentication/cram_md5/authenticator.hpp
@@ -36,6 +36,8 @@
 
 #include <stout/check.hpp>
 
+#include "authentication/authenticator.hpp"
+
 #include "authentication/cram_md5/auxprop.hpp"
 
 #include "messages/messages.hpp"
@@ -45,14 +47,19 @@ namespace internal {
 namespace cram_md5 {
 
 // Forward declaration.
-class AuthenticatorProcess;
+class CRAMMD5AuthenticatorProcess;
 
 
-class Authenticator
+class CRAMMD5Authenticator : public Authenticator
 {
 public:
-  explicit Authenticator(const process::UPID& pid);
-  ~Authenticator();
+  CRAMMD5Authenticator();
+  virtual ~CRAMMD5Authenticator();
+
+  virtual Try<Nothing> initialize(
+      const process::UPID& clientPid,
+      const Option<Credentials>& credentials);
+
 
   // Returns the principal of the Authenticatee if successfully
   // authenticated otherwise None or an error. Note that we
@@ -62,23 +69,24 @@ 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.
-  process::Future<Option<std::string> > authenticate();
+  virtual process::Future<Option<std::string>> authenticate();
 
 private:
-  AuthenticatorProcess* process;
+  CRAMMD5AuthenticatorProcess* process;
 };
 
 
-class AuthenticatorProcess : public ProtobufProcess<AuthenticatorProcess>
+class CRAMMD5AuthenticatorProcess
+  : public ProtobufProcess<CRAMMD5AuthenticatorProcess>
 {
 public:
-  explicit AuthenticatorProcess(const process::UPID& _pid)
-    : ProcessBase(process::ID::generate("authenticator")),
+  explicit CRAMMD5AuthenticatorProcess(const process::UPID& _pid)
+    : ProcessBase(process::ID::generate("crammd5_authenticator")),
       status(READY),
       pid(_pid),
       connection(NULL) {}
 
-  virtual ~AuthenticatorProcess()
+  virtual ~CRAMMD5AuthenticatorProcess()
   {
     if (connection != NULL) {
       sasl_dispose(&connection);
@@ -90,7 +98,7 @@ public:
     discarded(); // Fail the promise.
   }
 
-  process::Future<Option<std::string> > authenticate()
+  process::Future<Option<std::string>> authenticate()
   {
     static process::Once* initialize = new process::Once();
     static bool initialized = false;
@@ -236,12 +244,12 @@ protected:
 
     // Anticipate start and steps messages from the client.
     install<AuthenticationStartMessage>(
-        &AuthenticatorProcess::start,
+        &CRAMMD5AuthenticatorProcess::start,
         &AuthenticationStartMessage::mechanism,
         &AuthenticationStartMessage::data);
 
     install<AuthenticationStepMessage>(
-        &AuthenticatorProcess::step,
+        &CRAMMD5AuthenticatorProcess::step,
         &AuthenticationStepMessage::data);
   }
 
@@ -341,7 +349,7 @@ private:
   }
 
   // Callback for canonicalizing the username (principal). We use it
-  // to record the principal in Authenticator.
+  // to record the principal in CRAMMD5Authenticator.
   static int canonicalize(
       sasl_conn_t* connection,
       void* context,
@@ -425,38 +433,11 @@ private:
 
   sasl_conn_t* connection;
 
-  process::Promise<Option<std::string> > promise;
+  process::Promise<Option<std::string>> promise;
 
   Option<std::string> principal;
 };
 
-
-Authenticator::Authenticator(const process::UPID& pid)
-{
-  process = new AuthenticatorProcess(pid);
-  process::spawn(process);
-}
-
-
-Authenticator::~Authenticator()
-{
-  // 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
-  // AuthenticatorProcess queue instead of at the front.
-  // The long term fix for this https://reviews.apache.org/r/25945/.
-  process::terminate(process, false);
-
-  process::wait(process);
-  delete process;
-}
-
-
-process::Future<Option<std::string> > Authenticator::authenticate()
-{
-  return process::dispatch(process, &AuthenticatorProcess::authenticate);
-}
-
-
 namespace secrets {
 
 // Loads secrets (principal -> secret) into the in-memory auxiliary
@@ -487,6 +468,50 @@ void load(const Credentials& credentials)
 
 } // namespace secrets {
 
+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/.
+    process::terminate(process, false);
+
+    process::wait(process);
+    delete process;
+  }
+}
+
+
+Try<Nothing> CRAMMD5Authenticator::initialize(
+    const process::UPID& pid,
+    const Option<Credentials>& credentials)
+{
+  if (credentials.isSome()) {
+    // Load "registration" credentials into SASL based Authenticator.
+    secrets::load(credentials.get());
+  } else {
+    return Error("Authentication requires credentials");
+  }
+
+  CHECK(process == NULL) << "Authenticator has already been initialized";
+  process = new CRAMMD5AuthenticatorProcess(pid);
+  process::spawn(process);
+
+  return Nothing();
+}
+
+
+process::Future<Option<std::string>> CRAMMD5Authenticator::authenticate(void)
+{
+  CHECK(process != NULL) << "Authenticator has not been initialized";
+  return process::dispatch(
+      process, &CRAMMD5AuthenticatorProcess::authenticate);
+}
+
 } // namespace cram_md5 {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/880ea7da/src/examples/test_authenticator_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_authenticator_module.cpp b/src/examples/test_authenticator_module.cpp
new file mode 100644
index 0000000..4398d71
--- /dev/null
+++ b/src/examples/test_authenticator_module.cpp
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <mesos/mesos.hpp>
+#include <mesos/module.hpp>
+
+#include "authentication/authenticator.hpp"
+
+#include "authentication/cram_md5/authenticator.hpp"
+
+#include "module/authenticator.hpp"
+
+using namespace mesos;
+
+using mesos::internal::Authenticator;
+
+static bool compatible()
+{
+  return true;
+}
+
+
+static Authenticator* createCRAMMD5Authenticator(const Parameters& parameters)
+{
+  return new mesos::internal::cram_md5::CRAMMD5Authenticator();
+}
+
+
+mesos::modules::Module<Authenticator> org_apache_mesos_TestCRAMMD5Authenticator(
+    MESOS_MODULE_API_VERSION,
+    MESOS_VERSION,
+    "Apache Mesos",
+    "modules@mesos.apache.org",
+    "Test CRAM-MD5 SASL authenticator module.",
+    compatible,
+    createCRAMMD5Authenticator);

http://git-wip-us.apache.org/repos/asf/mesos/blob/880ea7da/src/module/authenticator.hpp
----------------------------------------------------------------------
diff --git a/src/module/authenticator.hpp b/src/module/authenticator.hpp
new file mode 100644
index 0000000..9562338
--- /dev/null
+++ b/src/module/authenticator.hpp
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __MODULE_AUTHENTICATOR_HPP__
+#define __MODULE_AUTHENTICATOR_HPP__
+
+#include <mesos/module.hpp>
+
+#include "authentication/authenticator.hpp"
+
+namespace mesos {
+namespace modules {
+
+template<>
+inline const char* kind<mesos::internal::Authenticator>()
+{
+  return "Authenticator";
+}
+
+
+template <>
+struct Module<mesos::internal::Authenticator> : ModuleBase
+{
+  Module(
+      const char* _moduleApiVersion,
+      const char* _mesosVersion,
+      const char* _authorName,
+      const char* _authorEmail,
+      const char* _description,
+      bool (*_compatible)(),
+      mesos::internal::Authenticator* (*_create)(const Parameters& parameters))
+    : ModuleBase(
+        _moduleApiVersion,
+        _mesosVersion,
+        mesos::modules::kind<mesos::internal::Authenticator>(),
+        _authorName,
+        _authorEmail,
+        _description,
+        _compatible),
+      create(_create)
+  { }
+
+  mesos::internal::Authenticator* (*create)(const Parameters& parameters);
+};
+
+} // namespace modules {
+} // namespace mesos {
+
+#endif // __MODULE_AUTHENTICATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/880ea7da/src/module/manager.cpp
----------------------------------------------------------------------
diff --git a/src/module/manager.cpp b/src/module/manager.cpp
index 7a6c884..3062c41 100644
--- a/src/module/manager.cpp
+++ b/src/module/manager.cpp
@@ -57,8 +57,9 @@ void ModuleManager::initialize()
   // current change.  Typically that should be the version currently
   // under development.
 
-  kindToVersion["TestModule"] = MESOS_VERSION;
+  kindToVersion["Authenticator"] = MESOS_VERSION;
   kindToVersion["Isolator"] = MESOS_VERSION;
+  kindToVersion["TestModule"] = MESOS_VERSION;
 
   // What happens then when Mesos is built with a certain version,
   // 'kindToVersion' states a certain other minimum version, and a


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

Posted by me...@apache.org.
Integrated CRAM-MD5 Authenticator module into tests.

The CRAM-MD5 authenticator module is now usable via mesos-tests.sh

Example usage:
bin/mesos-tests.sh \
--authenticators=org_apache_mesos_TestCRAMMD5Authenticator \
--modules="{\"libraries\": [{\"file\": \
\"src/.libs/libtestauthenticator.so\",\"modules\": [{\"name\": \
\"org_apache_mesos_TestCRAMMD5Authenticator\"}]}]}"

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


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

Branch: refs/heads/master
Commit: 0756d185ab095d88d7313305417939260067a17f
Parents: 9ca757f
Author: Till Toenshoff <to...@me.com>
Authored: Mon Nov 3 14:56:43 2014 -0800
Committer: Adam B <ad...@mesosphere.io>
Committed: Mon Nov 3 14:57:09 2014 -0800

----------------------------------------------------------------------
 src/tests/flags.hpp | 12 ++++++++++++
 src/tests/mesos.cpp |  2 ++
 2 files changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0756d185/src/tests/flags.hpp
----------------------------------------------------------------------
diff --git a/src/tests/flags.hpp b/src/tests/flags.hpp
index 2886e89..e988b6f 100644
--- a/src/tests/flags.hpp
+++ b/src/tests/flags.hpp
@@ -28,6 +28,7 @@
 #include "common/parse.hpp"
 #include "common/type_utils.hpp"
 #include "logging/logging.hpp"
+#include "master/constants.hpp"
 #include "messages/messages.hpp"
 
 namespace mesos {
@@ -130,6 +131,16 @@ public:
         "'cgroups/cpu,cgroups/mem', or network/port_mapping\n"
         "(configure with flag: --with-network-isolator to enable),\n"
         "or 'external'.");
+
+    // This help message is duplicated from master/flags.hpp and
+    // should always be kept in sync with that.
+    add(&Flags::authenticators,
+        "authenticators",
+        "Authenticator implementation to use when authenticating frameworks\n"
+        "and/or slaves. "
+        "Use the default '" + master::DEFAULT_AUTHENTICATOR + "', or\n"
+        "load an alternate authenticator module using --modules.",
+        master::DEFAULT_AUTHENTICATOR);
   }
 
   bool verbose;
@@ -139,6 +150,7 @@ public:
   std::string docker;
   Option<Modules> modules;
   Option<std::string> isolation;
+  std::string authenticators;
 };
 
 // Global flags for running the tests.

http://git-wip-us.apache.org/repos/asf/mesos/blob/0756d185/src/tests/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index 2e8e39a..3b98c69 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -123,6 +123,8 @@ master::Flags MesosTest::CreateMasterFlags()
   // On many test VMs, this default is too small.
   flags.registry_store_timeout = flags.registry_store_timeout * 5;
 
+  flags.authenticators = tests::flags.authenticators;
+
   return flags;
 }