You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ti...@apache.org on 2016/01/06 18:32:04 UTC

[1/5] mesos git commit: Quota: Added "RemoveQuota" message to ACL protobuf.

Repository: mesos
Updated Branches:
  refs/heads/master f00402885 -> 263dc3e94


Quota: Added "RemoveQuota" message to ACL protobuf.

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


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

Branch: refs/heads/master
Commit: 3b4aa918921b2f22781ccadade0be6ad48848633
Parents: f004028
Author: Jan Schlicht <ja...@mesosphere.io>
Authored: Wed Jan 6 18:19:57 2016 +0100
Committer: Till Toenshoff <to...@me.com>
Committed: Wed Jan 6 18:19:57 2016 +0100

----------------------------------------------------------------------
 include/mesos/authorizer/authorizer.proto | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3b4aa918/include/mesos/authorizer/authorizer.proto
----------------------------------------------------------------------
diff --git a/include/mesos/authorizer/authorizer.proto b/include/mesos/authorizer/authorizer.proto
index 7b729e1..84727e6 100644
--- a/include/mesos/authorizer/authorizer.proto
+++ b/include/mesos/authorizer/authorizer.proto
@@ -116,6 +116,15 @@ message ACL {
     // 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;
+  }
 }
 
 
@@ -148,4 +157,5 @@ message ACLs {
   repeated ACL.CreateVolume create_volumes = 7;
   repeated ACL.DestroyVolume destroy_volumes = 8;
   repeated ACL.SetQuota set_quotas = 9;
+  repeated ACL.RemoveQuota remove_quotas = 10;
 }


[5/5] mesos git commit: Quota: Added tests for quota removal authorization.

Posted by ti...@apache.org.
Quota: Added tests for quota removal authorization.

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


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

Branch: refs/heads/master
Commit: 263dc3e94a5cc8b02e314dd0e98df81937a75133
Parents: a114ab1
Author: Jan Schlicht <ja...@mesosphere.io>
Authored: Wed Jan 6 18:20:22 2016 +0100
Committer: Till Toenshoff <to...@me.com>
Committed: Wed Jan 6 18:20:22 2016 +0100

----------------------------------------------------------------------
 src/tests/authorization_tests.cpp |  72 ++++++++++
 src/tests/master_quota_tests.cpp  | 240 ++++++++++++++++++++-------------
 2 files changed, 215 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/263dc3e9/src/tests/authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/authorization_tests.cpp b/src/tests/authorization_tests.cpp
index 1d11a02..9d046e8 100644
--- a/src/tests/authorization_tests.cpp
+++ b/src/tests/authorization_tests.cpp
@@ -761,6 +761,78 @@ TYPED_TEST(AuthorizationTest, SetQuota)
   AWAIT_EXPECT_FALSE(authorizer.get()->authorize(request7));
 }
 
+
+// This tests the authorization of requests to remove quotas.
+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");
+
+  // "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);
+
+  // Create an `Authorizer` with the ACLs.
+  Try<Authorizer*> create = TypeParam::create();
+  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");
+  request1.mutable_quota_principals()->add_values("foo");
+  AWAIT_EXPECT_TRUE(authorizer.get()->authorize(request1));
+
+  // 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));
+
+  // 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));
+
+  // 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));
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/263dc3e9/src/tests/master_quota_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_quota_tests.cpp b/src/tests/master_quota_tests.cpp
index 81f0386..bc8a117 100644
--- a/src/tests/master_quota_tests.cpp
+++ b/src/tests/master_quota_tests.cpp
@@ -1203,145 +1203,191 @@ TEST_F(MasterQuotaTest, UnauthenticatedQuotaRequest)
 }
 
 
