You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by vi...@apache.org on 2017/03/15 18:36:14 UTC

[1/3] mesos git commit: Added the 'CombinedAuthenticator'.

Repository: mesos
Updated Branches:
  refs/heads/master 306e379be -> a68e1f28d


Added the 'CombinedAuthenticator'.

This patch adds a new default authenticator, the
`CombinedAuthenticator`, which can load multiple authenticators.
It calls installed authenticators serially, returning the first
successful result. When no results are successful, it returns a
single result obtained by merging all unsuccessful results.

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


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

Branch: refs/heads/master
Commit: 0804be3164273a4f71dc0623c18b218290635266
Parents: 306e379
Author: Greg Mann <gr...@mesosphere.io>
Authored: Wed Mar 15 19:35:45 2017 +0100
Committer: Vinod Kone <vi...@gmail.com>
Committed: Wed Mar 15 19:35:45 2017 +0100

----------------------------------------------------------------------
 .../http/combined_authenticator.hpp             | 141 +++++++
 src/CMakeLists.txt                              |   1 +
 src/Makefile.am                                 |   4 +-
 .../http/combined_authenticator.cpp             | 371 +++++++++++++++++++
 4 files changed, 516 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0804be31/include/mesos/authentication/http/combined_authenticator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/authentication/http/combined_authenticator.hpp b/include/mesos/authentication/http/combined_authenticator.hpp
new file mode 100644
index 0000000..f863453
--- /dev/null
+++ b/include/mesos/authentication/http/combined_authenticator.hpp
@@ -0,0 +1,141 @@
+// 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 __MESOS_AUTHENTICATION_HTTP_COMBINED_AUTHENTICATOR_HPP__
+#define __MESOS_AUTHENTICATION_HTTP_COMBINED_AUTHENTICATOR_HPP__
+
+#include <string>
+#include <vector>
+
+#include <mesos/mesos.hpp>
+
+#include <process/authenticator.hpp>
+#include <process/future.hpp>
+#include <process/http.hpp>
+#include <process/owned.hpp>
+
+#include <stout/hashset.hpp>
+
+namespace mesos {
+namespace http {
+namespace authentication {
+
+class CombinedAuthenticatorProcess;
+
+
+/**
+ * An authenticator which holds references to multiple authenticators.
+ */
+class CombinedAuthenticator
+  : public process::http::authentication::Authenticator
+{
+public:
+  CombinedAuthenticator(
+      const std::string& realm,
+      std::vector<process::Owned<
+        process::http::authentication::Authenticator>>&& authenticators);
+
+  /**
+   * Non-copyable to protect the ownership of the installed authenticators.
+   */
+  CombinedAuthenticator(const CombinedAuthenticator&) = delete;
+
+  ~CombinedAuthenticator() override;
+
+  /**
+   * Authenticates using the installed authenticators. When called, the
+   * `authenticate()` method of each authenticator is called serially, and the
+   * first successful result is returned. If all authentication attempts fail,
+   * the failed results are combined as follows:
+   *
+   * - If any results are Unauthorized, then the Unauthorized results are merged
+   *   and returned.
+   * - If no Unauthorized results are found, then multiple Forbidden results
+   *   may be present. The Forbidden results are combined and returned.
+   * - If no Forbidden results are found, then failed futures may be present.
+   *   Their error messages are combined and returned in a failed future.
+   *
+   * Below are examples illustrating various combinations of authentication
+   * results when two authenticators are installed. In these examples, the
+   * `scheme()` method of one authenticator returns "Basic". The other returns
+   * "Bearer". These schemes are used when combining response bodies.
+   *
+   * Both authentication results are Unauthorized:
+   *   - First result:
+   *                   Status code: 401
+   *     'WWW-Authenticate' header: 'Basic realm="mesos"'
+   *                 Response body: 'Incorrect credentials'
+   *   - Second result:
+   *                   Status code: 401
+   *     'WWW-Authenticate' header: 'Bearer realm="mesos"'
+   *                 Response body: 'Invalid token'
+   *
+   *   - Returned result:
+   *                   Status code: 401
+   *     'WWW-Authenticate' header: 'Basic realm="mesos",Bearer realm="mesos"'
+   *                 Response body: '"Basic" authenticator returned:\n'
+   *                                'Incorrect credentials\n\n'
+   *                                '"Bearer" authenticator returned:\n'
+   *                                'Invalid token'
+   *
+   * One Unauthorized result and one Forbidden:
+   *   - First result:
+   *                   Status code: 401
+   *     'WWW-Authenticate' header: 'Basic realm="mesos"'
+   *                 Response body: 'Incorrect credentials'
+   *   - Second result:
+   *                   Status code: 403
+   *                 Response body: 'Not authorized'
+   *
+   *   - Returned result:
+   *                   Status code: 401
+   *     'WWW-Authenticate' header: 'Basic realm="mesos"'
+   *                 Response body: 'Incorrect credentials'
+   *
+   * Both authentication results are Forbidden:
+   *   - First result:
+   *                   Status code: 403
+   *                 Response body: 'Basic: not authorized'
+   *   - Second result:
+   *                   Status code: 403
+   *                 Response body: 'Bearer: not authorized'
+   *
+   *   - Returned result:
+   *                   Status code: 403
+   *                 Response body: '"Basic" authenticator returned:\n'
+   *                                'Basic: not authorized\n\n'
+   *                                '"Bearer" authenticator returned:\n'
+   *                                'Bearer: not authorized'
+   */
+  process::Future<process::http::authentication::AuthenticationResult>
+    authenticate(const process::http::Request& request) override;
+
+  /**
+   * Returns the authentication schemes offered by the
+   * installed authenticators, separated by whitespace.
+   */
+  std::string scheme() const override;
+
+private:
+  process::Owned<CombinedAuthenticatorProcess> process;
+  hashset<std::string> schemes;
+};
+
+} // namespace authentication {
+} // namespace http {
+} // namespace mesos {
+
+#endif // __MESOS_AUTHENTICATION_HTTP_COMBINED_AUTHENTICATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/0804be31/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index e1f81a1..b67b512 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -210,6 +210,7 @@ set(APPC_SRC
 
 set(AUTHENTICATION_SRC
   authentication/http/basic_authenticator_factory.cpp
+  authentication/http/combined_authenticator.cpp
   )
 
 if (NOT WIN32)

