You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by al...@apache.org on 2016/04/27 17:29:34 UTC

[1/2] mesos git commit: Enabled authorization of endpoints on agents.

Repository: mesos
Updated Branches:
  refs/heads/master 321cd2aa6 -> fbbcc6af6


Enabled authorization of endpoints on agents.

Add a method that checks with the authorizer whether a GET request
shall be granted. Use this method to authorize '/flags' endpoint.

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


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

Branch: refs/heads/master
Commit: 721a414f64dbb90920202e674b87b42a2c7876f9
Parents: 321cd2a
Author: Jan Schlicht <ja...@mesosphere.io>
Authored: Wed Apr 27 16:26:33 2016 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Wed Apr 27 17:28:25 2016 +0200

----------------------------------------------------------------------
 docs/configuration.md                     |  10 ++
 include/mesos/authorizer/acls.proto       |  10 ++
 include/mesos/authorizer/authorizer.proto |   1 +
 src/Makefile.am                           |   1 +
 src/authorizer/local/authorizer.cpp       |  11 ++
 src/slave/flags.cpp                       |  12 +-
 src/slave/http.cpp                        |  72 ++++++++++-
 src/slave/slave.hpp                       |  15 ++-
 src/tests/authorization_tests.cpp         |  11 --
 src/tests/mesos.hpp                       |  11 ++
 src/tests/slave_authorization_tests.cpp   | 162 +++++++++++++++++++++++++
 11 files changed, 300 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/721a414f/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index 2796a81..34271c7 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -891,6 +891,16 @@ other than <code>local</code>, the ACLs contents will be
 ignored.
 <p/>
 See the ACLs protobuf in acls.proto for the expected format.
+<p/>
+Example:
+<pre><code>{
+  "get_endpoints": [
+    {
+      "principals": { "values": ["a"] },
+      "paths": { "values": ["/flags", "/monitor/statistics"] }
+    }
+  ]
+}</code></pre>
   </td>
 </tr>
 <tr>

http://git-wip-us.apache.org/repos/asf/mesos/blob/721a414f/include/mesos/authorizer/acls.proto
----------------------------------------------------------------------
diff --git a/include/mesos/authorizer/acls.proto b/include/mesos/authorizer/acls.proto
index c50deeb..9adae8c 100644
--- a/include/mesos/authorizer/acls.proto
+++ b/include/mesos/authorizer/acls.proto
@@ -146,6 +146,15 @@ message ACL {
     // Objects: The list of roles whose weights can be updated.
     optional Entity roles = 2;
   }
+
+  // Which principals are authorized to GET HTTP endpoints at the given paths.
+  message GetEndpoint {
+    // Subjects: HTTP username.
+    required Entity principals = 1;
+
+    // Objects: The list of paths that can be retrieved.
+    optional Entity paths = 2;
+  }
 }
 
 
