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 2016/07/01 17:40:39 UTC

[1/2] mesos git commit: Added validation for the `get_endpoints` ACL.

Repository: mesos
Updated Branches:
  refs/heads/master c1239fbd5 -> 926e8d25c


Added validation for the `get_endpoints` ACL.

The fact that not all endpoints can be secure through ACLs, yet
the ACL is called \`get_endpoints\`, can be confusing for operators.
Therefore, if an operator tries to launch an agent/master with an
invalid configuration an error is generated.

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


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

Branch: refs/heads/master
Commit: ec45ae5aeab53693442d836980dd95149079d54c
Parents: c1239fb
Author: Alexander Rojas <al...@mesosphere.io>
Authored: Fri Jul 1 10:39:21 2016 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Jul 1 10:40:17 2016 -0700

----------------------------------------------------------------------
 src/authorizer/local/authorizer.cpp  | 11 ++++++
 src/common/http.cpp                  | 59 ++++++++++++++++++++++++++++++-
 src/common/http.hpp                  | 17 +++++++++
 src/slave/http.cpp                   | 46 ++++++------------------
 src/slave/slave.hpp                  | 12 -------
 src/tests/authorization_tests.cpp    | 47 ++++++++++++++++++++++++
 src/tests/scheduler_driver_tests.cpp |  2 +-
 7 files changed, 144 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ec45ae5a/src/authorizer/local/authorizer.cpp
----------------------------------------------------------------------
diff --git a/src/authorizer/local/authorizer.cpp b/src/authorizer/local/authorizer.cpp
index aadb7f6..e5d4add 100644
--- a/src/authorizer/local/authorizer.cpp
+++ b/src/authorizer/local/authorizer.cpp
@@ -37,6 +37,7 @@
 #include <stout/try.hpp>
 
 #include "common/parse.hpp"
+#include "common/http.hpp"
 
 using process::dispatch;
 using process::Failure;
@@ -784,6 +785,16 @@ Option<Error> LocalAuthorizer::validate(const ACLs& acls)
     }
   }
 
+  foreach (const ACL::GetEndpoint& acl, acls.get_endpoints()) {
+    if (acl.paths().type() == ACL::Entity::SOME) {
+      foreach (const string& path, acl.paths().values()) {
+        if (!AUTHORIZABLE_ENDPOINTS.contains(path)) {
+          return Error("Path: '" + path + "' is not an authorizable path");
+        }
+      }
+    }
+  }
+
   // TODO(alexr): Consider validating not only protobuf, but also the original
   // JSON in order to spot misspelled names. A misspelled action may affect
   // authorization result and hence lead to a security issue (e.g. when there

http://git-wip-us.apache.org/repos/asf/mesos/blob/ec45ae5a/src/common/http.cpp
----------------------------------------------------------------------
diff --git a/src/common/http.cpp b/src/common/http.cpp
index 2ef264f..fffa24c 100644
--- a/src/common/http.cpp
+++ b/src/common/http.cpp
@@ -47,6 +47,7 @@ using std::set;
 using std::string;
 using std::vector;
 
+using process::Failure;
 using process::Owned;
 
 using process::http::authorization::AuthorizationCallbacks;
@@ -69,6 +70,16 @@ ostream& operator<<(ostream& stream, ContentType contentType)
 
 namespace internal {
 
+// Set of endpoint whose access is protected with the authorization
+// action `GET_EDNPOINTS_WITH_PATH`.
+hashset<string> AUTHORIZABLE_ENDPOINTS{
+    "/containers",
+    "/logging/toggle",
+    "/metrics/snapshot",
+    "/monitor/statistics",
+    "/monitor/statistics.json"};
+
+
 string serialize(
     ContentType contentType,
     const google::protobuf::Message& message)
@@ -599,6 +610,13 @@ const AuthorizationCallbacks createAuthorizationCallbacks(
   Callback getEndpoint = [authorizer](
       const process::http::Request& httpRequest,
       const Option<string>& principal) -> process::Future<bool> {
+        const string path = httpRequest.url.path;
+
+        if (!internal::AUTHORIZABLE_ENDPOINTS.contains(path)) {
+          return Failure(
+              "Endpoint '" + path + "' is not an authorizable endpoint.");
+        }
+
         authorization::Request authRequest;
         authRequest.set_action(mesos::authorization::GET_ENDPOINT_WITH_PATH);
 
@@ -606,7 +624,6 @@ const AuthorizationCallbacks createAuthorizationCallbacks(
           authRequest.mutable_subject()->set_value(principal.get());
         }
 
-        const string path = httpRequest.url.path;
         authRequest.mutable_object()->set_value(path);
 
         LOG(INFO) << "Authorizing principal '"
@@ -713,4 +730,44 @@ bool approveViewFlags(
   return approved.get();
 }
 
+
+process::Future<bool> authorizeEndpoint(
+    const std::string& endpoint,
+    const std::string& method,
+    const Option<Authorizer*>& authorizer,
+    const Option<std::string>& principal)
+{
+  if (authorizer.isNone()) {
+    return true;
+  }
+
+  authorization::Request request;
+
+  // TODO(nfnt): Add an additional case when POST requests
+  // need to be authorized separately from GET requests.
+  if (method == "GET") {
+    request.set_action(authorization::GET_ENDPOINT_WITH_PATH);
+  } else {
+    return Failure("Unexpected request method '" + method + "'");
+  }
+
+  if (!internal::AUTHORIZABLE_ENDPOINTS.contains(endpoint)) {
+    return Failure(
+        "Endpoint '" + endpoint + "' is not an authorizable endpoint.");
+  }
+
+  if (principal.isSome()) {
+    request.mutable_subject()->set_value(principal.get());
+  }
+
+  request.mutable_object()->set_value(endpoint);
+
+  LOG(INFO) << "Authorizing principal '"
+            << (principal.isSome() ? principal.get() : "ANY")
+            << "' to " <<  method
+            << " the '" << endpoint << "' endpoint";
+
+  return authorizer.get()->authorized(request);
+}
+
 }  // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/ec45ae5a/src/common/http.hpp
----------------------------------------------------------------------
diff --git a/src/common/http.hpp b/src/common/http.hpp
index eb2d015..c410d03 100644
--- a/src/common/http.hpp
+++ b/src/common/http.hpp
@@ -29,6 +29,7 @@
 #include <process/owned.hpp>
 
 #include <stout/hashmap.hpp>
+#include <stout/hashset.hpp>
 #include <stout/json.hpp>
 #include <stout/jsonify.hpp>
 #include <stout/protobuf.hpp>
@@ -41,6 +42,8 @@ class Task;
 
 namespace internal {
 
+extern hashset<std::string> AUTHORIZABLE_ENDPOINTS;
+
 // Serializes a protobuf message for transmission
 // based on the HTTP content type.
 std::string serialize(
@@ -139,6 +142,20 @@ bool approveViewTask(
 
 bool approveViewFlags(const process::Owned<ObjectApprover>& flagsApprover);
 
+
+// Authorizes access to an HTTP endpoint. The `method` parameter
+// determines which ACL action will be used in the authorization.
+// It is expected that the caller has validated that `method` is
+// supported by this function. Currently "GET" is supported.
+//
+// TODO(nfnt): Prefer types instead of strings
+// for `endpoint` and `method`, see MESOS-5300.
+process::Future<bool> authorizeEndpoint(
+    const std::string& endpoint,
+    const std::string& method,
+    const Option<Authorizer*>& authorizer,
+    const Option<std::string>& principal);
+
 } // namespace mesos {
 
 #endif // __COMMON_HTTP_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/ec45ae5a/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 60a780c..67ed67e 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -1057,7 +1057,11 @@ Future<Response> Slave::Http::statistics(
     return Failure("Failed to extract endpoint: " + endpoint.error());
   }
 
-  return authorizeEndpoint(principal, endpoint.get(), request.method)
+  return authorizeEndpoint(
+      endpoint.get(),
+      request.method,
+      slave->authorizer,
+      principal)
     .then(defer(
         slave->self(),
         [this, request](bool authorized) -> Future<Response> {
@@ -1162,7 +1166,11 @@ Future<Response> Slave::Http::containers(
     return Failure("Failed to extract endpoint: " + endpoint.error());
   }
 
-  return authorizeEndpoint(principal, endpoint.get(), request.method)
+  return authorizeEndpoint(
+      endpoint.get(),
+      request.method,
+      slave->authorizer,
+      principal)
     .then(defer(
         slave->self(),
         [this, request](bool authorized) -> Future<Response> {
@@ -1323,40 +1331,6 @@ Try<string> Slave::Http::extractEndpoint(const process::http::URL& url) const
   return "/" + pathComponents[1];
 }
 
-
-Future<bool> Slave::Http::authorizeEndpoint(
-    const Option<string>& principal,
-    const string& endpoint,
-    const string& method) const
-{
-  if (slave->authorizer.isNone()) {
-    return true;
-  }
-
-  authorization::Request request;
-
-  // TODO(nfnt): Add an additional case when POST requests
-  // need to be authorized separately from GET requests.
-  if (method == "GET") {
-    request.set_action(authorization::GET_ENDPOINT_WITH_PATH);
-  } else {
-    return Failure("Unexpected request method '" + method + "'");
-  }
-
-  if (principal.isSome()) {
-    request.mutable_subject()->set_value(principal.get());
-  }
-
-  request.mutable_object()->set_value(endpoint);
-
-  LOG(INFO) << "Authorizing principal '"
-            << (principal.isSome() ? principal.get() : "ANY")
-            << "' to " <<  method
-            << " the '" << endpoint << "' endpoint";
-
-  return slave->authorizer.get()->authorized(request);
-}
-
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/ec45ae5a/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 2afd7d1..484ba75 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -502,18 +502,6 @@ private:
     // Helper routines for endpoint authorization.
     Try<std::string> extractEndpoint(const process::http::URL& url) const;
 
-    // Authorizes access to an HTTP endpoint. The `method` parameter
-    // determines which ACL action will be used in the authorization.
-    // It is expected that the caller has validated that `method` is
-    // supported by this function. Currently "GET" is supported.
-    //
-    // TODO(nfnt): Prefer types instead of strings
-    // for `endpoint` and `method`, see MESOS-5300.
-    process::Future<bool> authorizeEndpoint(
-        const Option<std::string>& principal,
-        const std::string& endpoint,
-        const std::string& method) const;
-
     // Agent API handlers.
 
     process::Future<process::http::Response> getFlags(

http://git-wip-us.apache.org/repos/asf/mesos/blob/ec45ae5a/src/tests/authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/authorization_tests.cpp b/src/tests/authorization_tests.cpp
index c1e8ea6..ddd1f6f 100644
--- a/src/tests/authorization_tests.cpp
+++ b/src/tests/authorization_tests.cpp
@@ -2344,6 +2344,53 @@ TYPED_TEST(AuthorizationTest, ViewFlags)
   }
 }
 
+
+// This tests the authorization of ACLs used for unreserve
+// operations on dynamically reserved resources.
+TYPED_TEST(AuthorizationTest, ValidateEndpoints)
+{
+  {
+    ACLs acls;
+
+    mesos::ACL::GetEndpoint* acl = acls.add_get_endpoints();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_paths()->add_values("/frameworks");
+
+    // Create an `Authorizer` with the ACLs.
+    Try<Authorizer*> create = TypeParam::create(parameterize(acls));
+    EXPECT_ERROR(create);
+  }
+
+  {
+    ACLs acls;
+
+    mesos::ACL::GetEndpoint* acl = acls.add_get_endpoints();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_paths()->add_values("/frameworks");
+    acl->mutable_paths()->add_values("/monitor/statistics");
+    acl->mutable_paths()->add_values("/containers");
+
+    // Create an `Authorizer` with the ACLs.
+    Try<Authorizer*> create = TypeParam::create(parameterize(acls));
+    EXPECT_ERROR(create);
+  }
+
+  {
+    ACLs acls;
+
+    mesos::ACL::GetEndpoint* acl = acls.add_get_endpoints();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_paths()->add_values("/monitor/statistics");
+    acl->mutable_paths()->add_values("/monitor/statistics.json");
+    acl->mutable_paths()->add_values("/containers");
+
+    // Create an `Authorizer` with the ACLs.
+    Try<Authorizer*> create = TypeParam::create(parameterize(acls));
+    EXPECT_SOME(create);
+    delete create.get();
+  }
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/ec45ae5a/src/tests/scheduler_driver_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/scheduler_driver_tests.cpp b/src/tests/scheduler_driver_tests.cpp
index 2171857..faf2e6c 100644
--- a/src/tests/scheduler_driver_tests.cpp
+++ b/src/tests/scheduler_driver_tests.cpp
@@ -96,7 +96,7 @@ TEST_F(MesosSchedulerDriverTest, MetricsEndpoint)
   AWAIT_READY(registered);
 
   Future<process::http::Response> response =
-    process::http::get(MetricsProcess::instance()->self(), "/snapshot");
+    process::http::get(MetricsProcess::instance()->self(), "snapshot");
 
   AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
   AWAIT_EXPECT_RESPONSE_HEADER_EQ(APPLICATION_JSON, "Content-Type", response);


[2/2] mesos git commit: Added documentation on coarse grain authorization for endpoints.

Posted by vi...@apache.org.
Added documentation on coarse grain authorization for endpoints.

Coarse grained authorization for endpoints landed a while ago, however
no documentation is available.

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


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

Branch: refs/heads/master
Commit: 926e8d25c5b5cc570faeb2a4494ac8bc4e03d773
Parents: ec45ae5
Author: Alexander Rojas <al...@mesosphere.io>
Authored: Fri Jul 1 10:39:27 2016 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Jul 1 10:40:21 2016 -0700

----------------------------------------------------------------------
 docs/authorization.md | 45 +++++++++++++++++++++++++++++++++++++++++++++
 docs/upgrades.md      |  8 ++++++++
 2 files changed, 53 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/926e8d25/docs/authorization.md
----------------------------------------------------------------------
diff --git a/docs/authorization.md b/docs/authorization.md
index 9bd6031..fb56cdb 100644
--- a/docs/authorization.md
+++ b/docs/authorization.md
@@ -216,6 +216,13 @@ entries, each representing an authorizable action:
   </td>
 </tr>
 <tr>
+  <td><code>get_endpoints</code></td>
+  <td>HTTP username.</td>
+  <td>HTTP endpoints the user should be able to access using the HTTP "GET"
+      method.</td>
+  <td>Performing an HTTP "GET" on an endpoint.</td>
+</tr>
+<tr>
   <td><code>update_weights</code></td>
   <td>Operator username.</td>
   <td>Resource roles whose weights can be updated by the operator.</td>
@@ -258,6 +265,14 @@ entries, each representing an authorizable action:
 </tbody>
 </table>
 
+### Authorizable HTTP endpoints
+
+The `get_endpoints` action covers:
+
+* `/logging/toggle`
+* `/metrics/snapshot`
+* `/slave(id)/containers`
+* `/slave(id)/monitor/statistics`
 
 ### Examples
 
@@ -708,6 +723,36 @@ principal can update quota.
 }
 ```
 