http://git-wip-us.apache.org/repos/asf/mesos/blob/0804be31/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 2eea11a..8c67e47 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -561,7 +561,8 @@ authentication_HEADERS =						\
   $(top_srcdir)/include/mesos/authentication/authentication.proto	\
   $(top_srcdir)/include/mesos/authentication/authenticator.hpp		\
   $(top_srcdir)/include/mesos/authentication/secret_generator.hpp	\
-  $(top_srcdir)/include/mesos/authentication/http/basic_authenticator_factory.hpp
+  $(top_srcdir)/include/mesos/authentication/http/basic_authenticator_factory.hpp	\
+  $(top_srcdir)/include/mesos/authentication/http/combined_authenticator.hpp
 
 nodist_authentication_HEADERS =						\
   ../include/mesos/authentication/authentication.pb.h
@@ -843,6 +844,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   authentication/cram_md5/authenticator.cpp				\
   authentication/cram_md5/auxprop.cpp					\
   authentication/http/basic_authenticator_factory.cpp			\
+  authentication/http/combined_authenticator.cpp			\
   authorizer/acls.cpp							\
   authorizer/authorizer.cpp						\
   authorizer/local/authorizer.cpp					\

http://git-wip-us.apache.org/repos/asf/mesos/blob/0804be31/src/authentication/http/combined_authenticator.cpp
----------------------------------------------------------------------
diff --git a/src/authentication/http/combined_authenticator.cpp b/src/authentication/http/combined_authenticator.cpp
new file mode 100644
index 0000000..c734e76
--- /dev/null
+++ b/src/authentication/http/combined_authenticator.cpp
@@ -0,0 +1,371 @@
+// 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/authentication/http/combined_authenticator.hpp>
+
+#include <list>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <process/id.hpp>
+#include <process/loop.hpp>
+#include <process/process.hpp>
+
+#include <stout/foreach.hpp>
+#include <stout/hashset.hpp>
+#include <stout/strings.hpp>
+
+namespace mesos {
+namespace http {
+namespace authentication {
+
+using std::list;
+using std::make_pair;
+using std::pair;
+using std::string;
+using std::vector;
+
+using process::Break;
+using process::Continue;
+using process::ControlFlow;
+using process::Failure;
+using process::Future;
+using process::Owned;
+using process::Process;
+using process::UPID;
+
+using process::http::Forbidden;
+using process::http::Request;
+using process::http::Unauthorized;
+
+using process::http::authentication::AuthenticationResult;
+using process::http::authentication::Authenticator;
+
+
+class CombinedAuthenticatorProcess
+  : public Process<CombinedAuthenticatorProcess>
+{
+public:
+  CombinedAuthenticatorProcess(
+      const string& _realm,
+      vector<Owned<Authenticator>>&& _authenticators);
+
+  Future<AuthenticationResult> authenticate(const Request& request);
+
+private:
+  typedef pair<string, Try<AuthenticationResult>> SchemeResultPair;
+
+  static bool anyUnauthorized(
+      const list<SchemeResultPair>& authenticationResults);
+
+  static bool anyForbidden(const list<SchemeResultPair>& authenticationResults);
+
+  static bool anyError(const list<SchemeResultPair>& authenticationResults);
+
+  static vector<string> extractUnauthorizedHeaders(
+      const list<SchemeResultPair>& authenticationResults);
+
+  static vector<string> extractUnauthorizedBodies(
+      const list<SchemeResultPair>& authenticationResults);
+
+  static vector<string> extractForbiddenBodies(
+      const list<SchemeResultPair>& authenticationResults);
+
+  static vector<string> extractErrorMessages(
+      const list<SchemeResultPair>& authenticationResults);
+
+  static Future<ControlFlow<AuthenticationResult>> combineFailed(
+      const list<SchemeResultPair>& results);
+
+  const vector<Owned<Authenticator>> authenticators;
+  const string realm;
+};
+
+
+CombinedAuthenticatorProcess::CombinedAuthenticatorProcess(
+    const string& _realm,
+    vector<Owned<Authenticator>>&& _authenticators)
+  : ProcessBase(process::ID::generate("__combined_authenticator__")),
+    authenticators(_authenticators),
+    realm(_realm) {}
+
+
+bool CombinedAuthenticatorProcess::anyUnauthorized(
+    const list<SchemeResultPair>& authenticationResults)
+{
+  foreach (const SchemeResultPair& result, authenticationResults) {
+    if (result.second.isSome() && result.second->unauthorized.isSome()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+
+bool CombinedAuthenticatorProcess::anyForbidden(
+    const list<SchemeResultPair>& authenticationResults)
+{
+  foreach (const SchemeResultPair& result, authenticationResults) {
+    if (result.second.isSome() && result.second->forbidden.isSome()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+
+bool CombinedAuthenticatorProcess::anyError(
+    const list<SchemeResultPair>& authenticationResults)
+{
+  foreach (const SchemeResultPair& result, authenticationResults) {
+    if (result.second.isError()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+
+vector<string> CombinedAuthenticatorProcess::extractUnauthorizedHeaders(
+    const list<SchemeResultPair>& authenticationResults)
+{
+  vector<string> headers;
+
+  foreach (const SchemeResultPair& result, authenticationResults) {
+    if (result.second.isSome() &&
+        result.second->unauthorized.isSome() &&
+        result.second->unauthorized->headers.contains("WWW-Authenticate")) {
+      headers.push_back(
+          result.second->unauthorized->headers.at("WWW-Authenticate"));
+    }
+  }
+
+  return headers;
+}
+
+
+vector<string> CombinedAuthenticatorProcess::extractUnauthorizedBodies(
+    const list<SchemeResultPair>& authenticationResults)
+{
+  vector<string> bodies;
+
+  foreachpair (
+      const string& scheme,
+      const Try<AuthenticationResult>& result,
+      authenticationResults) {
+    if (result.isSome() &&
+        result->unauthorized.isSome() &&
+        result->unauthorized->body != "") {
+      bodies.push_back(
+          "\"" + scheme + "\" authenticator returned:\n" +
+          result->unauthorized->body);
+    }
+  }
+
+  return bodies;
+}
+
+
+vector<string> CombinedAuthenticatorProcess::extractForbiddenBodies(
+    const list<SchemeResultPair>& authenticationResults)
+{
+  vector<string> bodies;
+
+  foreachpair (
+      const string& scheme,
+      const Try<AuthenticationResult>& result,
+      authenticationResults) {
+    if (result.isSome() &&
+        result->forbidden.isSome() &&
+        result->forbidden->body != "") {
+      bodies.push_back(
+          "\"" + scheme + "\" authenticator returned:\n" +
+          result->forbidden->body);
+    }
+  }
+
+  return bodies;
+}
+
+
+vector<string> CombinedAuthenticatorProcess::extractErrorMessages(
+    const list<SchemeResultPair>& authenticationResults)
+{
+  vector<string> messages;
+
+  foreachpair (
+      const string& scheme,
+      const Try<AuthenticationResult>& result,
+      authenticationResults) {
+    if (result.isError()) {
+      messages.push_back(
+          "\"" + scheme + "\" authenticator returned:\n" +
+          result.error());
+    }
+  }
+
+  return messages;
+}
+
+
+// Creates a single authentication result for the authenticator to return
+// in the case that all authentication attempts have failed.
+Future<ControlFlow<AuthenticationResult>>
+  CombinedAuthenticatorProcess::combineFailed(
+      const list<SchemeResultPair>& results)
+{
+  AuthenticationResult combinedResult;
+
+  if (anyUnauthorized(results)) {
+    combinedResult.unauthorized = Unauthorized(
+        {strings::join(",", extractUnauthorizedHeaders(results))},
+        strings::join("\n\n", extractUnauthorizedBodies(results)));
+
+    return Break(combinedResult);
+  }
+
+  if (anyForbidden(results)) {
+    combinedResult.forbidden =
+      Forbidden(strings::join("\n\n", extractForbiddenBodies(results)));
+
+    return Break(combinedResult);
+  }
+
+  // This case serves to surface errors from failed futures to libprocess.
+  if (anyError(results)) {
+    return Failure(strings::join("\n\n", extractErrorMessages(results)));
+  }
+
+  // Here, it is possible that we return a default-initialized
+  // `AuthenticationResult`. Libprocess considers such a result invalid and will
+  // fail the HTTP request in that case. We allow it here to maintain existing
+  // behavior, in which libprocess is responsible for enforcing this constraint.
+  return Break(combinedResult);
+}
+
+
+Future<AuthenticationResult> CombinedAuthenticatorProcess::authenticate(
+    const Request& request)
+{
+  // Variables to hold the state of the authentication loop.
+  auto authenticator = authenticators.begin();
+  auto end = authenticators.end();
+  // Each pair contains a string representing the scheme of the authenticator
+  // and a `Try<AuthenticationResult>` which is used to capture failure messages
+  // in the event that an authenticator returns a failed future.
+  list<SchemeResultPair> results;
+
+  UPID self_ = self();
+
+  // Loop over all installed authenticators.
+  return loop(
+      self(),
+      [authenticator]() mutable {
+        return authenticator++;
+      },
+      [request, results, end, self_](
+          vector<Owned<Authenticator>>::const_iterator authenticator) mutable
+              -> Future<ControlFlow<AuthenticationResult>> {
+        // All authentication attempts have failed. Combine them and return.
+        if (authenticator == end) {
+          return combineFailed(results);
+        }
+
+        return authenticator->get()->authenticate(request)
+          .then(defer(
+              self_,
+              [&results, authenticator](const AuthenticationResult& result)
+                  -> ControlFlow<AuthenticationResult> {
+                // Validate that exactly 1 member is set.
+                size_t count =
+                  (result.principal.isSome()    ? 1 : 0) +
+                  (result.unauthorized.isSome() ? 1 : 0) +
+                  (result.forbidden.isSome()    ? 1 : 0);
+
+                if (count != 1) {
+                  LOG(WARNING) << "HTTP authenticator for scheme '"
+                               << authenticator->get()->scheme()
+                               << "' returned a result with " << count
+                               << " members set, which is an error";
+                  return Continue();
+                }
+
+                if (result.principal.isSome()) {
+                  // Authentication successful; break and return the result.
+                  return Break(result);
+                }
+
+                // Authentication unsuccessful; append the result and continue.
+                results.push_back(make_pair(
+                    authenticator->get()->scheme(),
+                    result));
+                return Continue();
+              }))
+          .repair([&results, authenticator](
+              const Future<ControlFlow<AuthenticationResult>>& failedResult)
+                  -> ControlFlow<AuthenticationResult> {
+            results.push_back(make_pair(
+                authenticator->get()->scheme(),
+                Error(failedResult.failure())));
+            return Continue();
+          });
+      });
+}
+
+
+CombinedAuthenticator::CombinedAuthenticator(
+    const string& _realm,
+    vector<Owned<Authenticator>>&& _authenticators)
+{
+  // Initialize the set of offered authentication schemes.
+  foreach (const Owned<Authenticator>& authenticator, _authenticators) {
+    schemes.insert(authenticator->scheme());
+  }
+
+  process = Owned<CombinedAuthenticatorProcess>(
+      new CombinedAuthenticatorProcess(_realm, std::move(_authenticators)));
+
+  spawn(process.get());
+}
+
+
+CombinedAuthenticator::~CombinedAuthenticator()
+{
+  terminate(process.get());
+  wait(process.get());
+}
+
+
+Future<AuthenticationResult> CombinedAuthenticator::authenticate(
+    const Request& request)
+{
+  return dispatch(
+      process.get(), &CombinedAuthenticatorProcess::authenticate, request);
+}
+
+
+string CombinedAuthenticator::scheme() const
+{
+  return strings::join(" ", schemes);
+}
+
+} // namespace authentication {
+} // namespace http {
+} // namespace mesos {


[3/3] mesos git commit: Added a test for the 'CombinedAuthenticator'.

Posted by vi...@apache.org.
Added a test for the 'CombinedAuthenticator'.

This patch adds a unit test to verify the functionality of
the `CombinedAuthenticator`. The new test is called
`CombinedAuthenticatorTest.MultipleAuthenticators`.

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


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

Branch: refs/heads/master
Commit: a68e1f28d92e2f9b0c1ea3a6701b041709aaad8a
Parents: af5af3f
Author: Greg Mann <gr...@mesosphere.io>
Authored: Wed Mar 15 19:36:01 2017 +0100
Committer: Vinod Kone <vi...@gmail.com>
Committed: Wed Mar 15 19:36:01 2017 +0100

----------------------------------------------------------------------
 src/tests/http_authentication_tests.cpp | 306 ++++++++++++++++++++++++++-
 1 file changed, 303 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a68e1f28/src/tests/http_authentication_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/http_authentication_tests.cpp b/src/tests/http_authentication_tests.cpp
index 0eeed9d..36d2b73 100644
--- a/src/tests/http_authentication_tests.cpp
+++ b/src/tests/http_authentication_tests.cpp
@@ -19,6 +19,7 @@
 #include <vector>
 
 #include <mesos/authentication/http/basic_authenticator_factory.hpp>
+#include <mesos/authentication/http/combined_authenticator.hpp>
 
 #include <mesos/module/http_authenticator.hpp>
 
@@ -33,6 +34,7 @@
 
 using namespace process;
 
+using std::string;
 using std::vector;
 
 namespace process {
@@ -52,6 +54,27 @@ bool operator==(const Unauthorized &left, const Unauthorized &right)
 }
 
 
+bool operator==(const URL &left, const URL &right)
+{
+  return left.scheme == right.scheme &&
+         left.domain == right.domain &&
+         left.ip == right.ip &&
+         left.path == right.path &&
+         left.query == right.query &&
+         left.fragment == right.fragment &&
+         left.port == right.port;
+}
+
+
+bool operator==(const Request &left, const Request &right)
+{
+  return left.headers == right.headers &&
+         left.url == right.url &&
+         left.method == right.method &&
+         left.body == right.body;
+}
+
+
 namespace authentication {
 
 bool operator==(
@@ -73,19 +96,21 @@ namespace internal {
 namespace tests {
 
 using mesos::http::authentication::BasicAuthenticatorFactory;
+using mesos::http::authentication::CombinedAuthenticator;
 
+using process::http::Forbidden;
 using process::http::Request;
 using process::http::Unauthorized;
 
 using process::http::authentication::Authenticator;
-using process::http::authentication::Principal;
 using process::http::authentication::AuthenticationResult;
+using process::http::authentication::Principal;
 
 
-static const std::string REALM = "tatooine";
+static const string REALM = "tatooine";
 
 static Parameters createBasicAuthenticatorParameters(
-    const Option<std::string>& realm,
+    const Option<string>& realm,
     const Option<Credentials>& credentials)
 {
   Parameters parameters;
@@ -235,6 +260,281 @@ TYPED_TEST(HttpAuthenticationTest, BasicWithoutRealm)
   ASSERT_ERROR(create);
 }
 
+
+class MockAuthenticator : public Authenticator
+{
+public:
+  MockAuthenticator(string scheme) : mockScheme(scheme) {}
+
+  MockAuthenticator(const MockAuthenticator& authenticator)
+    : mockScheme(authenticator.mockScheme) {}
+
+  MOCK_METHOD1(authenticate, Future<AuthenticationResult>(const Request&));
+
+  virtual string scheme() const { return mockScheme; }
+
+private:
+  const string mockScheme;
+};
+
+
+AuthenticationResult createUnauthorized(MockAuthenticator& authenticator)
+{
+  AuthenticationResult result;
+  result.unauthorized = Unauthorized(
+      {authenticator.scheme() + " realm=\"" + REALM + "\""},
+      authenticator.scheme() + " unauthorized");
+
+  return result;
+}
+
+
+AuthenticationResult createForbidden(MockAuthenticator& authenticator)
+{
+  AuthenticationResult result;
+  result.forbidden = Forbidden(authenticator.scheme() + " forbidden");
+
+  return result;
+}
+
+
+AuthenticationResult createCombinedUnauthorized(
+    vector<MockAuthenticator> authenticators)
+{
+  AuthenticationResult result;
+  vector<string> headers;
+  vector<string> bodies;
+
+  foreach (const MockAuthenticator& authenticator, authenticators) {
+    headers.push_back(authenticator.scheme() + " realm=\"" + REALM + "\"");
+    bodies.push_back(
+        "\"" + authenticator.scheme() + "\" authenticator returned:\n" +
+        authenticator.scheme() + " unauthorized");
+  }
+
+  result.unauthorized = Unauthorized(
+      {strings::join(",", headers)},
+      strings::join("\n\n", bodies));
+
+  return result;
+}
+
+
+AuthenticationResult createCombinedForbidden(
+    vector<MockAuthenticator> authenticators)
+{
+  AuthenticationResult result;
+  vector<string> bodies;
+
+  foreach (const MockAuthenticator& authenticator, authenticators) {
+    bodies.push_back(
+        "\"" + authenticator.scheme() + "\" authenticator returned:\n" +
+        authenticator.scheme() + " forbidden");
+  }
+
+  result.forbidden = Forbidden(strings::join("\n\n", bodies));
+
+  return result;
+}
+
+
+// Verifies the functionality of the `CombinedAuthenticator`.
+//
+// Note: This test relies on the order of invocation of the installed
+// authenticators. If the `CombinedAuthenticator` is changed in the future to
+// call them in a different order, this test must be udpated.
+TEST(CombinedAuthenticatorTest, MultipleAuthenticators)
+{
+  // Create two mock HTTP authenticators to install.
+  MockAuthenticator* basicAuthenticator = new MockAuthenticator("Basic");
+  MockAuthenticator* bearerAuthenticator = new MockAuthenticator("Bearer");
+
+  // Create a `CombinedAuthenticator` containing multiple authenticators.
+  Owned<Authenticator> combinedAuthenticator(
+      new CombinedAuthenticator(
+          REALM,
+          {
+            Owned<Authenticator>(basicAuthenticator),
+            Owned<Authenticator>(bearerAuthenticator)
+          }
+      ));
+
+  Request request;
+  request.headers.put(
+      "Authorization",
+      "Basic " + base64::encode("user:password"));
+
+  // The first authenticator succeeds.
+  {
+    AuthenticationResult successfulResult;
+    successfulResult.principal = Principal("user");
+
+    EXPECT_CALL(*basicAuthenticator, authenticate(request))
+      .WillOnce(Return(successfulResult));
+
+    Future<AuthenticationResult> result =
+      combinedAuthenticator->authenticate(request);
+    AWAIT_EXPECT_EQ(successfulResult, result);
+  }
+
+  // The first authenticator fails but the second one succeeds.
+  {
+    AuthenticationResult successfulResult;
+    successfulResult.principal = Principal("user");
+
+    EXPECT_CALL(*basicAuthenticator, authenticate(request))
+      .WillOnce(Return(createUnauthorized(*basicAuthenticator)));
+    EXPECT_CALL(*bearerAuthenticator, authenticate(request))
+      .WillOnce(Return(successfulResult));
+
+    Future<AuthenticationResult> result =
+      combinedAuthenticator->authenticate(request);
+    AWAIT_EXPECT_EQ(successfulResult, result);
+  }
+
+  // Two Unauthorized results.
+  {
+    EXPECT_CALL(*basicAuthenticator, authenticate(request))
+      .WillOnce(Return(createUnauthorized(*basicAuthenticator)));
+    EXPECT_CALL(*bearerAuthenticator, authenticate(request))
+      .WillOnce(Return(createUnauthorized(*bearerAuthenticator)));
+
+    Future<AuthenticationResult> result =
+      combinedAuthenticator->authenticate(request);
+    AWAIT_EXPECT_EQ(
+        createCombinedUnauthorized({*basicAuthenticator, *bearerAuthenticator}),
+        result);
+  }
+
+  // One Unauthorized and one Forbidden result.
+  {
+    EXPECT_CALL(*basicAuthenticator, authenticate(request))
+      .WillOnce(Return(createUnauthorized(*basicAuthenticator)));
+    EXPECT_CALL(*bearerAuthenticator, authenticate(request))
+      .WillOnce(Return(createForbidden(*bearerAuthenticator)));
+
+    Future<AuthenticationResult> result =
+      combinedAuthenticator->authenticate(request);
+    AWAIT_EXPECT_EQ(createCombinedUnauthorized({*basicAuthenticator}), result);
+  }
+
+  // Two Forbidden results.
+  {
+    EXPECT_CALL(*basicAuthenticator, authenticate(request))
+      .WillOnce(Return(createForbidden(*basicAuthenticator)));
+    EXPECT_CALL(*bearerAuthenticator, authenticate(request))
+      .WillOnce(Return(createForbidden(*bearerAuthenticator)));
+
+    Future<AuthenticationResult> result =
+      combinedAuthenticator->authenticate(request);
+    AWAIT_EXPECT_EQ(
+        createCombinedForbidden({*basicAuthenticator, *bearerAuthenticator}),
+        result);
+  }
+
+  // Two empty results.
+  {
+    AuthenticationResult emptyResult;
+
+    EXPECT_CALL(*basicAuthenticator, authenticate(request))
+      .WillOnce(Return(emptyResult));
+    EXPECT_CALL(*bearerAuthenticator, authenticate(request))
+      .WillOnce(Return(emptyResult));
+
+    Future<AuthenticationResult> result =
+      combinedAuthenticator->authenticate(request);
+    AWAIT_EXPECT_EQ(emptyResult, result);
+  }
+
+  // One empty and one Unauthorized result.
+  {
+    AuthenticationResult emptyResult;
+
+    EXPECT_CALL(*basicAuthenticator, authenticate(request))
+      .WillOnce(Return(emptyResult));
+    EXPECT_CALL(*bearerAuthenticator, authenticate(request))
+      .WillOnce(Return(createUnauthorized(*bearerAuthenticator)));
+
+    Future<AuthenticationResult> result =
+      combinedAuthenticator->authenticate(request);
+    AWAIT_EXPECT_EQ(createCombinedUnauthorized({*bearerAuthenticator}), result);
+  }
+
+  // One empty and one successful result.
+  {
+    AuthenticationResult emptyResult;
+    AuthenticationResult successfulResult;
+    successfulResult.principal = Principal("user");
+
+    EXPECT_CALL(*basicAuthenticator, authenticate(request))
+      .WillOnce(Return(emptyResult));
+    EXPECT_CALL(*bearerAuthenticator, authenticate(request))
+      .WillOnce(Return(successfulResult));
+
+    Future<AuthenticationResult> result =
+      combinedAuthenticator->authenticate(request);
+    AWAIT_EXPECT_EQ(successfulResult, result);
+  }
+
+  // Two failed futures.
+  {
+    Future<AuthenticationResult> failedResult(Failure("Failed result"));
+
+    EXPECT_CALL(*basicAuthenticator, authenticate(request))
+      .WillOnce(Return(failedResult));
+    EXPECT_CALL(*bearerAuthenticator, authenticate(request))
+      .WillOnce(Return(failedResult));
+
+    Future<AuthenticationResult> result =
+      combinedAuthenticator->authenticate(request);
+    AWAIT_EXPECT_FAILED(result);
+  }
+
+  // One failed future and one Unauthorized result.
+  {
+    Future<AuthenticationResult> failedResult(Failure("Failed result"));
+
+    EXPECT_CALL(*basicAuthenticator, authenticate(request))
+      .WillOnce(Return(failedResult));
+    EXPECT_CALL(*bearerAuthenticator, authenticate(request))
+      .WillOnce(Return(createUnauthorized(*bearerAuthenticator)));
+
+    Future<AuthenticationResult> result =
+      combinedAuthenticator->authenticate(request);
+    AWAIT_EXPECT_EQ(createCombinedUnauthorized({*bearerAuthenticator}), result);
+  }
+
+  // One failed future and one Forbidden result.
+  {
+    Future<AuthenticationResult> failedResult(Failure("Failed result"));
+
+    EXPECT_CALL(*basicAuthenticator, authenticate(request))
+      .WillOnce(Return(failedResult));
+    EXPECT_CALL(*bearerAuthenticator, authenticate(request))
+      .WillOnce(Return(createForbidden(*bearerAuthenticator)));
+
+    Future<AuthenticationResult> result =
+      combinedAuthenticator->authenticate(request);
+    AWAIT_EXPECT_EQ(createCombinedForbidden({*bearerAuthenticator}), result);
+  }
+
+  // One failed future and one successful result.
+  {
+    Future<AuthenticationResult> failedResult(Failure("Failed result"));
+    AuthenticationResult successfulResult;
+    successfulResult.principal = Principal("user");
+
+    EXPECT_CALL(*basicAuthenticator, authenticate(request))
+      .WillOnce(Return(failedResult));
+    EXPECT_CALL(*bearerAuthenticator, authenticate(request))
+      .WillOnce(Return(successfulResult));
+
+    Future<AuthenticationResult> result =
+      combinedAuthenticator->authenticate(request);
+    AWAIT_EXPECT_EQ(successfulResult, result);
+  }
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[2/3] mesos git commit: Enabled loading multiple HTTP authenticators in Mesos.

Posted by vi...@apache.org.
Enabled loading multiple HTTP authenticators in Mesos.

This patch updates the Mesos code to allow master and agent
to load multiple HTTP authenticator modules.

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


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

Branch: refs/heads/master
Commit: af5af3f9f8e2d736f3aaa244a4e254230a8467f0
Parents: 0804be3
Author: Greg Mann <gr...@mesosphere.io>
Authored: Wed Mar 15 19:35:55 2017 +0100
Committer: Vinod Kone <vi...@gmail.com>
Committed: Wed Mar 15 19:35:55 2017 +0100

----------------------------------------------------------------------
 src/common/http.cpp | 127 +++++++++++++++++++++++++++++++----------------
 1 file changed, 83 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/af5af3f9/src/common/http.cpp
----------------------------------------------------------------------
diff --git a/src/common/http.cpp b/src/common/http.cpp
index 0848f70..ce32ff3 100644
--- a/src/common/http.cpp
+++ b/src/common/http.cpp
@@ -26,6 +26,7 @@
 #include <mesos/resources.hpp>
 
 #include <mesos/authentication/http/basic_authenticator_factory.hpp>
+#include <mesos/authentication/http/combined_authenticator.hpp>
 #include <mesos/authorizer/authorizer.hpp>
 #include <mesos/module/http_authenticator.hpp>
 
@@ -63,6 +64,7 @@ using process::http::authentication::Principal;
 using process::http::authorization::AuthorizationCallbacks;
 
 using mesos::http::authentication::BasicAuthenticatorFactory;
+using mesos::http::authentication::CombinedAuthenticator;
 
 namespace mesos {
 
@@ -938,72 +940,109 @@ bool approveViewRole(
   return approved.get();
 }
 
+namespace {
 
-Try<Nothing> initializeHttpAuthenticators(
+Result<Authenticator*> createBasicAuthenticator(
     const string& realm,
-    const vector<string>& httpAuthenticatorNames,
+    const string& authenticatorName,
     const Option<Credentials>& credentials)
 {
-  if (httpAuthenticatorNames.empty()) {
-    return Error("No HTTP authenticator specified for realm '" + realm + "'");
+  if (credentials.isNone()) {
+    return Error(
+        "No credentials provided for the default '" +
+        string(internal::DEFAULT_HTTP_AUTHENTICATOR) +
+        "' HTTP authenticator for realm '" + realm + "'");
   }
 
-  if (httpAuthenticatorNames.size() > 1) {
-    return Error("Multiple HTTP authenticators not supported");
+  LOG(INFO) << "Creating default '" << internal::DEFAULT_HTTP_AUTHENTICATOR
+            << "' HTTP authenticator for realm '" << realm << "'";
+
+  return BasicAuthenticatorFactory::create(realm, credentials.get());
+}
+
+
+Result<Authenticator*> createCustomAuthenticator(
+    const string& realm,
+    const string& authenticatorName)
+{
+  if (!modules::ModuleManager::contains<Authenticator>(authenticatorName)) {
+    return Error(
+        "HTTP authenticator '" + authenticatorName + "' not found. "
+        "Check the spelling (compare to '" +
+        string(internal::DEFAULT_HTTP_AUTHENTICATOR) +
+        "') or verify that the authenticator was loaded "
+        "successfully (see --modules)");
   }
 
-  Option<Authenticator*> httpAuthenticator;
-  if (httpAuthenticatorNames[0] == internal::DEFAULT_HTTP_AUTHENTICATOR) {
-    if (credentials.isNone()) {
-      return Error(
-          "No credentials provided for the default '" +
-          string(internal::DEFAULT_HTTP_AUTHENTICATOR) +
-          "' HTTP authenticator for realm '" + realm + "'");
-    }
+  LOG(INFO) << "Creating '" << authenticatorName << "' HTTP authenticator "
+            << "for realm '" << realm << "'";
 
-    LOG(INFO) << "Using default '" << internal::DEFAULT_HTTP_AUTHENTICATOR
-              << "' HTTP authenticator for realm '" << realm << "'";
+  return modules::ModuleManager::create<Authenticator>(authenticatorName);
+}
 
-    Try<Authenticator*> authenticator =
-      BasicAuthenticatorFactory::create(realm, credentials.get());
-    if (authenticator.isError()) {
-      return Error(
-          "Could not create HTTP authenticator module '" +
-          httpAuthenticatorNames[0] + "': " + authenticator.error());
+} // namespace {
+
+Try<Nothing> initializeHttpAuthenticators(
+    const string& realm,
+    const vector<string>& authenticatorNames,
+    const Option<Credentials>& credentials)
+{
+  if (authenticatorNames.empty()) {
+    return Error(
+        "No HTTP authenticators specified for realm '" + realm + "'");
+  }
+
+  Option<Authenticator*> authenticator;
+
+  if (authenticatorNames.size() == 1) {
+    Result<Authenticator*> authenticator_ = None();
+    if (authenticatorNames[0] == internal::DEFAULT_HTTP_AUTHENTICATOR) {
+      authenticator_ =
+        createBasicAuthenticator(realm, authenticatorNames[0], credentials);
+    } else {
+      authenticator_ = createCustomAuthenticator(realm, authenticatorNames[0]);
     }
 
-    httpAuthenticator = authenticator.get();
-  } else {
-    if (!modules::ModuleManager::contains<Authenticator>(
-          httpAuthenticatorNames[0])) {
+    if (authenticator_.isError()) {
       return Error(
-          "HTTP authenticator '" + httpAuthenticatorNames[0] +
-          "' not found. Check the spelling (compare to '" +
-          string(internal::DEFAULT_HTTP_AUTHENTICATOR) +
-          "') or verify that the authenticator was loaded "
-          "successfully (see --modules)");
+          "Failed to create HTTP authenticator module '" +
+          authenticatorNames[0] + "': " + authenticator_.error());
     }
 
-    Try<Authenticator*> module =
-      modules::ModuleManager::create<Authenticator>(httpAuthenticatorNames[0]);
-    if (module.isError()) {
-      return Error(
-          "Could not create HTTP authenticator module '" +
-          httpAuthenticatorNames[0] + "': " + module.error());
+    CHECK_SOME(authenticator_);
+    authenticator = authenticator_.get();
+  } else {
+    // There are multiple authenticators loaded for this realm,
+    // so construct a `CombinedAuthenticator` to handle them.
+    vector<Owned<Authenticator>> authenticators;
+    foreach (const string& name, authenticatorNames) {
+      Result<Authenticator*> authenticator_ = None();
+      if (name == internal::DEFAULT_HTTP_AUTHENTICATOR) {
+        authenticator_ = createBasicAuthenticator(realm, name, credentials);
+      } else {
+        authenticator_ = createCustomAuthenticator(realm, name);
+      }
+
+      if (authenticator_.isError()) {
+        return Error(
+            "Failed to create HTTP authenticator module '" +
+            name + "': " + authenticator_.error());
+      }
+
+      CHECK_SOME(authenticator_);
+      authenticators.push_back(Owned<Authenticator>(authenticator_.get()));
     }
-    LOG(INFO) << "Using '" << httpAuthenticatorNames[0]
-              << "' HTTP authenticator for realm '" << realm << "'";
-    httpAuthenticator = module.get();
+
+    authenticator = new CombinedAuthenticator(realm, std::move(authenticators));
   }
 
-  CHECK(httpAuthenticator.isSome());
+  CHECK(authenticator.isSome());
 
-  // Ownership of the `httpAuthenticator` is passed to libprocess.
+  // Ownership of the authenticator is passed to libprocess.
   process::http::authentication::setAuthenticator(
-    realm, Owned<Authenticator>(httpAuthenticator.get()));
+      realm, Owned<Authenticator>(authenticator.get()));
 
   return Nothing();
 }
 
-
 }  // namespace mesos {