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

[2/2] mesos git commit: Added persistent volume HTTP endpoint authorization.

Added persistent volume HTTP endpoint authorization.

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


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

Branch: refs/heads/master
Commit: d9b9bcfc13dbec04c1e869827e583ea41eca5eb7
Parents: b24318a
Author: Greg Mann <gr...@mesosphere.io>
Authored: Tue Jan 5 17:14:54 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Jan 5 17:14:54 2016 -0800

----------------------------------------------------------------------
 src/master/http.cpp                             |  30 +-
 src/tests/persistent_volume_endpoints_tests.cpp | 375 +++++++++++++++++++
 2 files changed, 398 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d9b9bcfc/src/master/http.cpp
----------------------------------------------------------------------
diff --git a/src/master/http.cpp b/src/master/http.cpp
index deb0c8f..d7afa2a 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -595,12 +595,20 @@ Future<Response> Master::Http::createVolumes(const Request& request) const
     return BadRequest("Invalid CREATE operation: " + validate.get().message);
   }
 
-  // TODO(neilc): Add a create-volumes ACL for authorization.
+  Option<string> principal =
+    credential.isSome() ? credential.get().principal() : Option<string>::none();
+
+  return master->authorizeCreateVolume(operation.create(), principal)
+    .then(defer(master->self(), [=](bool authorized) -> Future<Response> {
+      if (!authorized) {
+        return Unauthorized("Mesos master");
+      }
 
-  // The resources required for this operation are equivalent to the
-  // volumes specified by the user minus any DiskInfo (DiskInfo will
-  // be created when this operation is applied).
-  return _operation(slaveId, removeDiskInfos(volumes), operation);
+      // The resources required for this operation are equivalent to the
+      // volumes specified by the user minus any DiskInfo (DiskInfo will
+      // be created when this operation is applied).
+      return _operation(slaveId, removeDiskInfos(volumes), operation);
+    }));
 }
 
 
@@ -684,9 +692,17 @@ Future<Response> Master::Http::destroyVolumes(const Request& request) const
     return BadRequest("Invalid DESTROY operation: " + validate.get().message);
   }
 
-  // TODO(neilc): Add a destroy-volumes ACL for authorization.
+  Option<string> principal =
+    credential.isSome() ? credential.get().principal() : Option<string>::none();
+
+  return master->authorizeDestroyVolume(operation.destroy(), principal)
+    .then(defer(master->self(), [=](bool authorized) -> Future<Response> {
+      if (!authorized) {
+        return Unauthorized("Mesos master");
+      }
 
-  return _operation(slaveId, volumes, operation);
+      return _operation(slaveId, volumes, operation);
+    }));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/d9b9bcfc/src/tests/persistent_volume_endpoints_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/persistent_volume_endpoints_tests.cpp b/src/tests/persistent_volume_endpoints_tests.cpp
index c0feede..f0cce19 100644
--- a/src/tests/persistent_volume_endpoints_tests.cpp
+++ b/src/tests/persistent_volume_endpoints_tests.cpp
@@ -22,6 +22,7 @@
 #include <mesos/executor.hpp>
 #include <mesos/scheduler.hpp>
 
+#include <process/clock.hpp>
 #include <process/future.hpp>
 #include <process/gmock.hpp>
 #include <process/http.hpp>
@@ -31,6 +32,7 @@
 #include <stout/hashmap.hpp>
 #include <stout/option.hpp>
 
+#include "master/constants.hpp"
 #include "master/flags.hpp"
 #include "master/master.hpp"
 
@@ -43,9 +45,11 @@ using std::vector;
 
 using google::protobuf::RepeatedPtrField;
 
+using mesos::internal::master::DEFAULT_ALLOCATION_INTERVAL;
 using mesos::internal::master::Master;
 using mesos::internal::slave::Slave;
 
+using process::Clock;
 using process::Future;
 using process::PID;
 
@@ -704,6 +708,377 @@ TEST_F(PersistentVolumeEndpointsTest, BadCredentials)
 }
 
 
+// This tests that correct setup of CreateVolume/DestroyVolume ACLs allows an
+// operator to perform volume creation/destruction operations successfully.
+TEST_F(PersistentVolumeEndpointsTest, GoodCreateAndDestroyACL)
+{
+  // Pause the clock to gain control over the offer cycle.
+  Clock::pause();
+
+  TestAllocator<> allocator;
+  ACLs acls;
+
+  // This ACL asserts that the principal of `DEFAULT_CREDENTIAL`
+  // can create ANY volumes.
+  mesos::ACL::CreateVolume* create = acls.add_create_volumes();
+  create->mutable_principals()->add_values(DEFAULT_CREDENTIAL.principal());
+  create->mutable_volume_types()->set_type(mesos::ACL::Entity::ANY);
+
+  // This ACL asserts that the principal of `DEFAULT_CREDENTIAL`
+  // can destroy volumes that it created.
+  mesos::ACL::DestroyVolume* destroy = acls.add_destroy_volumes();
+  destroy->mutable_principals()->add_values(DEFAULT_CREDENTIAL.principal());
+  destroy->mutable_creator_principals()->add_values(
+      DEFAULT_CREDENTIAL.principal());
+
+  // Create a master.
+  master::Flags masterFlags = CreateMasterFlags();
+  masterFlags.acls = acls;
+
+  EXPECT_CALL(allocator, initialize(_, _, _, _));
+
+  Try<PID<Master>> master = StartMaster(&allocator, masterFlags);
+  ASSERT_SOME(master);
+
+  // Create a slave. Disk resources are statically reserved to allow the
+  // creation of a persistent volume.
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.resources = "cpus:1;mem:512;disk(role1):1024";
+
+  Future<SlaveID> slaveId;
+  EXPECT_CALL(allocator, addSlave(_, _, _, _, _))
+    .WillOnce(DoAll(InvokeAddSlave(&allocator), FutureArg<0>(&slaveId)));
+
+  Try<PID<Slave>> slave = StartSlave(slaveFlags);
+  ASSERT_SOME(slave);
+
+  process::http::Headers headers = createBasicAuthHeaders(DEFAULT_CREDENTIAL);
+
+  Resources volume = createPersistentVolume(
+      Megabytes(64),
+      "role1",
+      "id1",
+      "path1");
+
+  Future<Response> createResponse = process::http::post(
+      master.get(),
+      "create-volumes",
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+      createRequestBody(slaveId.get(), "volumes", volume));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, createResponse);
+
+  FrameworkInfo frameworkInfo = createFrameworkInfo();
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, frameworkInfo, master.get(), DEFAULT_CREDENTIAL);
+
+  Future<vector<Offer>> offers;
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  driver.start();
+
+  Clock::settle();
+  Clock::advance(DEFAULT_ALLOCATION_INTERVAL);
+
+  AWAIT_READY(offers);
+
+  ASSERT_EQ(1u, offers.get().size());
+  Offer offer = offers.get()[0];
+
+  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+
+  Future<OfferID> rescindedOfferId;
+
+  EXPECT_CALL(sched, offerRescinded(&driver, _))
+    .WillOnce(FutureArg<1>(&rescindedOfferId));
+
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  Future<Response> destroyResponse = process::http::post(
+      master.get(),
+      "destroy-volumes",
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+      createRequestBody(slaveId.get(), "volumes", volume));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, destroyResponse);
+
+  Clock::settle();
+  Clock::advance(DEFAULT_ALLOCATION_INTERVAL);
+
+  AWAIT_READY(rescindedOfferId);
+
+  EXPECT_EQ(rescindedOfferId.get(), offer.id());
+
+  AWAIT_READY(offers);
+
+  ASSERT_EQ(1u, offers.get().size());
+  offer = offers.get()[0];
+
+  EXPECT_FALSE(Resources(offer.resources()).contains(volume));
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
+
+
+// This tests that an ACL prohibiting the creation of a persistent volume by a
+// principal will lead to a properly failed request.
+TEST_F(PersistentVolumeEndpointsTest, BadCreateAndDestroyACL)
+{
+  // Pause the clock to gain control over the offer cycle.
+  Clock::pause();
+
+  TestAllocator<> allocator;
+  ACLs acls;
+
+  // This ACL asserts that the principal of `DEFAULT_CREDENTIAL`
+  // cannot create persistent volumes.
+  mesos::ACL::CreateVolume* cannotCreate = acls.add_create_volumes();
+  cannotCreate->mutable_principals()->add_values(
+      DEFAULT_CREDENTIAL.principal());
+  cannotCreate->mutable_volume_types()->set_type(mesos::ACL::Entity::NONE);
+
+  // This ACL asserts that the principal of `DEFAULT_CREDENTIAL_2`
+  // can create persistent volumes.
+  mesos::ACL::CreateVolume* canCreate = acls.add_create_volumes();
+  canCreate->mutable_principals()->add_values(DEFAULT_CREDENTIAL_2.principal());
+  canCreate->mutable_volume_types()->set_type(mesos::ACL::Entity::ANY);
+
+  // This ACL asserts that the principal of `DEFAULT_CREDENTIAL`
+  // cannot destroy persistent volumes.
+  mesos::ACL::DestroyVolume* cannotDestroy = acls.add_destroy_volumes();
+  cannotDestroy->mutable_principals()->add_values(
+      DEFAULT_CREDENTIAL.principal());
+  cannotDestroy->mutable_creator_principals()->set_type(
+      mesos::ACL::Entity::NONE);
+
+  // Create a master.
+  master::Flags masterFlags = CreateMasterFlags();
+  masterFlags.acls = acls;
+
+  EXPECT_CALL(allocator, initialize(_, _, _, _));
+
+  Try<PID<Master>> master = StartMaster(&allocator, masterFlags);
+  ASSERT_SOME(master);
+
+  // Create a slave. Disk resources are statically reserved to allow the
+  // creation of a persistent volume.
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.resources = "cpus:1;mem:512;disk(role1):1024";
+
+  Future<SlaveID> slaveId;
+  EXPECT_CALL(allocator, addSlave(_, _, _, _, _))
+    .WillOnce(DoAll(InvokeAddSlave(&allocator),
+                    FutureArg<0>(&slaveId)));
+
+  Try<PID<Slave>> slave = StartSlave(slaveFlags);
+  ASSERT_SOME(slave);
+
+  Resources volume = createPersistentVolume(
+      Megabytes(64),
+      "role1",
+      "id1",
+      "path1");
+
+  // The failed creation attempt.
+  Future<Response> createResponse = process::http::post(
+      master.get(),
+      "create-volumes",
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+      createRequestBody(slaveId.get(), "volumes", volume));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      Unauthorized("Mesos master").status,
+      createResponse);
+
+  // The successful creation attempt.
+  createResponse = process::http::post(
+      master.get(),
+      "create-volumes",
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL_2),
+      createRequestBody(slaveId.get(), "volumes", volume));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, createResponse);
+
+  FrameworkInfo frameworkInfo = createFrameworkInfo();
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, frameworkInfo, master.get(), DEFAULT_CREDENTIAL);
+
+  Future<vector<Offer>> offers;
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  driver.start();
+
+  Clock::settle();
+  Clock::advance(DEFAULT_ALLOCATION_INTERVAL);
+
+  AWAIT_READY(offers);
+
+  ASSERT_EQ(1u, offers.get().size());
+  Offer offer = offers.get()[0];
+
+  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+
+  // The failed destruction attempt.
+  Future<Response> destroyResponse = process::http::post(
+      master.get(),
+      "destroy-volumes",
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+      createRequestBody(slaveId.get(), "volumes", volume));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      Unauthorized("Mesos master").status,
+      destroyResponse);
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
+
+
+// This tests that a request containing a credential which is not listed in the
+// master for authentication will not succeed, even if a good authorization ACL
+// is provided.
+TEST_F(PersistentVolumeEndpointsTest, GoodCreateAndDestroyACLBadCredential)
+{
+  // Pause the clock to gain control over the offer cycle.
+  Clock::pause();
+
+  // Create a credential which will not be listed
+  // for valid authentication with the master.
+  Credential failedCredential;
+  failedCredential.set_principal("awesome-principal");
+  failedCredential.set_secret("super-secret-secret");
+
+  TestAllocator<> allocator;
+  ACLs acls;
+
+  // This ACL asserts that the principal of `failedCredential`
+  // can create persistent volumes.
+  mesos::ACL::CreateVolume* failedCreate = acls.add_create_volumes();
+  failedCreate->mutable_principals()->add_values(failedCredential.principal());
+  failedCreate->mutable_volume_types()->set_type(mesos::ACL::Entity::ANY);
+
+  // This ACL asserts that the principal of `failedCredential`
+  // can destroy persistent volumes.
+  mesos::ACL::DestroyVolume* failedDestroy = acls.add_destroy_volumes();
+  failedDestroy->mutable_principals()->add_values(failedCredential.principal());
+  failedDestroy->mutable_creator_principals()->set_type(
+      mesos::ACL::Entity::ANY);
+
+  // This ACL asserts that the principal of `DEFAULT_CREDENTIAL`
+  // can create persistent volumes.
+  mesos::ACL::CreateVolume* canCreate = acls.add_create_volumes();
+  canCreate->mutable_principals()->add_values(DEFAULT_CREDENTIAL.principal());
+  canCreate->mutable_volume_types()->set_type(mesos::ACL::Entity::ANY);
+
+  // Create a master.
+  master::Flags masterFlags = CreateMasterFlags();
+  masterFlags.acls = acls;
+
+  EXPECT_CALL(allocator, initialize(_, _, _, _));
+
+  Try<PID<Master>> master = StartMaster(&allocator, masterFlags);
+  ASSERT_SOME(master);
+
+  // Create a slave. Disk resources are statically reserved to allow the
+  // creation of a persistent volume.
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.resources = "cpus:1;mem:512;disk(role1):1024";
+
+  Future<SlaveID> slaveId;
+  EXPECT_CALL(allocator, addSlave(_, _, _, _, _))
+    .WillOnce(DoAll(InvokeAddSlave(&allocator),
+                    FutureArg<0>(&slaveId)));
+
+  Try<PID<Slave>> slave = StartSlave(slaveFlags);
+  ASSERT_SOME(slave);
+
+  Resources volume = createPersistentVolume(
+      Megabytes(64),
+      "role1",
+      "id1",
+      "path1");
+
+  // The failed creation attempt.
+  Future<Response> createResponse = process::http::post(
+      master.get(),
+      "create-volumes",
+      createBasicAuthHeaders(failedCredential),
+      createRequestBody(slaveId.get(), "volumes", volume));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      Unauthorized("Mesos master").status,
+      createResponse);
+
+  // The successful creation attempt.
+  createResponse = process::http::post(
+      master.get(),
+      "create-volumes",
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+      createRequestBody(slaveId.get(), "volumes", volume));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, createResponse);
+
+  FrameworkInfo frameworkInfo = createFrameworkInfo();
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, frameworkInfo, master.get(), DEFAULT_CREDENTIAL);
+
+  Future<vector<Offer>> offers;
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  driver.start();
+
+  Clock::settle();
+  Clock::advance(DEFAULT_ALLOCATION_INTERVAL);
+
+  AWAIT_READY(offers);
+
+  ASSERT_EQ(1u, offers.get().size());
+  Offer offer = offers.get()[0];
+
+  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+
+  // The failed destruction attempt.
+  Future<Response> destroyResponse = process::http::post(
+      master.get(),
+      "destroy-volumes",
+      createBasicAuthHeaders(failedCredential),
+      createRequestBody(slaveId.get(), "volumes", volume));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(
+      Unauthorized("Mesos master").status,
+      destroyResponse);
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
+
+
 // This tests that an attempt to create or destroy a volume with no
 // 'slaveId' results in a 'BadRequest' HTTP error.
 TEST_F(PersistentVolumeEndpointsTest, NoSlaveId)