@@ -183,4 +192,5 @@ message ACLs {
   repeated ACL.RemoveQuota remove_quotas = 10;
   repeated ACL.TeardownFramework teardown_frameworks = 11;
   repeated ACL.UpdateWeights update_weights = 12;
+  repeated ACL.GetEndpoint get_endpoints = 13;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/721a414f/include/mesos/authorizer/authorizer.proto
----------------------------------------------------------------------
diff --git a/include/mesos/authorizer/authorizer.proto b/include/mesos/authorizer/authorizer.proto
index 40d93ea..32492a5 100644
--- a/include/mesos/authorizer/authorizer.proto
+++ b/include/mesos/authorizer/authorizer.proto
@@ -58,6 +58,7 @@ enum Action {
   SET_QUOTA_WITH_ROLE = 8;
   DESTROY_QUOTA_WITH_PRINCIPAL = 9;
   UPDATE_WEIGHTS_WITH_ROLE = 10;
+  GET_ENDPOINT_WITH_PATH = 11;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/721a414f/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index e024c6d..12276a9 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1989,6 +1989,7 @@ mesos_tests_SOURCES =						\
   tests/scheduler_http_api_tests.cpp				\
   tests/scheduler_tests.cpp					\
   tests/script.cpp						\
+  tests/slave_authorization_tests.cpp				\
   tests/slave_recovery_tests.cpp				\
   tests/slave_tests.cpp						\
   tests/sorter_tests.cpp					\

http://git-wip-us.apache.org/repos/asf/mesos/blob/721a414f/src/authorizer/local/authorizer.cpp
----------------------------------------------------------------------
diff --git a/src/authorizer/local/authorizer.cpp b/src/authorizer/local/authorizer.cpp
index 0a3805f..e59c112 100644
--- a/src/authorizer/local/authorizer.cpp
+++ b/src/authorizer/local/authorizer.cpp
@@ -201,6 +201,17 @@ public:
 
         return authorized(request, acls_);
         break;
+      case authorization::GET_ENDPOINT_WITH_PATH:
+        for (const ACL::GetEndpoint& acl : acls.get_endpoints()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.paths();
+
+          acls_.push_back(acl_);
+        }
+
+        return authorized(request, acls_);
+        break;
       case authorization::UNKNOWN:
         LOG(WARNING) << "Authorization request for action '" << request.action()
                      << "' is not defined and therefore not authorized";

http://git-wip-us.apache.org/repos/asf/mesos/blob/721a414f/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index a319d60..6fde51f 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -454,7 +454,17 @@ mesos::internal::slave::Flags::Flags()
       "other than `" + string(DEFAULT_AUTHORIZER) + "`, the ACLs contents\n"
       "will be ignored.\n"
       "\n"
-      "See the ACLs protobuf in acls.proto for the expected format.");
+      "See the ACLs protobuf in acls.proto for the expected format.\n"
+      "\n"
+      "Example:\n"
+      "{\n"
+      "  \"get_endpoints\": [\n"
+      "    {\n"
+      "      \"principals\": { \"values\": [\"a\"] },\n"
+      "      \"paths\": { \"values\": [\"/flags\", \"/monitor/statistics\"] }\n"
+      "    }\n"
+      "  ]\n"
+      "}");
 
   add(&Flags::containerizer_path,
       "containerizer_path",

http://git-wip-us.apache.org/repos/asf/mesos/blob/721a414f/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 537736d..1196c93 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -19,6 +19,9 @@
 #include <sstream>
 #include <string>
 #include <tuple>
+#include <vector>
+
+#include <mesos/authorizer/authorizer.hpp>
 
 #include <mesos/executor/executor.hpp>
 
@@ -80,6 +83,7 @@ using process::metrics::internal::MetricsProcess;
 using std::list;
 using std::string;
 using std::tuple;
+using std::vector;
 
 
 namespace mesos {
@@ -351,14 +355,32 @@ string Slave::Http::FLAGS_HELP()
 
 Future<Response> Slave::Http::flags(
     const Request& request,
-    const Option<string>& /* principal */) const
+    const Option<string>& principal) const
+{
+  const Flags slaveFlags = slave->flags;
+
+  return authorizeEndpoint(request, principal)
+    .then(defer(
+        slave->self(),
+        [request, slaveFlags](bool authorized) -> Future<Response> {
+          if (!authorized) {
+            return Forbidden();
+          }
+
+          return _flags(request, slaveFlags);
+        }));
+}
+
+Future<Response> Slave::Http::_flags(
+  const Request& request,
+  const Flags& slaveFlags)
 {
   JSON::Object object;
 
   {
     JSON::Object flags;
-    foreachpair (const string& name, const flags::Flag& flag, slave->flags) {
-      Option<string> value = flag.stringify(slave->flags);
+    foreachpair (const string& name, const flags::Flag& flag, slaveFlags) {
+      Option<string> value = flag.stringify(slaveFlags);
       if (value.isSome()) {
         flags.values[name] = value.get();
       }
@@ -762,6 +784,50 @@ Future<Response> Slave::Http::containers(const Request& request) const
       });
 }
 
+
+Future<bool> Slave::Http::authorizeEndpoint(
+    const Request& httpRequest,
+    const Option<string>& principal) const
+{
+  if (slave->authorizer.isNone()) {
+    return true;
+  }
+
+  // Paths are of the form "/slave(n)/endpoint". We're only interested
+  // in the part after "/slave(n)" and tokenize the path accordingly.
+  const vector<string> pathComponents =
+    strings::tokenize(httpRequest.url.path, "/", 2);
+
+  if (pathComponents.size() != 2u ||
+      pathComponents[0] != slave->self().id) {
+    return Failure("Unexpected path '" + httpRequest.url.path + "'");
+  }
+  const string endpoint = "/" + pathComponents[1];
+
+  authorization::Request authorizationRequest;
+
+  // TODO(nfnt): Add an additional case when POST requests need to be
+  // authorized separately from GET requests.
+  if (httpRequest.method == "GET") {
+    authorizationRequest.set_action(authorization::GET_ENDPOINT_WITH_PATH);
+  } else {
+    return Failure("Unexpected request method '" + httpRequest.method + "'");
+  }
+
+  if (principal.isSome()) {
+    authorizationRequest.mutable_subject()->set_value(principal.get());
+  }
+
+  authorizationRequest.mutable_object()->set_value(endpoint);
+
+  LOG(INFO) << "Authorizing principal '"
+            << (principal.isSome() ? principal.get() : "ANY")
+            << "' to " <<  httpRequest.method
+            << " the '" << endpoint << "' endpoint";
+
+  return slave->authorizer.get()->authorized(authorizationRequest);
+}
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/721a414f/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 57b1888..4b8bb07 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -439,7 +439,7 @@ private:
     // /slave/flags
     process::Future<process::http::Response> flags(
         const process::http::Request& request,
-        const Option<std::string>& /* principal */) const;
+        const Option<std::string>& principal) const;
 
     // /slave/health
     process::Future<process::http::Response> health(
@@ -468,6 +468,19 @@ private:
     static std::string CONTAINERS_HELP();
 
   private:
+    // Continuations.
+    static process::Future<process::http::Response> _flags(
+        const process::http::Request& request,
+        const Flags& flags);
+
+    // Authorizes access to an HTTP endpoint. It extracts the endpoint
+    // from the URL of the request by removing the "/slave(n)" part of
+    // the URL's path. The request's `method` determines which ACL action
+    // will be used in the authorization.
+    process::Future<bool> authorizeEndpoint(
+        const process::http::Request& request,
+        const Option<std::string>& principal) const;
+
     Slave* slave;
 
     // Used to rate limit the statistics endpoint.

http://git-wip-us.apache.org/repos/asf/mesos/blob/721a414f/src/tests/authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/authorization_tests.cpp b/src/tests/authorization_tests.cpp
index d4ef0f9..f50ac69 100644
--- a/src/tests/authorization_tests.cpp
+++ b/src/tests/authorization_tests.cpp
@@ -40,17 +40,6 @@ namespace tests {
 using std::string;
 
 
-static Parameters parameterize(const ACLs& acls)
-{
-  Parameters parameters;
-  Parameter *parameter = parameters.add_parameter();
-  parameter->set_key("acls");
-  parameter->set_value(string(jsonify(JSON::Protobuf(acls))));
-
-  return parameters;
-}
-
-
 template <typename T>
 class AuthorizationTest : public MesosTest {};
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/721a414f/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 78edab8..0f6f541 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -736,6 +736,17 @@ inline Offer::Operation LAUNCH(const std::vector<TaskInfo>& tasks)
 }
 
 
+inline Parameters parameterize(const ACLs& acls)
+{
+  Parameters parameters;
+  Parameter* parameter = parameters.add_parameter();
+  parameter->set_key("acls");
+  parameter->set_value(std::string(jsonify(JSON::Protobuf(acls))));
+
+  return parameters;
+}
+
+
 // Definition of a mock Scheduler to be used in tests with gmock.
 class MockScheduler : public Scheduler
 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/721a414f/src/tests/slave_authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_authorization_tests.cpp b/src/tests/slave_authorization_tests.cpp
new file mode 100644
index 0000000..61e94c8
--- /dev/null
+++ b/src/tests/slave_authorization_tests.cpp
@@ -0,0 +1,162 @@
+// 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 <string>
+
+#include <gtest/gtest.h>
+
+#include <mesos/authorizer/authorizer.hpp>
+
+#include <mesos/module/authorizer.hpp>
+
+#include <process/http.hpp>
+#include <process/owned.hpp>
+
+#include <stout/try.hpp>
+
+#include "authorizer/local/authorizer.hpp"
+
+#include "master/detector/standalone.hpp"
+
+#include "tests/mesos.hpp"
+#include "tests/module.hpp"
+
+namespace http = process::http;
+
+using mesos::internal::master::Master;
+
+using mesos::internal::slave::Slave;
+
+using mesos::master::detector::StandaloneMasterDetector;
+
+using process::Future;
+using process::Owned;
+
+using process::http::Forbidden;
+using process::http::OK;
+using process::http::Response;
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+template <typename T>
+class SlaveAuthorizationTest : public MesosTest {};
+
+
+typedef ::testing::Types<
+  LocalAuthorizer,
+  tests::Module<Authorizer, TestLocalAuthorizer>>
+  AuthorizerTypes;
+
+
+TYPED_TEST_CASE(SlaveAuthorizationTest, AuthorizerTypes);
+
+
+// This test verifies that only authorized principals
+// can access the '/flags' endpoint.
+TYPED_TEST(SlaveAuthorizationTest, AuthorizeFlagsEndpoint)
+{
+  const string endpoint = "flags";
+
+  // Setup ACLs so that only the default principal
+  // can access the '/flags' endpoint.
+  ACLs acls;
+  acls.set_permissive(false);
+
+  mesos::ACL::GetEndpoint* acl = acls.add_get_endpoints();
+  acl->mutable_principals()->add_values(DEFAULT_CREDENTIAL.principal());
+  acl->mutable_paths()->add_values("/" + endpoint);
+
+  // Create an `Authorizer` with the ACLs.
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
+  ASSERT_SOME(create);
+  Owned<Authorizer> authorizer(create.get());
+
+  StandaloneMasterDetector detector;
+  Try<Owned<cluster::Slave>> agent =
+    this->StartSlave(&detector, authorizer.get());
+  ASSERT_SOME(agent);
+
+  Future<Response> response = http::get(
+      agent.get()->pid,
+      endpoint,
+      None(),
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response)
+    << response.get().body;
+
+  response = http::get(
+      agent.get()->pid,
+      endpoint,
+      None(),
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL_2));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(Forbidden().status, response)
+    << response.get().body;
+}
+
+
+// This test verifies that access to the '/flags' endpoint can be authorized
+// without authentication if an authorization rule exists that applies to
+// anyone. The authorizer will map the absence of a principal to "ANY".
+TYPED_TEST(SlaveAuthorizationTest, AuthorizeFlagsEndpointWithoutPrincipal)
+{
+  const string endpoint = "flags";
+
+  // Because the authenticators' lifetime is tied to libprocess's lifetime,
+  // it may already be set by other tests. We have to unset it here to disable
+  // HTTP authentication.
+  // TODO(nfnt): Fix this behavior. The authenticator should be unset by
+  // every test case that sets it, similar to how it's done for the master.
+  http::authentication::unsetAuthenticator(
+      slave::DEFAULT_HTTP_AUTHENTICATION_REALM);
+
+  // Setup ACLs so that any principal can access the '/flags' endpoint.
+  ACLs acls;
+  acls.set_permissive(false);
+
+  mesos::ACL::GetEndpoint* acl = acls.add_get_endpoints();
+  acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+  acl->mutable_paths()->add_values("/" + endpoint);
+
+  slave::Flags agentFlags = this->CreateSlaveFlags();
+  agentFlags.acls = acls;
+  agentFlags.authenticate_http = false;
+  agentFlags.http_credentials = None();
+
+  // Create an `Authorizer` with the ACLs.
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
+  ASSERT_SOME(create);
+  Owned<Authorizer> authorizer(create.get());
+
+  StandaloneMasterDetector detector;
+  Try<Owned<cluster::Slave>> agent = this->StartSlave(
+      &detector, authorizer.get(), agentFlags);
+  ASSERT_SOME(agent);
+
+  Future<Response> response = http::get(agent.get()->pid, endpoint);
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response)
+    << response.get().body;
+}
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {


[2/2] mesos git commit: Added authorization to agents' `/monitor/statistics` endpoints.

Posted by al...@apache.org.
Added authorization to agents' `/monitor/statistics` endpoints.

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


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

Branch: refs/heads/master
Commit: fbbcc6af67fe146bb1a12ead11ee96861280247b
Parents: 721a414
Author: Benjamin Bannier <be...@mesosphere.io>
Authored: Wed Apr 27 16:53:46 2016 +0200
Committer: Alexander Rukletsov <al...@apache.org>
Committed: Wed Apr 27 17:28:27 2016 +0200

----------------------------------------------------------------------
 src/slave/http.cpp                      |  70 +++++++++++------
 src/slave/slave.hpp                     |   9 ++-
 src/tests/slave_authorization_tests.cpp | 112 +++++++++++++++++++++++++++
 3 files changed, 165 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fbbcc6af/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 1196c93..9b55886 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -32,8 +32,9 @@
 
 #include <process/collect.hpp>
 #include <process/help.hpp>
-#include <process/owned.hpp>
+#include <process/http.hpp>
 #include <process/limiter.hpp>
+#include <process/owned.hpp>
 
 #include <process/metrics/metrics.hpp>
 
@@ -621,40 +622,59 @@ string Slave::Http::STATISTICS_HELP()
 
 Future<Response> Slave::Http::statistics(
     const Request& request,
-    const Option<string>& /* principal */) const
+    const Option<string>& principal) const
 {
-  return statisticsLimiter->acquire()
-    .then(defer(slave->self(), &Slave::usage))
-    .then([=](const Future<ResourceUsage>& usage) -> Future<Response> {
-      JSON::Array result;
-
-      foreach (const ResourceUsage::Executor& executor,
-               usage.get().executors()) {
-        if (executor.has_statistics()) {
-          const ExecutorInfo info = executor.executor_info();
-
-          JSON::Object entry;
-          entry.values["framework_id"] = info.framework_id().value();
-          entry.values["executor_id"] = info.executor_id().value();
-          entry.values["executor_name"] = info.name();
-          entry.values["source"] = info.source();
-          entry.values["statistics"] = JSON::protobuf(executor.statistics());
+  const PID<Slave> pid = slave->self();
+  Shared<RateLimiter> limiter = statisticsLimiter;
 
-          result.values.push_back(entry);
-        }
-      }
+  return authorizeEndpoint(request, principal)
+    .then(defer(
+        pid,
+        [pid, limiter, request](bool authorized) -> Future<Response> {
+          if (!authorized) {
+            return Forbidden();
+          }
 
-      return process::http::OK(result, request.url.query.get("jsonp"));
-    })
+          return limiter->acquire()
+            .then(defer(pid, &Slave::usage))
+            .then(defer(pid, [request](const ResourceUsage& usage) {
+              return _statistics(usage, request);
+            }));
+        }))
     .repair([](const Future<Response>& future) {
-      LOG(WARNING) << "Could not collect resource usage: "
+      LOG(WARNING) << "Could not collect statistics: "
                    << (future.isFailed() ? future.failure() : "discarded");
 
-      return process::http::InternalServerError();
+      return InternalServerError();
     });
 }
 
 