-// Checks that an authorized principal can set quota.
-TEST_F(MasterQuotaTest, AuthorizedQuotaSetRequest)
+// Checks that an authorized principal can set and remove quota while
+// unauthorized principals cannot.
+TEST_F(MasterQuotaTest, AuthorizeQuotaRequests)
 {
   TestAllocator<> allocator;
   EXPECT_CALL(allocator, initialize(_, _, _, _));
 
-  // Setup ACLs so that the default principal can set quotas for `ROLE1`.
+  // Setup ACLs so that only the default principal can set quotas for `ROLE1`
+  // and can remove its own quotas.
   ACLs acls;
-  mesos::ACL::SetQuota* acl = acls.add_set_quotas();
-  acl->mutable_principals()->add_values(DEFAULT_CREDENTIAL.principal());
-  acl->mutable_roles()->add_values(ROLE1);
 
-  master::Flags masterFlags = CreateMasterFlags();
-  masterFlags.acls = acls;
+  mesos::ACL::SetQuota* acl1 = acls.add_set_quotas();
+  acl1->mutable_principals()->add_values(DEFAULT_CREDENTIAL.principal());
+  acl1->mutable_roles()->add_values(ROLE1);
 
-  Try<PID<Master>> master = StartMaster(&allocator, masterFlags);
-  ASSERT_SOME(master);
+  mesos::ACL::SetQuota* acl2 = acls.add_set_quotas();
+  acl2->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+  acl2->mutable_roles()->set_type(mesos::ACL::Entity::NONE);
 
-  // Start an agent and wait until its resources are available.
-  Future<Resources> agentTotalResources;
-  EXPECT_CALL(allocator, addSlave(_, _, _, _, _))
-    .WillOnce(DoAll(InvokeAddSlave(&allocator),
-                    FutureArg<3>(&agentTotalResources)));
-
-  Try<PID<Slave>> agent = StartSlave();
-  ASSERT_SOME(agent);
-
-  AWAIT_READY(agentTotalResources);
-  EXPECT_EQ(defaultAgentResources, agentTotalResources.get());
+  mesos::ACL::RemoveQuota* acl3 = acls.add_remove_quotas();
+  acl3->mutable_principals()->add_values(DEFAULT_CREDENTIAL.principal());
+  acl3->mutable_quota_principals()->add_values(DEFAULT_CREDENTIAL.principal());
 
-  // Request quota for a portion of the resources available on the agent.
-  Resources quotaResources = Resources::parse("cpus:1;mem:512;", ROLE1).get();
-  EXPECT_TRUE(agentTotalResources.get().contains(quotaResources.flatten()));
+  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);
 
-  Future<QuotaInfo> quotaInfo;
-  EXPECT_CALL(allocator, setQuota(Eq(ROLE1), _))
-    .WillOnce(DoAll(InvokeSetQuota(&allocator),
-                    FutureArg<1>(&quotaInfo)));
+  master::Flags masterFlags = CreateMasterFlags();
+  masterFlags.acls = acls;
 
-  Future<Response> response = process::http::post(
-      master.get(),
-      "quota",
-      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
-      createRequestBody(quotaResources));
+  Try<PID<Master>> master = StartMaster(&allocator, masterFlags);
+  ASSERT_SOME(master);
 
-  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response) << response.get().body;
+  // Try to request quota using a principal that is not the default principal.
+  // This request will fail because only the default principal is authorized
+  // to do that.
+  {
+    // As we don't care about the enforcement of quota but only the
+    // authorization of the quota request we set the force flag in the post
+    // request below to override the capacity heuristic check.
+    Resources quotaResources = Resources::parse("cpus:1;mem:512;", ROLE1).get();
+
+    // Note that we set the force flag because we are setting a quota that
+    // cannot currently be satisfied by the resources in the cluster (because
+    // there are no agents).
+    Future<Response> response = process::http::post(
+        master.get(),
+        "quota",
+        createBasicAuthHeaders(DEFAULT_CREDENTIAL_2),
+        createRequestBody(quotaResources, true));
 
-  AWAIT_READY(quotaInfo);
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+        Unauthorized("Mesos master").status, response) << response.get().body;
+  }
 
