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/12 21:53:26 UTC

[1/3] mesos git commit: Added CRAM-MD5 Test Authenticatee Module.

Repository: mesos
Updated Branches:
  refs/heads/master b930d5ce3 -> 3f693f23a


Added CRAM-MD5 Test Authenticatee Module.

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


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

Branch: refs/heads/master
Commit: da3f41a42d35da02752feb7e6666d1f56364b5d8
Parents: b930d5c
Author: Till Toenshoff <to...@me.com>
Authored: Wed Nov 12 12:48:56 2014 -0800
Committer: Adam B <ad...@mesosphere.io>
Committed: Wed Nov 12 12:48:56 2014 -0800

----------------------------------------------------------------------
 src/Makefile.am                               | 13 ++--
 src/authentication/authenticatee.hpp          | 58 ++++++++++++++
 src/authentication/cram_md5/authenticatee.hpp | 88 +++++++++++++---------
 src/examples/test_authentication_modules.cpp  | 71 +++++++++++++++++
 src/examples/test_authenticator_module.cpp    | 51 -------------
 src/module/authenticatee.hpp                  | 64 ++++++++++++++++
 src/module/manager.cpp                        |  1 +
 7 files changed, 252 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/da3f41a4/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 8ecaa54..d5cad0a 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -389,6 +389,7 @@ if WITH_NETWORK_ISOLATOR
 endif
 
 libmesos_no_3rdparty_la_SOURCES +=					\
+	authentication/authenticatee.hpp				\
 	authentication/authenticator.hpp				\
 	authentication/cram_md5/authenticatee.hpp			\
 	authentication/cram_md5/authenticator.hpp			\
@@ -430,6 +431,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	master/registrar.hpp						\
 	master/sorter.hpp						\
 	messages/messages.hpp						\
+	module/authenticatee.hpp					\
 	module/authenticator.hpp					\
 	module/isolator.hpp						\
 	module/manager.hpp						\
@@ -1184,15 +1186,14 @@ 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.
+# Library containing the test CRAM-MD5 authentication modules.
 # 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
+lib_LTLIBRARIES += libtestauthentication.la
+libtestauthentication_la_SOURCES = examples/test_authentication_modules.cpp
+libtestauthentication_la_CPPFLAGS = $(MESOS_CPPFLAGS)
+libtestauthentication_la_LDFLAGS = -release $(PACKAGE_VERSION) -shared
 
 mesos_tests_SOURCES =				\
   tests/allocator_tests.cpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/da3f41a4/src/authentication/authenticatee.hpp
