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/03/11 22:48:12 UTC

[1/4] mesos git commit: Removed initialize method from the authorizer interface.

Repository: mesos
Updated Branches:
  refs/heads/master 46ef6bd5c -> 482dc141c


Removed initialize method from the authorizer interface.

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


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

Branch: refs/heads/master
Commit: fb28db30d9abc8d02f78f62a81c827c2de923eb1
Parents: 46ef6bd
Author: Alexander Rojas <al...@mesosphere.io>
Authored: Fri Mar 11 16:47:11 2016 -0500
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Mar 11 16:47:11 2016 -0500

----------------------------------------------------------------------
 include/mesos/authorizer/authorizer.hpp |  38 +++++-----
 src/authorizer/authorizer.cpp           |  21 +++---
 src/authorizer/local/authorizer.cpp     |  62 ++++++++-------
 src/authorizer/local/authorizer.hpp     |  30 +++++---
 src/examples/test_authorizer_module.cpp |  17 ++++-
 src/local/local.cpp                     |  58 ++++++--------
 src/master/main.cpp                     |  59 ++++++---------
 src/tests/authorization_tests.cpp       | 109 ++++++++++-----------------
 src/tests/cluster.cpp                   |  43 ++++++++---
 9 files changed, 216 insertions(+), 221 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fb28db30/include/mesos/authorizer/authorizer.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/authorizer/authorizer.hpp b/include/mesos/authorizer/authorizer.hpp
index 7054826..bcb1068 100644
--- a/include/mesos/authorizer/authorizer.hpp
+++ b/include/mesos/authorizer/authorizer.hpp
@@ -20,6 +20,8 @@
 #include <iosfwd>
 #include <string>
 
+#include <mesos/mesos.hpp>
+
 // ONLY USEFUL AFTER RUNNING PROTOC.
 #include <mesos/authorizer/authorizer.pb.h>
 
