You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ti...@apache.org on 2017/10/10 18:59:27 UTC

[1/4] mesos git commit: Updated scheduler library for modularized HTTP authenticatee use.

Repository: mesos
Updated Branches:
  refs/heads/master a3faf6cf8 -> 28f882795


Updated scheduler library for modularized HTTP authenticatee use.

Allows the scheduler library to load an HTTP authenticatee module
providing custom mechanisms for authentication.

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


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

Branch: refs/heads/master
Commit: 28f882795014ca4f47e3efc99a41453f5a3b68a9
Parents: e89cdc2
Author: Till Toenshoff <to...@me.com>
Authored: Tue Oct 10 20:17:51 2017 +0200
Committer: Till Toenshoff <to...@me.com>
Committed: Tue Oct 10 20:59:13 2017 +0200

----------------------------------------------------------------------
 src/scheduler/flags.hpp     |  83 +++++++++++++++++++++++--
 src/scheduler/scheduler.cpp | 130 +++++++++++++++++++++++++++++----------
 2 files changed, 176 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/28f88279/src/scheduler/flags.hpp
----------------------------------------------------------------------
diff --git a/src/scheduler/flags.hpp b/src/scheduler/flags.hpp
index 1e8efc0..f567224 100644
--- a/src/scheduler/flags.hpp
+++ b/src/scheduler/flags.hpp
@@ -17,6 +17,12 @@
 #ifndef __SCHEDULER_FLAGS_HPP__
 #define __SCHEDULER_FLAGS_HPP__
 
+#include <string>
+
+#include <mesos/mesos.hpp>
+
+#include "common/parse.hpp"
+
 #include "local/flags.hpp"
 
 #include "scheduler/constants.hpp"
@@ -32,14 +38,83 @@ public:
   {
     add(&Flags::connectionDelayMax,
         "connection_delay_max",
-        "The maximum amount of time to wait before trying to initiate a "
-        "connection with the master. The library waits for a random amount of "
-        "time between [0, b], where `b = connection_delay_max` before "
-        "initiating a (re-)connection attempt with the master",
+        "The maximum amount of time to wait before trying to initiate a\n"
+        "connection with the master. The library waits for a random amount of\n"
+        "time between [0, b], where `b = connection_delay_max` before\n"
+        "initiating a (re-)connection attempt with the master.",
         DEFAULT_CONNECTION_DELAY_MAX);
+
+    add(&Flags::httpAuthenticatee,
+        "http_authenticatee",
+        "HTTP authenticatee implementation to use when authenticating against\n"
+        "the master. Use the default '" +
+          std::string(mesos::internal::DEFAULT_BASIC_HTTP_AUTHENTICATEE) +
+          "' or load an alternate\n"
+        "authenticatee module using MESOS_MODULES.",
+        mesos::internal::DEFAULT_BASIC_HTTP_AUTHENTICATEE);
+
+    // This help message for --modules flag is the same for
+    // {master,slave,sched,tests}/flags.[ch]pp and should always be kept
+    // in sync.
+    // TODO(karya): Remove the JSON example and add reference to the
+    // doc file explaining the --modules flag.
+    add(&Flags::modules,
+        "modules",
+        "List of modules to be loaded and be available to the internal\n"
+        "subsystems.\n"
+        "\n"
+        "Use --modules=filepath to specify the list of modules via a\n"
+        "file containing a JSON formatted string. 'filepath' can be\n"
+        "of the form 'file:///path/to/file' or '/path/to/file'.\n"
+        "\n"
+        "Use --modules=\"{...}\" to specify the list of modules inline.\n"
+        "\n"
+        "Example:\n"
+        "{\n"
+        "  \"libraries\": [\n"
+        "    {\n"
+        "      \"file\": \"/path/to/libfoo.so\",\n"
+        "      \"modules\": [\n"
+        "        {\n"
+        "          \"name\": \"org_apache_mesos_bar\",\n"
+        "          \"parameters\": [\n"
+        "            {\n"
+        "              \"key\": \"X\",\n"
+        "              \"value\": \"Y\"\n"
+        "            }\n"
+        "          ]\n"
+        "        },\n"
+        "        {\n"
+        "          \"name\": \"org_apache_mesos_baz\"\n"
+        "        }\n"
+        "      ]\n"
+        "    },\n"
+        "    {\n"
+        "      \"name\": \"qux\",\n"
+        "      \"modules\": [\n"
+        "        {\n"
+        "          \"name\": \"org_apache_mesos_norf\"\n"
+        "        }\n"
+        "      ]\n"
+        "    }\n"
+        "  ]\n"
+        "}");
+
+    // This help message for --modules_dir flag is the same for
+    // {master,slave,sched,tests}/flags.[ch]pp and should always be kept
+    // in sync.
+    add(&Flags::modulesDir,
+        "modules_dir",
+        "Directory path of the module manifest files.\n"
+        "The manifest files are processed in alphabetical order.\n"
+        "(See --modules for more information on module manifest files).\n"
+        "Cannot be used in conjunction with --modules.\n");
   }
 
   Duration connectionDelayMax;