----------------------------------------------------------------------
diff --git a/src/authentication/authenticatee.hpp b/src/authentication/authenticatee.hpp
new file mode 100644
index 0000000..361083b
--- /dev/null
+++ b/src/authentication/authenticatee.hpp
@@ -0,0 +1,58 @@
+/**
+ * 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 __AUTHENTICATEE_HPP__
+#define __AUTHENTICATEE_HPP__
+
+#include <mesos/mesos.hpp>
+
+#include <process/future.hpp>
+
+#include <stout/nothing.hpp>
+#include <stout/try.hpp>
+
+namespace mesos {
+namespace internal {
+
+class Authenticatee
+{
+public:
+  Authenticatee() {}
+
+  virtual ~Authenticatee() {}
+
+  // 'pid' is the process to authenticate against (master).
+  // 'client' is the process to be authenticated (slave / framework).
+  // 'credential' is used to authenticate the 'client'.
+  // 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.
+  virtual process::Future<bool> authenticate(
+    const process::UPID& pid,
+    const process::UPID& client,
+    const mesos::Credential& credential) = 0;
+};
+
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __AUTHENTICATEE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/da3f41a4/src/authentication/cram_md5/authenticatee.hpp
----------------------------------------------------------------------
diff --git a/src/authentication/cram_md5/authenticatee.hpp b/src/authentication/cram_md5/authenticatee.hpp
index 3088a77..5d35598 100644
--- a/src/authentication/cram_md5/authenticatee.hpp
+++ b/src/authentication/cram_md5/authenticatee.hpp
@@ -19,8 +19,12 @@
 #ifndef __AUTHENTICATION_CRAM_MD5_AUTHENTICATEE_HPP__
 #define __AUTHENTICATION_CRAM_MD5_AUTHENTICATEE_HPP__
 
+#include <stddef.h>   // For size_t needed by sasl.h.
+
 #include <sasl/sasl.h>
 
+#include <string>
+
 #include <mesos/mesos.hpp>
 
 #include <process/defer.hpp>
@@ -32,6 +36,8 @@
 
 #include <stout/strings.hpp>
 
+#include "authentication/authenticatee.hpp"
+
 #include "messages/messages.hpp"
 
 namespace mesos {
@@ -39,36 +45,35 @@ namespace internal {
 namespace cram_md5 {
 
 // Forward declaration.
-class AuthenticateeProcess;
+class CRAMMD5AuthenticateeProcess;
 
 
-class Authenticatee
+class CRAMMD5Authenticatee : public 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);
+  // Factory to allow for typed tests.
+  static Try<Authenticatee*> create();
+
+  CRAMMD5Authenticatee();
+
+  virtual ~CRAMMD5Authenticatee();
+
+  process::Future<bool> authenticate(const process::UPID& pid,
+                                     const process::UPID& client,
+                                     const Credential& credential);
 
 private:
-  AuthenticateeProcess* process;
+  CRAMMD5AuthenticateeProcess* process;
 };
 
 
-class AuthenticateeProcess : public ProtobufProcess<AuthenticateeProcess>
+class CRAMMD5AuthenticateeProcess
+  : public ProtobufProcess<CRAMMD5AuthenticateeProcess>
 {
 public:
-  AuthenticateeProcess(const Credential& _credential,
-                       const process::UPID& _client)
-    : ProcessBase(process::ID::generate("authenticatee")),
+  CRAMMD5AuthenticateeProcess(const Credential& _credential,
+                              const process::UPID& _client)
+    : ProcessBase(process::ID::generate("crammd5_authenticatee")),
       credential(_credential),
       client(_client),
       status(READY),
@@ -87,7 +92,7 @@ public:
     secret->len = length;
   }
 
-  virtual ~AuthenticateeProcess()
+  virtual ~CRAMMD5AuthenticateeProcess()
   {
     if (connection != NULL) {
       sasl_dispose(&connection);
@@ -192,21 +197,21 @@ protected:
   {
     // Anticipate mechanisms and steps from the server.
     install<AuthenticationMechanismsMessage>(
-        &AuthenticateeProcess::mechanisms,
+        &CRAMMD5AuthenticateeProcess::mechanisms,
         &AuthenticationMechanismsMessage::mechanisms);
 
     install<AuthenticationStepMessage>(
-        &AuthenticateeProcess::step,
+        &CRAMMD5AuthenticateeProcess::step,
         &AuthenticationStepMessage::data);
 
     install<AuthenticationCompletedMessage>(
-        &AuthenticateeProcess::completed);
+        &CRAMMD5AuthenticateeProcess::completed);
 
     install<AuthenticationFailedMessage>(
-        &AuthenticateeProcess::failed);
+        &CRAMMD5AuthenticateeProcess::failed);
 
     install<AuthenticationErrorMessage>(
-        &AuthenticateeProcess::error,
+        &CRAMMD5AuthenticateeProcess::error,
         &AuthenticationErrorMessage::error);
   }
 
@@ -382,27 +387,36 @@ private:
 };
 
 
-inline Authenticatee::Authenticatee(
-    const Credential& credential,
-    const process::UPID& client)
+inline Try<Authenticatee*> CRAMMD5Authenticatee::create()
 {
-  process = new AuthenticateeProcess(credential, client);
-  process::spawn(process);
+  return new CRAMMD5Authenticatee();
 }
 
 
-inline Authenticatee::~Authenticatee()
+inline CRAMMD5Authenticatee::CRAMMD5Authenticatee() : process(NULL) {}
+
+
+inline CRAMMD5Authenticatee::~CRAMMD5Authenticatee()
 {
-  process::terminate(process);
-  process::wait(process);
-  delete process;
+  if (process != NULL) {
+    process::terminate(process);
+    process::wait(process);
+    delete process;
+  }
 }
 
 
-inline process::Future<bool> Authenticatee::authenticate(
-    const process::UPID& pid)
+inline process::Future<bool> CRAMMD5Authenticatee::authenticate(
+  const process::UPID& pid,
+  const process::UPID& client,
+  const mesos::Credential& credential)
 {
-  return process::dispatch(process, &AuthenticateeProcess::authenticate, pid);
+  CHECK(process == NULL);
+  process = new CRAMMD5AuthenticateeProcess(credential, client);
+  process::spawn(process);
+
+  return process::dispatch(
+      process, &CRAMMD5AuthenticateeProcess::authenticate, pid);
 }
 
 } // namespace cram_md5 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/da3f41a4/src/examples/test_authentication_modules.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_authentication_modules.cpp b/src/examples/test_authentication_modules.cpp
new file mode 100644
index 0000000..6d32573
--- /dev/null
+++ b/src/examples/test_authentication_modules.cpp
@@ -0,0 +1,71 @@
+/**
+ * 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/authenticatee.hpp"
+#include "authentication/authenticator.hpp"
+
+#include "authentication/cram_md5/authenticatee.hpp"
+#include "authentication/cram_md5/authenticator.hpp"
+
+#include "module/authenticatee.hpp"
+#include "module/authenticator.hpp"
+
+using namespace mesos;
+
+using mesos::internal::Authenticatee;
+using mesos::internal::Authenticator;
+
+static bool compatible()
+{
+  return true;
+}
+
+
+static Authenticatee* createCRAMMD5Authenticatee(const Parameters& parameters)
+{
+  return new mesos::internal::cram_md5::CRAMMD5Authenticatee();
+}
+
+
+mesos::modules::Module<Authenticatee> org_apache_mesos_TestCRAMMD5Authenticatee(
+    MESOS_MODULE_API_VERSION,
+    MESOS_VERSION,
+    "Apache Mesos",
+    "modules@mesos.apache.org",
+    "Test CRAM-MD5 SASL authenticatee module.",
+    compatible,
+    createCRAMMD5Authenticatee);
+
+
+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/da3f41a4/src/examples/test_authenticator_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_authenticator_module.cpp b/src/examples/test_authenticator_module.cpp
deleted file mode 100644
index 4398d71..0000000
--- a/src/examples/test_authenticator_module.cpp
+++ /dev/null
@@ -1,51 +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 <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/da3f41a4/src/module/authenticatee.hpp
----------------------------------------------------------------------
diff --git a/src/module/authenticatee.hpp b/src/module/authenticatee.hpp
new file mode 100644
index 0000000..bae37ce
--- /dev/null
+++ b/src/module/authenticatee.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_AUTHENTICATEE_HPP__
+#define __MODULE_AUTHENTICATEE_HPP__
+
+#include <mesos/module.hpp>
+
+#include "authentication/authenticatee.hpp"
+
+namespace mesos {
+namespace modules {
+
+template<>
+inline const char* kind<mesos::internal::Authenticatee>()
+{
+  return "Authenticatee";
+}
+
+
+template <>
+struct Module<mesos::internal::Authenticatee> : ModuleBase
+{
+  Module(
+      const char* _moduleApiVersion,
+      const char* _mesosVersion,
+      const char* _authorName,
+      const char* _authorEmail,
+      const char* _description,
+      bool (*_compatible)(),
+      mesos::internal::Authenticatee* (*_create)(const Parameters& parameters))
+    : ModuleBase(
+        _moduleApiVersion,
+        _mesosVersion,
+        mesos::modules::kind<mesos::internal::Authenticatee>(),
+        _authorName,
+        _authorEmail,
+        _description,
+        _compatible),
+      create(_create)
+  { }
+
+  mesos::internal::Authenticatee* (*create)(const Parameters& parameters);
+};
+
+} // namespace modules {
+} // namespace mesos {
+
+#endif // __MODULE_AUTHENTICATEE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/da3f41a4/src/module/manager.cpp
----------------------------------------------------------------------
diff --git a/src/module/manager.cpp b/src/module/manager.cpp
index 0d71e33..b15b0fc 100644
--- a/src/module/manager.cpp
+++ b/src/module/manager.cpp
@@ -57,6 +57,7 @@ void ModuleManager::initialize()
   // current change.  Typically that should be the version currently
   // under development.
 
+  kindToVersion["Authenticatee"] = MESOS_VERSION;
   kindToVersion["Authenticator"] = MESOS_VERSION;
   kindToVersion["Isolator"] = MESOS_VERSION;
   kindToVersion["TestModule"] = MESOS_VERSION;


[2/3] mesos git commit: Integrated CRAM-MD5 Test Authenticatee module into slave.

Posted by me...@apache.org.
Integrated CRAM-MD5 Test Authenticatee module into slave.

Also fixes messages.proto to use a raw bytestream instead of a string
for AuthenticationStartMessage as non CRAM-MD5 authentication may
transmit binary data.
Note that the change of AuthenticationStartMessage does basically have
no impact on C++ based proto code other than the prevention of a
warning due to non-UTF8 characters being encoded. That does in fact
occur when using non CRAM-MD5 based SASL authentication mechanisms.

Note that this patch covers modularized slave authentication only.
Framework authentication is currently covered by the default (built-in)
implementation. There will be a subsequent patch for modularized
framework authentication.

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


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

Branch: refs/heads/master
Commit: 6451af3ee26cba317fa9b65ef0d6e381378e9099
Parents: da3f41a
Author: Till Toenshoff <to...@me.com>
Authored: Wed Nov 12 12:49:19 2014 -0800
Committer: Adam B <ad...@mesosphere.io>
Committed: Wed Nov 12 12:49:19 2014 -0800

----------------------------------------------------------------------
 src/messages/messages.proto |  2 +-
 src/sched/sched.cpp         | 10 ++++++----
 src/scheduler/scheduler.cpp | 10 ++++++----
 src/slave/constants.cpp     |  1 +
 src/slave/constants.hpp     |  3 +++
 src/slave/flags.hpp         |  8 ++++++++
 src/slave/slave.cpp         | 29 ++++++++++++++++++++++++-----
 src/slave/slave.hpp         |  7 ++++---
 8 files changed, 53 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/6451af3e/src/messages/messages.proto
----------------------------------------------------------------------
diff --git a/src/messages/messages.proto b/src/messages/messages.proto
index de0e2a2..566ce53 100644
--- a/src/messages/messages.proto
+++ b/src/messages/messages.proto
@@ -372,7 +372,7 @@ message AuthenticationMechanismsMessage {
 
 message AuthenticationStartMessage {
   required string mechanism = 1;
-  optional string data = 2;
+  optional bytes data = 2;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/6451af3e/src/sched/sched.cpp
----------------------------------------------------------------------
diff --git a/src/sched/sched.cpp b/src/sched/sched.cpp
index 8ca0526..4981dfb 100644
--- a/src/sched/sched.cpp
+++ b/src/sched/sched.cpp
@@ -59,6 +59,7 @@
 #include <stout/utils.hpp>
 #include <stout/uuid.hpp>
 
+#include "authentication/authenticatee.hpp"
 #include "authentication/cram_md5/authenticatee.hpp"
 
 #include "common/lock.hpp"
@@ -297,7 +298,7 @@ protected:
     CHECK_SOME(credential);
 
     CHECK(authenticatee == NULL);
-    authenticatee = new cram_md5::Authenticatee(credential.get(), self());
+    authenticatee = new cram_md5::CRAMMD5Authenticatee();
 
     // NOTE: We do not pass 'Owned<Authenticatee>' here because doing
     // so could make 'AuthenticateeProcess' responsible for deleting
@@ -312,8 +313,9 @@ protected:
     //     'Authenticatee'.
     // --> '~Authenticatee()' is invoked by 'AuthenticateeProcess'.
     // TODO(vinod): Consider using 'Shared' to 'Owned' upgrade.
-    authenticating = authenticatee->authenticate(master.get())
-      .onAny(defer(self(), &Self::_authenticate));
+    authenticating =
+      authenticatee->authenticate(master.get(), self(), credential.get())
+        .onAny(defer(self(), &Self::_authenticate));
 
     delay(Seconds(5),
           self(),
@@ -1062,7 +1064,7 @@ private:
 
   const Option<Credential> credential;
 
-  cram_md5::Authenticatee* authenticatee;
+  Authenticatee* authenticatee;
 
   // Indicates if an authentication attempt is in progress.
   Option<Future<bool> > authenticating;

http://git-wip-us.apache.org/repos/asf/mesos/blob/6451af3e/src/scheduler/scheduler.cpp
----------------------------------------------------------------------
diff --git a/src/scheduler/scheduler.cpp b/src/scheduler/scheduler.cpp
index c74187c..cbb982a 100644
--- a/src/scheduler/scheduler.cpp
+++ b/src/scheduler/scheduler.cpp
@@ -53,6 +53,7 @@
 #include <stout/os.hpp>
 #include <stout/uuid.hpp>
 
+#include "authentication/authenticatee.hpp"
 #include "authentication/cram_md5/authenticatee.hpp"
 
 #include "common/type_utils.hpp"
@@ -465,7 +466,7 @@ protected:
     CHECK_SOME(credential);
 
     CHECK(authenticatee == NULL);
-    authenticatee = new cram_md5::Authenticatee(credential.get(), self());
+    authenticatee = new cram_md5::CRAMMD5Authenticatee();
 
     // NOTE: We do not pass 'Owned<Authenticatee>' here because doing
     // so could make 'AuthenticateeProcess' responsible for deleting
@@ -480,8 +481,9 @@ protected:
     //     'Authenticatee'.
     // --> '~Authenticatee()' is invoked by 'AuthenticateeProcess'.
     // TODO(vinod): Consider using 'Shared' to 'Owned' upgrade.
-    authenticating = authenticatee->authenticate(master.get())
-      .onAny(defer(self(), &Self::_authenticate));
+    authenticating =
+      authenticatee->authenticate(master.get(), self(), credential.get())
+        .onAny(defer(self(), &Self::_authenticate));
 
     delay(Seconds(5),
           self(),
@@ -808,7 +810,7 @@ private:
 
   Option<UPID> master;
 
-  cram_md5::Authenticatee* authenticatee;
+  Authenticatee* authenticatee;
 
   // Indicates if an authentication attempt is in progress.
   Option<Future<bool> > authenticating;

http://git-wip-us.apache.org/repos/asf/mesos/blob/6451af3e/src/slave/constants.cpp
----------------------------------------------------------------------
diff --git a/src/slave/constants.cpp b/src/slave/constants.cpp
index d6ad78c..2a99b11 100644
--- a/src/slave/constants.cpp
+++ b/src/slave/constants.cpp
@@ -50,6 +50,7 @@ const std::string DEFAULT_PORTS = "[31000-32000]";
 const uint16_t DEFAULT_EPHEMERAL_PORTS_PER_CONTAINER = 1024;
 #endif
 const Duration DOCKER_REMOVE_DELAY = Hours(6);
+const std::string DEFAULT_AUTHENTICATEE = "crammd5";
 
 Duration MASTER_PING_TIMEOUT()
 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/6451af3e/src/slave/constants.hpp
----------------------------------------------------------------------
diff --git a/src/slave/constants.hpp b/src/slave/constants.hpp
index 701dd89..fd1c1ab 100644
--- a/src/slave/constants.hpp
+++ b/src/slave/constants.hpp
@@ -97,6 +97,9 @@ extern const uint16_t DEFAULT_EPHEMERAL_PORTS_PER_CONTAINER;
 // Default duration that docker containers will be removed after exit.
 extern const Duration DOCKER_REMOVE_DELAY;
 
+// Name of the default, CRAM-MD5 authenticatee.
+extern const std::string DEFAULT_AUTHENTICATEE;
+
 // If no pings received within this timeout, then the slave will
 // trigger a re-detection of the master to cause a re-registration.
 Duration MASTER_PING_TIMEOUT();

http://git-wip-us.apache.org/repos/asf/mesos/blob/6451af3e/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index efbd35d..4ec5954 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -404,6 +404,13 @@ public:
         "    }\n"
         "  ]\n"
         "}");
+
+    add(&Flags::authenticatee,
+        "authenticatee",
+        "Authenticatee implementation to use when authenticating against the\n"
+        "master. Use the default '" + DEFAULT_AUTHENTICATEE + "', or\n"
+        "load an alternate authenticatee module using --modules.",
+        DEFAULT_AUTHENTICATEE);
   }
 
   bool version;
@@ -454,6 +461,7 @@ public:
   Option<Bytes> egress_rate_limit_per_container;
 #endif
   Option<Modules> modules;
+  std::string authenticatee;
 };
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/6451af3e/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index ba00bf5..99fd055 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -73,6 +73,9 @@
 
 #include "logging/logging.hpp"
 
+#include "module/authenticatee.hpp"
+#include "module/manager.hpp"
+
 #include "slave/constants.hpp"
 #include "slave/flags.hpp"
 #include "slave/paths.hpp"
@@ -261,6 +264,8 @@ void Slave::initialize()
             << "Must be less than " << REGISTER_RETRY_INTERVAL_MAX;
   }
 
+  authenticateeName = flags.authenticatee;
+
   if (flags.credential.isSome()) {
     const string& path =
       strings::remove(flags.credential.get(), "file://", strings::PREFIX);
@@ -664,13 +669,27 @@ void Slave::authenticate()
 
   LOG(INFO) << "Authenticating with master " << master.get();
 
-  CHECK_SOME(credential);
-
   CHECK(authenticatee == NULL);
-  authenticatee = new cram_md5::Authenticatee(credential.get(), self());
 
-  authenticating = authenticatee->authenticate(master.get())
-    .onAny(defer(self(), &Self::_authenticate));
+  if (authenticateeName == DEFAULT_AUTHENTICATEE) {
+    LOG(INFO) << "Using default CRAM-MD5 authenticatee";
+    authenticatee = new cram_md5::CRAMMD5Authenticatee();
+  } else {
+    Try<Authenticatee*> module =
+      modules::ModuleManager::create<Authenticatee>(authenticateeName);
+    if (module.isError()) {
+      EXIT(1) << "Could not create authenticatee module '"
+              << authenticateeName << "': " << module.error();
+    }
+    LOG(INFO) << "Using '" << authenticateeName << "' authenticatee";
+    authenticatee = module.get();
+  }
+
+  CHECK_SOME(credential);
+
+  authenticating =
+    authenticatee->authenticate(master.get(), self(), credential.get())
+      .onAny(defer(self(), &Self::_authenticate));
 
   delay(Seconds(5),
         self(),

http://git-wip-us.apache.org/repos/asf/mesos/blob/6451af3e/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index d9f9d2e..29bea65 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -70,9 +70,7 @@ namespace internal {
 
 class MasterDetector; // Forward declaration.
 
-namespace cram_md5 {
 class Authenticatee;
-} // namespace cram_md5 {
 
 namespace slave {
 
@@ -496,7 +494,10 @@ private:
 
   Option<Credential> credential;
 
-  cram_md5::Authenticatee* authenticatee;
+  // Authenticatee name as supplied via flags.
+  std::string authenticateeName;
+
+  Authenticatee* authenticatee;
 
   // Indicates if an authentication attempt is in progress.
   Option<Future<bool> > authenticating;


[3/3] mesos git commit: Integrated CRAM-MD5 Authenticatee module into tests.

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

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


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

Branch: refs/heads/master
Commit: 3f693f23ac1a9792a7f6fc91e76cf875847867bb
Parents: 6451af3
Author: Till Toenshoff <to...@me.com>
Authored: Wed Nov 12 12:51:20 2014 -0800
Committer: Adam B <ad...@mesosphere.io>
Committed: Wed Nov 12 12:51:20 2014 -0800

----------------------------------------------------------------------
 src/tests/cram_md5_authentication_tests.cpp | 20 ++++++++------------
 1 file changed, 8 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3f693f23/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 1ed2312..f91f1f9 100644
--- a/src/tests/cram_md5_authentication_tests.cpp
+++ b/src/tests/cram_md5_authentication_tests.cpp
@@ -61,12 +61,11 @@ TEST(CRAMMD5Authentication, success)
 
   secrets::load(credentials);
 
-  Authenticatee authenticatee(credential1, UPID());
-
   Future<Message> message =
     FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
 
-  Future<bool> client = authenticatee.authenticate(pid);
+  CRAMMD5Authenticatee authenticatee;
+  Future<bool> client = authenticatee.authenticate(pid, UPID(), credential1);
 
   AWAIT_READY(message);
 
@@ -100,12 +99,11 @@ TEST(CRAMMD5Authentication, failed1)
 
   secrets::load(credentials);
 
-  Authenticatee authenticatee(credential1, UPID());
-
   Future<Message> message =
     FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
 
-  Future<bool> client = authenticatee.authenticate(pid);
+  CRAMMD5Authenticatee authenticatee;
+  Future<bool> client = authenticatee.authenticate(pid, UPID(), credential1);
 
   AWAIT_READY(message);
 
@@ -139,12 +137,11 @@ TEST(CRAMMD5Authentication, failed2)
 
   secrets::load(credentials);
 
-  Authenticatee authenticatee(credential1, UPID());
-
   Future<Message> message =
     FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
 
-  Future<bool> client = authenticatee.authenticate(pid);
+  CRAMMD5Authenticatee authenticatee;
+  Future<bool> client = authenticatee.authenticate(pid, UPID(), credential1);
 
   AWAIT_READY(message);
 
@@ -180,12 +177,11 @@ TEST(CRAMMD5Authentication, AuthenticatorDestructionRace)
 
   secrets::load(credentials);
 
-  Authenticatee authenticatee(credential1, UPID());
-
   Future<Message> message =
     FUTURE_MESSAGE(Eq(AuthenticateMessage().GetTypeName()), _, _);
 
-  Future<bool> client = authenticatee.authenticate(pid);
+  CRAMMD5Authenticatee authenticatee;
+  Future<bool> client = authenticatee.authenticate(pid, UPID(), credential1);
 
   AWAIT_READY(message);