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 2014/05/10 18:54:14 UTC

git commit: Implemented a basic Authorizer interface.

Repository: mesos
Updated Branches:
  refs/heads/master 2a11d35d8 -> 50bbbf5f8


Implemented a basic Authorizer interface.

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


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

Branch: refs/heads/master
Commit: 50bbbf5f8e751d721aaf494eb40358d2c51eadc9
Parents: 2a11d35
Author: Vinod Kone <vi...@twitter.com>
Authored: Mon Mar 3 22:55:21 2014 -0800
Committer: Vinod Kone <vi...@twitter.com>
Committed: Sat May 10 09:53:49 2014 -0700

----------------------------------------------------------------------
 include/mesos/mesos.proto                     | 115 ++++-
 src/Makefile.am                               |   2 +
 src/authorizer/authorizer.hpp                 | 394 +++++++++++++++
 src/tests/authorization_tests.cpp             | 541 +++++++++++++++++++++
 src/tests/master_contender_detector_tests.cpp |   2 +-
 5 files changed, 1039 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/50bbbf5f/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index a5826d7..8012873 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -94,20 +94,21 @@ message ContainerID {
 
 
 /**
- * Describes a framework. If the user field is set to an empty string
- * Mesos will automagically set it to the current user. Note that the
- * ID is only available after a framework has registered, however, it
- * is included here in order to facilitate scheduler failover (i.e.,
- * if it is set then the MesosSchedulerDriver expects the scheduler is
- * performing failover). The amount of time that the master will wait
- * for the scheduler to failover before removing the framework is
- * specified by failover_timeout.
- * If checkpoint is set, framework pid, executor pids and status updates
- * are checkpointed to disk by the slaves.
- * Checkpointing allows a restarted slave to reconnect with old executors
- * and recover status updates, at the cost of disk I/O.
- * The role field is used to group frameworks for allocation decisions,
- * depending on the allocation policy being used.
+ * Describes a framework. The user field is used to determine the
+ * Unix user that an executor/task should be launched as. If the user
+ * field is set to an empty string Mesos will automagically set it
+ * to the current user. Note that the ID is only available after a
+ * framework has registered, however, it is included here in order to
+ * facilitate scheduler failover (i.e., if it is set then the
+ * MesosSchedulerDriver expects the scheduler is performing failover).
+ * The amount of time that the master will wait for the scheduler to
+ * failover before removing the framework is specified by
+ * failover_timeout. If checkpoint is set, framework pid, executor
+ * pids and status updates are checkpointed to disk by the slaves.
+ * Checkpointing allows a restarted slave to reconnect with old
+ * executors and recover status updates, at the cost of disk I/O.
+ * The role field is used to group frameworks for allocation
+ * decisions, depending on the allocation policy being used.
  * If the hostname field is set to an empty string Mesos will
  * automagically set it to the current hostname.
  */
@@ -563,3 +564,89 @@ message Credential {
   required string principal = 1;
   optional bytes secret = 2;
 }
+
+
+/**
+ * ACLs used for authorization.
+ */
+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 RunTasks {
+    // Subjects.
+    required Entity principals = 1; // Framework principals.
+
+    // Objects.
+    required Entity users = 2; // Users to run the tasks/executors as.
+  }
+
+  message ReceiveOffers {
+    // Subjects.
+    required Entity principals = 1; // Framework principals.
+
+    // Objects.
+    required Entity roles = 2; // Resource roles that can be offered.
+  }
+
+  message HTTPGet {
+    // Subjects (At least one of these should be set).
+    optional Entity usernames = 1; // HTTP authentication based usernames.
+    optional Entity ips = 2;
+    optional Entity hostnames = 3;
+
+    // Objects.
+    required Entity urls = 4;
+  }
+
+  message HTTPPut {
+    // Subjects (At least one of these should be set).
+    optional Entity usernames = 1; // HTTP authentication based usernames.
+    optional Entity ips = 2;
+    optional Entity hostnames = 3;
+
+    // Objects.
+    required Entity urls = 4;
+  }
+}
+
+
+/*
+ * 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 {
+  required bool permissive = 1 [default = true];
+  repeated ACL.RunTasks run_tasks = 2;
+  repeated ACL.ReceiveOffers receive_offers = 3;
+  repeated ACL.HTTPGet http_get = 4;
+  repeated ACL.HTTPPut http_put = 5;
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/50bbbf5f/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index f461a15..812ad2c 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -186,6 +186,7 @@ nodist_libmesos_no_3rdparty_la_SOURCES =				\
   $(REGISTRY_PROTOS)
 
 libmesos_no_3rdparty_la_SOURCES =					\
+	authorizer/authorizer.hpp					\
 	sasl/authenticatee.hpp						\
 	sasl/authenticator.hpp						\
 	sasl/auxprop.hpp						\
@@ -887,6 +888,7 @@ mesos_tests_SOURCES =				\
   tests/allocator_tests.cpp			\
   tests/attributes_tests.cpp			\
   tests/authentication_tests.cpp		\
+  tests/authorization_tests.cpp		        \
   tests/containerizer.cpp			\
   tests/containerizer_tests.cpp			\
   tests/environment.cpp				\

http://git-wip-us.apache.org/repos/asf/mesos/blob/50bbbf5f/src/authorizer/authorizer.hpp
----------------------------------------------------------------------
diff --git a/src/authorizer/authorizer.hpp b/src/authorizer/authorizer.hpp
new file mode 100644
index 0000000..b542c96
--- /dev/null
+++ b/src/authorizer/authorizer.hpp
@@ -0,0 +1,394 @@
+/**
+ * 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 __AUTHORIZER_AUTHORIZER_HPP__
+#define __AUTHORIZER_AUTHORIZER_HPP__
+
+#include <string>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include <process/dispatch.hpp>
+#include <process/future.hpp>
+#include <process/id.hpp>
+#include <process/owned.hpp>
+#include <process/process.hpp>
+#include <process/protobuf.hpp>
+
+#include <stout/check.hpp>
+#include <stout/hashmap.hpp>
+#include <stout/hashset.hpp>
+#include <stout/try.hpp>
+
+#include "mesos/mesos.hpp"
+
+#include "master/flags.hpp"
+
+namespace mesos {
+namespace internal {
+
+// Forward declaration.
+class LocalAuthorizerProcess;
+
+
+class Authorizer
+{
+public:
+  virtual ~Authorizer() {}
+
+  // Attempts to create an Authorizer based on the flags.
+  static Try<process::Owned<Authorizer> > create(const master::Flags& flags);
+
+  // Returns true if the ACL can be satisfied or false otherwise.
+  // A failed future indicates a transient failure and the user
+  // can (should) retry.
+  virtual process::Future<bool> authorize(
+      const ACL::RunTasks& request) = 0;
+  virtual process::Future<bool> authorize(
+      const ACL::ReceiveOffers& request) = 0;
+  virtual process::Future<bool> authorize(
+      const ACL::HTTPGet& request) = 0;
+  virtual process::Future<bool> authorize(
+      const ACL::HTTPPut& request) = 0;
+
+protected:
+  Authorizer() {}
+};
+
+
+// This Authorizer is constructed with all the required ACLs upfront.
+class LocalAuthorizer : public Authorizer
+{
+public:
+  // Validates the ACLs and creates a LocalAuthorizer.
+  static Try<process::Owned<LocalAuthorizer> > create(const ACLs& acls);
+  virtual ~LocalAuthorizer();
+
+  // Implementation of Authorizer interface.
+  virtual process::Future<bool> authorize(const ACL::RunTasks& request);
+  virtual process::Future<bool> authorize(const ACL::ReceiveOffers& request);
+  virtual process::Future<bool> authorize(const ACL::HTTPGet& request);
+  virtual process::Future<bool> authorize(const ACL::HTTPPut& request);
+
+private:
+  LocalAuthorizer(const ACLs& acls);
+  LocalAuthorizerProcess* process;
+};
+
+
+class LocalAuthorizerProcess : public ProtobufProcess<LocalAuthorizerProcess>
+{
+public:
+  LocalAuthorizerProcess(const ACLs& _acls)
+    : ProcessBase(process::ID::generate("authorizer")), acls(_acls) {}
+
+  process::Future<bool> authorize(const ACL::RunTasks& request)
+  {
+    foreach (const ACL::RunTasks& 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());
+      }
+    }
+
+    return acls.permissive(); // None of the ACLs match.
+  }
+
+  process::Future<bool> authorize(const ACL::ReceiveOffers& request)
+  {
+    foreach (const ACL::ReceiveOffers& acl, acls.receive_offers()) {
+      // 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());
+      }
+    }
+
+    return acls.permissive(); // None of the ACLs match.
+  }
+
+  process::Future<bool> authorize(const ACL::HTTPGet& request)
+  {
+    foreach (const ACL::HTTPGet& acl, acls.http_get()) {
+      // ACL matches if both subjects and objects match.
+      if (matches(request.usernames(), acl.usernames()) &&
+          matches(request.ips(), acl.ips()) &&
+          matches(request.hostnames(), acl.hostnames()) &&
+          matches(request.urls(), acl.urls())) {
+
+        // ACL is allowed if both subjects and objects are allowed.
+        return allows(request.usernames(), acl.usernames()) &&
+               allows(request.ips(), acl.ips()) &&
+               allows(request.hostnames(), acl.hostnames()) &&
+               allows(request.urls(), acl.urls());
+      }
+    }
+
+    return acls.permissive(); // None of the ACLs match.
+  }
+
+  process::Future<bool> authorize(const ACL::HTTPPut& request)
+  {
+    foreach (const ACL::HTTPPut& acl, acls.http_put()) {
+      // ACL matches if both subjects and objects match.
+      if (matches(request.usernames(), acl.usernames()) &&
+          matches(request.ips(), acl.ips()) &&
+          matches(request.hostnames(), acl.hostnames()) &&
+          matches(request.urls(), acl.urls())) {
+
+        // ACL is allowed if both subjects and objects are allowed.
+        return allows(request.usernames(), acl.usernames()) &&
+               allows(request.ips(), acl.ips()) &&
+               allows(request.hostnames(), acl.hostnames()) &&
+               allows(request.urls(), acl.urls());
+      }
+    }
+
+    return acls.permissive(); // None of the ACLs match.
+  }
+private:
+  // Match matrix:
+  //
+  //                  -----------ACL----------
+  //
+  //                    SOME    NONE    ANY
+  //          -------|-------|-------|-------
+  //  |        SOME  | Yes/No|  Yes  |   Yes
+  //  |       -------|-------|-------|-------
+  // Request   NONE  |  No   |  Yes  |   No
+  //  |       -------|-------|-------|-------
+  //  |        ANY   |  No   |  Yes  |   Yes
+  //          -------|-------|-------|-------
+  bool matches(const ACL::Entity& request, const ACL::Entity& acl)
+  {
+    // NONE only matches with NONE.
+    if (request.type() == ACL::Entity::NONE) {
+      return acl.type() == ACL::Entity::NONE;
+    }
+
+    // ANY matches with ANY or NONE.
+    if (request.type() == ACL::Entity::ANY) {
+      return acl.type() == ACL::Entity::ANY || acl.type() == ACL::Entity::NONE;
+    }
+
+    if (request.type() == ACL::Entity::SOME) {
+      // SOME matches with ANY or NONE.
+      if (acl.type() == ACL::Entity::ANY || acl.type() == ACL::Entity::NONE) {
+        return true;
+      }
+
+      // SOME is allowed if the request values are a subset of ACL
+      // values.
+      foreach (const std::string& value, request.values()) {
+        bool found = false;
+        foreach (const std::string& value_, acl.values()) {
+          if (value == value_) {
+            found = true;
+            break;
+          }
+        }
+
+        if (!found) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    return false;
+  }
+
+  // Allow matrix:
+  //
+  //                 -----------ACL----------
+  //
+  //                    SOME    NONE    ANY
+  //          -------|-------|-------|-------
+  //  |        SOME  | Yes/No|  No   |   Yes
+  //  |       -------|-------|-------|-------
+  // Request   NONE  |  No   |  Yes  |   No
+  //  |       -------|-------|-------|-------
+  //  |        ANY   |  No   |  No   |   Yes
+  //          -------|-------|-------|-------
+  bool allows(const ACL::Entity& request, const ACL::Entity& acl)
+  {
+    // NONE is only allowed by NONE.
+    if (request.type() == ACL::Entity::NONE) {
+      return acl.type() == ACL::Entity::NONE;
+    }
+
+    // ANY is only allowed by ANY.
+    if (request.type() == ACL::Entity::ANY) {
+      return acl.type() == ACL::Entity::ANY;
+    }
+
+    if (request.type() == ACL::Entity::SOME) {
+      // SOME is allowed by ANY.
+      if (acl.type() == ACL::Entity::ANY) {
+        return true;
+      }
+
+      // SOME is not allowed by NONE.
+      if (acl.type() == ACL::Entity::NONE) {
+        return false;
+      }
+
+      // SOME is allowed if the request values are a subset of ACL
+      // values.
+      foreach (const std::string& value, request.values()) {
+        bool found = false;
+        foreach (const std::string& value_, acl.values()) {
+          if (value == value_) {
+            found = true;
+            break;
+          }
+        }
+
+        if (!found) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    return false;
+  }
+
+  ACLs acls;
+};
+
+
+Try<process::Owned<Authorizer> > Authorizer::create(const master::Flags& flags)
+{
+  // TODO(vinod): Parse "flags.acls" from JSON to "ACLs" protobuf.
+  Try<process::Owned<LocalAuthorizer> > authorizer =
+    LocalAuthorizer::create(ACLs());
+
+  if (authorizer.isError()) {
+    return Error(authorizer.error());
+  }
+
+  return authorizer.get().get();
+}
+
+
+LocalAuthorizer::LocalAuthorizer(const ACLs& acls)
+{
+  process = new LocalAuthorizerProcess(acls);
+  process::spawn(process);
+}
+
+
+LocalAuthorizer::~LocalAuthorizer()
+{
+  process::terminate(process);
+  process::wait(process);
+  delete process;
+}
+
+
+Try<process::Owned<LocalAuthorizer> > LocalAuthorizer::create(const ACLs& acls)
+{
+  // Validate ACLs.
+  foreach (const ACL::HTTPGet& acl, acls.http_get()) {
+    // At least one of the subjects should be set.
+    if (acl.has_usernames() + acl.has_ips() + acl.has_hostnames() < 1) {
+      return Error("At least one of the subjects should be set for ACL: " +
+                    acl.DebugString());
+    }
+  }
+
+  foreach (const ACL::HTTPPut& acl, acls.http_put()) {
+    // At least one of the subjects should be set.
+    if (acl.has_usernames() + acl.has_ips() + acl.has_hostnames() < 1) {
+       return Error("At least one of the subjects should be set for ACL: " +
+                     acl.DebugString());
+     }
+   }
+
+  return new LocalAuthorizer(acls);
+}
+
+
+process::Future<bool> LocalAuthorizer::authorize(
+    const ACL::RunTasks& request)
+{
+  // Necessary to disambiguate.
+  typedef process::Future<bool>(LocalAuthorizerProcess::*F)(
+      const ACL::RunTasks&);
+
+  return process::dispatch(
+      process,
+      static_cast<F>(&LocalAuthorizerProcess::authorize),
+      request);
+}
+
+
+process::Future<bool> LocalAuthorizer::authorize(
+    const ACL::ReceiveOffers& request)
+{
+  // Necessary to disambiguate.
+  typedef process::Future<bool>(LocalAuthorizerProcess::*F)(
+      const ACL::ReceiveOffers&);
+
+  return process::dispatch(
+      process,
+      static_cast<F>(&LocalAuthorizerProcess::authorize),
+      request);
+}
+
+
+process::Future<bool> LocalAuthorizer::authorize(
+    const ACL::HTTPGet& request)
+{
+  // Necessary to disambiguate.
+  typedef process::Future<bool>(LocalAuthorizerProcess::*F)(
+      const ACL::HTTPGet&);
+
+  return process::dispatch(
+      process,
+      static_cast<F>(&LocalAuthorizerProcess::authorize),
+      request);
+}
+
+
+process::Future<bool> LocalAuthorizer::authorize(
+    const ACL::HTTPPut& request)
+{
+  // Necessary to disambiguate.
+  typedef process::Future<bool>(LocalAuthorizerProcess::*F)(
+      const ACL::HTTPPut&);
+
+  return process::dispatch(
+      process,
+      static_cast<F>(&LocalAuthorizerProcess::authorize),
+      request);
+}
+
+} // namespace internal {
+} // namespace mesos {
+
+#endif //__AUTHORIZER_AUTHORIZER_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/50bbbf5f/src/tests/authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/authorization_tests.cpp b/src/tests/authorization_tests.cpp
new file mode 100644
index 0000000..611f371
--- /dev/null
+++ b/src/tests/authorization_tests.cpp
@@ -0,0 +1,541 @@
+/**
+ * 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 <gtest/gtest.h>
+
+#include <process/future.hpp>
+
+#include "authorizer/authorizer.hpp"
+
+#include "tests/mesos.hpp"
+
+using namespace mesos;
+using namespace mesos::internal;
+using namespace mesos::internal::tests;
+
+using namespace process;
+
+class AuthorizationTest : public MesosTest {};
+
+
+TEST_F(AuthorizationTest, AnyPrincipalRunAsUser)
+{
+  // Any principal can run as "guest" user.
+  ACLs acls;
+  mesos::ACL::RunTasks* 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<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Principals "foo" and "bar" can run as "guest".
+  mesos::ACL::RunTasks request;
+  request.mutable_principals()->add_values("foo");
+  request.mutable_principals()->add_values("bar");
+  request.mutable_users()->add_values("guest");
+  AWAIT_EXPECT_EQ(true, authorizer.get()->authorize(request));
+
+  // Principal "foo" can run as "root" since the ACLs are permissive.
+  mesos::ACL::RunTasks request2;
+  request2.mutable_principals()->add_values("foo");
+  request2.mutable_users()->add_values("root");
+  AWAIT_EXPECT_EQ(true, authorizer.get()->authorize(request2));
+}
+
+
+TEST_F(AuthorizationTest, NoPrincipalRunAsUser)
+{
+  // No principal can run as "root" user.
+  ACLs acls;
+  mesos::ACL::RunTasks* 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<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Principal "foo" cannot run as "root".
+  mesos::ACL::RunTasks request;
+  request.mutable_principals()->add_values("foo");
+  request.mutable_users()->add_values("root");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request));
+}
+
+
+TEST_F(AuthorizationTest, PrincipalRunAsAnyUser)
+{
+  // A principal "foo" can run as any user.
+  ACLs acls;
+  mesos::ACL::RunTasks* 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<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Principal "foo" can run as "user1" and "user2".
+  mesos::ACL::RunTasks request;
+  request.mutable_principals()->add_values("foo");
+  request.mutable_users()->add_values("user1");
+  request.mutable_users()->add_values("user2");
+  AWAIT_EXPECT_EQ(true, authorizer.get()->authorize(request));
+}
+
+
+TEST_F(AuthorizationTest, AnyPrincipalRunAsAnyUser)
+{
+  // Any principal can run as any user.
+  ACLs acls;
+  mesos::ACL::RunTasks* 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<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Principals "foo" and "bar" can run as "user1" and "user2".
+  mesos::ACL::RunTasks 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_EQ(true, authorizer.get()->authorize(request));
+}
+
+
+TEST_F(AuthorizationTest, OnlySomePrincipalsRunAsSomeUsers)
+{
+  // Only some principals can run as some users.
+  ACLs acls;
+
+  // ACL for some principals to run as some users.
+  mesos::ACL::RunTasks* 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::RunTasks* 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");
+
+  // Create an Authorizer with the ACLs.
+  Try<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Principals "foo" and "bar" can run as "user1" and "user2".
+  mesos::ACL::RunTasks 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_EQ(true, authorizer.get()->authorize(request));
+
+  // Principal "baz" cannot run as "user1".
+  mesos::ACL::RunTasks request2;
+  request2.mutable_principals()->add_values("baz");
+  request2.mutable_users()->add_values("user1");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request2));
+
+  // Principal "baz" cannot run as "user2".
+  mesos::ACL::RunTasks request3;
+  request3.mutable_principals()->add_values("baz");
+  request3.mutable_users()->add_values("user1");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request3));
+}
+
+
+TEST_F(AuthorizationTest, SomePrincipalOnlySomeUser)
+{
+  // Some principal can run as only some user.
+  ACLs acls;
+
+  // ACL for some principal to run as some user.
+  mesos::ACL::RunTasks* 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::RunTasks* acl2 = acls.add_run_tasks();
+  acl2->mutable_principals()->add_values("foo");
+  acl2->mutable_users()->set_type(mesos::ACL::Entity::NONE);
+
+  // Create an Authorizer with the ACLs.
+  Try<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Principal "foo" can run as "user1".
+  mesos::ACL::RunTasks request;
+  request.mutable_principals()->add_values("foo");
+  request.mutable_users()->add_values("user1");
+  AWAIT_EXPECT_EQ(true, authorizer.get()->authorize(request));
+
+  // Principal "foo" cannot run as "user2".
+  mesos::ACL::RunTasks request2;
+  request2.mutable_principals()->add_values("foo");
+  request2.mutable_users()->add_values("user2");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request2));
+
+  // Principal "bar" can run as "user1" and "user2".
+  mesos::ACL::RunTasks request3;
+  request3.mutable_principals()->add_values("bar");
+  request3.mutable_users()->add_values("user1");
+  request3.mutable_users()->add_values("user2");
+  AWAIT_EXPECT_EQ(true, authorizer.get()->authorize(request3));
+}
+
+
+TEST_F(AuthorizationTest, PrincipalRunAsSomeUserRestrictive)
+{
+  // A principal can run as "user1";
+  ACLs acls;
+  acls.set_permissive(false); // Restrictive.
+  mesos::ACL::RunTasks* acl = acls.add_run_tasks();
+  acl->mutable_principals()->add_values("foo");
+  acl->mutable_users()->add_values("user1");
+
+  // Create an Authorizer with the ACLs.
+  Try<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Principal "foo" can run as "user1".
+  mesos::ACL::RunTasks request;
+  request.mutable_principals()->add_values("foo");
+  request.mutable_users()->add_values("user1");
+  AWAIT_EXPECT_EQ(true, authorizer.get()->authorize(request));
+
+  // Principal "foo" cannot run as "user2".
+  mesos::ACL::RunTasks request2;
+  request2.mutable_principals()->add_values("foo");
+  request2.mutable_users()->add_values("user2");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request2));
+
+  // Principal "bar" cannot run as "user2" since no ACL is set.
+  mesos::ACL::RunTasks request3;
+  request3.mutable_principals()->add_values("bar");
+  request3.mutable_users()->add_values("user2");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request3));
+}
+
+
+TEST_F(AuthorizationTest, AnyPrincipalOfferedRole)
+{
+  // Any principal can be offered "*" role's resources.
+  ACLs acls;
+  mesos::ACL::ReceiveOffers* acl = acls.add_receive_offers();
+  acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+  acl->mutable_roles()->add_values("*");
+
+  // Create an Authorizer with the ACLs.
+  Try<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Principals "foo" and "bar" can be offered "*" role's resources.
+  mesos::ACL::ReceiveOffers request;
+  request.mutable_principals()->add_values("foo");
+  request.mutable_principals()->add_values("bar");
+  request.mutable_roles()->add_values("*");
+  AWAIT_EXPECT_EQ(true, authorizer.get()->authorize(request));
+}
+
+
+TEST_F(AuthorizationTest, SomePrincipalsOfferedRole)
+{
+  // Some principals can be offered "ads" role's resources.
+  ACLs acls;
+  mesos::ACL::ReceiveOffers* acl = acls.add_receive_offers();
+  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<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Principals "foo", "bar" and "baz" (no ACL) can be offered "ads"
+  // role's resources.
+  mesos::ACL::ReceiveOffers 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_EQ(true, authorizer.get()->authorize(request));
+}
+
+
+TEST_F(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::ReceiveOffers* acl = acls.add_receive_offers();
+  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::ReceiveOffers* acl2 = acls.add_receive_offers();
+  acl2->mutable_principals()->set_type(mesos::ACL::Entity::NONE);
+  acl2->mutable_roles()->add_values("analytics");
+
+  // Create an Authorizer with the ACLs.
+  Try<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Principal "foo" can be offered "analytics" role's resources.
+  mesos::ACL::ReceiveOffers request;
+  request.mutable_principals()->add_values("foo");
+  request.mutable_roles()->add_values("analytics");
+  AWAIT_EXPECT_EQ(true, authorizer.get()->authorize(request));
+
+  // Principal "bar" cannot be offered "analytics" role's resources.
+  mesos::ACL::ReceiveOffers request2;
+  request2.mutable_principals()->add_values("bar");
+  request2.mutable_roles()->add_values("analytics");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request2));
+}
+
+
+TEST_F(AuthorizationTest, PrincipalNotOfferedAnyRoleRestrictive)
+{
+  // A principal "foo" can be offered "analytics" role's resources.
+  ACLs acls;
+  acls.set_permissive(false);
+  mesos::ACL::ReceiveOffers* acl = acls.add_receive_offers();
+  acl->mutable_principals()->add_values("foo");
+  acl->mutable_roles()->add_values("analytics");
+
+  // Create an Authorizer with the ACLs.
+  Try<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Principal "foo" can be offered "analytics" role's resources.
+  mesos::ACL::ReceiveOffers request;
+  request.mutable_principals()->add_values("foo");
+  request.mutable_roles()->add_values("analytics");
+  AWAIT_EXPECT_EQ(true, authorizer.get()->authorize(request));
+
+  // Principal "bar" cannot be offered "analytics" role's resources.
+  mesos::ACL::ReceiveOffers request2;
+  request2.mutable_principals()->add_values("bar");
+  request2.mutable_roles()->add_values("analytics");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request2));
+
+  // Principal "bar" cannot be offered "ads" role's resources because no ACL.
+  mesos::ACL::ReceiveOffers request3;
+  request3.mutable_principals()->add_values("bar");
+  request3.mutable_roles()->add_values("ads");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request3));
+}
+
+
+TEST_F(AuthorizationTest, AnyClientGETSomeURL)
+{
+  // Any client can GET access "/help".
+  ACLs acls;
+  mesos::ACL::HTTPGet* acl = acls.add_http_get();
+  acl->mutable_usernames()->set_type(mesos::ACL::Entity::ANY);
+  acl->mutable_ips()->set_type(mesos::ACL::Entity::ANY);
+  acl->mutable_hostnames()->set_type(mesos::ACL::Entity::ANY);
+  acl->mutable_urls()->add_values("/help");
+
+  // Create an Authorizer with the ACLs.
+  Try<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Clients "foo", "127.0.0.1", "localhost" can GET access "/help".
+  mesos::ACL::HTTPGet request;
+  request.mutable_usernames()->add_values("foo");
+  request.mutable_ips()->add_values("127.0.0.1");
+  request.mutable_hostnames()->add_values("localhost");
+  request.mutable_urls()->add_values("/help");
+  AWAIT_EXPECT_EQ(true, authorizer.get()->authorize(request));
+}
+
+
+TEST_F(AuthorizationTest, SomeClientsPUTSomeURL)
+{
+  // Only some clients can PUT access "/admin".
+  ACLs acls;
+
+  // Some clients can PUT access "/admin".
+  mesos::ACL::HTTPPut* acl = acls.add_http_put();
+  acl->mutable_ips()->add_values("127.0.0.1");
+  acl->mutable_hostnames()->add_values("localhost");
+  acl->mutable_urls()->add_values("/admin");
+
+  // No one else can PUT access "/admin".
+  mesos::ACL::HTTPPut* acl2 = acls.add_http_put();
+  acl2->mutable_usernames()->set_type(mesos::ACL::Entity::NONE);
+  acl2->mutable_ips()->set_type(mesos::ACL::Entity::NONE);
+  acl2->mutable_hostnames()->set_type(mesos::ACL::Entity::NONE);
+  acl2->mutable_urls()->add_values("/admin");
+
+  // Create an Authorizer with the ACLs.
+  Try<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Clients "127.0.0.1" and "localhost" can PUT access "/admin".
+  mesos::ACL::HTTPPut request;
+  request.mutable_ips()->add_values("127.0.0.1");
+  request.mutable_hostnames()->add_values("localhost");
+  request.mutable_urls()->add_values("/admin");
+  AWAIT_EXPECT_EQ(true, authorizer.get()->authorize(request));
+
+  // Client "10.0.0.0" cannot PUT access "/admin".
+  mesos::ACL::HTTPPut request2;
+  request2.mutable_ips()->add_values("10.0.0.0");
+  request2.mutable_urls()->add_values("/admin");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request2));
+}
+
+
+TEST_F(AuthorizationTest, NoClientGETPUTSomeURL)
+{
+  // No client can GET access "/secret".
+  ACLs acls;
+  mesos::ACL::HTTPGet* acl = acls.add_http_get();
+  acl->mutable_usernames()->set_type(mesos::ACL::Entity::NONE);
+  acl->mutable_ips()->set_type(mesos::ACL::Entity::NONE);
+  acl->mutable_hostnames()->set_type(mesos::ACL::Entity::NONE);
+  acl->mutable_urls()->add_values("/secret");
+
+  // No client can PUT access "/secret".
+  mesos::ACL::HTTPPut* acl2 = acls.add_http_put();
+  acl2->mutable_usernames()->set_type(mesos::ACL::Entity::NONE);
+  acl2->mutable_ips()->set_type(mesos::ACL::Entity::NONE);
+  acl2->mutable_hostnames()->set_type(mesos::ACL::Entity::NONE);
+  acl2->mutable_urls()->add_values("/secret");
+
+  // Create an Authorizer with the ACLs.
+  Try<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Clients "127.0.0.1" and "localhost" cannot GET access "/secret".
+  mesos::ACL::HTTPGet request;
+  request.mutable_ips()->add_values("127.0.0.1");
+  request.mutable_hostnames()->add_values("localhost");
+  request.mutable_urls()->add_values("/secret");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request));
+
+  // Clients "127.0.0.1" and "localhost" cannot PUT access "/secret".
+  mesos::ACL::HTTPPut request2;
+  request2.mutable_ips()->add_values("127.0.0.1");
+  request2.mutable_hostnames()->add_values("localhost");
+  request2.mutable_urls()->add_values("/secret");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request2));
+}
+
+
+TEST_F(AuthorizationTest, SomeClientsCannotGETAnyURL)
+{
+  // Some clients cannot GET access any URL.
+  ACLs acls;
+  mesos::ACL::HTTPGet* acl = acls.add_http_get();
+  acl->mutable_ips()->add_values("127.0.0.1");
+  acl->mutable_hostnames()->add_values("localhost");
+  acl->mutable_urls()->set_type(mesos::ACL::Entity::NONE);
+
+  // Create an Authorizer with the ACLs.
+  Try<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Clients "127.0.0.1" and "localhost" cannot GET access "/help".
+  mesos::ACL::HTTPGet request;
+  request.mutable_ips()->add_values("127.0.0.1");
+  request.mutable_hostnames()->add_values("localhost");
+  request.mutable_urls()->add_values("/help");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request));
+}
+
+
+TEST_F(AuthorizationTest, NoClientsCanGETPUTAnyURLRestrictive)
+{
+  // No clients can GET/PUT access any URL.
+  ACLs acls;
+  acls.set_permissive(false);
+
+  // Create an Authorizer with the ACLs.
+  Try<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // Clients "foo", "127.0.0.1" cannot GET access "/help".
+  mesos::ACL::HTTPGet request;
+  request.mutable_usernames()->add_values("foo");
+  request.mutable_ips()->add_values("127.0.0.1");
+  request.mutable_urls()->add_values("/help");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request));
+
+  // Clients "127.0.0.1", "localhost" cannot PUT access "/help".
+  mesos::ACL::HTTPPut request2;
+  request2.mutable_ips()->add_values("127.0.0.1");
+  request2.mutable_hostnames()->add_values("localhost");
+  request2.mutable_urls()->add_values("/help");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request2));
+}
+
+
+TEST_F(AuthorizationTest, SomeClientsAggregatePUTRequestRestrictive)
+{
+  // Some clients can PUT access "/admin" but ACLs are setup
+  // separately.
+  ACLs acls;
+  acls.set_permissive(false);
+
+  // "foo" can PUT access "/admin".
+  mesos::ACL::HTTPPut* acl = acls.add_http_put();
+  acl->mutable_usernames()->add_values("foo");
+  acl->mutable_urls()->add_values("/admin");
+
+  // "bar" can PUT access "/admin".
+  mesos::ACL::HTTPPut* acl2 = acls.add_http_put();
+  acl2->mutable_usernames()->add_values("bar");
+  acl2->mutable_urls()->add_values("/admin");
+
+  // Create an Authorizer with the ACLs.
+  Try<Owned<LocalAuthorizer> > authorizer = LocalAuthorizer::create(acls);
+  ASSERT_SOME(authorizer);
+
+  // "foo" can PUT access "/admin".
+  mesos::ACL::HTTPPut request;
+  request.mutable_usernames()->add_values("foo");
+  request.mutable_urls()->add_values("/admin");
+  AWAIT_EXPECT_EQ(true, authorizer.get()->authorize(request));
+
+  // "bar" can PUT access "/admin".
+  mesos::ACL::HTTPPut request2;
+  request2.mutable_usernames()->add_values("bar");
+  request2.mutable_urls()->add_values("/admin");
+  AWAIT_EXPECT_EQ(true, authorizer.get()->authorize(request2));
+
+  // Aggregate request for clients "foo" and "bar" for PUT access to
+  // "/admin" is not allowed because ACLs are not aggregated.
+  mesos::ACL::HTTPPut request3;
+  request3.mutable_usernames()->add_values("foo");
+  request3.mutable_usernames()->add_values("bar");
+  request3.mutable_urls()->add_values("/admin");
+  AWAIT_EXPECT_EQ(false, authorizer.get()->authorize(request3));
+}
+

http://git-wip-us.apache.org/repos/asf/mesos/blob/50bbbf5f/src/tests/master_contender_detector_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_contender_detector_tests.cpp b/src/tests/master_contender_detector_tests.cpp
index 42051bf..231648d 100644
--- a/src/tests/master_contender_detector_tests.cpp
+++ b/src/tests/master_contender_detector_tests.cpp
@@ -368,7 +368,7 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, NonRetryableFrrors)
 
   // Creator of the base path restricts the all accesses to be
   // itself.
-  ACL onlyCreatorCanAccess[] = {{ ZOO_PERM_ALL, ZOO_AUTH_IDS }};
+  ::ACL onlyCreatorCanAccess[] = {{ ZOO_PERM_ALL, ZOO_AUTH_IDS }};
   authenticatedZk.create("/test",
                          "42",
                          (ACL_vector) {1, onlyCreatorCanAccess},