-  // TODO(nfnt): Quota removal authorization will add a principal field to
-  // `QuotaInfo`. Check it for the correct principal value.
-  EXPECT_EQ(ROLE1, quotaInfo.get().role());
-  EXPECT_EQ(quotaResources.flatten(), quotaInfo.get().guarantee());
+  // Request quota using the default principal.
+  {
+    // As we don't care about the enforcement of quota but only the
+    // authorization of the quota request we set the force flag in the post
+    // request below to override the capacity heuristic check.
+    Resources quotaResources = Resources::parse("cpus:1;mem:512;", ROLE1).get();
 
-  Shutdown();
-}
+    Future<QuotaInfo> quotaInfo;
+    EXPECT_CALL(allocator, setQuota(Eq(ROLE1), _))
+      .WillOnce(DoAll(InvokeSetQuota(&allocator),
+                      FutureArg<1>(&quotaInfo)));
 
+    // Note that we set the force flag because we are setting a quota that
+    // cannot currently be satisfied by the resources in the cluster (because
+    // there are no agents).
+    Future<Response> response = process::http::post(
+        master.get(),
+        "quota",
+        createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+        createRequestBody(quotaResources, true));
 
-// Checks that set quota requests can be authorized without authentication
-// if an authorization rule exists that applies to anyone. The authorizer
-// will map the absence of a principal to "ANY".
-TEST_F(MasterQuotaTest, AuthorizedQuotaSetRequestWithoutPrincipal)
-{
-  TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+        OK().status, response) << response.get().body;
 
-  // Setup ACLs so that the default principal can set quotas for `ROLE1`.
-  ACLs acls;
-  mesos::ACL::SetQuota* acl = acls.add_set_quotas();
-  acl->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
-  acl->mutable_roles()->add_values(ROLE1);
+    AWAIT_READY(quotaInfo);
 
-  // Disable authentication by not providing credentials.
-  master::Flags masterFlags = CreateMasterFlags();
-  masterFlags.acls = acls;
-  masterFlags.credentials = None();
+    // Extract the principal from `DEFAULT_CREDENTIAL` because `EXPECT_EQ`
+    // does not compile if `DEFAULT_CREDENTIAL.principal()` is used as an
+    // argument.
+    const string principal = DEFAULT_CREDENTIAL.principal();
 
-  Try<PID<Master>> master = StartMaster(&allocator, masterFlags);
-  ASSERT_SOME(master);
+    EXPECT_EQ(ROLE1, quotaInfo.get().role());
+    EXPECT_EQ(principal, quotaInfo.get().principal());
+    EXPECT_EQ(quotaResources.flatten(), quotaInfo.get().guarantee());
+  }
 
-  // Start an agent and wait until its resources are available.
-  Future<Resources> agentTotalResources;
-  EXPECT_CALL(allocator, addSlave(_, _, _, _, _))
-    .WillOnce(DoAll(InvokeAddSlave(&allocator),
-                    FutureArg<3>(&agentTotalResources)));
+  // Try to remove the previously requested quota using a principal that is
+  // not the default principal. This will fail because only the default
+  // principal is authorized to do that.
+  {
+    Future<Response> response = process::http::requestDelete(
+        master.get(),
+        "quota/" + ROLE1,
+        createBasicAuthHeaders(DEFAULT_CREDENTIAL_2));
 
-  Try<PID<Slave>> agent = StartSlave();
-  ASSERT_SOME(agent);
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+        Unauthorized("Mesos master").status, response) << response.get().body;
+  }
 