+Response Slave::Http::_statistics(
+    const ResourceUsage& usage,
+    const Request& request)
+{
+  JSON::Array result;
+
+  foreach (const ResourceUsage::Executor& executor, usage.executors()) {
+    if (executor.has_statistics()) {
+      const ExecutorInfo info = executor.executor_info();
+
+      JSON::Object entry;
+      entry.values["framework_id"] = info.framework_id().value();
+      entry.values["executor_id"] = info.executor_id().value();
+      entry.values["executor_name"] = info.name();
+      entry.values["source"] = info.source();
+      entry.values["statistics"] = JSON::protobuf(executor.statistics());
+
+      result.values.push_back(entry);
+    }
+  }
+
+  return OK(result, request.url.query.get("jsonp"));
+}
+
+
 string Slave::Http::CONTAINERS_HELP()
 {
   return HELP(

http://git-wip-us.apache.org/repos/asf/mesos/blob/fbbcc6af/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 4b8bb07..b724380 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -454,7 +454,7 @@ private:
     // /slave/monitor/statistics.json
     process::Future<process::http::Response> statistics(
         const process::http::Request& request,
-        const Option<std::string>& /* principal */) const;
+        const Option<std::string>& principal) const;
 
     // /slave/containers
     process::Future<process::http::Response> containers(
@@ -481,6 +481,13 @@ private:
         const process::http::Request& request,
         const Option<std::string>& principal) const;
 
+
+    // Make continuation for `statistics` `static` as it might
+    // execute when the invoking `Http` is already destructed.
+    static process::http::Response _statistics(
+        const ResourceUsage& usage,
+        const process::http::Request& request);
+
     Slave* slave;
 
     // Used to rate limit the statistics endpoint.

http://git-wip-us.apache.org/repos/asf/mesos/blob/fbbcc6af/src/tests/slave_authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_authorization_tests.cpp b/src/tests/slave_authorization_tests.cpp
index 61e94c8..fc3aef7 100644
--- a/src/tests/slave_authorization_tests.cpp
+++ b/src/tests/slave_authorization_tests.cpp
@@ -16,6 +16,8 @@
 
 #include <string>
 
+#include <gmock/gmock.h>
+
 #include <gtest/gtest.h>
 
 #include <mesos/authorizer/authorizer.hpp>
@@ -51,6 +53,8 @@ using process::http::Response;
 
 using std::string;
 
+using testing::DoAll;
+
 namespace mesos {
 namespace internal {
 namespace tests {
@@ -157,6 +161,114 @@ TYPED_TEST(SlaveAuthorizationTest, AuthorizeFlagsEndpointWithoutPrincipal)
     << response.get().body;
 }
 
+
+// Parameterized fixture for agent-specific authorization tests. The
+// path of the tested endpoint is passed as the only parameter.
+class SlaveEndpointTest:
+    public MesosTest,
+    public ::testing::WithParamInterface<string> {};
+
+
+// The tests are parameterized by the endpoint being queried.
+//
+// TODO(bbannier): Once agent endpoint handlers use more than just
+// `GET_ENDPOINT_WITH_PATH`, we should consider parameterizing
+// `SlaveEndpointTest` by the authorization action as well.
+INSTANTIATE_TEST_CASE_P(
+    Endpoint,
+    SlaveEndpointTest,
+    ::testing::Values(
+        "monitor/statistics", "monitor/statistics.json", "flags"));
+
+
+// Tests that an agent endpoint handler form
+// correct queries against the authorizer.
+TEST_P(SlaveEndpointTest, AuthorizedRequest)
+{
+  const string endpoint = GetParam();
+
+  StandaloneMasterDetector detector;
+
+  MockAuthorizer mockAuthorizer;
+
+  Try<Owned<cluster::Slave>> agent = StartSlave(&detector, &mockAuthorizer);
+  ASSERT_SOME(agent);
+
+  Future<authorization::Request> request;
+  EXPECT_CALL(mockAuthorizer, authorized(_))
+    .WillOnce(DoAll(FutureArg<0>(&request),
+                    Return(true)));
+
+  Future<Response> response = http::get(
+      agent.get()->pid,
+      endpoint,
+      None(),
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL));
+
+  AWAIT_READY(request);
+
+  EXPECT_EQ(DEFAULT_CREDENTIAL.principal(), request.get().subject().value());
+
+  // TODO(bbannier): Once agent endpoint handlers use more than just
+  // `GET_ENDPOINT_WITH_PATH` we should factor out the request method
+  // and expected authorization action and parameterize
+  // `SlaveEndpointTest` on that as well in addition to the endpoint.
+  EXPECT_EQ(authorization::GET_ENDPOINT_WITH_PATH, request.get().action());
+
+  EXPECT_EQ("/" + endpoint, request.get().object().value());
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response)
+    << response.get().body;
+}
+
+
+// Tests that unauthorized requests for an agent endpoint are properly rejected.
+TEST_P(SlaveEndpointTest, UnauthorizedRequest)
+{
+  const string endpoint = GetParam();
+
+  StandaloneMasterDetector detector;
+
+  MockAuthorizer mockAuthorizer;
+
+  Try<Owned<cluster::Slave>> agent = StartSlave(&detector, &mockAuthorizer);
+  ASSERT_SOME(agent);
+
+  EXPECT_CALL(mockAuthorizer, authorized(_))
+    .WillOnce(Return(false));
+
+  Future<Response> response = http::get(
+      agent.get()->pid,
+      endpoint,
+      None(),
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(Forbidden().status, response)
+    << response.get().body;
+}
+
+
+// Tests that requests for an agent endpoint
+// always succeed if the authorizer is absent.
+TEST_P(SlaveEndpointTest, NoAuthorizer)
+{
+  const string endpoint = GetParam();
+
+  StandaloneMasterDetector detector;
+
+  Try<Owned<cluster::Slave>> agent = StartSlave(&detector, CreateSlaveFlags());
+  ASSERT_SOME(agent);
+
+  Future<Response> response = http::get(
+      agent.get()->pid,
+      endpoint,
+      None(),
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response)
+    << response.get().body;
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {