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:08 UTC

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

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();
 }