+  Option<Modules> modules;
+  Option<std::string> modulesDir;
+  std::string httpAuthenticatee;
 };
 
 } // namespace scheduler {

http://git-wip-us.apache.org/repos/asf/mesos/blob/28f88279/src/scheduler/scheduler.cpp
----------------------------------------------------------------------
diff --git a/src/scheduler/scheduler.cpp b/src/scheduler/scheduler.cpp
index 78f5370..fe37454 100644
--- a/src/scheduler/scheduler.cpp
+++ b/src/scheduler/scheduler.cpp
@@ -41,6 +41,8 @@
 
 #include <mesos/master/detector.hpp>
 
+#include <mesos/module/http_authenticatee.hpp>
+
 #include <process/async.hpp>
 #include <process/collect.hpp>
 #include <process/defer.hpp>
@@ -73,6 +75,8 @@
 #include <stout/unreachable.hpp>
 #include <stout/uuid.hpp>
 
+#include "authentication/http/basic_authenticatee.hpp"
+
 #include "common/http.hpp"
 #include "common/recordio.hpp"
 
@@ -87,6 +91,8 @@
 
 #include "messages/messages.hpp"
 
+#include "module/manager.hpp"
+
 #include "scheduler/flags.hpp"
 
 using namespace mesos;
@@ -254,37 +260,9 @@ public:
     request.headers = {{"Accept", stringify(contentType)},
                        {"Content-Type", stringify(contentType)}};
 
-    // TODO(anand): Add support for other authentication schemes.
-
-    if (credential.isSome()) {
-      request.headers["Authorization"] =
-        "Basic " +
-        base64::encode(credential->principal() + ":" + credential->secret());
-    }
-
-    CHECK_SOME(connections);
-
-    Future<Response> response;
-    if (call.type() == Call::SUBSCRIBE) {
-      state = SUBSCRIBING;
-
-      // Send a streaming request for Subscribe call.
-      response = connections->subscribe.send(request, true);
-    } else {
-      CHECK_SOME(streamId);
-
-      // Set the stream ID associated with this connection.
-      request.headers["Mesos-Stream-Id"] = streamId->toString();
-
-      response = connections->nonSubscribe.send(request);
-    }
-
-    CHECK_SOME(connectionId);
-    response.onAny(defer(self(),
-                         &Self::_send,
-                         connectionId.get(),
-                         call,
-                         lambda::_1));
+    // TODO(tillt): Add support for multi-step authentication protocols.
+    authenticatee->authenticate(request, credential)
+      .onAny(defer(self(), &Self::_send, call, lambda::_1));
   }
 
   void reconnect()
@@ -307,6 +285,54 @@ public:
 protected:
   virtual void initialize()
   {
+    // Initialize modules.
+    if (flags.modules.isSome() && flags.modulesDir.isSome()) {
+      EXIT(EXIT_FAILURE) << "Only one of MESOS_MODULES or MESOS_MODULES_DIR "
+                         << "should be specified";
+    }
+
+    if (flags.modulesDir.isSome()) {
+      Try<Nothing> result =
+        modules::ModuleManager::load(flags.modulesDir.get());
+
+      if (result.isError()) {
+        EXIT(EXIT_FAILURE) << "Error loading modules: " << result.error();
+      }
+    }
+
+    if (flags.modules.isSome()) {
+      Try<Nothing> result = modules::ModuleManager::load(flags.modules.get());
+
+      if (result.isError()) {
+        EXIT(EXIT_FAILURE) << "Error loading modules: " << result.error();
+      }
+    }
+
+    // Initialize authenticatee.
+    if (flags.httpAuthenticatee == DEFAULT_BASIC_HTTP_AUTHENTICATEE) {
+      LOG(INFO) << "Using default '" << DEFAULT_BASIC_HTTP_AUTHENTICATEE
+                << "' HTTP authenticatee";
+
+      authenticatee = Owned<mesos::http::authentication::Authenticatee>(
+          new mesos::http::authentication::BasicAuthenticatee);
+    } else {
+      LOG(INFO) << "Using '" << flags.httpAuthenticatee
+                << "' HTTP authenticatee";
+
+      Try<mesos::http::authentication::Authenticatee*> createdAuthenticatee =
+        modules::ModuleManager::create<
+            mesos::http::authentication::Authenticatee>(
+                flags.httpAuthenticatee);
+
+      if (createdAuthenticatee.isError()) {
+        EXIT(EXIT_FAILURE) << "Failed to load HTTP authenticatee: "
+                           << createdAuthenticatee.error();
+      }
+
+      authenticatee = Owned<mesos::http::authentication::Authenticatee>(
+          createdAuthenticatee.get());
+    }
+
     // Start detecting masters.
     detection = detector->detect()
       .onAny(defer(self(), &MesosProcess::detected, lambda::_1));
@@ -513,7 +539,43 @@ protected:
     LOG(WARNING) << "Dropping " << call.type() << ": " << message;
   }
 
-  void _send(
+  void _send(const Call& call, const Future<::Request>& future)
+  {
+    if (!future.isReady()) {
+      LOG(ERROR) << "HTTP authenticatee "
+                 << (future.isFailed() ? "failed: " + future.failure()
+                                       : "discarded");
+      return;
+    }
+
+    ::Request request = future.get();
+
+    CHECK_SOME(connections);
+
+    Future<Response> response;
+    if (call.type() == Call::SUBSCRIBE) {
+      state = SUBSCRIBING;
+
+      // Send a streaming request for Subscribe call.
+      response = connections->subscribe.send(request, true);
+    } else {
+      CHECK_SOME(streamId);
+
+      // Set the stream ID associated with this connection.
+      request.headers["Mesos-Stream-Id"] = streamId->toString();
+
+      response = connections->nonSubscribe.send(request);
+    }
+
+    CHECK_SOME(connectionId);
+    response.onAny(defer(self(),
+                         &Self::__send,
+                         connectionId.get(),
+                         call,
+                         lambda::_1));
+  }
+
+  void __send(
       const UUID& _connectionId,
       const Call& call,
       const Future<Response>& response)
@@ -539,7 +601,7 @@ protected:
     if (response->code == process::http::Status::OK) {
       // Only SUBSCRIBE call should get a "200 OK" response.
       CHECK_EQ(Call::SUBSCRIBE, call.type());
-      CHECK_EQ(response->type, http::Response::PIPE);
+      CHECK_EQ(response->type, process::http::Response::PIPE);
       CHECK_SOME(response->reader);
 
       state = SUBSCRIBED;
@@ -784,6 +846,8 @@ private:
   Option<UUID> streamId;
   const Flags flags;
 
+  Owned<mesos::http::authentication::Authenticatee> authenticatee;
+
   // Master detection future.
   process::Future<Option<mesos::MasterInfo>> detection;
 };


[4/4] mesos git commit: Added HTTP authenticatee interface definition.

Posted by ti...@apache.org.
Added HTTP authenticatee interface definition.

Defines an interface for the scheduler library authentication,
providing the means of an HTTP authenticatee. This interface allows
consumers of HTTP APIs to use replaceable authentication mechanisms.

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


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

Branch: refs/heads/master
Commit: 185b64fa05194b4350fcadc1fce6c9b9620ed34b
Parents: a3faf6c
Author: Till Toenshoff <to...@me.com>
Authored: Tue Oct 10 20:17:31 2017 +0200
Committer: Till Toenshoff <to...@me.com>
Committed: Tue Oct 10 20:59:13 2017 +0200

----------------------------------------------------------------------
 .../mesos/authentication/http/authenticatee.hpp | 77 ++++++++++++++++++++
 src/Makefile.am                                 | 11 ++-
 2 files changed, 85 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/185b64fa/include/mesos/authentication/http/authenticatee.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/authentication/http/authenticatee.hpp b/include/mesos/authentication/http/authenticatee.hpp
new file mode 100644
index 0000000..00676fe
--- /dev/null
+++ b/include/mesos/authentication/http/authenticatee.hpp
@@ -0,0 +1,77 @@
+// 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_AUTHENTICATEE_HPP__
+#define __MESOS_AUTHENTICATION_HTTP_AUTHENTICATEE_HPP__
+
+#include <mesos/v1/mesos.hpp>
+
+#include <process/future.hpp>
+#include <process/http.hpp>
+
+#include <stout/option.hpp>
+
+namespace mesos {
+namespace http {
+namespace authentication {
+
+/**
+ * An abstraction enabling any HTTP API consumer to authenticate.
+ */
+class Authenticatee
+{
+public:
+  virtual ~Authenticatee() = default;
+
+  /**
+   * Reset the authenticatee to its initial state.
+   *
+   * Allows the implementation to invalidate possibly cached state.
+   * This is useful if for example token-based authentication is
+   * performed and the authenticator signaled an expired token.
+   */
+  virtual void reset() {};
+
+  /**
+   * Name of the authentication scheme implemented.
+   *
+   * @return Authentication scheme.
+   */
+  virtual std::string scheme() const = 0;
+
+  /**
+   * Create an HTTP request for authentication.
+   *
+   * Used for mutating a provided `Request` with any means of
+   * authentication-related headers or other additions and changes.
+   *
+   * @param request The HTTP payload intended to be sent to an
+   *     authenticated endpoint.
+   * @param credential The principal and secret optionally used to
+   *     create the authentication request.
+   * @return The mutated HTTP request object containing all information
+   *     needed for authenticating.
+   */
+  virtual process::Future<process::http::Request> authenticate(
+      const process::http::Request& request,
+      const Option<mesos::v1::Credential>& credential) = 0;
+};
+
+} // namespace authentication {
+} // namespace http {
+} // namespace mesos {
+
+#endif // __MESOS_AUTHENTICATION_HTTP_AUTHENTICATEE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/185b64fa/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index da8af91..edfc16d 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -627,13 +627,18 @@ authentication_HEADERS =						\
   $(top_srcdir)/include/mesos/authentication/authentication.hpp		\
   $(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/combined_authenticator.hpp
+  $(top_srcdir)/include/mesos/authentication/secret_generator.hpp
 
 nodist_authentication_HEADERS =						\
   ../include/mesos/authentication/authentication.pb.h
 
+httpauthenticationdir = $(pkgincludedir)/authentication/http
+
+httpauthentication_HEADERS =						\
+  $(top_srcdir)/include/mesos/authentication/http/authenticatee.hpp	\
+  $(top_srcdir)/include/mesos/authentication/http/basic_authenticator_factory.hpp	\
+  $(top_srcdir)/include/mesos/authentication/http/combined_authenticator.hpp
+
 authorizerdir = $(pkgincludedir)/authorizer
 
 authorizer_HEADERS =							\


[3/4] mesos git commit: Added basic HTTP authenticatee implementation.

Posted by ti...@apache.org.
Added basic HTTP authenticatee implementation.

Moves the hardcoded basic HTTP authentication code from within the
scheduler library into the modularized HTTP authenticatee interface.

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


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

Branch: refs/heads/master
Commit: e89cdc2551c77c6951a30e9007c85cbfef58de32
Parents: a687d97
Author: Till Toenshoff <to...@me.com>
Authored: Tue Oct 10 20:17:46 2017 +0200
Committer: Till Toenshoff <to...@me.com>
Committed: Tue Oct 10 20:59:13 2017 +0200

----------------------------------------------------------------------
 src/CMakeLists.txt                              |   1 +
 src/Makefile.am                                 |   2 +
 src/authentication/http/basic_authenticatee.cpp | 103 +++++++++++++++++++
 src/authentication/http/basic_authenticatee.hpp |  66 ++++++++++++
 src/common/http.hpp                             |   3 +
 5 files changed, 175 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e89cdc25/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 1a0dff3..219252f 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -310,6 +310,7 @@ set(AUTHENTICATION_SRC
   authentication/cram_md5/authenticatee.cpp
   authentication/cram_md5/authenticator.cpp
   authentication/cram_md5/auxprop.cpp
+  authentication/http/basic_authenticatee.cpp
   authentication/http/basic_authenticator_factory.cpp
   authentication/http/combined_authenticator.cpp)
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/e89cdc25/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index ca43f25..085ff3b 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -941,6 +941,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   authentication/cram_md5/authenticatee.cpp				\
   authentication/cram_md5/authenticator.cpp				\
   authentication/cram_md5/auxprop.cpp					\
+  authentication/http/basic_authenticatee.cpp				\
   authentication/http/basic_authenticator_factory.cpp			\
   authentication/http/combined_authenticator.cpp			\
   authorizer/acls.cpp							\
@@ -1085,6 +1086,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   authentication/cram_md5/authenticatee.hpp				\
   authentication/cram_md5/authenticator.hpp				\
   authentication/cram_md5/auxprop.hpp					\
+  authentication/http/basic_authenticatee.hpp				\
   authorizer/local/authorizer.hpp					\
   checks/checker.hpp							\
   checks/checker_process.hpp						\

http://git-wip-us.apache.org/repos/asf/mesos/blob/e89cdc25/src/authentication/http/basic_authenticatee.cpp
----------------------------------------------------------------------
diff --git a/src/authentication/http/basic_authenticatee.cpp b/src/authentication/http/basic_authenticatee.cpp
new file mode 100644
index 0000000..5faeec6
--- /dev/null
+++ b/src/authentication/http/basic_authenticatee.cpp
@@ -0,0 +1,103 @@
+// 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/http/basic_authenticatee.hpp"
+
+#include <string>
+
+#include <mesos/v1/mesos.hpp>
+
+#include <process/id.hpp>
+#include <process/dispatch.hpp>
+#include <process/future.hpp>
+#include <process/http.hpp>
+#include <process/process.hpp>
+
+#include <stout/base64.hpp>
+#include <stout/option.hpp>
+
+namespace mesos {
+namespace http {
+namespace authentication {
+
+class BasicAuthenticateeProcess
+  : public process::Process<BasicAuthenticateeProcess>
+{
+public:
+  BasicAuthenticateeProcess()
+    : ProcessBase(process::ID::generate("basic_authenticatee")) {}
+
+  process::Future<process::http::Request> authenticate(
+      const process::http::Request& request,
+      const Option<mesos::v1::Credential>& credential)
+  {
+    // Without credential we can and should not try to authenticate.
+    if (credential.isNone()) {
+      return request;
+    }
+
+    process::http::Request decoratedRequest(request);
+
+    // As per https://tools.ietf.org/html/rfc7230#section-3.2.2 we
+    // must not return an additional "Authorization" header if there
+    // was one present already - for those cases, we need to combine
+    // them.
+    // TODO(tillt): Update this code to combine multiple schemes once
+    // our HTTP authenticator implementations do support multiple
+    // authentication schemes for the same request. See MESOS-8059.
+    decoratedRequest.headers["Authorization"] =
+      "Basic " +
+      base64::encode(credential->principal() + ":" + credential->secret());
+
+    return decoratedRequest;
+  }
+};
+
+
+BasicAuthenticatee::BasicAuthenticatee()
+  : process_(new BasicAuthenticateeProcess())
+{
+  spawn(*process_);
+}
+
+
+BasicAuthenticatee::~BasicAuthenticatee()
+{
+  terminate(*process_);
+  wait(*process_);
+}
+
+
+std::string BasicAuthenticatee::scheme() const
+{
+  return "Basic";
+}
+
+
+process::Future<process::http::Request> BasicAuthenticatee::authenticate(
+    const process::http::Request& request,
+    const Option<mesos::v1::Credential>& credential)
+{
+  return dispatch(
+      *process_,
+      &BasicAuthenticateeProcess::authenticate,
+      request,
+      credential);
+}
+
+} // namespace authentication {
+} // namespace http {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/e89cdc25/src/authentication/http/basic_authenticatee.hpp
----------------------------------------------------------------------
diff --git a/src/authentication/http/basic_authenticatee.hpp b/src/authentication/http/basic_authenticatee.hpp
new file mode 100644
index 0000000..59c1ba6
--- /dev/null
+++ b/src/authentication/http/basic_authenticatee.hpp
@@ -0,0 +1,66 @@
+// 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_HTTP_BASIC_AUTHENTICATEE_HPP__
+#define __AUTHENTICATION_HTTP_BASIC_AUTHENTICATEE_HPP__
+
+#include <mesos/v1/mesos.hpp>
+
+#include <mesos/authentication/http/authenticatee.hpp>
+
+#include <process/future.hpp>
+#include <process/http.hpp>
+
+#include <stout/option.hpp>
+
+namespace mesos {
+namespace http {
+namespace authentication {
+
+class BasicAuthenticateeProcess; // Forward declaration.
+
+/**
+ * Authenticatee implementing the client side of basic HTTP
+ * authentication.
+ */
+class BasicAuthenticatee : public Authenticatee
+{
+public:
+  BasicAuthenticatee();
+
+  ~BasicAuthenticatee() override;
+
+  // Not copy-constructable.
+  BasicAuthenticatee(const BasicAuthenticatee&) = delete;
+
+  // Not copyable.
+  BasicAuthenticatee& operator=(const BasicAuthenticatee&) = delete;
+
+  std::string scheme() const override;
+
+  process::Future<process::http::Request> authenticate(
+      const process::http::Request& request,
+      const Option<mesos::v1::Credential>& credential) override;
+
+private:
+  process::Owned<BasicAuthenticateeProcess> process_;
+};
+
+} // namespace authentication {
+} // namespace http {
+} // namespace mesos {
+
+#endif // __AUTHENTICATION_HTTP_BASIC_AUTHENTICATEE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/e89cdc25/src/common/http.hpp
----------------------------------------------------------------------
diff --git a/src/common/http.hpp b/src/common/http.hpp
index 0e6b1c5..505c6d7 100644
--- a/src/common/http.hpp
+++ b/src/common/http.hpp
@@ -48,6 +48,9 @@ namespace internal {
 // Name of the default, basic authenticator.
 constexpr char DEFAULT_BASIC_HTTP_AUTHENTICATOR[] = "basic";
 
+// Name of the default, basic authenticatee.
+constexpr char DEFAULT_BASIC_HTTP_AUTHENTICATEE[] = "basic";
+
 // Name of the default, JWT authenticator.
 constexpr char DEFAULT_JWT_HTTP_AUTHENTICATOR[] = "jwt";
 


[2/4] mesos git commit: Modularized HTTP authenticatee.

Posted by ti...@apache.org.
Modularized HTTP authenticatee.

Introduces an HTTP authenticatee module kind.

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


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

Branch: refs/heads/master
Commit: a687d975e41ce162cf5806278f3ca78a69f67571
Parents: 185b64f
Author: Till Toenshoff <to...@me.com>
Authored: Tue Oct 10 20:17:38 2017 +0200
Committer: Till Toenshoff <to...@me.com>
Committed: Tue Oct 10 20:59:13 2017 +0200

----------------------------------------------------------------------
 include/mesos/module/http_authenticatee.hpp | 64 ++++++++++++++++++++++++
 src/Makefile.am                             |  1 +
 src/module/manager.cpp                      |  1 +
 3 files changed, 66 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a687d975/include/mesos/module/http_authenticatee.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/module/http_authenticatee.hpp b/include/mesos/module/http_authenticatee.hpp
new file mode 100644
index 0000000..554b45b
--- /dev/null
+++ b/include/mesos/module/http_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 __MESOS_MODULE_HTTP_AUTHENTICATEE_HPP__
+#define __MESOS_MODULE_HTTP_AUTHENTICATEE_HPP__
+
+#include <mesos/mesos.hpp>
+#include <mesos/module.hpp>
+
+#include <mesos/authentication/http/authenticatee.hpp>
+
+namespace mesos {
+namespace modules {
+
+template <>
+inline const char* kind<mesos::http::authentication::Authenticatee>()
+{
+  return "HttpAuthenticatee";
+}
+
+
+template <>
+struct Module<mesos::http::authentication::Authenticatee> : ModuleBase
+{
+  Module(
+      const char* _moduleApiVersion,
+      const char* _mesosVersion,
+      const char* _authorName,
+      const char* _authorEmail,
+      const char* _description,
+      bool (*_compatible)(),
+      mesos::http::authentication::Authenticatee* (*_create)(
+          const Parameters& parameters))
+    : ModuleBase(
+        _moduleApiVersion,
+        _mesosVersion,
+        mesos::modules::kind<mesos::http::authentication::Authenticatee>(),
+        _authorName,
+        _authorEmail,
+        _description,
+        _compatible),
+      create(_create) {}
+
+  mesos::http::authentication::Authenticatee* (*create)(
+      const Parameters& parameters);
+};
+
+} // namespace modules {
+} // namespace mesos {
+
+#endif // __MESOS_MODULE_HTTP_AUTHENTICATEE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/a687d975/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index edfc16d..ca43f25 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -722,6 +722,7 @@ module_HEADERS =							\
   $(top_srcdir)/include/mesos/module/detector.hpp			\
   $(top_srcdir)/include/mesos/module/hook.hpp				\
   $(top_srcdir)/include/mesos/module/hook.proto				\
+  $(top_srcdir)/include/mesos/module/http_authenticatee.hpp		\
   $(top_srcdir)/include/mesos/module/http_authenticator.hpp		\
   $(top_srcdir)/include/mesos/module/isolator.hpp			\
   $(top_srcdir)/include/mesos/module/module.hpp				\

http://git-wip-us.apache.org/repos/asf/mesos/blob/a687d975/src/module/manager.cpp
----------------------------------------------------------------------
diff --git a/src/module/manager.cpp b/src/module/manager.cpp
index ec5614a..d93648a 100644
--- a/src/module/manager.cpp
+++ b/src/module/manager.cpp
@@ -72,6 +72,7 @@ void ModuleManager::initialize()
   kindToVersion["Authorizer"] = MESOS_VERSION;
   kindToVersion["ContainerLogger"] = MESOS_VERSION;
   kindToVersion["Hook"] = MESOS_VERSION;
+  kindToVersion["HttpAuthenticatee"] = MESOS_VERSION;
   kindToVersion["HttpAuthenticator"] = MESOS_VERSION;
   kindToVersion["Isolator"] = MESOS_VERSION;
   kindToVersion["MasterContender"] = MESOS_VERSION;