@@ -53,28 +55,30 @@ namespace mesos {
 class Authorizer
 {
 public:
-  static Try<Authorizer*> create(const std::string& name);
-
-  virtual ~Authorizer() {}
-
   /**
-   * Sets the Access Control Lists for the current instance of the interface.
-   * The contents of the `acls` parameter are used to define the rules which
-   * apply to the authorization actions.
+   * Factory method used to create instances of authorizer which are loaded from
+   * the `ModuleManager`. The parameters necessary to instantiate the authorizer
+   * are taken from the contents of the `--modules` flag.
+   *
+   * @param name The name of the module to be loaded as registered in the
+   *     `--modules` flag.
    *
-   * @param acls The access control lists used to initialize the authorizer
-   *     instance. See "authorizer.proto" for a description of their format.
+   * @return An instance of `Authorizer*` if the module with the given name
+   *     could be constructed. An error otherwise.
+   */
+  static Try<Authorizer*> create(const std::string &name);
+
+  /**
+   * Factory method used to create instances of the default 'local'  authorizer.
    *
-   * @return `Nothing` if the instance of the authorizer was successfully
-   *     initialized, an `Error` otherwise.
+   * @param acls The access control lists used to initialize the 'local'
+   *     authorizer.
    *
-   * TODO(arojas): This function is relevant for the default implementation
-   * of the `Authorizer` class only (see MESOS-3072) and it will not be
-   * called for any other implementation. Remove it once we have a module-only
-   * initialization which relies on module-specific parameters supplied via
-   * the modules JSON blob.
+   * @return An instance of the default 'local'  authorizer.
    */
-  virtual Try<Nothing> initialize(const Option<ACLs>& acls) = 0;
+  static Try<Authorizer*> create(const ACLs& acls);
+
+  virtual ~Authorizer() {}
 
   /**
    * Verifies whether a principal can register a framework with a specific role.

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb28db30/src/authorizer/authorizer.cpp
----------------------------------------------------------------------
diff --git a/src/authorizer/authorizer.cpp b/src/authorizer/authorizer.cpp
index 54278b0..74dfccd 100644
--- a/src/authorizer/authorizer.cpp
+++ b/src/authorizer/authorizer.cpp
@@ -20,8 +20,12 @@
 
 #include <mesos/module/authorizer.hpp>
 
+#include <stout/path.hpp>
+
 #include "authorizer/local/authorizer.hpp"
 
+#include "common/parse.hpp"
+
 #include "master/constants.hpp"
 
 #include "module/manager.hpp"
@@ -33,20 +37,17 @@ using mesos::internal::LocalAuthorizer;
 
 namespace mesos {
 
-Try<Authorizer*> Authorizer::create(const string& name)
-{
-  // Create an instance of the default authorizer. If other than the
-  // default authorizer is requested, search for it in loaded modules.
-  // NOTE: We do not need an extra not-null check, because both
-  // ModuleManager and built-in authorizer factory do that already.
-  if (name == mesos::internal::master::DEFAULT_AUTHORIZER) {
-    return LocalAuthorizer::create();
-  }
-
+Try<Authorizer*> Authorizer::create(const string &name) {
   return modules::ModuleManager::create<Authorizer>(name);
 }
 
 
+Try<Authorizer*> Authorizer::create(const ACLs& acls)
+{
+  return LocalAuthorizer::create(acls);
+}
+
+
 ostream& operator<<(ostream& stream, const ACLs& acls)
 {
   return stream << acls.DebugString();

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb28db30/src/authorizer/local/authorizer.cpp
----------------------------------------------------------------------
diff --git a/src/authorizer/local/authorizer.cpp b/src/authorizer/local/authorizer.cpp
index 15c857d..3dc69f0 100644
--- a/src/authorizer/local/authorizer.cpp
+++ b/src/authorizer/local/authorizer.cpp
@@ -18,15 +18,22 @@
 
 #include <string>
 
+#include <mesos/mesos.hpp>
+
 #include <process/dispatch.hpp>
 #include <process/future.hpp>
 #include <process/id.hpp>
 #include <process/process.hpp>
 #include <process/protobuf.hpp>
 
+#include <stout/foreach.hpp>
+#include <stout/option.hpp>
+#include <stout/path.hpp>
 #include <stout/protobuf.hpp>
 #include <stout/try.hpp>
 
+#include "common/parse.hpp"
+
 using process::Failure;
 using process::Future;
 using process::dispatch;
@@ -333,50 +340,51 @@ private:
 };
 
 
-Try<Authorizer*> LocalAuthorizer::create()
+Try<Authorizer*> LocalAuthorizer::create(const ACLs& acls)
 {
-  Authorizer* local = new LocalAuthorizer;
+  Authorizer* local = new LocalAuthorizer(acls);
 
   return local;
 }
 
 
-LocalAuthorizer::LocalAuthorizer() : process(NULL)
+Try<Authorizer*> LocalAuthorizer::create(const Parameters& parameters)
 {
-}
+  Option<string> acls;
+  foreach (const Parameter& parameter, parameters.parameter()) {
+    if (parameter.key() == "acls") {
+      acls = parameter.value();
+    }
+  }
 
+  if (acls.isNone()) {
+    return Error("No ACLs for default authorizer provided");
+  }
 
-LocalAuthorizer::~LocalAuthorizer()
-{
-  if (process != NULL) {
-    terminate(process);
-    wait(process);
-    delete process;
+  Try<ACLs> acls_ = flags::parse<ACLs>(acls.get());
+  if (acls_.isError()) {
+    return Error("Contents of 'acls' parameter could not be parsed into a "
+                 "valid ACLs object");
   }
+
+  return LocalAuthorizer::create(acls_.get());
 }
 
 
-Try<Nothing> LocalAuthorizer::initialize(const Option<ACLs>& acls)
+LocalAuthorizer::LocalAuthorizer(const ACLs& acls)
+    : process(new LocalAuthorizerProcess(acls))
 {
-  if (!acls.isSome()) {
-    return Error("ACLs need to be specified for local authorizer");
-  }
-
-  if (!initialized.once()) {
-    if (process != NULL) {
-      return Error("Authorizer already initialized");
-    }
+  spawn(process);
+}
 
-    // Process initialization needs to be done here because default
-    // implementations of modules need to be default constructible. So
-    // actual construction is delayed until initialization.
-    process = new LocalAuthorizerProcess(acls.get());
-    spawn(process);
 
-    initialized.done();
+LocalAuthorizer::~LocalAuthorizer()
+{
+  if (process != NULL) {
+    terminate(process);
+    wait(process);
+    delete process;
   }
-
-  return Nothing();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb28db30/src/authorizer/local/authorizer.hpp
----------------------------------------------------------------------
diff --git a/src/authorizer/local/authorizer.hpp b/src/authorizer/local/authorizer.hpp
index c87a991..361b213 100644
--- a/src/authorizer/local/authorizer.hpp
+++ b/src/authorizer/local/authorizer.hpp
@@ -27,6 +27,10 @@
 #include <stout/try.hpp>
 
 namespace mesos {
+
+// Forward declaration.
+class Parameters;
+
 namespace internal {
 
 // Forward declaration.
@@ -39,16 +43,22 @@ public:
   // Creates a LocalAuthorizer.
   // Never returns a nullptr, instead sets the Try to error.
   //
-  // This factory needs to return a raw pointer so its signature matches that
-  // of tests::Module<T,N>::create() so typed tests can be performed.
-  static Try<Authorizer*> create();
+  // This factory needs to return a raw pointer so it can be
+  // used in typed tests.
+  static Try<Authorizer*> create(const ACLs& acls);
 
-  virtual ~LocalAuthorizer();
+  // Creates a LocalAuthorizer.
+  // Never returns a nullptr, instead sets the Try to error.
+  //
+  // This factory needs to return a raw pointer so it can be
+  // used in typed tests.
+  //
+  // It extracts its ACLs from a parameter with key 'acls'.
+  // If such key does not exists or its contents cannot be
+  // parse, an error is returned.
+  static Try<Authorizer*> create(const Parameters& parameters);
 
-  // Initialization is needed since this class is required to be default
-  // constructible, however the ACLs still need to be provided. MESOS-3072
-  // tries to address this requirement.
-  virtual Try<Nothing> initialize(const Option<ACLs>& acls);
+  virtual ~LocalAuthorizer();
 
   // Implementation of Authorizer interface.
   virtual process::Future<bool> authorize(
@@ -73,11 +83,9 @@ public:
       const ACL::UpdateWeights& request);
 
 private:
-  LocalAuthorizer();
+  LocalAuthorizer(const ACLs& acls);
 
   LocalAuthorizerProcess* process;
-
-  process::Once initialized;
 };
 
 } // namespace internal {

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb28db30/src/examples/test_authorizer_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_authorizer_module.cpp b/src/examples/test_authorizer_module.cpp
index 95d77fb..1f0a770 100644
--- a/src/examples/test_authorizer_module.cpp
+++ b/src/examples/test_authorizer_module.cpp
@@ -14,6 +14,8 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+#include <string>
+
 #include <mesos/mesos.hpp>
 #include <mesos/module.hpp>
 
@@ -21,13 +23,22 @@
 
 #include <mesos/module/authorizer.hpp>
 
+#include <stout/foreach.hpp>
+#include <stout/option.hpp>
+#include <stout/path.hpp>
+#include <stout/try.hpp>
+
 #include "authorizer/local/authorizer.hpp"
 
+#include "common/parse.hpp"
+
+using std::string;
+
 using namespace mesos;
 
-static Authorizer* createAuthorizer(const Parameters& parameters)
+static Authorizer* createLocalAuthorizer(const Parameters& parameters)
 {
-  Try<Authorizer*> local = mesos::internal::LocalAuthorizer::create();
+  Try<Authorizer*> local = mesos::internal::LocalAuthorizer::create(parameters);
   if (local.isError()) {
     return NULL;
   }
@@ -45,4 +56,4 @@ mesos::modules::Module<Authorizer> org_apache_mesos_TestLocalAuthorizer(
     "modules@mesos.apache.org",
     "Test Authorizer module.",
     NULL,
-    createAuthorizer);
+    createLocalAuthorizer);

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb28db30/src/local/local.cpp
----------------------------------------------------------------------
diff --git a/src/local/local.cpp b/src/local/local.cpp
index 359fc54..f8599e7 100644
--- a/src/local/local.cpp
+++ b/src/local/local.cpp
@@ -122,7 +122,7 @@ static Master* master = NULL;
 static map<Containerizer*, Slave*> slaves;
 static StandaloneMasterDetector* detector = NULL;
 static MasterContender* contender = NULL;
-static Option<Authorizer*> authorizer = None();
+static Option<Authorizer*> authorizer_ = None();
 static Files* files = NULL;
 static vector<GarbageCollector*>* garbageCollectors = NULL;
 static vector<StatusUpdateManager*>* statusUpdateManagers = NULL;
@@ -224,42 +224,28 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
     // NOTE: The flag --authorizers overrides the flag --acls, i.e. if
     // a non default authorizer is requested, it will be used and
     // the contents of --acls will be ignored.
-    // TODO(arojas): Add support for multiple authorizers.
-    if (authorizerName != master::DEFAULT_AUTHORIZER ||
-        flags.acls.isSome()) {
-      Try<Authorizer*> create = Authorizer::create(authorizerName);
+    // TODO(arojas): Consider adding support for multiple authorizers.
+    Result<Authorizer*> authorizer((None()));
+    if (authorizerName != master::DEFAULT_AUTHORIZER) {
+      LOG(INFO) << "Creating '" << authorizerName << "' authorizer";
 
-      if (create.isError()) {
-        EXIT(EXIT_FAILURE) << "Could not create '" << authorizerName
-                           << "' authorizer: " << create.error();
-      }
+      authorizer = Authorizer::create(authorizerName);
+    } else {
+      // `authorizerName` is `DEFAULT_AUTHORIZER` at this point.
+      if (flags.acls.isSome()) {
+        LOG(INFO) << "Creating default '" << authorizerName << "' authorizer";
 
-      authorizer = create.get();
-
-      LOG(INFO) << "Using '" << authorizerName << "' authorizer";
-
-      if (authorizerName == master::DEFAULT_AUTHORIZER) {
-        Try<Nothing> initialize =
-          authorizer.get()->initialize(flags.acls.get());
-
-        if (initialize.isError()) {
-          // Failing to initialize the authorizer leads to undefined
-          // behavior, therefore we default to skip authorization
-          // altogether.
-          EXIT(EXIT_FAILURE) << "Failed to initialize '"
-                             << authorizerName << "' authorizer: "
-                             << initialize.error();
-
-          delete authorizer.get();
-          authorizer = None();
-        }
-      } else if (flags.acls.isSome()) {
-        LOG(WARNING) << "Ignoring contents of --acls flag, because '"
-                     << authorizerName << "' authorizer will be used instead"
-                     << " of the default.";
+        authorizer = Authorizer::create(flags.acls.get());
       }
     }
 
+    if (authorizer.isError()) {
+      EXIT(EXIT_FAILURE) << "Could not create '" << authorizerName
+                         << "' authorizer: " << authorizer.error();
+    } else if (authorizer.isSome()) {
+      authorizer_ = authorizer.get();
+    }
+
     Option<shared_ptr<RateLimiter>> slaveRemovalLimiter = None();
     if (flags.slave_removal_rate_limit.isSome()) {
       // Parse the flag value.
@@ -316,7 +302,7 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
         files,
         contender,
         detector,
-        authorizer,
+        authorizer_,
         slaveRemovalLimiter,
         flags);
 
@@ -427,9 +413,9 @@ void shutdown()
 
     slaves.clear();
 
-    if (authorizer.isSome()) {
-      delete authorizer.get();
-      authorizer = None();
+    if (authorizer_.isSome()) {
+      delete authorizer_.get();
+      authorizer_ = None();
     }
 
     delete detector;

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb28db30/src/master/main.cpp
----------------------------------------------------------------------
diff --git a/src/master/main.cpp b/src/master/main.cpp
index 4263110..7c1656b 100644
--- a/src/master/main.cpp
+++ b/src/master/main.cpp
@@ -81,6 +81,8 @@ using namespace zookeeper;
 
 using mesos::Authorizer;
 using mesos::MasterInfo;
+using mesos::Parameter;
+using mesos::Parameters;
 
 using mesos::master::allocator::Allocator;
 
@@ -346,7 +348,7 @@ int main(int argc, char** argv)
   }
   detector = detector_.get();
 
-  Option<Authorizer*> authorizer = None();
+  Option<Authorizer*> authorizer_ = None();
 
   auto authorizerNames = strings::split(flags.authorizers, ",");
   if (authorizerNames.empty()) {
@@ -360,43 +362,28 @@ int main(int argc, char** argv)
   // NOTE: The flag --authorizers overrides the flag --acls, i.e. if
   // a non default authorizer is requested, it will be used and
   // the contents of --acls will be ignored.
-  // TODO(arojas): Add support for multiple authorizers.
-  if (authorizerName != master::DEFAULT_AUTHORIZER ||
-      flags.acls.isSome()) {
-    Try<Authorizer*> create = Authorizer::create(authorizerName);
+  // TODO(arojas): Consider adding support for multiple authorizers.
+  Result<Authorizer*> authorizer((None()));
+  if (authorizerName != master::DEFAULT_AUTHORIZER) {
+    LOG(INFO) << "Creating '" << authorizerName << "' authorizer";
 
-    if (create.isError()) {
-      EXIT(EXIT_FAILURE) << "Could not create '" << authorizerName
-                         << "' authorizer: " << create.error();
-    }
-
-    authorizer = create.get();
-
-    LOG(INFO) << "Using '" << authorizerName << "' authorizer";
-
-    // Only default authorizer requires initialization, see the comment
-    // for `initialize()` in "mesos/authorizer/authorizer.hpp".
-    if (authorizerName == master::DEFAULT_AUTHORIZER) {
-      Try<Nothing> initialize = authorizer.get()->initialize(flags.acls.get());
-
-      if (initialize.isError()) {
-        // Failing to initialize the authorizer leads to undefined
-        // behavior, therefore we default to skip authorization
-        // altogether.
-        LOG(WARNING) << "Authorization disabled: Failed to initialize '"
-                     << authorizerName << "' authorizer: "
-                     << initialize.error();
+    authorizer = Authorizer::create(authorizerName);
+  } else {
+    // `authorizerName` is `DEFAULT_AUTHORIZER` at this point.
+    if (flags.acls.isSome()) {
+      LOG(INFO) << "Creating default '" << authorizerName << "' authorizer";
 
-        delete authorizer.get();
-        authorizer = None();
-      }
-    } else if (flags.acls.isSome()) {
-      LOG(WARNING) << "Ignoring contents of --acls flag, because '"
-                   << authorizerName << "' authorizer will be used instead "
-                   << " of the default.";
+      authorizer = Authorizer::create(flags.acls.get());
     }
   }
 
+  if (authorizer.isError()) {
+    EXIT(EXIT_FAILURE) << "Could not create '" << authorizerName
+                       << "' authorizer: " << authorizer.error();
+  } else if (authorizer.isSome()) {
+    authorizer_ = authorizer.get();
+  }
+
   Option<shared_ptr<RateLimiter>> slaveRemovalLimiter = None();
   if (flags.slave_removal_rate_limit.isSome()) {
     // Parse the flag value.
@@ -478,7 +465,7 @@ int main(int argc, char** argv)
       &files,
       contender,
       detector,
-      authorizer,
+      authorizer_,
       slaveRemovalLimiter,
       flags);
 
@@ -503,8 +490,8 @@ int main(int argc, char** argv)
   delete contender;
   delete detector;
 
-  if (authorizer.isSome()) {
-    delete authorizer.get();
+  if (authorizer_.isSome()) {
+    delete authorizer_.get();
   }
 
   return EXIT_SUCCESS;

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb28db30/src/tests/authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/authorization_tests.cpp b/src/tests/authorization_tests.cpp
index 2b22970..1c4f7e2 100644
--- a/src/tests/authorization_tests.cpp
+++ b/src/tests/authorization_tests.cpp
@@ -14,12 +14,18 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+#include <string>
+
 #include <gtest/gtest.h>
 
+#include <mesos/mesos.hpp>
+
 #include <mesos/authorizer/authorizer.hpp>
 
 #include <mesos/module/authorizer.hpp>
 
+#include <stout/try.hpp>
+
 #include "authorizer/local/authorizer.hpp"
 
 #include "tests/mesos.hpp"
@@ -31,6 +37,20 @@ namespace mesos {
 namespace internal {
 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 {};
 
@@ -52,13 +72,10 @@ TYPED_TEST(AuthorizationTest, AnyPrincipalRunAsUser)
   acl->mutable_users()->add_values("guest");
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principals "foo" and "bar" can run as "guest".
   mesos::ACL::RunTask request;
   request.mutable_principals()->add_values("foo");
@@ -83,13 +100,10 @@ TYPED_TEST(AuthorizationTest, NoPrincipalRunAsUser)
   acl->mutable_users()->add_values("root");
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principal "foo" cannot run as "root".
   mesos::ACL::RunTask request;
   request.mutable_principals()->add_values("foo");
@@ -107,13 +121,10 @@ TYPED_TEST(AuthorizationTest, PrincipalRunAsAnyUser)
   acl->mutable_users()->set_type(mesos::ACL::Entity::ANY);
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principal "foo" can run as "user1" and "user2".
   mesos::ACL::RunTask request;
   request.mutable_principals()->add_values("foo");
@@ -132,13 +143,10 @@ TYPED_TEST(AuthorizationTest, AnyPrincipalRunAsAnyUser)
   acl->mutable_users()->set_type(mesos::ACL::Entity::ANY);
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principals "foo" and "bar" can run as "user1" and "user2".
   mesos::ACL::RunTask request;
   request.mutable_principals()->add_values("foo");
@@ -168,13 +176,10 @@ TYPED_TEST(AuthorizationTest, OnlySomePrincipalsRunAsSomeUsers)
   acl2->mutable_users()->add_values("user2");
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principals "foo" and "bar" can run as "user1" and "user2".
   mesos::ACL::RunTask request;
   request.mutable_principals()->add_values("foo");
@@ -213,13 +218,10 @@ TYPED_TEST(AuthorizationTest, SomePrincipalOnlySomeUser)
   acl2->mutable_users()->set_type(mesos::ACL::Entity::NONE);
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principal "foo" can run as "user1".
   mesos::ACL::RunTask request;
   request.mutable_principals()->add_values("foo");
@@ -251,13 +253,10 @@ TYPED_TEST(AuthorizationTest, PrincipalRunAsSomeUserRestrictive)
   acl->mutable_users()->add_values("user1");
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principal "foo" can run as "user1".
   mesos::ACL::RunTask request;
   request.mutable_principals()->add_values("foo");
@@ -287,13 +286,10 @@ TYPED_TEST(AuthorizationTest, AnyPrincipalOfferedRole)
   acl->mutable_roles()->add_values("*");
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principals "foo" and "bar" can be offered "*" role's resources.
   mesos::ACL::RegisterFramework request;
   request.mutable_principals()->add_values("foo");
@@ -313,13 +309,10 @@ TYPED_TEST(AuthorizationTest, SomePrincipalsOfferedRole)
   acl->mutable_roles()->add_values("ads");
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principals "foo", "bar" and "baz" (no ACL) can be offered "ads"
   // role's resources.
   mesos::ACL::RegisterFramework request;
@@ -347,13 +340,10 @@ TYPED_TEST(AuthorizationTest, PrincipalOfferedRole)
   acl2->mutable_roles()->add_values("analytics");
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principal "foo" can be offered "analytics" role's resources.
   mesos::ACL::RegisterFramework request;
   request.mutable_principals()->add_values("foo");
@@ -378,13 +368,10 @@ TYPED_TEST(AuthorizationTest, PrincipalNotOfferedAnyRoleRestrictive)
   acl->mutable_roles()->add_values("analytics");
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principal "foo" can be offered "analytics" role's resources.
   mesos::ACL::RegisterFramework request;
   request.mutable_principals()->add_values("foo");
@@ -427,13 +414,10 @@ TYPED_TEST(AuthorizationTest, Reserve)
   acl3->mutable_roles()->set_type(mesos::ACL::Entity::NONE);
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principals "foo" and "bar" can reserve resources for any role,
   // so requests 1 and 2 will pass.
   mesos::ACL::ReserveResources request1;
@@ -502,13 +486,10 @@ TYPED_TEST(AuthorizationTest, Unreserve)
   acl4->mutable_reserver_principals()->set_type(mesos::ACL::Entity::NONE);
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principal "foo" can unreserve its own resources.
   mesos::ACL::UnreserveResources request1;
   request1.mutable_principals()->add_values("foo");
@@ -576,14 +557,11 @@ TYPED_TEST(AuthorizationTest, CreateVolume)
   acl4->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
   acl4->mutable_roles()->set_type(mesos::ACL::Entity::NONE);
 
-  // Create an Authorizer with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  // Create an `Authorizer` with the ACLs.
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principal "foo" can create volumes for any role, so this request will pass.
   mesos::ACL::CreateVolume request1;
   request1.mutable_principals()->add_values("foo");
@@ -647,14 +625,11 @@ TYPED_TEST(AuthorizationTest, DestroyVolume)
   acl4->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
   acl4->mutable_creator_principals()->set_type(mesos::ACL::Entity::NONE);
 
-  // Create an Authorizer with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  // Create an `Authorizer` with the ACLs.
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principal "foo" can destroy its own volumes, so this will pass.
   mesos::ACL::DestroyVolume request1;
   request1.mutable_principals()->add_values("foo");
@@ -722,13 +697,10 @@ TYPED_TEST(AuthorizationTest, SetQuota)
   acl4->mutable_roles()->set_type(mesos::ACL::Entity::NONE);
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principal "foo" can set quota for all roles, so requests 1 and 2 will pass.
   mesos::ACL::SetQuota request1;
   request1.mutable_principals()->add_values("foo");
@@ -800,13 +772,10 @@ TYPED_TEST(AuthorizationTest, RemoveQuota)
   acl4->mutable_quota_principals()->set_type(mesos::ACL::Entity::NONE);
 
   // Create an `Authorizer` with the ACLs.
-  Try<Authorizer*> create = TypeParam::create();
+  Try<Authorizer*> create = TypeParam::create(parameterize(acls));
   ASSERT_SOME(create);
   Owned<Authorizer> authorizer(create.get());
 
-  Try<Nothing> initialized = authorizer.get()->initialize(acls);
-  ASSERT_SOME(initialized);
-
   // Principal "foo" can remove its own quotas, so request 1 will pass.
   mesos::ACL::RemoveQuota request1;
   request1.mutable_principals()->add_values("foo");

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb28db30/src/tests/cluster.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cluster.cpp b/src/tests/cluster.cpp
index 084fb1c..e5796d3 100644
--- a/src/tests/cluster.cpp
+++ b/src/tests/cluster.cpp
@@ -14,6 +14,9 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+#include <string>
+#include <vector>
+
 #include "tests/cluster.hpp"
 
 #include <process/clock.hpp>
@@ -42,6 +45,9 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
+using std::string;
+using std::vector;
+
 Cluster::Masters::Masters(
     Cluster* _cluster,
     const Option<zookeeper::URL>& _url)
@@ -154,23 +160,38 @@ Try<process::PID<master::Master>> Cluster::Masters::start(
 
   if (authorizer.isSome()) {
     CHECK_NOTNULL(authorizer.get());
-  } else if (flags.acls.isSome()) {
-    Try<Authorizer*> local = Authorizer::create(master::DEFAULT_AUTHORIZER);
+  } else {
+    vector<string> authorizerNames = strings::split(flags.authorizers, ",");
+    if (authorizerNames.empty()) {
+      return Error("No authorizer specified");
+    }
 
-    if (local.isError()) {
-      EXIT(EXIT_FAILURE)
-        << "Failed to instantiate the local authorizer: "
-        << local.error();
+    if (authorizerNames.size() > 1) {
+      return Error("Multiple authorizers not supported");
     }
 
-    Try<Nothing> initialized = local.get()->initialize(flags.acls.get());
+    string authorizerName = authorizerNames[0];
+
+    Result<Authorizer*> authorizer((None()));
+    if (authorizerName != master::DEFAULT_AUTHORIZER) {
+      LOG(INFO) << "Creating '" << authorizerName << "' authorizer";
 
-    if (initialized.isError()) {
-      return Error("Failed to initialize the authorizer: " +
-                   initialized.error() + " (see --acls flag)");
+      authorizer = Authorizer::create(authorizerName);
+    } else {
+      // `authorizerName` is `DEFAULT_AUTHORIZER` at this point.
+      if (flags.acls.isSome()) {
+        LOG(INFO) << "Creating default '" << authorizerName << "' authorizer";
+
+        authorizer = Authorizer::create(flags.acls.get());
+      }
     }
 
-    master.authorizer.reset(local.get());
+    if (authorizer.isError()) {
+      return Error("Could not create '" + authorizerName +
+                   "' authorizer: " + authorizer.error());
+    } else if (authorizer.isSome()) {
+      master.authorizer.reset(authorizer.get());
+    }
   }
 
   if (slaveRemovalLimiter.isNone() &&


[2/4] mesos git commit: Moved authorizer.proto to acls.proto.

Posted by vi...@apache.org.
Moved authorizer.proto to acls.proto.

This is the first step towards separating the language used to define
the ACLs from the mechanism to query them.

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


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

Branch: refs/heads/master
Commit: 24c242f6c9b0c5563d1cbd2c8c1748f02e4d4cee
Parents: fb28db3
Author: Alexander Rojas <al...@mesosphere.io>
Authored: Fri Mar 11 16:47:22 2016 -0500
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Mar 11 16:47:22 2016 -0500

----------------------------------------------------------------------
 include/mesos/authorizer/acls.hpp            |  34 ++++
 include/mesos/authorizer/acls.proto          | 186 ++++++++++++++++++++++
 include/mesos/authorizer/authorizer.hpp      |   9 +-
 include/mesos/authorizer/authorizer.proto    | 184 ---------------------
 src/CMakeLists.txt                           |   5 +-
 src/Makefile.am                              |  16 +-
 src/authorizer/acls.cpp                      |  30 ++++
 src/authorizer/authorizer.cpp                |   8 -
 src/common/parse.hpp                         |   2 +-
 src/examples/persistent_volume_framework.cpp |   2 +-
 src/examples/test_authorizer_module.cpp      |   1 +
 src/master/flags.hpp                         |   2 +-
 src/master/quota_handler.cpp                 |   2 +
 13 files changed, 269 insertions(+), 212 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/24c242f6/include/mesos/authorizer/acls.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/authorizer/acls.hpp b/include/mesos/authorizer/acls.hpp
new file mode 100644
index 0000000..c201cf7
--- /dev/null
+++ b/include/mesos/authorizer/acls.hpp
@@ -0,0 +1,34 @@
+// 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_AUTHORIZER_ACLS_HPP__
+#define __MESOS_AUTHORIZER_ACLS_HPP__
+
+// TODO(arojas): Consider removing this file from the public interface.
+
+#include <iosfwd>
+#include <string>
+
+// ONLY USEFUL AFTER RUNNING PROTOC.
+#include <mesos/authorizer/acls.pb.h>
+
+namespace mesos {
+
+std::ostream& operator<<(std::ostream& stream, const ACLs& acls);
+
+} // namespace mesos {
+
+#endif //  __MESOS_AUTHORIZER_ACLS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c242f6/include/mesos/authorizer/acls.proto
----------------------------------------------------------------------
diff --git a/include/mesos/authorizer/acls.proto b/include/mesos/authorizer/acls.proto
new file mode 100644
index 0000000..c50deeb
--- /dev/null
+++ b/include/mesos/authorizer/acls.proto
@@ -0,0 +1,186 @@
+// 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.
+
+// TODO(arojas): Consider removing this file from the public interface.
+
+import "mesos/mesos.proto";
+
+package mesos;
+
+option java_package = "org.apache.mesos";
+option java_outer_classname = "Protos";
+
+
+/**
+ * ACLs used for local authorization (See authorization.md file in the docs).
+ */
+message ACL {
+  // Entity is used to describe a subject(s) or an object(s) of an ACL.
+  // NOTE:
+  // To allow everyone access to an Entity set its type to 'ANY'.
+  // To deny access to an Entity set its type to 'NONE'.
+  message Entity {
+    enum Type {
+      SOME = 0;
+      ANY = 1;
+      NONE = 2;
+    }
+    optional Type type = 1 [default = SOME];
+    repeated string values = 2; // Ignored for ANY/NONE.
+  }
+
+  // ACLs.
+  message RegisterFramework {
+    // Subjects: Framework principals.
+    required Entity principals = 1;
+
+    // Objects: Roles for resource offers.
+    required Entity roles = 2;
+  }
+
+  message RunTask {
+    // Subjects: Framework principals.
+    required Entity principals = 1;
+
+    // Objects: Users to run the tasks/executors as.
+    required Entity users = 2;
+  }
+
+  // Which principals are authorized to shutdown frameworks of other
+  // principals.
+  // TODO(gyliu513): Remove this message at the end of the deprecation
+  // cycle which started with 0.27. It will be fully replaced by
+  // TeardownFramework then.
+  message ShutdownFramework {
+    // Subjects.
+    required Entity principals = 1;
+
+    // Objects.
+    required Entity framework_principals = 2;
+  }
+
+  // Which principals are authorized to teardown frameworks of other
+  // principals.
+  message TeardownFramework {
+    // Subjects.
+    required Entity principals = 1;
+
+    // Objects.
+    required Entity framework_principals = 2;
+  }
+
+  // Specifies which roles a principal can reserve resources for.
+  message ReserveResources {
+    // Subjects: Framework principal or Operator username.
+    required Entity principals = 1;
+
+    // Objects: The principal(s) can reserve resources for these roles.
+    required Entity roles = 2;
+  }
+
+  // Specifies which principals can unreserve which principals'
+  // reserved resources.
+  message UnreserveResources {
+    // Subjects: Framework principal or Operator username.
+    required Entity principals = 1;
+
+    // Objects: Principal of the entity that reserved the resources.
+    required Entity reserver_principals = 2;
+  }
+
+  // Specifies which roles a principal can create volumes for.
+  message CreateVolume {
+    // Subjects: Framework principal or Operator username.
+    required Entity principals = 1;
+
+    // Objects: The principal(s) can create volumes for these roles.
+    required Entity roles = 2;
+  }
+
+  // Specifies which principals can destroy volumes
+  // created by which other principals.
+  message DestroyVolume {
+    // Subjects: Framework principal or Operator username.
+    required Entity principals = 1;
+
+    // Objects: Principal of the entity that created the volume.
+    required Entity creator_principals = 2;
+  }
+
+  // Which principals are authorized to set quotas for given roles.
+  message SetQuota {
+    // Subjects: Operator username.
+    required Entity principals = 1;
+
+    // Objects: The list of roles for which a quota can be set.
+    required Entity roles = 2;
+  }
+
+  // Which principals can remove quotas set by which other principals.
+  message RemoveQuota {
+    // Subjects: Operator username.
+    required Entity principals = 1;
+
+    // Objects: Principal of the entity that set the quota.
+    required Entity quota_principals = 2;
+  }
+
+  // Which principals are authorized to update weights for the given roles.
+  message UpdateWeights {
+    // Subjects: Operator username.
+    required Entity principals = 1;
+
+    // Objects: The list of roles whose weights can be updated.
+    optional Entity roles = 2;
+  }
+}
+
+
+/**
+ * Collection of ACL.
+ *
+ * Each authorization request is evaluated against the ACLs in the order
+ * they are defined.
+ *
+ * For simplicity, the ACLs for a given action are not aggregated even
+ * when they have the same subjects or objects. The first ACL that
+ * matches the request determines whether that request should be
+ * permitted or not. An ACL matches iff both the subjects
+ * (e.g., clients, principals) and the objects (e.g., urls, users,
+ * roles) of the ACL match the request.
+ *
+ * If none of the ACLs match the request, the 'permissive' field
+ * determines whether the request should be permitted or not.
+ *
+ * TODO(vinod): Do aggregation of ACLs when possible.
+ *
+ */
+message ACLs {
+  optional bool permissive = 1 [default = true];
+  repeated ACL.RegisterFramework register_frameworks = 2;
+  repeated ACL.RunTask run_tasks = 3;
+  // TODO(gyliu513): Remove this shutdown_frameworks at the
+  // end of the deprecation cycle which started on 0.27.
+  repeated ACL.ShutdownFramework shutdown_frameworks = 4;
+  repeated ACL.ReserveResources reserve_resources = 5;
+  repeated ACL.UnreserveResources unreserve_resources = 6;
+  repeated ACL.CreateVolume create_volumes = 7;
+  repeated ACL.DestroyVolume destroy_volumes = 8;
+  repeated ACL.SetQuota set_quotas = 9;
+  repeated ACL.RemoveQuota remove_quotas = 10;
+  repeated ACL.TeardownFramework teardown_frameworks = 11;
+  repeated ACL.UpdateWeights update_weights = 12;
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c242f6/include/mesos/authorizer/authorizer.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/authorizer/authorizer.hpp b/include/mesos/authorizer/authorizer.hpp
index bcb1068..3e838fa 100644
--- a/include/mesos/authorizer/authorizer.hpp
+++ b/include/mesos/authorizer/authorizer.hpp
@@ -17,13 +17,9 @@
 #ifndef __MESOS_AUTHORIZER_AUTHORIZER_HPP__
 #define __MESOS_AUTHORIZER_AUTHORIZER_HPP__
 
-#include <iosfwd>
-#include <string>
-
 #include <mesos/mesos.hpp>
 
-// ONLY USEFUL AFTER RUNNING PROTOC.
-#include <mesos/authorizer/authorizer.pb.h>
+#include <mesos/authorizer/acls.hpp>
 
 #include <process/future.hpp>
 
@@ -223,9 +219,6 @@ protected:
   Authorizer() {}
 };
 
-
-std::ostream& operator<<(std::ostream& stream, const ACLs& acls);
-
 } // namespace mesos {
 
 #endif // __MESOS_AUTHORIZER_AUTHORIZER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c242f6/include/mesos/authorizer/authorizer.proto
----------------------------------------------------------------------
diff --git a/include/mesos/authorizer/authorizer.proto b/include/mesos/authorizer/authorizer.proto
deleted file mode 100644
index d65377c..0000000
--- a/include/mesos/authorizer/authorizer.proto
+++ /dev/null
@@ -1,184 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-import "mesos/mesos.proto";
-
-package mesos;
-
-option java_package = "org.apache.mesos";
-option java_outer_classname = "Protos";
-
-
-/**
- * ACLs used for local authorization (See authorization.md file in the docs).
- */
-message ACL {
-  // Entity is used to describe a subject(s) or an object(s) of an ACL.
-  // NOTE:
-  // To allow everyone access to an Entity set its type to 'ANY'.
-  // To deny access to an Entity set its type to 'NONE'.
-  message Entity {
-    enum Type {
-      SOME = 0;
-      ANY = 1;
-      NONE = 2;
-    }
-    optional Type type = 1 [default = SOME];
-    repeated string values = 2; // Ignored for ANY/NONE.
-  }
-
-  // ACLs.
-  message RegisterFramework {
-    // Subjects: Framework principals.
-    required Entity principals = 1;
-
-    // Objects: Roles for resource offers.
-    required Entity roles = 2;
-  }
-
-  message RunTask {
-    // Subjects: Framework principals.
-    required Entity principals = 1;
-
-    // Objects: Users to run the tasks/executors as.
-    required Entity users = 2;
-  }
-
-  // Which principals are authorized to shutdown frameworks of other
-  // principals.
-  // TODO(gyliu513): Remove this message at the end of the deprecation
-  // cycle which started with 0.27. It will be fully replaced by
-  // TeardownFramework then.
-  message ShutdownFramework {
-    // Subjects.
-    required Entity principals = 1;
-
-    // Objects.
-    required Entity framework_principals = 2;
-  }
-
-  // Which principals are authorized to teardown frameworks of other
-  // principals.
-  message TeardownFramework {
-    // Subjects.
-    required Entity principals = 1;
-
-    // Objects.
-    required Entity framework_principals = 2;
-  }
-
-  // Specifies which roles a principal can reserve resources for.
-  message ReserveResources {
-    // Subjects: Framework principal or Operator username.
-    required Entity principals = 1;
-
-    // Objects: The principal(s) can reserve resources for these roles.
-    required Entity roles = 2;
-  }
-
-  // Specifies which principals can unreserve which principals'
-  // reserved resources.
-  message UnreserveResources {
-    // Subjects: Framework principal or Operator username.
-    required Entity principals = 1;
-
-    // Objects: Principal of the entity that reserved the resources.
-    required Entity reserver_principals = 2;
-  }
-
-  // Specifies which roles a principal can create volumes for.
-  message CreateVolume {
-    // Subjects: Framework principal or Operator username.
-    required Entity principals = 1;
-
-    // Objects: The principal(s) can create volumes for these roles.
-    required Entity roles = 2;
-  }
-
-  // Specifies which principals can destroy volumes
-  // created by which other principals.
-  message DestroyVolume {
-    // Subjects: Framework principal or Operator username.
-    required Entity principals = 1;
-
-    // Objects: Principal of the entity that created the volume.
-    required Entity creator_principals = 2;
-  }
-
-  // Which principals are authorized to set quotas for given roles.
-  message SetQuota {
-    // Subjects: Operator username.
-    required Entity principals = 1;
-
-    // Objects: The list of roles for which a quota can be set.
-    required Entity roles = 2;
-  }
-
-  // Which principals can remove quotas set by which other principals.
-  message RemoveQuota {
-    // Subjects: Operator username.
-    required Entity principals = 1;
-
-    // Objects: Principal of the entity that set the quota.
-    required Entity quota_principals = 2;
-  }
-
-  // Which principals are authorized to update weights for the given roles.
-  message UpdateWeights {
-    // Subjects: Operator username.
-    required Entity principals = 1;
-
-    // Objects: The list of roles whose weights can be updated.
-    optional Entity roles = 2;
-  }
-}
-
-
-/**
- * Collection of ACL.
- *
- * Each authorization request is evaluated against the ACLs in the order
- * they are defined.
- *
- * For simplicity, the ACLs for a given action are not aggregated even
- * when they have the same subjects or objects. The first ACL that
- * matches the request determines whether that request should be
- * permitted or not. An ACL matches iff both the subjects
- * (e.g., clients, principals) and the objects (e.g., urls, users,
- * roles) of the ACL match the request.
- *
- * If none of the ACLs match the request, the 'permissive' field
- * determines whether the request should be permitted or not.
- *
- * TODO(vinod): Do aggregation of ACLs when possible.
- *
- */
-message ACLs {
-  optional bool permissive = 1 [default = true];
-  repeated ACL.RegisterFramework register_frameworks = 2;
-  repeated ACL.RunTask run_tasks = 3;
-  // TODO(gyliu513): Remove this shutdown_frameworks at the
-  // end of the deprecation cycle which started on 0.27.
-  repeated ACL.ShutdownFramework shutdown_frameworks = 4;
-  repeated ACL.ReserveResources reserve_resources = 5;
-  repeated ACL.UnreserveResources unreserve_resources = 6;
-  repeated ACL.CreateVolume create_volumes = 7;
-  repeated ACL.DestroyVolume destroy_volumes = 8;
-  repeated ACL.SetQuota set_quotas = 9;
-  repeated ACL.RemoveQuota remove_quotas = 10;
-  repeated ACL.TeardownFramework teardown_frameworks = 11;
-  repeated ACL.UpdateWeights update_weights = 12;
-}

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c242f6/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index e9f7c3a..d101181 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -21,9 +21,9 @@ include(MesosProtobuf)
 # Build the protobuf structs.
 PROTOC_TO_INCLUDE_DIR(MESOS            mesos/mesos)
 PROTOC_TO_INCLUDE_DIR(V1_MESOS         mesos/v1/mesos)
+PROTOC_TO_INCLUDE_DIR(ACLS             mesos/authorizer/acls)
 PROTOC_TO_INCLUDE_DIR(APPC_SPEC        mesos/appc/spec)
 PROTOC_TO_INCLUDE_DIR(AUTHENTICATION   mesos/authentication/authentication)
-PROTOC_TO_INCLUDE_DIR(AUTHORIZATION    mesos/authorizer/authorizer)
 PROTOC_TO_INCLUDE_DIR(CONTAINERIZER    mesos/containerizer/containerizer)
 PROTOC_TO_INCLUDE_DIR(DOCKER_SPEC      mesos/docker/spec)
 PROTOC_TO_INCLUDE_DIR(DOCKER_V1        mesos/docker/v1)
@@ -53,9 +53,9 @@ PROTOC_TO_SRC_DIR(MESSAGE slave/containerizer/mesos/provisioner/docker/message)
 set(MESOS_PROTOBUF_SRC
   ${MESOS_PROTO_CC}
   ${V1_MESOS_PROTO_CC}
+  ${ACLS_PROTO_CC}
   ${APPC_SPEC_PROTO_CC}
   ${AUTHENTICATION_PROTO_CC}
-  ${AUTHORIZATION_PROTO_CC}
   ${CONTAINERIZER_PROTO_CC}
   ${DOCKER_SPEC_PROTO_CC}
   ${DOCKER_V1_PROTO_CC}
@@ -101,6 +101,7 @@ set(AUTHENTICATION_SRC
   )
 
 set(AUTHORIZER_SRC
+  authorizer/acls.cpp
   authorizer/authorizer.cpp
   authorizer/local/authorizer.cpp
   )

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c242f6/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index b24f0f5..24d5ba1 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -155,10 +155,10 @@ endif
 
 # First, let's define necessary protocol buffer files.
 
+ACLS_PROTO = $(top_srcdir)/include/mesos/authorizer/acls.proto
 ALLOCATOR_PROTO = $(top_srcdir)/include/mesos/master/allocator.proto
 APPC_SPEC_PROTO = $(top_srcdir)/include/mesos/appc/spec.proto
 AUTHENTICATION_PROTO = $(top_srcdir)/include/mesos/authentication/authentication.proto
-AUTHORIZATION_PROTO = $(top_srcdir)/include/mesos/authorizer/authorizer.proto
 CONTAINERIZER_PROTO = $(top_srcdir)/include/mesos/containerizer/containerizer.proto
 DOCKER_SPEC_PROTO = $(top_srcdir)/include/mesos/docker/spec.proto
 DOCKER_V1_PROTO = $(top_srcdir)/include/mesos/docker/v1.proto
@@ -185,8 +185,8 @@ CXX_PROTOS =								\
   ../include/mesos/appc/spec.pb.h					\
   ../include/mesos/authentication/authentication.pb.cc			\
   ../include/mesos/authentication/authentication.pb.h			\
-  ../include/mesos/authorizer/authorizer.pb.cc				\
-  ../include/mesos/authorizer/authorizer.pb.h				\
+  ../include/mesos/authorizer/acls.pb.cc				\
+  ../include/mesos/authorizer/acls.pb.h					\
   ../include/mesos/containerizer/containerizer.pb.cc			\
   ../include/mesos/containerizer/containerizer.pb.h			\
   ../include/mesos/docker/spec.pb.cc					\
@@ -406,11 +406,12 @@ nodist_authentication_HEADERS =						\
 authorizerdir = $(pkgincludedir)/authorizer
 
 authorizer_HEADERS =							\
-  $(top_srcdir)/include/mesos/authorizer/authorizer.hpp			\
-  $(top_srcdir)/include/mesos/authorizer/authorizer.proto
+  $(top_srcdir)/include/mesos/authorizer/acls.hpp			\
+  $(top_srcdir)/include/mesos/authorizer/acls.proto			\
+  $(top_srcdir)/include/mesos/authorizer/authorizer.hpp
 
 nodist_authorizer_HEADERS =						\
-  ../include/mesos/authorizer/authorizer.pb.h
+  ../include/mesos/authorizer/acls.pb.h
 
 containerizerdir = $(pkgincludedir)/containerizer
 
@@ -591,6 +592,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   authentication/cram_md5/authenticator.cpp				\
   authentication/cram_md5/auxprop.cpp					\
   authentication/http/basic_authenticator_factory.cpp			\
+  authorizer/acls.cpp							\
   authorizer/authorizer.cpp						\
   authorizer/local/authorizer.cpp					\
   common/attributes.cpp							\
@@ -1031,10 +1033,10 @@ lib_LTLIBRARIES += libmesos.la
 
 # Include as part of the distribution.
 libmesos_la_SOURCES =							\
+  $(ACLS_PROTO)								\
   $(ALLOCATOR_PROTO)							\
   $(APPC_SPEC_PROTO)							\
   $(AUTHENTICATION_PROTO)						\
-  $(AUTHORIZATION_PROTO)						\
   $(CONTAINERIZER_PROTO)						\
   $(EXECUTOR_PROTO)							\
   $(DOCKER_SPEC_PROTO)							\

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c242f6/src/authorizer/acls.cpp
----------------------------------------------------------------------
diff --git a/src/authorizer/acls.cpp b/src/authorizer/acls.cpp
new file mode 100644
index 0000000..fde971b
--- /dev/null
+++ b/src/authorizer/acls.cpp
@@ -0,0 +1,30 @@
+// 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/authorizer/acls.hpp>
+
+#include <ostream>
+
+using std::ostream;
+
+namespace mesos {
+
+ostream& operator<<(ostream& stream, const ACLs& acls)
+{
+  return stream << acls.DebugString();
+}
+
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c242f6/src/authorizer/authorizer.cpp
----------------------------------------------------------------------
diff --git a/src/authorizer/authorizer.cpp b/src/authorizer/authorizer.cpp
index 74dfccd..752c25d 100644
--- a/src/authorizer/authorizer.cpp
+++ b/src/authorizer/authorizer.cpp
@@ -14,8 +14,6 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-#include <ostream>
-
 #include <mesos/authorizer/authorizer.hpp>
 
 #include <mesos/module/authorizer.hpp>
@@ -47,10 +45,4 @@ Try<Authorizer*> Authorizer::create(const ACLs& acls)
   return LocalAuthorizer::create(acls);
 }
 
-
-ostream& operator<<(ostream& stream, const ACLs& acls)
-{
-  return stream << acls.DebugString();
-}
-
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c242f6/src/common/parse.hpp
----------------------------------------------------------------------
diff --git a/src/common/parse.hpp b/src/common/parse.hpp
index 78c7cf1..9535fad 100644
--- a/src/common/parse.hpp
+++ b/src/common/parse.hpp
@@ -15,7 +15,7 @@
 
 #include <mesos/mesos.hpp>
 
-#include <mesos/authorizer/authorizer.hpp>
+#include <mesos/authorizer/acls.hpp>
 
 #include <mesos/module/module.hpp>
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c242f6/src/examples/persistent_volume_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/persistent_volume_framework.cpp b/src/examples/persistent_volume_framework.cpp
index 4218b15..3848e24 100644
--- a/src/examples/persistent_volume_framework.cpp
+++ b/src/examples/persistent_volume_framework.cpp
@@ -26,7 +26,7 @@
 #include <mesos/scheduler.hpp>
 #include <mesos/type_utils.hpp>
 
-#include <mesos/authorizer/authorizer.hpp>
+#include <mesos/authorizer/acls.hpp>
 
 #include <stout/flags.hpp>
 #include <stout/format.hpp>

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c242f6/src/examples/test_authorizer_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_authorizer_module.cpp b/src/examples/test_authorizer_module.cpp
index 1f0a770..19ec7cd 100644
--- a/src/examples/test_authorizer_module.cpp
+++ b/src/examples/test_authorizer_module.cpp
@@ -19,6 +19,7 @@
 #include <mesos/mesos.hpp>
 #include <mesos/module.hpp>
 
+#include <mesos/authorizer/acls.hpp>
 #include <mesos/authorizer/authorizer.hpp>
 
 #include <mesos/module/authorizer.hpp>

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c242f6/src/master/flags.hpp
----------------------------------------------------------------------
diff --git a/src/master/flags.hpp b/src/master/flags.hpp
index 6f53099..f8d2cc4 100644
--- a/src/master/flags.hpp
+++ b/src/master/flags.hpp
@@ -25,7 +25,7 @@
 
 #include <mesos/mesos.hpp>
 
-#include <mesos/authorizer/authorizer.hpp>
+#include <mesos/authorizer/acls.hpp>
 
 #include <mesos/module/module.hpp>
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/24c242f6/src/master/quota_handler.cpp
----------------------------------------------------------------------
diff --git a/src/master/quota_handler.cpp b/src/master/quota_handler.cpp
index a41c91f..f361abb 100644
--- a/src/master/quota_handler.cpp
+++ b/src/master/quota_handler.cpp
@@ -22,6 +22,8 @@
 
 #include <mesos/resources.hpp>
 
+#include <mesos/authorizer/authorizer.hpp>
+
 #include <mesos/quota/quota.hpp>
 
 #include <process/defer.hpp>


[4/4] mesos git commit: Implemented a generalized interface for the authorizer.

Posted by vi...@apache.org.
Implemented a generalized interface for the authorizer.

Implements the [Generic Authorizer Interface v
0.3.1](https://docs.google.com/document/d/1-XARWJFUq0r_TgRHz_472NvLZNjbqE4G8c2JL44OSMQ).

It effectively separates the language used to define ACLs from the
language used to query it allowing for arbitrary identity server
backends.

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


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

Branch: refs/heads/master
Commit: 482dc141c6989deb1318979c3978d38a1642d291
Parents: 24c242f
Author: Alexander Rojas <al...@mesosphere.io>
Authored: Fri Mar 11 16:47:31 2016 -0500
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Mar 11 16:47:31 2016 -0500

----------------------------------------------------------------------
 include/mesos/authorizer/authorizer.hpp   |  179 +---
 include/mesos/authorizer/authorizer.proto |   64 ++
 src/CMakeLists.txt                        |    2 +
 src/Makefile.am                           |   10 +-
 src/authorizer/authorizer.cpp             |    2 +
 src/authorizer/local/authorizer.cpp       |  405 +++-----
 src/authorizer/local/authorizer.hpp       |   24 +-
 src/master/http.cpp                       |   14 +-
 src/master/master.cpp                     |  145 ++-
 src/master/quota_handler.cpp              |   24 +-
 src/master/weights_handler.cpp            |   33 +-
 src/tests/authorization_tests.cpp         | 1186 +++++++++++++++---------
 src/tests/master_authorization_tests.cpp  |   20 +-
 src/tests/mesos.cpp                       |   32 +-
 src/tests/mesos.hpp                       |   22 +-
 src/tests/reconciliation_tests.cpp        |    2 +-
 16 files changed, 1177 insertions(+), 987 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/include/mesos/authorizer/authorizer.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/authorizer/authorizer.hpp b/include/mesos/authorizer/authorizer.hpp
index 3e838fa..ed5f9e7 100644
--- a/include/mesos/authorizer/authorizer.hpp
+++ b/include/mesos/authorizer/authorizer.hpp
@@ -19,7 +19,8 @@
 
 #include <mesos/mesos.hpp>
 
-#include <mesos/authorizer/acls.hpp>
+// ONLY USEFUL AFTER RUNNING PROTOC.
+#include <mesos/authorizer/authorizer.pb.h>
 
 #include <process/future.hpp>
 
@@ -29,24 +30,23 @@
 
 namespace mesos {
 
+class ACLs;
+
 /**
- * An interface for authorization of actions with ACLs. Refer to
- * "authorizer.proto" and "docs/authorization.md" for the details
- * regarding the authorization mechanism.
+ * This interface is used to enable an identity service or any other
+ * back end to check authorization policies for a set of predefined
+ * actions.
  *
- * Each `authorize()` function returns `Future<bool>`, which has the same
- * meaning for all functions. If the action is allowed, the future is set to
- * `true`, otherwise to `false`. A third possible outcome is that the future
- * fails, which indicates that the request could not be completed at the
- * moment. This may be a temporary condition.
+ * The `authorized()` method returns `Future<bool>`. If the action is
+ * allowed, the future is set to `true`, otherwise to `false`. A third
+ * possible outcome is that the future fails, which usually indicates
+ * that the back end could not be contacted or it does not understand
+ * the requested action. This may be a temporary condition.
  *
- * NOTE: Any request allows bundling multiple values for each entity, which
- * are often principals. Though the default authorizer implementation
- * (`LocalAuthorizer`) supports this feature, Mesos code currently does not
- * authorize multiple principals in a single call.
+ * A description of the behavior of the default implementation of this
+ * interface can be found in "docs/authorization.md".
  *
  * @see authorizer.proto
- * @see docs/authorization.md
  */
 class Authorizer
 {
@@ -77,143 +77,22 @@ public:
   virtual ~Authorizer() {}
 
   /**
-   * Verifies whether a principal can register a framework with a specific role.
-   *
-   * @param request `ACL::RegisterFramework` packing all the parameters
-   *     needed to verify if the given principal can register a framework
-   *     with the specified role.
-   *
-   * @return true if the principal can register the framework with the
-   *     specified role or false otherwise. A failed future indicates a
-   *     problem processing the request; the request can be retried.
-   */
-  virtual process::Future<bool> authorize(
-      const ACL::RegisterFramework& request) = 0;
-
-  /**
-   * Verifies whether a principal can run tasks as the given UNIX user.
-   *
-   * @param request `ACL::RunTask` packing all the parameters needed to verify
-   *     if the given principal can launch a task using the specified UNIX user.
-   *
-   * @return true if the principal can run a task using the given UNIX user
-   *     name, false otherwise. A failed future indicates a problem processing
-   *     the request; the request can be retried.
-   */
-  virtual process::Future<bool> authorize(
-      const ACL::RunTask& request) = 0;
-
-  /**
-   * Verifies whether a principal can teardown a framework launched by another
-   * principal.
-   *
-   * @param request `ACL::TeardownFramework` packing all the parameters needed
-   *     to verify the given principal can teardown a framework originally
-   *     registered by a (potentially different) framework principal.
-   *
-   * @return true if the principal can teardown a framework registered by the
-   *     framework principal, false otherwise. A failed future indicates a
-   *     problem processing the request; the request can be retried.
-   */
-  virtual process::Future<bool> authorize(
-      const ACL::TeardownFramework& request) = 0;
-
-  /**
-   * Verifies whether a principal can reserve particular resources.
-   *
-   * @param request `ACL::ReserveResources` packing all the parameters needed to
-   *     verify the given principal can reserve one or more types of resources.
-   *
-   * @return true if the principal can reserve the resources, false otherwise. A
-   *     failed future indicates a problem processing the request; the request
-   *     can be retried.
-   */
-  virtual process::Future<bool> authorize(
-      const ACL::ReserveResources& request) = 0;
-
-  /**
-   * Verifies whether a principal can unreserve resources reserved by another
-   * principal.
-   *
-   * @param request `ACL::UnreserveResources` packing all the parameters needed
-   *     to verify the given principal can unreserve resources which were
-   *     reserved by the reserver principal contained in the request.
-   *
-   * @return true if the principal can unreserve resources which were reserved
-   *     by the reserver principal, false otherwise. A failed future indicates
-   *     a problem processing the request; the request can be retried.
-   */
-  virtual process::Future<bool> authorize(
-      const ACL::UnreserveResources& request) = 0;
-
-  /**
-   * Verifies whether a principal can create a persistent volume.
-   *
-   * @param request `ACL::CreateVolume` packing all the parameters needed to
-   *     verify the given principal can create the given type of volume.
-   *
-   * @return true if the principal can create a persistent volume, false
-   *     otherwise. A failed future indicates a problem processing the
-   *     request; the request can be retried.
-   */
-  virtual process::Future<bool> authorize(
-      const ACL::CreateVolume& request) = 0;
-
-  /**
-   * Verifies whether a principal can destroy a volume created by another
-   * principal.
-   *
-   * @param request `ACL::DestroyVolume` packing all the parameters needed to
-   *     verify the given principal can destroy volumes which were created by
-   *     the creator principal contained in the request.
-   *
-   * @return true if the principal can destroy volumes which were created by
-   *     the creator principal, false otherwise. A failed future indicates a
-   *     problem processing the request; the request can be retried.
-   */
-  virtual process::Future<bool> authorize(
-      const ACL::DestroyVolume& request) = 0;
-
-  /**
-   * Verifies whether a principal can set a quota for a specific role.
-   *
-   * @param request `ACL::SetQuota` packing all the parameters needed to verify
-   *     if the given principal can set a quota for the specified role.
-   *
-   * @return true if the principal can set a quota for the specified role,
-   *     false otherwise. A failed future indicates a problem processing
-   *     the request; the request can be retried.
-   */
-  virtual process::Future<bool> authorize(
-      const ACL::SetQuota& request) = 0;
-
-  /**
-   * Verifies whether a principal can remove a quota set by another principal.
-   *
-   * @param request `ACL::RemoveQuota` packing all the parameters needed to
-   *     verify the given principal can remove quotas which were set by the
-   *     principal contained in the set request.
-   *
-   * @return true if the principal can remove quotas which were set by the quota
-   *     principal, false otherwise. A failed future indicates a problem
-   *     processing the request; the request can be retried.
-   */
-  virtual process::Future<bool> authorize(
-      const ACL::RemoveQuota& request) = 0;
-
-  /**
-   * Verifies whether a principal can update the weight for the specific roles.
-   *
-   * @param request `ACL::UpdateWeights` packing all the parameters needed to
-   *     verify if the given principal is allowed to update the weight of the
-   *     specified roles.
-   *
-   * @return true if the principal is allowed to update the weight for every
-   *     specified role, false otherwise. A failed future indicates a problem
-   *     processing the request; the request can be retried.
+   * Checks with the identity server back end whether `request` is
+   * allowed by the policies of the identity server, i.e. `request.subject`
+   * can perform `request.action` with `request.object`. For details
+   * on how the request is built and what its parts are, refer to
+   * "authorizer.proto".
+   *
+   * @param request `authorization::Request` instance packing all the
+   *     parameters needed to verify whether a subject can perform
+   *     a given action with an object.
+   *
+   * @return `true` if the action is allowed, the future is set to `true`,
+   *     otherwise `false`. A failed future indicates a problem processing
+   *     the request, and it might be retried in the future.
    */
-  virtual process::Future<bool> authorize(
-      const ACL::UpdateWeights& request) = 0;
+  virtual process::Future<bool> authorized(
+      const authorization::Request& request) = 0;
 
 protected:
   Authorizer() {}

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/include/mesos/authorizer/authorizer.proto
----------------------------------------------------------------------
diff --git a/include/mesos/authorizer/authorizer.proto b/include/mesos/authorizer/authorizer.proto
new file mode 100644
index 0000000..944a493
--- /dev/null
+++ b/include/mesos/authorizer/authorizer.proto
@@ -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.
+
+import "mesos/mesos.proto";
+
+package mesos.authorization;
+
+option java_package = "org.apache.mesos.authorization";
+option java_outer_classname = "Protos";
+
+
+// A `Subject` is the entity which desires to execute an `Action` on
+// an `Object`. The `Subject` is described by its `value`. When
+// `value` is not set the subject is unknown.
+message Subject {
+  optional string value = 1;
+}
+
+
+// An `Object` is the entity on which a `Subject` wishes to execute an
+// `Action`. An `Object` is described by its value and the action in
+// the request, e.g. when registering frameworks, the value represents
+// a role, while when running a task it is the user name the task will
+// be run as. If the value is not set, the object is unknown.
+message Object {
+  optional string value = 1;
+}
+
+
+// List of authorizable actions supported in Mesos.
+enum Action {
+  REGISTER_FRAMEWORK_WITH_ROLE = 1;
+  RUN_TASK_WITH_USER = 2;
+  TEARDOWN_FRAMEWORK_WITH_PRINCIPAL = 3;
+  RESERVE_RESOURCES_WITH_ROLE = 4;
+  UNRESERVE_RESOURCES_WITH_PRINCIPAL = 5;
+  CREATE_VOLUME_WITH_ROLE = 6;
+  DESTROY_VOLUME_WITH_PRINCIPAL = 7;
+  SET_QUOTA_WITH_ROLE = 8;
+  DESTROY_QUOTA_WITH_PRINCIPAL = 9;
+  UPDATE_WEIGHTS_WITH_ROLE = 10;
+}
+
+
+// A `Request` is a <subject, action, object> tuple which can be read
+// as "Can `subject` perform `action` with `object`?".
+message Request {
+  required Subject subject = 1;
+  required Action action = 2;
+  required Object object = 3;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index d101181..bfd2141 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -24,6 +24,7 @@ PROTOC_TO_INCLUDE_DIR(V1_MESOS         mesos/v1/mesos)
 PROTOC_TO_INCLUDE_DIR(ACLS             mesos/authorizer/acls)
 PROTOC_TO_INCLUDE_DIR(APPC_SPEC        mesos/appc/spec)
 PROTOC_TO_INCLUDE_DIR(AUTHENTICATION   mesos/authentication/authentication)
+PROTOC_TO_INCLUDE_DIR(AUTHORIZATION    mesos/authorizer/authorization)
 PROTOC_TO_INCLUDE_DIR(CONTAINERIZER    mesos/containerizer/containerizer)
 PROTOC_TO_INCLUDE_DIR(DOCKER_SPEC      mesos/docker/spec)
 PROTOC_TO_INCLUDE_DIR(DOCKER_V1        mesos/docker/v1)
@@ -56,6 +57,7 @@ set(MESOS_PROTOBUF_SRC
   ${ACLS_PROTO_CC}
   ${APPC_SPEC_PROTO_CC}
   ${AUTHENTICATION_PROTO_CC}
+  ${AUTHORIZATION_PROTO_CC}
   ${CONTAINERIZER_PROTO_CC}
   ${DOCKER_SPEC_PROTO_CC}
   ${DOCKER_V1_PROTO_CC}

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 24d5ba1..f59ae12 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -159,6 +159,7 @@ ACLS_PROTO = $(top_srcdir)/include/mesos/authorizer/acls.proto
 ALLOCATOR_PROTO = $(top_srcdir)/include/mesos/master/allocator.proto
 APPC_SPEC_PROTO = $(top_srcdir)/include/mesos/appc/spec.proto
 AUTHENTICATION_PROTO = $(top_srcdir)/include/mesos/authentication/authentication.proto
+AUTHORIZATION_PROTO = $(top_srcdir)/include/mesos/authorizer/authorizer.proto
 CONTAINERIZER_PROTO = $(top_srcdir)/include/mesos/containerizer/containerizer.proto
 DOCKER_SPEC_PROTO = $(top_srcdir)/include/mesos/docker/spec.proto
 DOCKER_V1_PROTO = $(top_srcdir)/include/mesos/docker/v1.proto
@@ -187,6 +188,8 @@ CXX_PROTOS =								\
   ../include/mesos/authentication/authentication.pb.h			\
   ../include/mesos/authorizer/acls.pb.cc				\
   ../include/mesos/authorizer/acls.pb.h					\
+  ../include/mesos/authorizer/authorizer.pb.cc				\
+  ../include/mesos/authorizer/authorizer.pb.h				\
   ../include/mesos/containerizer/containerizer.pb.cc			\
   ../include/mesos/containerizer/containerizer.pb.h			\
   ../include/mesos/docker/spec.pb.cc					\
@@ -408,10 +411,12 @@ authorizerdir = $(pkgincludedir)/authorizer
 authorizer_HEADERS =							\
   $(top_srcdir)/include/mesos/authorizer/acls.hpp			\
   $(top_srcdir)/include/mesos/authorizer/acls.proto			\
-  $(top_srcdir)/include/mesos/authorizer/authorizer.hpp
+  $(top_srcdir)/include/mesos/authorizer/authorizer.hpp			\
+  $(top_srcdir)/include/mesos/authorizer/authorizer.proto
 
 nodist_authorizer_HEADERS =						\
-  ../include/mesos/authorizer/acls.pb.h
+  ../include/mesos/authorizer/acls.pb.h					\
+  ../include/mesos/authorizer/authorizer.pb.h
 
 containerizerdir = $(pkgincludedir)/containerizer
 
@@ -1037,6 +1042,7 @@ libmesos_la_SOURCES =							\
   $(ALLOCATOR_PROTO)							\
   $(APPC_SPEC_PROTO)							\
   $(AUTHENTICATION_PROTO)						\
+  $(AUTHORIZATION_PROTO)						\
   $(CONTAINERIZER_PROTO)						\
   $(EXECUTOR_PROTO)							\
   $(DOCKER_SPEC_PROTO)							\

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/authorizer/authorizer.cpp
----------------------------------------------------------------------
diff --git a/src/authorizer/authorizer.cpp b/src/authorizer/authorizer.cpp
index 752c25d..1e1a9b1 100644
--- a/src/authorizer/authorizer.cpp
+++ b/src/authorizer/authorizer.cpp
@@ -16,6 +16,8 @@
 
 #include <mesos/authorizer/authorizer.hpp>
 
+#include <mesos/authorizer/acls.hpp>
+
 #include <mesos/module/authorizer.hpp>
 
 #include <stout/path.hpp>

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/authorizer/local/authorizer.cpp
----------------------------------------------------------------------
diff --git a/src/authorizer/local/authorizer.cpp b/src/authorizer/local/authorizer.cpp
index 3dc69f0..0f0d927 100644
--- a/src/authorizer/local/authorizer.cpp
+++ b/src/authorizer/local/authorizer.cpp
@@ -17,8 +17,10 @@
 #include "authorizer/local/authorizer.hpp"
 
 #include <string>
+#include <vector>
 
 #include <mesos/mesos.hpp>
+#include <mesos/authorizer/acls.hpp>
 
 #include <process/dispatch.hpp>
 #include <process/future.hpp>
@@ -39,10 +41,18 @@ using process::Future;
 using process::dispatch;
 
 using std::string;
+using std::vector;
 
 namespace mesos {
 namespace internal {
 
+struct GenericACL
+{
+  ACL::Entity subjects;
+  ACL::Entity objects;
+};
+
+
 class LocalAuthorizerProcess : public ProtobufProcess<LocalAuthorizerProcess>
 {
 public:
@@ -76,159 +86,157 @@ public:
     acls.clear_shutdown_frameworks();
   }
 
-  Future<bool> authorize(const ACL::RegisterFramework& request)
+  Future<bool> authorized(const authorization::Request& request)
   {
-    foreach (const ACL::RegisterFramework& acl, acls.register_frameworks()) {
-      // ACL matches if both subjects and objects match.
-      if (matches(request.principals(), acl.principals()) &&
-          matches(request.roles(), acl.roles())) {
-        // ACL is allowed if both subjects and objects are allowed.
-        return allows(request.principals(), acl.principals()) &&
-               allows(request.roles(), acl.roles());
-      }
-    }
+    vector<GenericACL> acls_;
 
-    return acls.permissive(); // None of the ACLs match.
-  }
+    switch (request.action()) {
+      case authorization::REGISTER_FRAMEWORK_WITH_ROLE:
+        for (const ACL::RegisterFramework& acl : acls.register_frameworks()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.roles();
 
-  Future<bool> authorize(const ACL::RunTask& request)
-  {
-    foreach (const ACL::RunTask& acl, acls.run_tasks()) {
-      // ACL matches if both subjects and objects match.
-      if (matches(request.principals(), acl.principals()) &&
-          matches(request.users(), acl.users())) {
-        // ACL is allowed if both subjects and objects are allowed.
-        return allows(request.principals(), acl.principals()) &&
-               allows(request.users(), acl.users());
-      }
-    }
+          acls_.push_back(acl_);
+        }
 
-    return acls.permissive(); // None of the ACLs match.
-  }
+        return authorized(request, acls_);
+        break;
+      case authorization::TEARDOWN_FRAMEWORK_WITH_PRINCIPAL:
+        for (const ACL::TeardownFramework& acl : acls.teardown_frameworks()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.framework_principals();
 
-  Future<bool> authorize(const ACL::TeardownFramework& request)
-  {
-    foreach (const ACL::TeardownFramework& acl, acls.teardown_frameworks()) {
-      // ACL matches if both subjects and objects match.
-      if (matches(request.principals(), acl.principals()) &&
-          matches(request.framework_principals(),
-                  acl.framework_principals())) {
-        // ACL is allowed if both subjects and objects are allowed.
-        return allows(request.principals(), acl.principals()) &&
-               allows(request.framework_principals(),
-                      acl.framework_principals());
-      }
-    }
+          acls_.push_back(acl_);
+        }
 
-    return acls.permissive(); // None of the ACLs match.
-  }
+        return authorized(request, acls_);
+        break;
+      case authorization::RUN_TASK_WITH_USER:
+        for (const ACL::RunTask& acl : acls.run_tasks()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.users();
 
-  Future<bool> authorize(const ACL::ReserveResources& request)
-  {
-    foreach (const ACL::ReserveResources& acl, acls.reserve_resources()) {
-      // ACL matches if both subjects and objects match.
-      if (matches(request.principals(), acl.principals()) &&
-          matches(request.roles(), acl.roles())) {
-        // ACL is allowed if both subjects and objects are allowed.
-        return allows(request.principals(), acl.principals()) &&
-               allows(request.roles(), acl.roles());
-      }
-    }
+          acls_.push_back(acl_);
+        }
 
-    return acls.permissive(); // None of the ACLs match.
-  }
+        return authorized(request, acls_);
+        break;
+      case authorization::RESERVE_RESOURCES_WITH_ROLE:
+        for (const ACL::ReserveResources& acl : acls.reserve_resources()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.roles();
 
-  Future<bool> authorize(const ACL::UnreserveResources& request)
-  {
-    foreach (const ACL::UnreserveResources& acl, acls.unreserve_resources()) {
-      // ACL matches if both subjects and objects match.
-      if (matches(request.principals(), acl.principals()) &&
-          matches(request.reserver_principals(), acl.reserver_principals())) {
-        // ACL is allowed if both subjects and objects are allowed.
-        return allows(request.principals(), acl.principals()) &&
-               allows(request.reserver_principals(), acl.reserver_principals());
-      }
-    }
+          acls_.push_back(acl_);
+        }
 
-    return acls.permissive(); // None of the ACLs match.
-  }
+        return authorized(request, acls_);
+        break;
+      case authorization::UNRESERVE_RESOURCES_WITH_PRINCIPAL:
+        for (const ACL::UnreserveResources& acl : acls.unreserve_resources()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.reserver_principals();
 
-  Future<bool> authorize(const ACL::CreateVolume& request)
-  {
-    foreach (const ACL::CreateVolume& acl, acls.create_volumes()) {
-      // ACL matches if both subjects and objects match.
-      if (matches(request.principals(), acl.principals()) &&
-          matches(request.roles(), acl.roles())) {
-        // ACL is allowed if both subjects and objects are allowed.
-        return allows(request.principals(), acl.principals()) &&
-               allows(request.roles(), acl.roles());
-      }
-    }
+          acls_.push_back(acl_);
+        }
 
-    return acls.permissive(); // None of the ACLs match.
-  }
+        return authorized(request, acls_);
+        break;
+      case authorization::CREATE_VOLUME_WITH_ROLE:
+        for (const ACL::CreateVolume& acl : acls.create_volumes()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.roles();
 
-  Future<bool> authorize(const ACL::DestroyVolume& request)
-  {
-    foreach (const ACL::DestroyVolume& acl, acls.destroy_volumes()) {
-      // ACL matches if both subjects and objects match.
-      if (matches(request.principals(), acl.principals()) &&
-          matches(request.creator_principals(), acl.creator_principals())) {
-        // ACL is allowed if both subjects and objects are allowed.
-        return allows(request.principals(), acl.principals()) &&
-               allows(request.creator_principals(), acl.creator_principals());
-      }
-    }
+          acls_.push_back(acl_);
+        }
 
-    return acls.permissive(); // None of the ACLs match.
-  }
+        return authorized(request, acls_);
+        break;
+      case authorization::DESTROY_VOLUME_WITH_PRINCIPAL:
+        for (const ACL::DestroyVolume& acl : acls.destroy_volumes()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.creator_principals();
 
-  Future<bool> authorize(const ACL::SetQuota& request)
-  {
-    foreach (const ACL::SetQuota& acl, acls.set_quotas()) {
-      // ACL matches if both subjects and objects match.
-      if (matches(request.principals(), acl.principals()) &&
-          matches(request.roles(), acl.roles())) {
-        // ACL is allowed if both subjects and objects are allowed.
-        return allows(request.principals(), acl.principals()) &&
-               allows(request.roles(), acl.roles());
-      }
-    }
+          acls_.push_back(acl_);
+        }
+
+        return authorized(request, acls_);
+        break;
+      case authorization::SET_QUOTA_WITH_ROLE:
+        for (const ACL::SetQuota& acl : acls.set_quotas()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.roles();
+
+          acls_.push_back(acl_);
+        }
+
+        return authorized(request, acls_);
+        break;
+      case authorization::DESTROY_QUOTA_WITH_PRINCIPAL:
+        for (const ACL::RemoveQuota& acl : acls.remove_quotas()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.quota_principals();
+
+          acls_.push_back(acl_);
+        }
+
+        return authorized(request, acls_);
+        break;
+      case authorization::UPDATE_WEIGHTS_WITH_ROLE:
+        for (const ACL::UpdateWeights& acl : acls.update_weights()) {
+          GenericACL acl_;
+          acl_.subjects = acl.principals();
+          acl_.objects = acl.roles();
+
+          acls_.push_back(acl_);
+        }
 
-    return acls.permissive(); // None of the ACLs match.
+        return authorized(request, acls_);
+        break;
+    }
+    UNREACHABLE();
   }
 
-  Future<bool> authorize(const ACL::RemoveQuota& request)
+private:
+  Future<bool> authorized(
+      const authorization::Request& request,
+      const vector<GenericACL>& acls)
   {
-    foreach (const ACL::RemoveQuota& acl, acls.remove_quotas()) {
-      // ACL matches if both subjects and objects match.
-      if (matches(request.principals(), acl.principals()) &&
-          matches(request.quota_principals(), acl.quota_principals())) {
-        // ACL is allowed if both subjects and objects are allowed.
-        return allows(request.principals(), acl.principals()) &&
-               allows(request.quota_principals(), acl.quota_principals());
-      }
+    ACL::Entity subject;
+    if (request.subject().has_value()) {
+      subject.add_values(request.subject().value());
+      subject.set_type(mesos::ACL::Entity::SOME);
+    } else {
+      subject.set_type(mesos::ACL::Entity::ANY);
     }
 
-    return acls.permissive(); // None of the ACLs match.
-  }
+    ACL::Entity object;
+    if (request.object().has_value()) {
+      object.add_values(request.object().value());
+      object.set_type(mesos::ACL::Entity::SOME);
+    } else {
+      object.set_type(mesos::ACL::Entity::ANY);
+    }
 
-  Future<bool> authorize(const ACL::UpdateWeights& request)
-  {
-    foreach (const ACL::UpdateWeights& acl, acls.update_weights()) {
-      // ACL matches if both subjects and objects match.
-      if (matches(request.principals(), acl.principals()) &&
-          matches(request.roles(), acl.roles())) {
-        // ACL is allowed if both subjects and objects are allowed.
-        return allows(request.principals(), acl.principals()) &&
-               allows(request.roles(), acl.roles());
+    for (const GenericACL& acl : acls) {
+      if (matches(subject, acl.subjects) &&
+          matches(object, acl.objects)) {
+        return allows(subject, acl.subjects) &&
+            allows(object, acl.objects);
       }
     }
 
-    return acls.permissive(); // None of the ACLs match.
+    return this->acls.permissive(); // None of the ACLs match.
   }
 
-private:
   // Match matrix:
   //
   //                  -----------ACL----------
@@ -388,147 +396,16 @@ LocalAuthorizer::~LocalAuthorizer()
 }
 
 
-Future<bool> LocalAuthorizer::authorize(const ACL::RegisterFramework& request)
-{
-  if (process == NULL) {
-    return Failure("Authorizer not initialized");
-  }
-
-  // Necessary to disambiguate.
-  typedef Future<bool>(LocalAuthorizerProcess::*F)(
-      const ACL::RegisterFramework&);
-
-  return dispatch(
-      process, static_cast<F>(&LocalAuthorizerProcess::authorize), request);
-}
-
-
-Future<bool> LocalAuthorizer::authorize(const ACL::RunTask& request)
+process::Future<bool> LocalAuthorizer::authorized(
+  const authorization::Request& request)
 {
-  if (process == NULL) {
-    return Failure("Authorizer not initialized");
-  }
-
-  // Necessary to disambiguate.
-  typedef Future<bool>(LocalAuthorizerProcess::*F)(const ACL::RunTask&);
-
-  return dispatch(
-      process, static_cast<F>(&LocalAuthorizerProcess::authorize), request);
-}
-
-
-Future<bool> LocalAuthorizer::authorize(const ACL::TeardownFramework& request)
-{
-  if (process == NULL) {
-    return Failure("Authorizer not initialized");
-  }
-
-  // Necessary to disambiguate.
-  typedef Future<bool>(LocalAuthorizerProcess::*F)(
-      const ACL::TeardownFramework&);
-
-  return dispatch(
-      process, static_cast<F>(&LocalAuthorizerProcess::authorize), request);
-}
-
-
-Future<bool> LocalAuthorizer::authorize(const ACL::ReserveResources& request)
-{
-  if (process == NULL) {
-    return Failure("Authorizer not initialized");
-  }
-
-  // Necessary to disambiguate.
-  typedef Future<bool>(
-      LocalAuthorizerProcess::*F)(const ACL::ReserveResources&);
-
-  return dispatch(
-      process, static_cast<F>(&LocalAuthorizerProcess::authorize), request);
-}
-
-
-Future<bool> LocalAuthorizer::authorize(const ACL::UnreserveResources& request)
-{
-  if (process == NULL) {
-    return Failure("Authorizer not initialized");
-  }
-
-  // Necessary to disambiguate.
-  typedef Future<bool>(
-      LocalAuthorizerProcess::*F)(const ACL::UnreserveResources&);
-
-  return dispatch(
-      process, static_cast<F>(&LocalAuthorizerProcess::authorize), request);
-}
-
-
-Future<bool> LocalAuthorizer::authorize(const ACL::CreateVolume& request)
-{
-  if (process == NULL) {
-    return Failure("Authorizer not initialized");
-  }
-
-  // Necessary to disambiguate.
-  typedef Future<bool>(LocalAuthorizerProcess::*F)(const ACL::CreateVolume&);
-
-  return dispatch(
-      process, static_cast<F>(&LocalAuthorizerProcess::authorize), request);
-}
-
-
-Future<bool> LocalAuthorizer::authorize(const ACL::DestroyVolume& request)
-{
-  if (process == NULL) {
-    return Failure("Authorizer not initialized");
-  }
-
-  // Necessary to disambiguate.
-  typedef Future<bool>(LocalAuthorizerProcess::*F)(const ACL::DestroyVolume&);
-
-  return dispatch(
-      process, static_cast<F>(&LocalAuthorizerProcess::authorize), request);
-}
-
-
-Future<bool> LocalAuthorizer::authorize(const ACL::SetQuota& request)
-{
-  if (process == NULL) {
-    return Failure("Authorizer not initialized");
-  }
-
-  // Necessary to disambiguate.
-  typedef Future<bool>(LocalAuthorizerProcess::*F)(const ACL::SetQuota&);
-
-  return dispatch(
-      process, static_cast<F>(&LocalAuthorizerProcess::authorize), request);
-}
-
-
-Future<bool> LocalAuthorizer::authorize(const ACL::RemoveQuota& request)
-{
-  if (process == NULL) {
-    return Failure("Authorizer not initialized");
-  }
-
-  // Necessary to disambiguate.
-  typedef Future<bool>(LocalAuthorizerProcess::*F)(const ACL::RemoveQuota&);
-
-  return dispatch(
-      process, static_cast<F>(&LocalAuthorizerProcess::authorize), request);
-}
-
-
-Future<bool> LocalAuthorizer::authorize(const ACL::UpdateWeights& request)
-{
-  if (process == NULL) {
-    return Failure("Authorizer not initialized");
-  }
-
-  // Necessary to disambiguate.
-  typedef Future<bool>(LocalAuthorizerProcess::*F)(const ACL::UpdateWeights&);
+  typedef Future<bool> (LocalAuthorizerProcess::*F)(
+      const authorization::Request&);
 
   return dispatch(
-      process, static_cast<F>(&LocalAuthorizerProcess::authorize), request);
+      process,
+      static_cast<F>(&LocalAuthorizerProcess::authorized),
+      request);
 }
 
 } // namespace internal {

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/authorizer/local/authorizer.hpp
----------------------------------------------------------------------
diff --git a/src/authorizer/local/authorizer.hpp b/src/authorizer/local/authorizer.hpp
index 361b213..d15d3a6 100644
--- a/src/authorizer/local/authorizer.hpp
+++ b/src/authorizer/local/authorizer.hpp
@@ -30,6 +30,7 @@ namespace mesos {
 
 // Forward declaration.
 class Parameters;
+class ACLs;
 
 namespace internal {
 
@@ -60,27 +61,8 @@ public:
 
   virtual ~LocalAuthorizer();
 
-  // Implementation of Authorizer interface.
-  virtual process::Future<bool> authorize(
-      const ACL::RegisterFramework& request);
-  virtual process::Future<bool> authorize(
-      const ACL::RunTask& request);
-  virtual process::Future<bool> authorize(
-      const ACL::TeardownFramework& request);
-  virtual process::Future<bool> authorize(
-      const ACL::ReserveResources& request);
-  virtual process::Future<bool> authorize(
-      const ACL::UnreserveResources& request);
-  virtual process::Future<bool> authorize(
-      const ACL::CreateVolume& request);
-  virtual process::Future<bool> authorize(
-      const ACL::DestroyVolume& request);
-  virtual process::Future<bool> authorize(
-      const ACL::SetQuota& request);
-  virtual process::Future<bool> authorize(
-      const ACL::RemoveQuota& request);
-  virtual process::Future<bool> authorize(
-      const ACL::UpdateWeights& request);
+  virtual process::Future<bool> authorized(
+      const authorization::Request& request);
 
 private:
   LocalAuthorizer(const ACLs& acls);

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/master/http.cpp
----------------------------------------------------------------------
diff --git a/src/master/http.cpp b/src/master/http.cpp
index 54a2569..893e065 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -1805,22 +1805,18 @@ Future<Response> Master::Http::teardown(
     return _teardown(id);
   }
 
-  mesos::ACL::TeardownFramework teardown;
+  authorization::Request teardown;
+  teardown.set_action(authorization::TEARDOWN_FRAMEWORK_WITH_PRINCIPAL);
 
   if (principal.isSome()) {
-    teardown.mutable_principals()->add_values(principal.get());
-  } else {
-    teardown.mutable_principals()->set_type(ACL::Entity::ANY);
+    teardown.mutable_subject()->set_value(principal.get());
   }
 
   if (framework->info.has_principal()) {
-    teardown.mutable_framework_principals()->add_values(
-        framework->info.principal());
-  } else {
-    teardown.mutable_framework_principals()->set_type(ACL::Entity::ANY);
+    teardown.mutable_object()->set_value(framework->info.principal());
   }
 
-  return master->authorizer.get()->authorize(teardown)
+  return master->authorizer.get()->authorized(teardown)
     .then(defer(master->self(), [=](bool authorized) -> Future<Response> {
       if (!authorized) {
         return Forbidden();

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 249e82f..5f66a8d 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -1815,16 +1815,16 @@ Future<bool> Master::authorizeFramework(
   LOG(INFO) << "Authorizing framework principal '" << frameworkInfo.principal()
             << "' to receive offers for role '" << frameworkInfo.role() << "'";
 
-  mesos::ACL::RegisterFramework request;
+  authorization::Request request;
+  request.set_action(authorization::REGISTER_FRAMEWORK_WITH_ROLE);
+
   if (frameworkInfo.has_principal()) {
-    request.mutable_principals()->add_values(frameworkInfo.principal());
-  } else {
-    // Framework doesn't have a principal set.
-    request.mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    request.mutable_subject()->set_value(frameworkInfo.principal());
   }
-  request.mutable_roles()->add_values(frameworkInfo.role());
 
-  return authorizer.get()->authorize(request);
+  request.mutable_object()->set_value(frameworkInfo.role());
+
+  return authorizer.get()->authorized(request);
 }
 
 
@@ -2842,16 +2842,16 @@ Future<bool> Master::authorizeTask(
     << "Authorizing framework principal '" << framework->info.principal()
     << "' to launch task " << task.task_id() << " as user '" << user << "'";
 
-  mesos::ACL::RunTask request;
+  authorization::Request request;
+  request.set_action(authorization::RUN_TASK_WITH_USER);
+
   if (framework->info.has_principal()) {
-    request.mutable_principals()->add_values(framework->info.principal());
-  } else {
-    // Framework doesn't have a principal set.
-    request.mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    request.mutable_subject()->set_value(framework->info.principal());
   }
-  request.mutable_users()->add_values(user);
 
-  return authorizer.get()->authorize(request);
+  request.mutable_object()->set_value(user);
+
+  return authorizer.get()->authorized(request);
 }
 
 
@@ -2863,22 +2863,24 @@ Future<bool> Master::authorizeReserveResources(
     return true; // Authorization is disabled.
   }
 
-  mesos::ACL::ReserveResources request;
+  authorization::Request request;
+  request.set_action(authorization::RESERVE_RESOURCES_WITH_ROLE);
 
   if (principal.isSome()) {
-    request.mutable_principals()->add_values(principal.get());
-  } else {
-    request.mutable_principals()->set_type(ACL::Entity::ANY);
+    request.mutable_subject()->set_value(principal.get());
   }
 
   // The operation will be authorized if the principal is allowed to make
   // reservations for all roles included in `reserve.resources`.
   // Add an element to `request.roles` for each unique role in the resources.
   hashset<string> roles;
+  list<Future<bool>> authorizations;
   foreach (const Resource& resource, reserve.resources()) {
     if (!roles.contains(resource.role())) {
-      request.mutable_roles()->add_values(resource.role());
       roles.insert(resource.role());
+
+      request.mutable_object()->set_value(resource.role());
+      authorizations.push_back(authorizer.get()->authorized(request));
     }
   }
 
@@ -2886,7 +2888,26 @@ Future<bool> Master::authorizeReserveResources(
             << (principal.isSome() ? principal.get() : "ANY")
             << "' to reserve resources '" << reserve.resources() << "'";
 
-  return authorizer.get()->authorize(request);
+  // NOTE: Empty authorizations are not valid and are checked by a validator.
+  // However under certain circumstances, this method can be called before
+  // the validation occur and the case must be considered non erroneous.
+  // TODO(arojas): Consider ensuring that `validate()` is called before
+  // `authorizeReserveResources` so a `CHECK(!roles.empty())` can be added.
+  if (authorizations.empty()) {
+    return authorizer.get()->authorized(request);
+  }
+
+  return await(authorizations)
+      .then([](const std::list<Future<bool>>& authorizations)
+            -> Future<bool> {
+        // Compute a disjunction.
+        for (const Future<bool>& authorization : authorizations) {
+          if (!authorization.get()) {
+            return false;
+          }
+        }
+        return true;
+      });
 }
 
 
@@ -2898,14 +2919,14 @@ Future<bool> Master::authorizeUnreserveResources(
     return true; // Authorization is disabled.
   }
 
-  mesos::ACL::UnreserveResources request;
+  authorization::Request request;
+  request.set_action(authorization::UNRESERVE_RESOURCES_WITH_PRINCIPAL);
 
   if (principal.isSome()) {
-    request.mutable_principals()->add_values(principal.get());
-  } else {
-    request.mutable_principals()->set_type(ACL::Entity::ANY);
+    request.mutable_subject()->set_value(principal.get());
   }
 
+  list<Future<bool>> authorizations;
   foreach (const Resource& resource, unreserve.resources()) {
     // NOTE: Since validation of this operation is performed after
     // authorization, we must check here that this resource is
@@ -2913,8 +2934,10 @@ Future<bool> Master::authorizeUnreserveResources(
     // during validation.
     if (Resources::isDynamicallyReserved(resource) &&
         resource.reservation().has_principal()) {
-      request.mutable_reserver_principals()->add_values(
+      request.mutable_object()->set_value(
           resource.reservation().principal());
+
+      authorizations.push_back(authorizer.get()->authorized(request));
     }
   }
 
@@ -2923,7 +2946,21 @@ Future<bool> Master::authorizeUnreserveResources(
     << (principal.isSome() ? principal.get() : "ANY")
     << "' to unreserve resources '" << unreserve.resources() << "'";
 
-  return authorizer.get()->authorize(request);
+  if (authorizations.empty()) {
+    return authorizer.get()->authorized(request);
+  }
+
+  return await(authorizations)
+      .then([](const std::list<Future<bool>>& authorizations)
+            -> Future<bool> {
+        // Compute a disjunction.
+        for (const Future<bool>& authorization : authorizations) {
+          if (!authorization.get()) {
+            return false;
+          }
+        }
+        return true;
+      });
 }
 
 
@@ -2935,22 +2972,24 @@ Future<bool> Master::authorizeCreateVolume(
     return true; // Authorization is disabled.
   }
 
-  mesos::ACL::CreateVolume request;
+  authorization::Request request;
+  request.set_action(authorization::CREATE_VOLUME_WITH_ROLE);
 
   if (principal.isSome()) {
-    request.mutable_principals()->add_values(principal.get());
-  } else {
-    request.mutable_principals()->set_type(ACL::Entity::ANY);
+    request.mutable_subject()->set_value(principal.get());
   }
 
   // The operation will be authorized if the principal is allowed to create
   // volumes for all roles included in `create.volumes`.
   // Add an element to `request.roles` for each unique role in the volumes.
   hashset<string> roles;
+  list<Future<bool>> authorizations;
   foreach (const Resource& volume, create.volumes()) {
     if (!roles.contains(volume.role())) {
-      request.mutable_roles()->add_values(volume.role());
       roles.insert(volume.role());
+
+      request.mutable_object()->set_value(volume.role());
+      authorizations.push_back(authorizer.get()->authorized(request));
     }
   }
 
@@ -2959,7 +2998,21 @@ Future<bool> Master::authorizeCreateVolume(
     << (principal.isSome() ? principal.get() : "ANY")
     << "' to create volumes";
 
-  return authorizer.get()->authorize(request);
+  if (authorizations.empty()) {
+    return authorizer.get()->authorized(request);
+  }
+
+  return await(authorizations)
+      .then([](const std::list<Future<bool>>& authorizations)
+            -> Future<bool> {
+        // Compute a disjunction.
+        for (const Future<bool>& authorization : authorizations) {
+          if (!authorization.get()) {
+            return false;
+          }
+        }
+        return true;
+      });
 }
 
 
@@ -2971,21 +3024,23 @@ Future<bool> Master::authorizeDestroyVolume(
     return true; // Authorization is disabled.
   }
 
-  mesos::ACL::DestroyVolume request;
+  authorization::Request request;
+  request.set_action(authorization::DESTROY_VOLUME_WITH_PRINCIPAL);
 
   if (principal.isSome()) {
-    request.mutable_principals()->add_values(principal.get());
-  } else {
-    request.mutable_principals()->set_type(ACL::Entity::ANY);
+    request.mutable_subject()->set_value(principal.get());
   }
 
+  list<Future<bool>> authorizations;
   foreach (const Resource& volume, destroy.volumes()) {
     // NOTE: Since validation of this operation may be performed after
     // authorization, we must check here that this resource is a persistent
     // volume. If it isn't, the error will be caught during validation.
     if (Resources::isPersistentVolume(volume)) {
-      request.mutable_creator_principals()->add_values(
+      request.mutable_object()->set_value(
           volume.disk().persistence().principal());
+
+      authorizations.push_back(authorizer.get()->authorized(request));
     }
   }
 
@@ -2995,7 +3050,21 @@ Future<bool> Master::authorizeDestroyVolume(
     << "' to destroy volumes '"
     << stringify(destroy.volumes()) << "'";
 
-  return authorizer.get()->authorize(request);
+  if (authorizations.empty()) {
+    return authorizer.get()->authorized(request);
+  }
+
+  return await(authorizations)
+      .then([](const std::list<Future<bool>>& authorizations)
+            -> Future<bool> {
+        // Compute a disjunction.
+        for (const Future<bool>& authorization : authorizations) {
+          if (!authorization.get()) {
+            return false;
+          }
+        }
+        return true;
+      });
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/master/quota_handler.cpp
----------------------------------------------------------------------
diff --git a/src/master/quota_handler.cpp b/src/master/quota_handler.cpp
index f361abb..88247d6 100644
--- a/src/master/quota_handler.cpp
+++ b/src/master/quota_handler.cpp
@@ -449,17 +449,16 @@ Future<bool> Master::QuotaHandler::authorizeSetQuota(
             << (principal.isSome() ? principal.get() : "ANY")
             << "' to request quota for role '" << role << "'";
 
-  mesos::ACL::SetQuota request;
+  authorization::Request request;
+  request.set_action(authorization::SET_QUOTA_WITH_ROLE);
 
   if (principal.isSome()) {
-    request.mutable_principals()->add_values(principal.get());
-  } else {
-    request.mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    request.mutable_subject()->set_value(principal.get());
   }
 
-  request.mutable_roles()->add_values(role);
+  request.mutable_object()->set_value(role);
 
-  return master->authorizer.get()->authorize(request);
+  return master->authorizer.get()->authorized(request);
 }
 
 
@@ -477,21 +476,18 @@ Future<bool> Master::QuotaHandler::authorizeRemoveQuota(
             << (quotaPrincipal.isSome() ? quotaPrincipal.get() : "ANY")
             << "'";
 
-  mesos::ACL::RemoveQuota request;
+  authorization::Request request;
+  request.set_action(authorization::DESTROY_QUOTA_WITH_PRINCIPAL);
 
   if (requestPrincipal.isSome()) {
-    request.mutable_principals()->add_values(requestPrincipal.get());
-  } else {
-    request.mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    request.mutable_subject()->set_value(requestPrincipal.get());
   }
 
   if (quotaPrincipal.isSome()) {
-    request.mutable_quota_principals()->add_values(quotaPrincipal.get());
-  } else {
-    request.mutable_quota_principals()->set_type(mesos::ACL::Entity::ANY);
+    request.mutable_object()->set_value(quotaPrincipal.get());
   }
 
-  return master->authorizer.get()->authorize(request);
+  return master->authorizer.get()->authorized(request);
 }
 
 } // namespace master {

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/master/weights_handler.cpp
----------------------------------------------------------------------
diff --git a/src/master/weights_handler.cpp b/src/master/weights_handler.cpp
index 9e4ab19..c9a1b0d 100644
--- a/src/master/weights_handler.cpp
+++ b/src/master/weights_handler.cpp
@@ -16,6 +16,11 @@
 
 #include "master/master.hpp"
 
+#include <list>
+
+#include <mesos/authorizer/authorizer.hpp>
+
+#include <process/collect.hpp>
 #include <process/future.hpp>
 #include <process/http.hpp>
 
@@ -28,6 +33,7 @@ namespace http = process::http;
 
 using google::protobuf::RepeatedPtrField;
 
+using std::list;
 using std::string;
 using std::vector;
 
@@ -142,19 +148,34 @@ Future<bool> Master::WeightsHandler::authorize(
             << (principal.isSome() ? principal.get() : "ANY")
             << "' to update weights for roles '" << stringify(roles) << "'";
 
-  mesos::ACL::UpdateWeights request;
+  authorization::Request request;
+  request.set_action(authorization::UPDATE_WEIGHTS_WITH_ROLE);
 
   if (principal.isSome()) {
-    request.mutable_principals()->add_values(principal.get());
-  } else {
-    request.mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    request.mutable_subject()->set_value(principal.get());
   }
 
+  list<Future<bool>> authorizations;
   foreach (const string& role, roles) {
-    request.mutable_roles()->add_values(role);
+    request.mutable_object()->set_value(role);
+    authorizations.push_back(master->authorizer.get()->authorized(request));
+  }
+
+  if (authorizations.empty()) {
+    return master->authorizer.get()->authorized(request);
   }
 
-  return master->authorizer.get()->authorize(request);
+  return await(authorizations)
+      .then([](const std::list<Future<bool>>& authorizations)
+            -> Future<bool> {
+        // Compute a disjunction.
+        for (const Future<bool>& authorization : authorizations) {
+          if (!authorization.get()) {
+            return false;
+          }
+        }
+        return true;
+      });
 }
 
 } // namespace master {


[3/4] mesos git commit: Implemented a generalized interface for the authorizer.

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/tests/authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/authorization_tests.cpp b/src/tests/authorization_tests.cpp
index 1c4f7e2..d4ef0f9 100644
--- a/src/tests/authorization_tests.cpp
+++ b/src/tests/authorization_tests.cpp
@@ -65,11 +65,14 @@ TYPED_TEST_CASE(AuthorizationTest, AuthorizerTypes);
 
 TYPED_TEST(AuthorizationTest, AnyPrincipalRunAsUser)
 {
-  // Any principal can run as "guest" user.
   ACLs acls;
-  mesos::ACL::RunTask* acl = acls.add_run_tasks();
-  acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
-  acl->mutable_users()->add_values("guest");
+
+  {
+    // Any principal can run as "guest" user.
+    mesos::ACL::RunTask* acl = acls.add_run_tasks();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    acl->mutable_users()->add_values("guest");
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -77,17 +80,30 @@ TYPED_TEST(AuthorizationTest, AnyPrincipalRunAsUser)
   Owned<Authorizer> authorizer(create.get());
 
   // Principals "foo" and "bar" can run as "guest".
-  mesos::ACL::RunTask request;
-  request.mutable_principals()->add_values("foo");
-  request.mutable_principals()->add_values("bar");
-  request.mutable_users()->add_values("guest");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("guest");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("guest");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "foo" can run as "root" since the ACLs are permissive.
-  mesos::ACL::RunTask request2;
-  request2.mutable_principals()->add_values("foo");
-  request2.mutable_users()->add_values("root");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request2));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("root");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 }
 
 
@@ -95,9 +111,11 @@ TYPED_TEST(AuthorizationTest, NoPrincipalRunAsUser)
 {
   // No principal can run as "root" user.
   ACLs acls;
-  mesos::ACL::RunTask* acl = acls.add_run_tasks();
-  acl->mutable_principals()->set_type(mesos::ACL::Entity::NONE);
-  acl->mutable_users()->add_values("root");
+  {
+    mesos::ACL::RunTask* acl = acls.add_run_tasks();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::NONE);
+    acl->mutable_users()->add_values("root");
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -105,10 +123,13 @@ TYPED_TEST(AuthorizationTest, NoPrincipalRunAsUser)
   Owned<Authorizer> authorizer(create.get());
 
   // Principal "foo" cannot run as "root".
-  mesos::ACL::RunTask request;
-  request.mutable_principals()->add_values("foo");
-  request.mutable_users()->add_values("root");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("root");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 }
 
 
@@ -116,9 +137,12 @@ TYPED_TEST(AuthorizationTest, PrincipalRunAsAnyUser)
 {
   // A principal "foo" can run as any user.
   ACLs acls;
-  mesos::ACL::RunTask* acl = acls.add_run_tasks();
-  acl->mutable_principals()->add_values("foo");
-  acl->mutable_users()->set_type(mesos::ACL::Entity::ANY);
+
+  {
+    mesos::ACL::RunTask* acl = acls.add_run_tasks();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_users()->set_type(mesos::ACL::Entity::ANY);
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -126,11 +150,21 @@ TYPED_TEST(AuthorizationTest, PrincipalRunAsAnyUser)
   Owned<Authorizer> authorizer(create.get());
 
   // Principal "foo" can run as "user1" and "user2".
-  mesos::ACL::RunTask request;
-  request.mutable_principals()->add_values("foo");
-  request.mutable_users()->add_values("user1");
-  request.mutable_users()->add_values("user2");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("user1");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("foo");;
+    request.mutable_object()->set_value("user2");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 }
 
 
@@ -138,9 +172,12 @@ TYPED_TEST(AuthorizationTest, AnyPrincipalRunAsAnyUser)
 {
   // Any principal can run as any user.
   ACLs acls;
-  mesos::ACL::RunTask* acl = acls.add_run_tasks();
-  acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
-  acl->mutable_users()->set_type(mesos::ACL::Entity::ANY);
+
+  {
+    mesos::ACL::RunTask* acl = acls.add_run_tasks();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    acl->mutable_users()->set_type(mesos::ACL::Entity::ANY);
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -148,12 +185,37 @@ TYPED_TEST(AuthorizationTest, AnyPrincipalRunAsAnyUser)
   Owned<Authorizer> authorizer(create.get());
 
   // Principals "foo" and "bar" can run as "user1" and "user2".
-  mesos::ACL::RunTask request;
-  request.mutable_principals()->add_values("foo");
-  request.mutable_principals()->add_values("bar");
-  request.mutable_users()->add_values("user1");
-  request.mutable_users()->add_values("user2");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("user1");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("user2");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("user1");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("user1");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 }
 
 
@@ -162,18 +224,22 @@ TYPED_TEST(AuthorizationTest, OnlySomePrincipalsRunAsSomeUsers)
   // Only some principals can run as some users.
   ACLs acls;
 
-  // ACL for some principals to run as some users.
-  mesos::ACL::RunTask* acl = acls.add_run_tasks();
-  acl->mutable_principals()->add_values("foo");
-  acl->mutable_principals()->add_values("bar");
-  acl->mutable_users()->add_values("user1");
-  acl->mutable_users()->add_values("user2");
-
-  // ACL for no one else to run as some users.
-  mesos::ACL::RunTask* acl2 = acls.add_run_tasks();
-  acl2->mutable_principals()->set_type(mesos::ACL::Entity::NONE);
-  acl2->mutable_users()->add_values("user1");
-  acl2->mutable_users()->add_values("user2");
+  {
+    // ACL for some principals to run as some users.
+    mesos::ACL::RunTask* acl = acls.add_run_tasks();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_principals()->add_values("bar");
+    acl->mutable_users()->add_values("user1");
+    acl->mutable_users()->add_values("user2");
+  }
+
+  {
+    // ACL for no one else to run as some users.
+    mesos::ACL::RunTask* acl = acls.add_run_tasks();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::NONE);
+    acl->mutable_users()->add_values("user1");
+    acl->mutable_users()->add_values("user2");
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -181,24 +247,52 @@ TYPED_TEST(AuthorizationTest, OnlySomePrincipalsRunAsSomeUsers)
   Owned<Authorizer> authorizer(create.get());
 
   // Principals "foo" and "bar" can run as "user1" and "user2".
-  mesos::ACL::RunTask request;
-  request.mutable_principals()->add_values("foo");
-  request.mutable_principals()->add_values("bar");
-  request.mutable_users()->add_values("user1");
-  request.mutable_users()->add_values("user2");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("user1");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("user2");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("user1");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("user2");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "baz" cannot run as "user1".
-  mesos::ACL::RunTask request2;
-  request2.mutable_principals()->add_values("baz");
-  request2.mutable_users()->add_values("user1");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request2));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("baz");
+    request.mutable_object()->set_value("user1");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 
   // Principal "baz" cannot run as "user2".
-  mesos::ACL::RunTask request3;
-  request3.mutable_principals()->add_values("baz");
-  request3.mutable_users()->add_values("user1");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request3));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("baz");
+    request.mutable_object()->set_value("user1");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 }
 
 
@@ -208,14 +302,17 @@ TYPED_TEST(AuthorizationTest, SomePrincipalOnlySomeUser)
   ACLs acls;
 
   // ACL for some principal to run as some user.
-  mesos::ACL::RunTask* acl = acls.add_run_tasks();
-  acl->mutable_principals()->add_values("foo");
-  acl->mutable_users()->add_values("user1");
-
-  // ACL for some principal to not run as any other user.
-  mesos::ACL::RunTask* acl2 = acls.add_run_tasks();
-  acl2->mutable_principals()->add_values("foo");
-  acl2->mutable_users()->set_type(mesos::ACL::Entity::NONE);
+  {
+    mesos::ACL::RunTask* acl = acls.add_run_tasks();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_users()->add_values("user1");
+  }
+
+  {
+    mesos::ACL::RunTask* acl = acls.add_run_tasks();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_users()->set_type(mesos::ACL::Entity::NONE);
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -223,34 +320,52 @@ TYPED_TEST(AuthorizationTest, SomePrincipalOnlySomeUser)
   Owned<Authorizer> authorizer(create.get());
 
   // Principal "foo" can run as "user1".
-  mesos::ACL::RunTask request;
-  request.mutable_principals()->add_values("foo");
-  request.mutable_users()->add_values("user1");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("user1");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "foo" cannot run as "user2".
-  mesos::ACL::RunTask request2;
-  request2.mutable_principals()->add_values("foo");
-  request2.mutable_users()->add_values("user2");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request2));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("user2");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 
   // Principal "bar" can run as "user1" and "user2".
-  mesos::ACL::RunTask request3;
-  request3.mutable_principals()->add_values("bar");
-  request3.mutable_users()->add_values("user1");
-  request3.mutable_users()->add_values("user2");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request3));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("user1");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("user2");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 }
 
 
 TYPED_TEST(AuthorizationTest, PrincipalRunAsSomeUserRestrictive)
 {
-  // A principal can run as "user1";
   ACLs acls;
   acls.set_permissive(false); // Restrictive.
-  mesos::ACL::RunTask* acl = acls.add_run_tasks();
-  acl->mutable_principals()->add_values("foo");
-  acl->mutable_users()->add_values("user1");
+
+  {
+    // A principal can run as "user1";
+    mesos::ACL::RunTask* acl = acls.add_run_tasks();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_users()->add_values("user1");
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -258,32 +373,44 @@ TYPED_TEST(AuthorizationTest, PrincipalRunAsSomeUserRestrictive)
   Owned<Authorizer> authorizer(create.get());
 
   // Principal "foo" can run as "user1".
-  mesos::ACL::RunTask request;
-  request.mutable_principals()->add_values("foo");
-  request.mutable_users()->add_values("user1");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("user1");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "foo" cannot run as "user2".
-  mesos::ACL::RunTask request2;
-  request2.mutable_principals()->add_values("foo");
-  request2.mutable_users()->add_values("user2");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request2));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("user2");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 
   // Principal "bar" cannot run as "user2" since no ACL is set.
-  mesos::ACL::RunTask request3;
-  request3.mutable_principals()->add_values("bar");
-  request3.mutable_users()->add_values("user2");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request3));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RUN_TASK_WITH_USER);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("user2");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 }
 
 
 TYPED_TEST(AuthorizationTest, AnyPrincipalOfferedRole)
 {
-  // Any principal can be offered "*" role's resources.
   ACLs acls;
-  mesos::ACL::RegisterFramework* acl = acls.add_register_frameworks();
-  acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
-  acl->mutable_roles()->add_values("*");
+
+  {
+    // Any principal can be offered "*" role's resources.
+    mesos::ACL::RegisterFramework* acl = acls.add_register_frameworks();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    acl->mutable_roles()->add_values("*");
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -291,22 +418,34 @@ TYPED_TEST(AuthorizationTest, AnyPrincipalOfferedRole)
   Owned<Authorizer> authorizer(create.get());
 
   // Principals "foo" and "bar" can be offered "*" role's resources.
-  mesos::ACL::RegisterFramework request;
-  request.mutable_principals()->add_values("foo");
-  request.mutable_principals()->add_values("bar");
-  request.mutable_roles()->add_values("*");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request));
+  {
+    authorization::Request request;
+    request.set_action(authorization::REGISTER_FRAMEWORK_WITH_ROLE);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("*");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+  {
+    authorization::Request request;
+    request.set_action(authorization::REGISTER_FRAMEWORK_WITH_ROLE);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("*");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 }
 
 
 TYPED_TEST(AuthorizationTest, SomePrincipalsOfferedRole)
 {
-  // Some principals can be offered "ads" role's resources.
   ACLs acls;
-  mesos::ACL::RegisterFramework* acl = acls.add_register_frameworks();
-  acl->mutable_principals()->add_values("foo");
-  acl->mutable_principals()->add_values("bar");
-  acl->mutable_roles()->add_values("ads");
+
+  {
+    // Some principals can be offered "ads" role's resources.
+    mesos::ACL::RegisterFramework* acl = acls.add_register_frameworks();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_principals()->add_values("bar");
+    acl->mutable_roles()->add_values("ads");
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -315,12 +454,27 @@ TYPED_TEST(AuthorizationTest, SomePrincipalsOfferedRole)
 
   // Principals "foo", "bar" and "baz" (no ACL) can be offered "ads"
   // role's resources.
-  mesos::ACL::RegisterFramework request;
-  request.mutable_principals()->add_values("foo");
-  request.mutable_principals()->add_values("bar");
-  request.mutable_principals()->add_values("baz");
-  request.mutable_roles()->add_values("ads");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request));
+  {
+    authorization::Request request;
+    request.set_action(authorization::REGISTER_FRAMEWORK_WITH_ROLE);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("ads");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+  {
+    authorization::Request request;
+    request.set_action(authorization::REGISTER_FRAMEWORK_WITH_ROLE);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("ads");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+  {
+    authorization::Request request;
+    request.set_action(authorization::REGISTER_FRAMEWORK_WITH_ROLE);
+    request.mutable_subject()->set_value("baz");
+    request.mutable_object()->set_value("ads");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 }
 
 
@@ -329,15 +483,19 @@ TYPED_TEST(AuthorizationTest, PrincipalOfferedRole)
   // Only a principal can be offered "analytics" role's resources.
   ACLs acls;
 
-  // ACL for a principal to be offered "analytics" role's resources.
-  mesos::ACL::RegisterFramework* acl = acls.add_register_frameworks();
-  acl->mutable_principals()->add_values("foo");
-  acl->mutable_roles()->add_values("analytics");
+  {
+    // ACL for a principal to be offered "analytics" role's resources.
+    mesos::ACL::RegisterFramework* acl = acls.add_register_frameworks();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_roles()->add_values("analytics");
+  }
 
-  // ACL for no one else to be offered "analytics" role's resources.
-  mesos::ACL::RegisterFramework* acl2 = acls.add_register_frameworks();
-  acl2->mutable_principals()->set_type(mesos::ACL::Entity::NONE);
-  acl2->mutable_roles()->add_values("analytics");
+  {
+    // ACL for no one else to be offered "analytics" role's resources.
+    mesos::ACL::RegisterFramework* acl = acls.add_register_frameworks();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::NONE);
+    acl->mutable_roles()->add_values("analytics");
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -345,27 +503,36 @@ TYPED_TEST(AuthorizationTest, PrincipalOfferedRole)
   Owned<Authorizer> authorizer(create.get());
 
   // Principal "foo" can be offered "analytics" role's resources.
-  mesos::ACL::RegisterFramework request;
-  request.mutable_principals()->add_values("foo");
-  request.mutable_roles()->add_values("analytics");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request));
+  {
+    authorization::Request request;
+    request.set_action(authorization::REGISTER_FRAMEWORK_WITH_ROLE);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("analytics");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "bar" cannot be offered "analytics" role's resources.
-  mesos::ACL::RegisterFramework request2;
-  request2.mutable_principals()->add_values("bar");
-  request2.mutable_roles()->add_values("analytics");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request2));
+  {
+    authorization::Request request;
+    request.set_action(authorization::REGISTER_FRAMEWORK_WITH_ROLE);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("analytics");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 }
 
 
 TYPED_TEST(AuthorizationTest, PrincipalNotOfferedAnyRoleRestrictive)
 {
-  // A principal "foo" can be offered "analytics" role's resources.
   ACLs acls;
   acls.set_permissive(false);
-  mesos::ACL::RegisterFramework* acl = acls.add_register_frameworks();
-  acl->mutable_principals()->add_values("foo");
-  acl->mutable_roles()->add_values("analytics");
+
+  {
+    // A principal "foo" can be offered "analytics" role's resources.
+    mesos::ACL::RegisterFramework* acl = acls.add_register_frameworks();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_roles()->add_values("analytics");
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -373,22 +540,31 @@ TYPED_TEST(AuthorizationTest, PrincipalNotOfferedAnyRoleRestrictive)
   Owned<Authorizer> authorizer(create.get());
 
   // Principal "foo" can be offered "analytics" role's resources.
-  mesos::ACL::RegisterFramework request;
-  request.mutable_principals()->add_values("foo");
-  request.mutable_roles()->add_values("analytics");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request));
+  {
+    authorization::Request request;
+    request.set_action(authorization::REGISTER_FRAMEWORK_WITH_ROLE);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("analytics");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "bar" cannot be offered "analytics" role's resources.
-  mesos::ACL::RegisterFramework request2;
-  request2.mutable_principals()->add_values("bar");
-  request2.mutable_roles()->add_values("analytics");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request2));
+  {
+    authorization::Request request;
+    request.set_action(authorization::REGISTER_FRAMEWORK_WITH_ROLE);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("analytics");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 
   // Principal "bar" cannot be offered "ads" role's resources because no ACL.
-  mesos::ACL::RegisterFramework request3;
-  request3.mutable_principals()->add_values("bar");
-  request3.mutable_roles()->add_values("ads");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request3));
+  {
+    authorization::Request request;
+    request.set_action(authorization::REGISTER_FRAMEWORK_WITH_ROLE);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("ads");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 }
 
 
@@ -397,21 +573,27 @@ TYPED_TEST(AuthorizationTest, Reserve)
 {
   ACLs acls;
 
-  // Principals "foo" and "bar" can reserve resources for any role.
-  mesos::ACL::ReserveResources* acl1 = acls.add_reserve_resources();
-  acl1->mutable_principals()->add_values("foo");
-  acl1->mutable_principals()->add_values("bar");
-  acl1->mutable_roles()->set_type(mesos::ACL::Entity::ANY);
-
-  // Principal "baz" can only reserve resources for the "ads" role.
-  mesos::ACL::ReserveResources* acl2 = acls.add_reserve_resources();
-  acl2->mutable_principals()->add_values("baz");
-  acl2->mutable_roles()->add_values("ads");
-
-  // No other principals can reserve resources.
-  mesos::ACL::ReserveResources* acl3 = acls.add_reserve_resources();
-  acl3->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
-  acl3->mutable_roles()->set_type(mesos::ACL::Entity::NONE);
+  {
+    // Principals "foo" and "bar" can reserve resources for any role.
+    mesos::ACL::ReserveResources* acl = acls.add_reserve_resources();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_principals()->add_values("bar");
+    acl->mutable_roles()->set_type(mesos::ACL::Entity::ANY);
+  }
+
+  {
+    // Principal "baz" can only reserve resources for the "ads" role.
+    mesos::ACL::ReserveResources* acl = acls.add_reserve_resources();
+    acl->mutable_principals()->add_values("baz");
+    acl->mutable_roles()->add_values("ads");
+  }
+
+  {
+    // No other principals can reserve resources.
+    mesos::ACL::ReserveResources* acl = acls.add_reserve_resources();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    acl->mutable_roles()->set_type(mesos::ACL::Entity::NONE);
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -420,42 +602,64 @@ TYPED_TEST(AuthorizationTest, Reserve)
 
   // Principals "foo" and "bar" can reserve resources for any role,
   // so requests 1 and 2 will pass.
-  mesos::ACL::ReserveResources request1;
-  request1.mutable_principals()->add_values("foo");
-  request1.mutable_principals()->add_values("bar");
-  request1.mutable_roles()->set_type(mesos::ACL::Entity::ANY);
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request1));
-
-  mesos::ACL::ReserveResources request2;
-  request2.mutable_principals()->add_values("foo");
-  request2.mutable_principals()->add_values("bar");
-  request2.mutable_roles()->add_values("awesome_role");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request2));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RESERVE_RESOURCES_WITH_ROLE);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_subject()->set_value("bar");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::RESERVE_RESOURCES_WITH_ROLE);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("awesome_role");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+  {
+    authorization::Request request;
+    request.set_action(authorization::RESERVE_RESOURCES_WITH_ROLE);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("awesome_role");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "baz" can only reserve resources for the "ads" role, so request 3
   // will pass, but requests 4 and 5 will fail.
-  mesos::ACL::ReserveResources request3;
-  request3.mutable_principals()->add_values("baz");
-  request3.mutable_roles()->add_values("ads");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request3));
-
-  mesos::ACL::ReserveResources request4;
-  request4.mutable_principals()->add_values("baz");
-  request4.mutable_roles()->add_values("awesome_role");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request4));
-
-  mesos::ACL::ReserveResources request5;
-  request5.mutable_principals()->add_values("baz");
-  request5.mutable_roles()->set_type(mesos::ACL::Entity::ANY);
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request5));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RESERVE_RESOURCES_WITH_ROLE);
+    request.mutable_subject()->set_value("baz");
+    request.mutable_object()->set_value("ads");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::RESERVE_RESOURCES_WITH_ROLE);
+    request.mutable_subject()->set_value("baz");
+    request.mutable_object()->set_value("awesome_role");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::RESERVE_RESOURCES_WITH_ROLE);
+    request.mutable_subject()->set_value("baz");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 
   // Principal "zelda" is not mentioned in the ACLs of the Authorizer, so it
   // will be caught by the final ACL, which provides a default case that denies
   // access for all other principals. This request will fail.