-  AWAIT_READY(agentTotalResources);
-  EXPECT_EQ(defaultAgentResources, agentTotalResources.get());
+  // Remove the previously requested quota using the default principal.
+  {
+    Future<Nothing> receivedRemoveRequest;
+    EXPECT_CALL(allocator, removeQuota(Eq(ROLE1)))
+      .WillOnce(DoAll(InvokeRemoveQuota(&allocator),
+                      FutureSatisfy(&receivedRemoveRequest)));
 
-  // Request quota for a portion of the resources available on the agent.
-  Resources quotaResources = Resources::parse("cpus:1;mem:512;", ROLE1).get();
-  EXPECT_TRUE(agentTotalResources.get().contains(quotaResources.flatten()));
+    Future<Response> response = process::http::requestDelete(
+        master.get(),
+        "quota/" + ROLE1,
+        createBasicAuthHeaders(DEFAULT_CREDENTIAL));
 
-  // Create a HTTP request without authorization headers.
-  Future<Response> response = process::http::post(
-      master.get(),
-      "quota",
-      None(),
-      createRequestBody(quotaResources));
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+        OK().status, response) << response.get().body;
 
-  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response) << response.get().body;
+    AWAIT_READY(receivedRemoveRequest);
+  }
 
   Shutdown();
 }
 
 
-// Checks that an unauthorized principal cannot set quota.
-TEST_F(MasterQuotaTest, UnauthorizedQuotaSetRequest)
+// Checks that set and remove quota requests can be authorized without
+// authentication if an authorization rule exists that applies to anyone.
+// The authorizer will map the absence of a principal to "ANY".
+TEST_F(MasterQuotaTest, AuthorizeQuotaRequestsWithoutPrincipal)
 {
-  // Setup ACLs so that no principal can set quotas for `ROLE1`.
+  TestAllocator<> allocator;
+  EXPECT_CALL(allocator, initialize(_, _, _, _));
+
+  // Setup ACLs so that any principal can set quotas for `ROLE1` and remove
+  // anyone's quotas.
   ACLs acls;
-  mesos::ACL::SetQuota* acl = acls.add_set_quotas();
-  acl->mutable_principals()->set_type(mesos::ACL::Entity::NONE);
-  acl->mutable_roles()->add_values(ROLE1);
 
+  mesos::ACL::SetQuota* acl1 = acls.add_set_quotas();
+  acl1->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+  acl1->mutable_roles()->add_values(ROLE1);
+
+  mesos::ACL::RemoveQuota* acl2 = acls.add_remove_quotas();
+  acl2->mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+  acl2->mutable_quota_principals()->set_type(mesos::ACL::Entity::ANY);
+
+  // Disable authentication by not providing credentials.
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
+  masterFlags.credentials = None();
 
-  Try<PID<Master>> master = StartMaster(masterFlags);
+  Try<PID<Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
 
-  // We do not need an agent since a request should be rejected before
-  // we start looking at available resources.
+  // Request quota without providing authorization headers.
+  {
+    // As we don't care about the enforcement of quota but only the
+    // authorization of the quota request we set the force flag in the post
+    // request below to override the capacity heuristic check.
+    Resources quotaResources = Resources::parse("cpus:1;mem:512;", ROLE1).get();
+
+    // Create a HTTP request without authorization headers. Note that we set the
+    // force flag because we are setting a quota that cannot currently be
+    // satisfied by the resources in the cluster (because there are no agents).
+    Future<Response> response = process::http::post(
+        master.get(),
+        "quota",
+        None(),
+        createRequestBody(quotaResources, true));
 
-  // A request can contain any amount of resources because it will be rejected
-  // before we start looking at available resources.
-  Resources quotaResources = Resources::parse("cpus:1;mem:512", ROLE1).get();
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+        OK().status, response) << response.get().body;
+  }
 
-  Future<Response> response = process::http::post(
-      master.get(),
-      "quota",
-      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
-      createRequestBody(quotaResources));
+  // Remove the previously requested quota without providing authorization
+  // headers.
+  {
+    Future<Response> response = process::http::requestDelete(
+        master.get(),
+        "quota/" + ROLE1,
+        None());
 
-  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
-      Unauthorized("Mesos master").status, response) << response.get().body;
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+        OK().status, response) << response.get().body;
+  }
 
   Shutdown();
 }