+
+The principal `ops` can reach all HTTP endpoints using the _GET_
+method. The principal `foo`, however, can only use the HTTP _GET_ on
+the `/logging/toggle` and `/monitor/statistics` endpoints.  No other
+principals can use _GET_ on any endpoints.
+
+```json
+{
+  "permissive": false,
+  "get_endpoints": [
+                     {
+                       "principals": {
+                         "values": ["ops"]
+                       },
+                       "paths": {
+                         "type": "ANY"
+                       }
+                     },
+                     {
+                       "principals": {
+                         "values": ["foo"]
+                       },
+                       "paths": {
+                         "values": ["/logging/toggle", "/monitor/statistics"]
+                       }
+                     }
+                   ]
+}
+```
+
 ## Implementing an Authorizer
 
 In case you plan to implement your own authorizer [module](modules.md), the

http://git-wip-us.apache.org/repos/asf/mesos/blob/926e8d25/docs/upgrades.md
----------------------------------------------------------------------
diff --git a/docs/upgrades.md b/docs/upgrades.md
index 079e04f..255b5bd 100644
--- a/docs/upgrades.md
+++ b/docs/upgrades.md
@@ -271,6 +271,14 @@ We categorize the changes as follows:
 
 * Mesos 1.0 introduces authorization support for several HTTP endpoints. Note that some of these endpoints are used by the web UI, and thus using the web UI in a cluster with authorization enabled will require that ACLs be set appropriately. Please refer to the [authorization documentation](authorization.md) for details.
 
+* The endpoints with coarse grained authorization enabled are:
+  - `/logging/toggle`
+  - `/metrics/snapshot`
+  - `/slave(id)/containers`
+  - `/slave(id)/monitor/statistics`
+
+* If the defined ACLs used `permissive: false`, the listed HTTP endpoints will stop working unless ACLs for the `get_endpoints` actions are defined.
+
 In order to upgrade a running cluster:
 
 1. Rebuild and install any modules so that upgraded masters/agents can use them.