-  mesos::ACL::ReserveResources request6;
-  request6.mutable_principals()->add_values("zelda");
-  request6.mutable_roles()->add_values("ads");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request6));
+  {
+    authorization::Request request;
+    request.set_action(authorization::RESERVE_RESOURCES_WITH_ROLE);
+    request.mutable_subject()->set_value("zelda");
+    request.mutable_object()->set_value("ads");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 }
 
 
@@ -465,25 +669,33 @@ TYPED_TEST(AuthorizationTest, Unreserve)
 {
   ACLs acls;
 
-  // "foo" principal can unreserve its own resources.
-  mesos::ACL::UnreserveResources* acl1 = acls.add_unreserve_resources();
-  acl1->mutable_principals()->add_values("foo");
-  acl1->mutable_reserver_principals()->add_values("foo");
-
-  // "bar" principal cannot unreserve anyone's resources.
-  mesos::ACL::UnreserveResources* acl2 = acls.add_unreserve_resources();
-  acl2->mutable_principals()->add_values("bar");
-  acl2->mutable_reserver_principals()->set_type(mesos::ACL::Entity::NONE);
-
-  // "ops" principal can unreserve anyone's resources.
-  mesos::ACL::UnreserveResources* acl3 = acls.add_unreserve_resources();
-  acl3->mutable_principals()->add_values("ops");
-  acl3->mutable_reserver_principals()->set_type(mesos::ACL::Entity::ANY);
-
-  // No other principals can unreserve resources.
-  mesos::ACL::UnreserveResources* acl4 = acls.add_unreserve_resources();
-  acl4->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
-  acl4->mutable_reserver_principals()->set_type(mesos::ACL::Entity::NONE);
+  {
+    // "foo" principal can unreserve its own resources.
+    mesos::ACL::UnreserveResources* acl = acls.add_unreserve_resources();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_reserver_principals()->add_values("foo");
+  }
+
+  {
+    // "bar" principal cannot unreserve anyone's resources.
+    mesos::ACL::UnreserveResources* acl = acls.add_unreserve_resources();
+    acl->mutable_principals()->add_values("bar");
+    acl->mutable_reserver_principals()->set_type(mesos::ACL::Entity::NONE);
+  }
+
+  {
+    // "ops" principal can unreserve anyone's resources.
+    mesos::ACL::UnreserveResources* acl = acls.add_unreserve_resources();
+    acl->mutable_principals()->add_values("ops");
+    acl->mutable_reserver_principals()->set_type(mesos::ACL::Entity::ANY);
+  }
+
+  {
+    // No other principals can unreserve resources.
+    mesos::ACL::UnreserveResources* acl = acls.add_unreserve_resources();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    acl->mutable_reserver_principals()->set_type(mesos::ACL::Entity::NONE);
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -491,44 +703,74 @@ TYPED_TEST(AuthorizationTest, Unreserve)
   Owned<Authorizer> authorizer(create.get());
 
   // Principal "foo" can unreserve its own resources.
-  mesos::ACL::UnreserveResources request1;
-  request1.mutable_principals()->add_values("foo");
-  request1.mutable_reserver_principals()->add_values("foo");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request1));
+  {
+    authorization::Request request;
+    request.set_action(authorization::UNRESERVE_RESOURCES_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("foo");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "bar" cannot unreserve anyone's
   // resources, so requests 2 and 3 will fail.
-  mesos::ACL::UnreserveResources request2;
-  request2.mutable_principals()->add_values("bar");
-  request2.mutable_reserver_principals()->add_values("foo");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request2));
-
-  mesos::ACL::UnreserveResources request3;
-  request3.mutable_principals()->add_values("bar");
-  request3.mutable_reserver_principals()->add_values("bar");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request3));
+  {
+    authorization::Request request;
+    request.set_action(authorization::UNRESERVE_RESOURCES_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("foo");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::UNRESERVE_RESOURCES_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("bar");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 
   // Principal "ops" can unreserve anyone's resources,
   // so requests 4 and 5 will succeed.
-  mesos::ACL::UnreserveResources request4;
-  request4.mutable_principals()->add_values("ops");
-  request4.mutable_reserver_principals()->add_values("foo");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request4));
-
-  mesos::ACL::UnreserveResources request5;
-  request5.mutable_principals()->add_values("ops");
-  request5.mutable_reserver_principals()->add_values("foo");
-  request5.mutable_reserver_principals()->add_values("bar");
-  request5.mutable_reserver_principals()->add_values("ops");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request5));
+  {
+    authorization::Request request;
+    request.set_action(authorization::UNRESERVE_RESOURCES_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("ops");
+    request.mutable_object()->set_value("foo");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::UNRESERVE_RESOURCES_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("ops");
+    request.mutable_object()->set_value("foo");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+  {
+    authorization::Request request;
+    request.set_action(authorization::UNRESERVE_RESOURCES_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("ops");
+    request.mutable_object()->set_value("bar");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+  {
+    authorization::Request request;
+    request.set_action(authorization::UNRESERVE_RESOURCES_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("ops");
+    request.mutable_object()->set_value("ops");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "zelda" is not mentioned in the ACLs of the Authorizer, so it
   // will be caught by the final ACL, which provides a default case that denies
   // access for all other principals. This case will fail.
-  mesos::ACL::UnreserveResources request6;
-  request6.mutable_principals()->add_values("zelda");
-  request6.mutable_reserver_principals()->add_values("foo");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request6));
+  {
+    authorization::Request request;
+    request.set_action(authorization::UNRESERVE_RESOURCES_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("zelda");
+    request.mutable_object()->set_value("foo");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 }
 
 
@@ -537,25 +779,33 @@ TYPED_TEST(AuthorizationTest, CreateVolume)
 {
   ACLs acls;
 
-  // Principal "foo" can create volumes for any role.
-  mesos::ACL::CreateVolume* acl1 = acls.add_create_volumes();
-  acl1->mutable_principals()->add_values("foo");
-  acl1->mutable_roles()->set_type(mesos::ACL::Entity::ANY);
-
-  // Principal "bar" can only create volumes for the "panda" role.
-  mesos::ACL::CreateVolume* acl2 = acls.add_create_volumes();
-  acl2->mutable_principals()->add_values("bar");
-  acl2->mutable_roles()->add_values("panda");
-
-  // Principal "baz" cannot create volumes.
-  mesos::ACL::CreateVolume* acl3 = acls.add_create_volumes();
-  acl3->mutable_principals()->add_values("baz");
-  acl3->mutable_roles()->set_type(mesos::ACL::Entity::NONE);
-
-  // No other principals can create volumes.
-  mesos::ACL::CreateVolume* acl4 = acls.add_create_volumes();
-  acl4->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
-  acl4->mutable_roles()->set_type(mesos::ACL::Entity::NONE);
+  {
+    // Principal "foo" can create volumes for any role.
+    mesos::ACL::CreateVolume* acl = acls.add_create_volumes();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_roles()->set_type(mesos::ACL::Entity::ANY);
+  }
+
+  {
+    // Principal "bar" can only create volumes for the "panda" role.
+    mesos::ACL::CreateVolume* acl = acls.add_create_volumes();
+    acl->mutable_principals()->add_values("bar");
+    acl->mutable_roles()->add_values("panda");
+  }
+
+  {
+    // Principal "baz" cannot create volumes.
+    mesos::ACL::CreateVolume* acl = acls.add_create_volumes();
+    acl->mutable_principals()->add_values("baz");
+    acl->mutable_roles()->set_type(mesos::ACL::Entity::NONE);
+  }
+
+  {
+    // No other principals can create volumes.
+    mesos::ACL::CreateVolume* acl = acls.add_create_volumes();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    acl->mutable_roles()->set_type(mesos::ACL::Entity::NONE);
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -563,39 +813,54 @@ TYPED_TEST(AuthorizationTest, CreateVolume)
   Owned<Authorizer> authorizer(create.get());
 
   // Principal "foo" can create volumes for any role, so this request will pass.
-  mesos::ACL::CreateVolume request1;
-  request1.mutable_principals()->add_values("foo");
-  request1.mutable_roles()->add_values("awesome_role");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request1));
+  {
+    authorization::Request request;
+    request.set_action(authorization::CREATE_VOLUME_WITH_ROLE);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("awesome_role");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "bar" can create volumes for the "panda" role,
   // so this request will pass.
-  mesos::ACL::CreateVolume request2;
-  request2.mutable_principals()->add_values("bar");
-  request2.mutable_roles()->add_values("panda");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request2));
+  {
+    authorization::Request request;
+    request.set_action(authorization::CREATE_VOLUME_WITH_ROLE);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("panda");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "bar" cannot create volumes for the "giraffe" role,
   // so this request will fail.
-  mesos::ACL::CreateVolume request3;
-  request3.mutable_principals()->add_values("bar");
-  request3.mutable_roles()->add_values("giraffe");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request3));
+  {
+    authorization::Request request;
+    request.set_action(authorization::CREATE_VOLUME_WITH_ROLE);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("giraffe");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 
   // Principal "baz" cannot create volumes for any role,
   // so this request will fail.
-  mesos::ACL::CreateVolume request4;
-  request4.mutable_principals()->add_values("baz");
-  request4.mutable_roles()->add_values("panda");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request4));
+  {
+    authorization::Request request;
+    request.set_action(authorization::CREATE_VOLUME_WITH_ROLE);
+    request.mutable_subject()->set_value("baz");
+    request.mutable_object()->set_value("panda");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 
   // Principal "zelda" is not mentioned in the ACLs of the Authorizer, so it
   // will be caught by the final ACL, which provides a default case that denies
   // access for all other principals. This request will fail.
-  mesos::ACL::CreateVolume request5;
-  request5.mutable_principals()->add_values("zelda");
-  request5.mutable_roles()->add_values("panda");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request5));
+  {
+    authorization::Request request;
+    request.set_action(authorization::CREATE_VOLUME_WITH_ROLE);
+    request.mutable_subject()->set_value("zelda");
+    request.mutable_object()->set_value("panda");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 }
 
 
@@ -605,25 +870,33 @@ TYPED_TEST(AuthorizationTest, DestroyVolume)
 {
   ACLs acls;
 
-  // "foo" principal can destroy its own volumes.
-  mesos::ACL::DestroyVolume* acl1 = acls.add_destroy_volumes();
-  acl1->mutable_principals()->add_values("foo");
-  acl1->mutable_creator_principals()->add_values("foo");
-
-  // "bar" principal cannot destroy anyone's volumes.
-  mesos::ACL::DestroyVolume* acl2 = acls.add_destroy_volumes();
-  acl2->mutable_principals()->add_values("bar");
-  acl2->mutable_creator_principals()->set_type(mesos::ACL::Entity::NONE);
-
-  // "ops" principal can destroy anyone's volumes.
-  mesos::ACL::DestroyVolume* acl3 = acls.add_destroy_volumes();
-  acl3->mutable_principals()->add_values("ops");
-  acl3->mutable_creator_principals()->set_type(mesos::ACL::Entity::ANY);
-
-  // No other principals can destroy volumes.
-  mesos::ACL::DestroyVolume* acl4 = acls.add_destroy_volumes();
-  acl4->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
-  acl4->mutable_creator_principals()->set_type(mesos::ACL::Entity::NONE);
+  {
+    // "foo" principal can destroy its own volumes.
+    mesos::ACL::DestroyVolume* acl = acls.add_destroy_volumes();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_creator_principals()->add_values("foo");
+  }
+
+  {
+    // "bar" principal cannot destroy anyone's volumes.
+    mesos::ACL::DestroyVolume* acl = acls.add_destroy_volumes();
+    acl->mutable_principals()->add_values("bar");
+    acl->mutable_creator_principals()->set_type(mesos::ACL::Entity::NONE);
+  }
+
+  {
+    // "ops" principal can destroy anyone's volumes.
+    mesos::ACL::DestroyVolume* acl = acls.add_destroy_volumes();
+    acl->mutable_principals()->add_values("ops");
+    acl->mutable_creator_principals()->set_type(mesos::ACL::Entity::ANY);
+  }
+
+  {
+    // No other principals can destroy volumes.
+    mesos::ACL::DestroyVolume* acl = acls.add_destroy_volumes();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    acl->mutable_creator_principals()->set_type(mesos::ACL::Entity::NONE);
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -631,43 +904,67 @@ TYPED_TEST(AuthorizationTest, DestroyVolume)
   Owned<Authorizer> authorizer(create.get());
 
   // Principal "foo" can destroy its own volumes, so this will pass.
-  mesos::ACL::DestroyVolume request1;
-  request1.mutable_principals()->add_values("foo");
-  request1.mutable_creator_principals()->add_values("foo");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request1));
+  {
+    authorization::Request request;
+    request.set_action(authorization::DESTROY_VOLUME_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("foo");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "bar" cannot destroy anyone's
   // volumes, so requests 2 and 3 will fail.
-  mesos::ACL::DestroyVolume request2;
-  request2.mutable_principals()->add_values("bar");
-  request2.mutable_creator_principals()->add_values("foo");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request2));
-
-  mesos::ACL::DestroyVolume request3;
-  request3.mutable_principals()->add_values("bar");
-  request3.mutable_creator_principals()->add_values("bar");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request3));
+  {
+    authorization::Request request;
+    request.set_action(authorization::DESTROY_VOLUME_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("foo");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::DESTROY_VOLUME_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("bar");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 
   // Principal "ops" can destroy anyone's volumes,
   // so requests 4 and 5 will succeed.
-  mesos::ACL::DestroyVolume request4;
-  request4.mutable_principals()->add_values("ops");
-  request4.mutable_creator_principals()->add_values("foo");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request4));
-
-  mesos::ACL::DestroyVolume request5;
-  request5.mutable_principals()->add_values("ops");
-  request5.mutable_creator_principals()->add_values("bar");
-  request5.mutable_creator_principals()->add_values("ops");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request5));
+  {
+    authorization::Request request;
+    request.set_action(authorization::DESTROY_VOLUME_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("ops");
+    request.mutable_object()->set_value("foo");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::DESTROY_VOLUME_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("ops");
+    request.mutable_object()->set_value("ops");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+  {
+    authorization::Request request;
+    request.set_action(authorization::DESTROY_VOLUME_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("ops");
+    request.mutable_object()->set_value("bar");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "zelda" is not mentioned in the ACLs of the Authorizer, so it
   // will be caught by the final ACL, which provides a default case that denies
   // access for all other principals. This case will fail.
-  mesos::ACL::DestroyVolume request6;
-  request6.mutable_principals()->add_values("zelda");
-  request6.mutable_creator_principals()->add_values("foo");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request6));
+  {
+    authorization::Request request;
+    request.set_action(authorization::DESTROY_VOLUME_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("zelda");
+    request.mutable_object()->set_value("foo");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 }
 
 
@@ -676,25 +973,32 @@ TYPED_TEST(AuthorizationTest, SetQuota)
 {
   ACLs acls;
 
-  // "foo" principal can set quotas for all roles.
-  mesos::ACL::SetQuota* acl1 = acls.add_set_quotas();
-  acl1->mutable_principals()->add_values("foo");
-  acl1->mutable_roles()->set_type(mesos::ACL::Entity::ANY);
-
-  // "bar" principal can set quotas for "dev" role.
-  mesos::ACL::SetQuota* acl2 = acls.add_set_quotas();
-  acl2->mutable_principals()->add_values("bar");
-  acl2->mutable_roles()->add_values("dev");
-
-  // Anyone can set quotas for "test" role.
-  mesos::ACL::SetQuota* acl3 = acls.add_set_quotas();
-  acl3->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
-  acl3->mutable_roles()->add_values("test");
-
-  // No other principal can set quotas.
-  mesos::ACL::SetQuota* acl4 = acls.add_set_quotas();
-  acl4->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
-  acl4->mutable_roles()->set_type(mesos::ACL::Entity::NONE);
+  {
+    // "foo" principal can set quotas for all roles.
+    mesos::ACL::SetQuota* acl = acls.add_set_quotas();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_roles()->set_type(mesos::ACL::Entity::ANY);
+  }
+
+  {
+    // "bar" principal can set quotas for "dev" role.
+    mesos::ACL::SetQuota* acl = acls.add_set_quotas();
+    acl->mutable_principals()->add_values("bar");
+    acl->mutable_roles()->add_values("dev");
+  }
+  {
+    // Anyone can set quotas for "test" role.
+    mesos::ACL::SetQuota* acl = acls.add_set_quotas();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    acl->mutable_roles()->add_values("test");
+  }
+
+  {
+    // No other principal can set quotas.
+    mesos::ACL::SetQuota* acl = acls.add_set_quotas();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    acl->mutable_roles()->set_type(mesos::ACL::Entity::NONE);
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -702,47 +1006,64 @@ TYPED_TEST(AuthorizationTest, SetQuota)
   Owned<Authorizer> authorizer(create.get());
 
   // Principal "foo" can set quota for all roles, so requests 1 and 2 will pass.
-  mesos::ACL::SetQuota request1;
-  request1.mutable_principals()->add_values("foo");
-  request1.mutable_roles()->set_type(mesos::ACL::Entity::ANY);
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request1));
-
-  mesos::ACL::SetQuota request2;
-  request2.mutable_principals()->add_values("foo");
-  request2.mutable_roles()->add_values("prod");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request2));
+  {
+    authorization::Request request;
+    request.set_action(authorization::SET_QUOTA_WITH_ROLE);
+    request.mutable_subject()->set_value("foo");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::SET_QUOTA_WITH_ROLE);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("prod");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "bar" can set quotas for role "dev", so this will pass.
-  mesos::ACL::SetQuota request3;
-  request3.mutable_principals()->add_values("bar");
-  request3.mutable_roles()->add_values("dev");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request3));
+  {
+    authorization::Request request;
+    request.set_action(authorization::SET_QUOTA_WITH_ROLE);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("dev");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "bar" can only set quotas for role "dev",
   // so request 4 and 5 will fail.
-  mesos::ACL::SetQuota request4;
-  request4.mutable_principals()->add_values("bar");
-  request4.mutable_roles()->add_values("prod");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request4));
-
-  mesos::ACL::SetQuota request5;
-  request5.mutable_principals()->add_values("bar");
-  request5.mutable_roles()->set_type(mesos::ACL::Entity::ANY);
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request5));
+  {
+    authorization::Request request;
+    request.set_action(authorization::SET_QUOTA_WITH_ROLE);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("prod");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::SET_QUOTA_WITH_ROLE);
+    request.mutable_subject()->set_value("bar");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 
   // Anyone can set quotas for role "test", so request 6 will pass.
-  mesos::ACL::SetQuota request6;
-  request6.mutable_principals()->set_type(mesos::ACL::Entity::ANY);
-  request6.mutable_roles()->add_values("test");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request6));
+  {
+    authorization::Request request;
+    request.set_action(authorization::SET_QUOTA_WITH_ROLE);
+    request.mutable_object()->set_value("test");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "jeff" is not mentioned in the ACLs of the `Authorizer`, so it
   // will be caught by the final ACL, which provides a default case that denies
   // access for all other principals. This case will fail.
-  mesos::ACL::SetQuota request7;
-  request7.mutable_principals()->add_values("jeff");
-  request7.mutable_roles()->set_type(mesos::ACL::Entity::ANY);
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request7));
+  {
+    authorization::Request request;
+    request.set_action(authorization::SET_QUOTA_WITH_ROLE);
+    request.mutable_subject()->set_value("jeff");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 }
 
 
@@ -751,25 +1072,33 @@ TYPED_TEST(AuthorizationTest, RemoveQuota)
 {
   ACLs acls;
 
-  // "foo" principal can remove its own quotas.
-  mesos::ACL::RemoveQuota* acl1 = acls.add_remove_quotas();
-  acl1->mutable_principals()->add_values("foo");
-  acl1->mutable_quota_principals()->add_values("foo");
+  {
+    // "foo" principal can remove its own quotas.
+    mesos::ACL::RemoveQuota* acl = acls.add_remove_quotas();
+    acl->mutable_principals()->add_values("foo");
+    acl->mutable_quota_principals()->add_values("foo");
+  }
 
+  {
   // "bar" principal cannot remove anyone's quotas.
-  mesos::ACL::RemoveQuota* acl2 = acls.add_remove_quotas();
-  acl2->mutable_principals()->add_values("bar");
-  acl2->mutable_quota_principals()->set_type(mesos::ACL::Entity::NONE);
-
-  // "ops" principal can remove anyone's quotas.
-  mesos::ACL::RemoveQuota* acl3 = acls.add_remove_quotas();
-  acl3->mutable_principals()->add_values("ops");
-  acl3->mutable_quota_principals()->set_type(mesos::ACL::Entity::ANY);
-
-  // No other principals can remove quotas.
-  mesos::ACL::RemoveQuota* acl4 = acls.add_remove_quotas();
-  acl4->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
-  acl4->mutable_quota_principals()->set_type(mesos::ACL::Entity::NONE);
+    mesos::ACL::RemoveQuota* acl = acls.add_remove_quotas();
+    acl->mutable_principals()->add_values("bar");
+    acl->mutable_quota_principals()->set_type(mesos::ACL::Entity::NONE);
+  }
+
+  {
+    // "ops" principal can remove anyone's quotas.
+    mesos::ACL::RemoveQuota* acl = acls.add_remove_quotas();
+    acl->mutable_principals()->add_values("ops");
+    acl->mutable_quota_principals()->set_type(mesos::ACL::Entity::ANY);
+  }
+
+  {
+    // No other principals can remove quotas.
+    mesos::ACL::RemoveQuota* acl = acls.add_remove_quotas();
+    acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+    acl->mutable_quota_principals()->set_type(mesos::ACL::Entity::NONE);
+  }
 
   // Create an `Authorizer` with the ACLs.
   Try<Authorizer*> create = TypeParam::create(parameterize(acls));
@@ -777,41 +1106,58 @@ TYPED_TEST(AuthorizationTest, RemoveQuota)
   Owned<Authorizer> authorizer(create.get());
 
   // Principal "foo" can remove its own quotas, so request 1 will pass.
-  mesos::ACL::RemoveQuota request1;
-  request1.mutable_principals()->add_values("foo");
-  request1.mutable_quota_principals()->add_values("foo");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request1));
+  {
+    authorization::Request request;
+    request.set_action(authorization::DESTROY_QUOTA_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("foo");
+    request.mutable_object()->set_value("foo");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "bar" cannot remove anyone's quotas, so requests 2 and 3 will
   // fail.
-  mesos::ACL::RemoveQuota request2;
-  request2.mutable_principals()->add_values("bar");
-  request2.mutable_quota_principals()->add_values("bar");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request2));
-
-  mesos::ACL::RemoveQuota request3;
-  request3.mutable_principals()->add_values("bar");
-  request3.mutable_quota_principals()->add_values("foo");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request3));
+  {
+    authorization::Request request;
+    request.set_action(authorization::DESTROY_QUOTA_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("bar");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::DESTROY_QUOTA_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("bar");
+    request.mutable_object()->set_value("foo");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 
   // Principal "ops" can remove anyone's quotas, so requests 4 and 5 will pass.
-  mesos::ACL::RemoveQuota request4;
-  request4.mutable_principals()->add_values("ops");
-  request4.mutable_quota_principals()->add_values("foo");
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request4));
-
-  mesos::ACL::RemoveQuota request5;
-  request5.mutable_principals()->add_values("ops");
-  request5.mutable_quota_principals()->set_type(mesos::ACL::Entity::ANY);
-  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request5));
+  {
+    authorization::Request request;
+    request.set_action(authorization::DESTROY_QUOTA_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("ops");
+    request.mutable_object()->set_value("foo");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
+
+  {
+    authorization::Request request;
+    request.set_action(authorization::DESTROY_QUOTA_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("ops");
+    AWAIT_EXPECT_TRUE(authorizer.get()->authorized(request));
+  }
 
   // Principal "jeff" is not mentioned in the ACLs of the `Authorizer`, so it
   // will be caught by the final rule, which provides a default case that denies
   // access for all other principals. This case will fail.
-  mesos::ACL::RemoveQuota request6;
-  request6.mutable_principals()->add_values("jeff");
-  request6.mutable_quota_principals()->add_values("foo");
-  AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request6));
+  {
+    authorization::Request request;
+    request.set_action(authorization::DESTROY_QUOTA_WITH_PRINCIPAL);
+    request.mutable_subject()->set_value("jeff");
+    request.mutable_object()->set_value("foo");
+    AWAIT_EXPECT_FALSE(authorizer.get()->authorized(request));
+  }
 }
 
 } // namespace tests {

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/tests/master_authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_authorization_tests.cpp b/src/tests/master_authorization_tests.cpp
index 29c89fb..6a4de47 100644
--- a/src/tests/master_authorization_tests.cpp
+++ b/src/tests/master_authorization_tests.cpp
@@ -244,7 +244,7 @@ TEST_F(MasterAuthorizationTest, KillTask)
   // Return a pending future from authorizer.
   Future<Nothing> authorize;
   Promise<bool> promise;