[3/5] mesos git commit: Quota: Implemented quota remove authorization.

Posted by ti...@apache.org.
Quota: Implemented quota remove authorization.

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


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

Branch: refs/heads/master
Commit: 4fe7880a44646f6659f6c0e9a013735cb6071e5d
Parents: 6998047
Author: Jan Schlicht <ja...@mesosphere.io>
Authored: Wed Jan 6 18:20:11 2016 +0100
Committer: Till Toenshoff <to...@me.com>
Committed: Wed Jan 6 18:20:11 2016 +0100

----------------------------------------------------------------------
 include/mesos/quota/quota.proto |  3 ++
 src/master/master.hpp           |  9 ++++-
 src/master/quota_handler.cpp    | 74 +++++++++++++++++++++++++++++++-----
 3 files changed, 75 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/4fe7880a/include/mesos/quota/quota.proto
----------------------------------------------------------------------
diff --git a/include/mesos/quota/quota.proto b/include/mesos/quota/quota.proto
index ab505b1..338412e 100644
--- a/include/mesos/quota/quota.proto
+++ b/include/mesos/quota/quota.proto
@@ -38,6 +38,9 @@ message QuotaInfo {
   // dynamic reservations.
   optional string role = 1;
 
+  // Principal which set the quota. Currently only operators can set quotas.
+  optional string principal = 2;
+
   // Resources which are guaranteed to be allocatable by role.
   // NOTE: `guarantee.role` should not specify any role except '*',
   // because quota does not reserve specific resources.

http://git-wip-us.apache.org/repos/asf/mesos/blob/4fe7880a/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 2936d32..f764915 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -997,14 +997,21 @@ private:
     // (including rescinding) is moved to allocator.
     void rescindOffers(const mesos::quota::QuotaInfo& request) const;
 
-    process::Future<bool> authorize(
+    process::Future<bool> authorizeSetQuota(
         const Option<std::string>& principal,
         const std::string& role) const;
 
+    process::Future<bool> authorizeRemoveQuota(
+        const Option<std::string>& requestPrincipal,
+        const Option<std::string>& quotaPrincipal) const;
+
     process::Future<process::http::Response> _set(
         const mesos::quota::QuotaInfo& quota,
         bool forced) const;
 
+    process::Future<process::http::Response> _remove(
+        const std::string& role) const;
+
     // To perform actions related to quota management, we require access to the
     // master data structures. No synchronization primitives are needed here
     // since `QuotaHandler`'s functions are invoked in the Master's actor.

http://git-wip-us.apache.org/repos/asf/mesos/blob/4fe7880a/src/master/quota_handler.cpp
----------------------------------------------------------------------
diff --git a/src/master/quota_handler.cpp b/src/master/quota_handler.cpp
index ac4da74..93960f3 100644
--- a/src/master/quota_handler.cpp
+++ b/src/master/quota_handler.cpp
@@ -299,7 +299,7 @@ Future<http::Response> Master::QuotaHandler::set(
         request.body + "': " + create.error());
   }
 
-  const QuotaInfo& quotaInfo = create.get();
+  QuotaInfo quotaInfo = create.get();
 
   // Check that the `QuotaInfo` is a valid quota request.
   Try<Nothing> validate = quota::validation::quotaInfo(quotaInfo);
@@ -335,13 +335,16 @@ Future<http::Response> Master::QuotaHandler::set(
         request.body + "': " + force.error());
   }
 
-  const bool forced = force.isSome() ? force.get().value : false;
-
   // Extract principal from request credentials.
-  Option<string> principal =
-    credential.isSome() ? credential.get().principal() : Option<string>::none();
+  Option<string> principal = None();
+  if (credential.isSome()) {
+    principal = credential.get().principal();
+    quotaInfo.set_principal(principal.get());
+  }
 