-  EXPECT_CALL(authorizer, authorize(An<const mesos::ACL::RunTask&>()))
+  EXPECT_CALL(authorizer, authorized(_))
     .WillOnce(DoAll(FutureSatisfy(&authorize),
                     Return(promise.future())));
 
@@ -316,7 +316,7 @@ TEST_F(MasterAuthorizationTest, SlaveRemoved)
   // Return a pending future from authorizer.
   Future<Nothing> authorize;
   Promise<bool> promise;
-  EXPECT_CALL(authorizer, authorize(An<const mesos::ACL::RunTask&>()))
+  EXPECT_CALL(authorizer, authorized(_))
     .WillOnce(DoAll(FutureSatisfy(&authorize),
                     Return(promise.future())));
 
@@ -404,7 +404,7 @@ TEST_F(MasterAuthorizationTest, SlaveDisconnected)
   // Return a pending future from authorizer.
   Future<Nothing> authorize;
   Promise<bool> promise;
-  EXPECT_CALL(authorizer, authorize(An<const mesos::ACL::RunTask&>()))
+  EXPECT_CALL(authorizer, authorized(_))
     .WillOnce(DoAll(FutureSatisfy(&authorize),
                     Return(promise.future())));
 
@@ -496,7 +496,7 @@ TEST_F(MasterAuthorizationTest, FrameworkRemoved)
   // Return a pending future from authorizer.
   Future<Nothing> authorize;
   Promise<bool> promise;
-  EXPECT_CALL(authorizer, authorize(An<const mesos::ACL::RunTask&>()))
+  EXPECT_CALL(authorizer, authorized(_))
     .WillOnce(DoAll(FutureSatisfy(&authorize),
                     Return(promise.future())));
 
@@ -573,7 +573,7 @@ TEST_F(MasterAuthorizationTest, PendingExecutorInfoDiffersOnDifferentSlaves)
   // Return a pending future from authorizer.
   Future<Nothing> authorize;
   Promise<bool> promise;
-  EXPECT_CALL(authorizer, authorize(An<const mesos::ACL::RunTask&>()))
+  EXPECT_CALL(authorizer, authorized(_))
     .WillOnce(DoAll(FutureSatisfy(&authorize),
                     Return(promise.future())));
 
@@ -615,7 +615,7 @@ TEST_F(MasterAuthorizationTest, PendingExecutorInfoDiffersOnDifferentSlaves)
   EXPECT_CALL(sched, statusUpdate(&driver, _))
     .WillOnce(FutureArg<1>(&status2));
 
-  EXPECT_CALL(authorizer, authorize(An<const mesos::ACL::RunTask&>()))
+  EXPECT_CALL(authorizer, authorized(_))
     .WillOnce(Return(true));
 
   driver.launchTasks(offers2.get()[0].id(), {task2});