-  return authorize(principal, quotaInfo.role())
+  const bool forced = force.isSome() ? force.get().value : false;
+
+  return authorizeSetQuota(principal, quotaInfo.role())
     .then(defer(master->self(), [=](bool authorized) -> Future<http::Response> {
       if (!authorized) {
         return Unauthorized("Mesos master");
@@ -406,14 +409,12 @@ Future<http::Response> Master::QuotaHandler::remove(
 {
   VLOG(1) << "Removing quota for request path: '" << request.url.path << "'";
 
-    // Authenticate the request.
+  // Authenticate the request.
   Result<Credential> credential = master->http.authenticate(request);
   if (credential.isError()) {
     return Unauthorized("Mesos master", credential.error());
   }
 
-  // TODO(nfnt): Authorize the request.
-
   // Check that the request type is DELETE which is guaranteed by the master.
   CHECK_EQ("DELETE", request.method);
 
@@ -451,6 +452,27 @@ Future<http::Response> Master::QuotaHandler::remove(
         "': Role '" + role + "' has no quota set");
   }
 
+  // Extract principal from request credentials.
+  Option<string> principal =
+    credential.isSome() ? credential.get().principal() : Option<string>::none();
+
+  Option<string> quota_principal = master->quotas[role].info.has_principal()
+    ? master->quotas[role].info.principal()
+    : Option<string>::none();
+
+  return authorizeRemoveQuota(principal, quota_principal)
+    .then(defer(master->self(), [=](bool authorized) -> Future<http::Response> {
+      if (!authorized) {
+        return Unauthorized("Mesos master");
+      }
+
+      return _remove(role);
+    }));
+}
+
+
+Future<http::Response> Master::QuotaHandler::_remove(const string& role) const
+{
   // Remove quota from the quota-related local state. We do this before
   // updating the registry in order to make sure that we are not already
   // trying to remove quota for this role (since this is a multi-phase event).
@@ -493,7 +515,7 @@ Future<http::Response> Master::QuotaHandler::status(
 }
 
 
-Future<bool> Master::QuotaHandler::authorize(
+Future<bool> Master::QuotaHandler::authorizeSetQuota(
     const Option<string>& principal,
     const string& role) const
 {
@@ -518,6 +540,38 @@ Future<bool> Master::QuotaHandler::authorize(
   return master->authorizer.get()->authorize(request);
 }
 
+
+Future<bool> Master::QuotaHandler::authorizeRemoveQuota(
+    const Option<string>& requestPrincipal,
+    const Option<string>& quotaPrincipal) const
+{
+  if (master->authorizer.isNone()) {
+    return true;
+  }
+
+  LOG(INFO) << "Authorizing principal '"
+            << (requestPrincipal.isSome() ? requestPrincipal.get() : "ANY")
+            << "' to remove quota set by '"
+            << (quotaPrincipal.isSome() ? quotaPrincipal.get() : "ANY")
+            << "'";
+
+  mesos::ACL::RemoveQuota request;
+
+  if (requestPrincipal.isSome()) {
+    request.mutable_principals()->add_values(requestPrincipal.get());
+  } else {
+    request.mutable_principals()->set_type(mesos::ACL::Entity::ANY);
+  }
+
+  if (quotaPrincipal.isSome()) {
+    request.mutable_quota_principals()->add_values(quotaPrincipal.get());
+  } else {
+    request.mutable_quota_principals()->set_type(mesos::ACL::Entity::ANY);
+  }
+
+  return master->authorizer.get()->authorize(request);
+}
+
 } // namespace master {
 } // namespace internal {
 } // namespace mesos {


[4/5] mesos git commit: Quota: Documented quota removal authorization.

Posted by ti...@apache.org.
Quota: Documented quota removal authorization.

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


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

Branch: refs/heads/master
Commit: a114ab192da1d5073c7e5184db9a998975019355
Parents: 4fe7880
Author: Jan Schlicht <ja...@mesosphere.io>
Authored: Wed Jan 6 18:20:16 2016 +0100
Committer: Till Toenshoff <to...@me.com>
Committed: Wed Jan 6 18:20:16 2016 +0100

----------------------------------------------------------------------
 docs/authorization.md | 14 ++++++++------
 src/master/flags.cpp  |  6 ++++++
 2 files changed, 14 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a114ab19/docs/authorization.md
----------------------------------------------------------------------
diff --git a/docs/authorization.md b/docs/authorization.md
index 9009228..6375c9a 100644
--- a/docs/authorization.md
+++ b/docs/authorization.md
@@ -9,7 +9,7 @@ Authorization currently allows
  1. Frameworks to (re-)register with authorized _roles_.
  2. Frameworks to launch tasks/executors as authorized _users_.
  3. Authorized _principals_ to shutdown frameworks through the "/teardown" HTTP endpoint.
- 4. Authorized _principals_ to set quotas through the "/quota" HTTP endpoint.
+ 4. Authorized _principals_ to set and remove quotas through the "/quota" HTTP endpoint.
  5. Authorized _principals_ to reserve and unreserve resources through the "/reserve" and "/unreserve" HTTP endpoints, as well as with the `RESERVE` and `UNRESERVE` offer operations.
  6. Authorized _principals_ to create and destroy persistent volumes through the `CREATE` and `DESTROY` offer operations.
 
@@ -26,16 +26,17 @@ The currently supported `Actions` are:
 2. "run_tasks": Run tasks/executors
 3. "shutdown_frameworks": Shutdown frameworks
 4. "set_quotas": Set quotas
-5. "reserve_resources": Reserve resources
-6. "unreserve_resources": Unreserve resources
-7. "create_volumes": Create persistent volumes
-8. "destroy_volumes": Destroy persistent volumes
+5. "remove_quotas": Remove quotas
+6. "reserve_resources": Reserve resources
+7. "unreserve_resources": Unreserve resources
+8. "create_volumes": Create persistent volumes
+9. "destroy_volumes": Destroy persistent volumes
 
 The currently supported `Subjects` are:
 
 1. "principals"
 	- Framework principals (used by "register_frameworks", "run_tasks", "reserve", "unreserve", "create_volumes", and "destroy_volumes" actions)
-	- Usernames (used by "shutdown_frameworks", "set_quotas", "reserve", "unreserve", "create_volumes", and "destroy_volumes" actions)
+	- Usernames (used by "shutdown_frameworks", "set_quotas", "remove_quotas", "reserve", "unreserve", "create_volumes", and "destroy_volumes" actions)
 
 The currently supported `Objects` are:
 
@@ -46,6 +47,7 @@ The currently supported `Objects` are:
 5. "reserver_principals": Framework principals whose reserved resources can be unreserved (used by "unreserves" action).
 6. "volume_types": Types of volumes that can be created by a given principal. Currently the only types considered by the default authorizer are `ANY` and `NONE` (used by "create_volumes" action).
 7. "creator_principals": Principals whose persistent volumes can be destroyed (used by "destroy_volumes" action).
+8. "quota_principals": Principals that set the quota to be removed (used by "remove_quotas" action)
 
 > NOTE: Both `Subjects` and `Objects` can be either an array of strings or one of the special values `ANY` or `NONE`.
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a114ab19/src/master/flags.cpp
----------------------------------------------------------------------
diff --git a/src/master/flags.cpp b/src/master/flags.cpp
index 800e54a..4aaa3e0 100644
--- a/src/master/flags.cpp
+++ b/src/master/flags.cpp
@@ -264,6 +264,12 @@ mesos::internal::master::Flags::Flags()
       "      \"principals\": { \"values\": [\"a\"] },\n"
       "      \"roles\": { \"values\": [\"a\", \"b\"] }\n"
       "    }\n"
+      "  ],\n"
+      "  \"remove_quotas\": [\n"
+      "    {\n"
+      "      \"principals\": { \"values\": [\"a\"] },\n"
+      "      \"quota_principals\": { \"values\": [\"a\"] }\n"
+      "    }\n"
       "  ]\n"
       "}");
 


[2/5] mesos git commit: Quota: Implemented authorization of "remove quota" requests.

Posted by ti...@apache.org.
Quota: Implemented authorization of "remove quota" requests.

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


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

Branch: refs/heads/master
Commit: 69980474c68b18a5d1d49f663260824ae597c89a
Parents: 3b4aa91
Author: Jan Schlicht <ja...@mesosphere.io>
Authored: Wed Jan 6 18:20:05 2016 +0100
Committer: Till Toenshoff <to...@me.com>
Committed: Wed Jan 6 18:20:05 2016 +0100

----------------------------------------------------------------------
 include/mesos/authorizer/authorizer.hpp | 14 ++++++++++++++
 src/authorizer/local/authorizer.cpp     | 29 ++++++++++++++++++++++++++++
 src/authorizer/local/authorizer.hpp     |  2 ++
 src/tests/mesos.cpp                     |  3 +++
 src/tests/mesos.hpp                     |  2 ++
 5 files changed, 50 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/69980474/include/mesos/authorizer/authorizer.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/authorizer/authorizer.hpp b/include/mesos/authorizer/authorizer.hpp
index f616139..5ee3c7a 100644
--- a/include/mesos/authorizer/authorizer.hpp
+++ b/include/mesos/authorizer/authorizer.hpp
@@ -187,6 +187,20 @@ public:
   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;
+
 protected:
   Authorizer() {}
 };

http://git-wip-us.apache.org/repos/asf/mesos/blob/69980474/src/authorizer/local/authorizer.cpp
----------------------------------------------------------------------
diff --git a/src/authorizer/local/authorizer.cpp b/src/authorizer/local/authorizer.cpp
index 1d135fb..c1db9c2 100644
--- a/src/authorizer/local/authorizer.cpp
+++ b/src/authorizer/local/authorizer.cpp
@@ -164,6 +164,21 @@ public:
     return acls.permissive(); // None of the ACLs match.
   }
 
+  Future<bool> authorize(const ACL::RemoveQuota& request)
+  {
+    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());
+      }
+    }
+
+    return acls.permissive(); // None of the ACLs match.
+  }
+
 private:
   // Match matrix:
   //
@@ -438,5 +453,19 @@ Future<bool> LocalAuthorizer::authorize(const ACL::SetQuota& request)
       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);
+}
+
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/69980474/src/authorizer/local/authorizer.hpp
----------------------------------------------------------------------
diff --git a/src/authorizer/local/authorizer.hpp b/src/authorizer/local/authorizer.hpp
index 1563c11..586f0da 100644
--- a/src/authorizer/local/authorizer.hpp
+++ b/src/authorizer/local/authorizer.hpp
@@ -67,6 +67,8 @@ public:
       const ACL::DestroyVolume& request);
   virtual process::Future<bool> authorize(
       const ACL::SetQuota& request);
+  virtual process::Future<bool> authorize(
+      const ACL::RemoveQuota& request);
 
 private:
   LocalAuthorizer();

http://git-wip-us.apache.org/repos/asf/mesos/blob/69980474/src/tests/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index 082e57b..3867ed2 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -710,6 +710,9 @@ MockAuthorizer::MockAuthorizer()
   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, initialize(An<const Option<ACLs>&>()))
     .WillRepeatedly(Return(Nothing()));
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/69980474/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 49a4c48..3d9ebc6 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -1220,6 +1220,8 @@ public:
       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));
 };