@@ -759,7 +759,7 @@ TEST_F(MasterAuthorizationTest, DuplicateRegistration)
   Promise<bool> promise1;
   Future<Nothing> authorize2;
   Promise<bool> promise2;
-  EXPECT_CALL(authorizer, authorize(An<const mesos::ACL::RegisterFramework&>()))
+  EXPECT_CALL(authorizer, authorized(_))
     .WillOnce(DoAll(FutureSatisfy(&authorize1),
                     Return(promise1.future())))
     .WillOnce(DoAll(FutureSatisfy(&authorize2),
@@ -829,7 +829,7 @@ TEST_F(MasterAuthorizationTest, DuplicateReregistration)
   Promise<bool> promise2;
   Future<Nothing> authorize3;
   Promise<bool> promise3;
-  EXPECT_CALL(authorizer, authorize(An<const mesos::ACL::RegisterFramework&>()))
+  EXPECT_CALL(authorizer, authorized(_))
     .WillOnce(Return(true))
     .WillOnce(DoAll(FutureSatisfy(&authorize2),
                     Return(promise2.future())))
@@ -901,7 +901,7 @@ TEST_F(MasterAuthorizationTest, FrameworkRemovedBeforeRegistration)
   // Return a pending future from authorizer.
   Future<Nothing> authorize;
   Promise<bool> promise;
-  EXPECT_CALL(authorizer, authorize(An<const mesos::ACL::RegisterFramework&>()))
+  EXPECT_CALL(authorizer, authorized(_))
     .WillOnce(DoAll(FutureSatisfy(&authorize),
                     Return(promise.future())))
     .WillRepeatedly(Return(true)); // Authorize subsequent registration retries.
@@ -962,7 +962,7 @@ TEST_F(MasterAuthorizationTest, FrameworkRemovedBeforeReregistration)
   // Return a pending future from authorizer after first attempt.
   Future<Nothing> authorize2;
   Promise<bool> promise2;
-  EXPECT_CALL(authorizer, authorize(An<const mesos::ACL::RegisterFramework&>()))
+  EXPECT_CALL(authorizer, authorized(_))
     .WillOnce(Return(true))
     .WillOnce(DoAll(FutureSatisfy(&authorize2),
                     Return(promise2.future())));

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/tests/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index 5770721..7cca4ed 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -735,38 +735,8 @@ MockAuthorizer::MockAuthorizer()
   // NOTE: We use 'EXPECT_CALL' and 'WillRepeatedly' here instead of
   // 'ON_CALL' and 'WillByDefault'. See 'TestContainerizer::SetUp()'
   // for more details.
-  EXPECT_CALL(*this, authorize(An<const mesos::ACL::RegisterFramework&>()))
+  EXPECT_CALL(*this, authorized(_))
     .WillRepeatedly(Return(true));
-
-  EXPECT_CALL(*this, authorize(An<const mesos::ACL::RunTask&>()))
-    .WillRepeatedly(Return(true));
-
-  EXPECT_CALL(*this, authorize(An<const mesos::ACL::TeardownFramework&>()))
-    .WillRepeatedly(Return(true));
-
-  EXPECT_CALL(*this, authorize(An<const mesos::ACL::ReserveResources&>()))
-    .WillRepeatedly(Return(true));
-
-  EXPECT_CALL(*this, authorize(An<const mesos::ACL::UnreserveResources&>()))
-    .WillRepeatedly(Return(true));
-
-  EXPECT_CALL(*this, authorize(An<const mesos::ACL::CreateVolume&>()))
-    .WillRepeatedly(Return(true));
-
-  EXPECT_CALL(*this, authorize(An<const mesos::ACL::DestroyVolume&>()))
-    .WillRepeatedly(Return(true));
-
-  EXPECT_CALL(*this, authorize(An<const mesos::ACL::SetQuota&>()))
-    .WillRepeatedly(Return(true));
-
-  EXPECT_CALL(*this, authorize(An<const mesos::ACL::RemoveQuota&>()))
-    .WillRepeatedly(Return(true));
-
-  EXPECT_CALL(*this, authorize(An<const mesos::ACL::UpdateWeights&>()))
-    .WillRepeatedly(Return(true));
-
-  EXPECT_CALL(*this, initialize(An<const Option<ACLs>&>()))
-    .WillRepeatedly(Return(Nothing()));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 9409da7..908a53a 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -1635,27 +1635,7 @@ public:
   virtual ~MockAuthorizer();
 
   MOCK_METHOD1(
-      initialize, Try<Nothing>(const Option<ACLs>& acls));
-  MOCK_METHOD1(
-      authorize, process::Future<bool>(const ACL::RegisterFramework& request));
-  MOCK_METHOD1(
-      authorize, process::Future<bool>(const ACL::RunTask& request));
-  MOCK_METHOD1(
-      authorize, process::Future<bool>(const ACL::TeardownFramework& request));
-  MOCK_METHOD1(
-      authorize, process::Future<bool>(const ACL::ReserveResources& request));
-  MOCK_METHOD1(
-      authorize, process::Future<bool>(const ACL::UnreserveResources& request));
-  MOCK_METHOD1(
-      authorize, process::Future<bool>(const ACL::CreateVolume& request));
-  MOCK_METHOD1(
-      authorize, process::Future<bool>(const ACL::DestroyVolume& request));
-  MOCK_METHOD1(
-      authorize, process::Future<bool>(const ACL::SetQuota& request));
-  MOCK_METHOD1(
-      authorize, process::Future<bool>(const ACL::RemoveQuota& request));
-  MOCK_METHOD1(
-      authorize, process::Future<bool>(const ACL::UpdateWeights& request));
+      authorized, process::Future<bool>(const authorization::Request& request));
 };
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/482dc141/src/tests/reconciliation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reconciliation_tests.cpp b/src/tests/reconciliation_tests.cpp
index 97112c4..e8f3f29 100644
--- a/src/tests/reconciliation_tests.cpp
+++ b/src/tests/reconciliation_tests.cpp
@@ -652,7 +652,7 @@ TEST_F(ReconciliationTest, PendingTask)
   // Return a pending future from authorizer.
   Future<Nothing> authorize;
   Promise<bool> promise;
-  EXPECT_CALL(authorizer, authorize(An<const mesos::ACL::RunTask&>()))
+  EXPECT_CALL(authorizer, authorized(_))
     .WillOnce(DoAll(FutureSatisfy(&authorize),
                     Return(promise.future())));