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

[1/3] mesos git commit: Allowed dynamic reservation without a principal.

Repository: mesos
Updated Branches:
  refs/heads/master ff3cc98df -> 866c52c51


Allowed dynamic reservation without a principal.

The `ReservationInfo.principal` field has been migrated to `optional`,
which means we can now allow dynamic reservation and unreservation
without a principal. This allows the use of the `/reserve` and
`/unreserve` HTTP endpoints when HTTP authentication is disabled.

Note that we still require that frameworks/operators set the
`ReservationInfo.principal` field to match their own principal,
if present. It may be desirable to remove this requirement;
this improvement is tracked in MESOS-4696.

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


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

Branch: refs/heads/master
Commit: efbdef8dfd96ff08c1342b171ef89dcb266bdce7
Parents: ff3cc98
Author: Greg Mann <gr...@mesosphere.io>
Authored: Mon Feb 22 16:43:09 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Mon Feb 22 19:06:04 2016 -0800

----------------------------------------------------------------------
 src/master/validation.cpp                 |  47 +++----
 src/tests/master_validation_tests.cpp     |   8 +-
 src/tests/reservation_endpoints_tests.cpp |  82 +-----------
 src/tests/reservation_tests.cpp           | 172 +++++++++++++++++++++----
 4 files changed, 173 insertions(+), 136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/efbdef8d/src/master/validation.cpp
----------------------------------------------------------------------
diff --git a/src/master/validation.cpp b/src/master/validation.cpp
index 66898e9..3d1b65c 100644
--- a/src/master/validation.cpp
+++ b/src/master/validation.cpp
@@ -660,41 +660,38 @@ Option<Error> validate(
     return Error("Invalid resources: " + error.get().message);
   }
 
-  // TODO(greggomann): Remove this check once dynamic reservation is
-  // allowed without a principal in 0.28.
-  if (principal.isNone()) {
-    return Error(
-        "Currently must have a principal associated with the request in order "
-        "to reserve resources. This will change in a future version. Note "
-        "that this is distinct from the principal contained in the resources");
-  }
-
   foreach (const Resource& resource, reserve.resources()) {
     if (!Resources::isDynamicallyReserved(resource)) {
       return Error(
           "Resource " + stringify(resource) + " is not dynamically reserved");
     }
 
-    // TODO(greggomann): Remove this check once dynamic reservation is
-    // allowed without a principal in 0.28.
-    if (!resource.reservation().has_principal()) {
-      return Error(
-          "Reserved resources currently must contain a principal. "
-          "This will change in a future version");
-    }
-
     if (role.isSome() && resource.role() != role.get()) {
       return Error(
           "The reserved resource's role '" + resource.role() +
           "' does not match the framework's role '" + role.get() + "'");
     }
 
-    if (resource.reservation().principal() != principal.get()) {
+    if (principal.isSome()) {
+      if (!resource.reservation().has_principal()) {
+        return Error(
+            "A reserve operation was attempted by principal '" +
+            principal.get() + "', but there is a reserved resource in the "
+            "request with no principal set in `ReservationInfo`");
+      }
+
+      if (resource.reservation().principal() != principal.get()) {
+        return Error(
+            "A reserve operation was attempted by principal '" +
+            principal.get() + "', but there is a reserved resource in the "
+            "request with principal '" + resource.reservation().principal() +
+            "' set in `ReservationInfo`");
+      }
+    } else if (resource.reservation().has_principal()) {
       return Error(
-          "The reserved resource's principal '" +
-          resource.reservation().principal() +
-          "' does not match the principal '" +
-          principal.get() + "'");
+          "A reserve operation was attempted with no principal, but there is a "
+          "reserved resource in the request with principal '" +
+          resource.reservation().principal() + "' set in `ReservationInfo`");
     }
 
     // NOTE: This check would be covered by 'contains' since there
@@ -720,12 +717,6 @@ Option<Error> validate(
     return Error("Invalid resources: " + error.get().message);
   }
 
-  if (!hasPrincipal) {
-    return Error(
-        "Currently cannot unreserve resources without a principal. "
-        "This will change in a future version");
-  }
-
   // NOTE: We don't check that 'FrameworkInfo.principal' matches
   // 'Resource.ReservationInfo.principal' here because the authorization
   // depends on the "unreserve" ACL which specifies which 'principal' can

http://git-wip-us.apache.org/repos/asf/mesos/blob/efbdef8d/src/tests/master_validation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_validation_tests.cpp b/src/tests/master_validation_tests.cpp
index 6fae01f..0037f73 100644
--- a/src/tests/master_validation_tests.cpp
+++ b/src/tests/master_validation_tests.cpp
@@ -381,17 +381,17 @@ TEST_F(UnreserveOperationValidationTest, WithoutACL)
 }
 
 
-// This test verifies that validation fails if the framework's
-// 'principal' is not set.
+// This test verifies that validation succeeds if the framework's
+// `principal` is not set.
 TEST_F(UnreserveOperationValidationTest, FrameworkMissingPrincipal)
 {
   Resource resource = Resources::parse("cpus", "8", "role").get();
-  resource.mutable_reservation()->CopyFrom(createReservationInfo("principal"));
+  resource.mutable_reservation()->CopyFrom(createReservationInfo());
 
   Offer::Operation::Unreserve unreserve;
   unreserve.add_resources()->CopyFrom(resource);
 
-  EXPECT_SOME(operation::validate(unreserve, false));
+  EXPECT_NONE(operation::validate(unreserve, false));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/efbdef8d/src/tests/reservation_endpoints_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reservation_endpoints_tests.cpp b/src/tests/reservation_endpoints_tests.cpp
index afe81b1..32b2af4 100644
--- a/src/tests/reservation_endpoints_tests.cpp
+++ b/src/tests/reservation_endpoints_tests.cpp
@@ -1264,13 +1264,8 @@ TEST_F(ReservationEndpointsTest, NonMatchingPrincipal)
 }
 
 
-// This tests the situation where framework and HTTP authentication are disabled
-// and no ACLs are set in the master. Currently, the reservation endpoints do
-// not work in this case because the master invalidates reserve and unreserve
-// operations with no authenticated principal or no principal set in
-// `ReservationInfo`. In 0.28.0, these endpoints will work in this case.
-//
-// TODO(greggomann): Change this test for 0.28.0; see comments below.
+// Tests the situation where framework and HTTP authentication are disabled
+// and no ACLs are set in the master.
 TEST_F(ReservationEndpointsTest, ReserveAndUnreserveNoAuthentication)
 {
   // Manipulate the clock manually in order to
@@ -1311,89 +1306,24 @@ TEST_F(ReservationEndpointsTest, ReserveAndUnreserveNoAuthentication)
       createReservationInfo());
 
   // Try a reservation with no principal in `ReservationInfo` and no
-  // authentication headers. This will fail because currently, dynamic
-  // reservations without either an authenticated principal or a principal in
-  // `ReservationInfo` are invalidated.
-  //
-  // TODO(greggomann): Update this request for 0.28.0. This request should
-  // succeed, but since `ReservationInfo.principal` is being migrated to
-  // `optional`, the request is currently invalidated.
+  // authentication headers.
   Future<Response> response = process::http::post(
       master.get(),
       "reserve",
       None(),
       createRequestBody(slaveId.get(), dynamicallyReservedWithNoPrincipal));
 
-  AWAIT_EXPECT_RESPONSE_STATUS_EQ(BadRequest().status, response);
-
-  // Create a framework that we can use to dynamically reserve some resources
-  // that we will then attempt to unreserve.
-  //
-  // TODO(greggomann): Remove this reserving framework for 0.28.0. It will no
-  // longer be needed once the HTTP reserve request succeeds.
-  MockScheduler sched;
-  MesosSchedulerDriver driver(&sched, frameworkInfo, master.get());
-
-  EXPECT_CALL(sched, registered(&driver, _, _));
-
-  // Expect an offer from the agent.
-  Future<vector<Offer>> offers;
-  EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(FutureArg<1>(&offers));
-
-  driver.start();
-
-  AWAIT_READY(offers);
-  EXPECT_FALSE(offers.get().empty());
-
-  Offer offer = offers.get()[0];
-
-  // We use this filter so that resources will not
-  // be filtered for 5 seconds (the default).
-  Filters filters;
-  filters.set_refuse_seconds(0);
-
-  Resources dynamicallyReservedWithDefaultPrincipal =
-    unreserved.flatten(
-        "role", createReservationInfo(frameworkInfo.principal()));
-
-  // Dynamically reserve resources using `acceptOffers`.
-  driver.acceptOffers(
-      {offer.id()},
-      {RESERVE(dynamicallyReservedWithDefaultPrincipal)},
-      filters);
-
-  // Expect another offer.
-  EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(FutureArg<1>(&offers));
-
-  Clock::settle();
-  Clock::advance(masterFlags.allocation_interval);
-
-  AWAIT_READY(offers);
-  EXPECT_FALSE(offers.get().empty());
-
-  offer = offers.get()[0];
-
-  // Check that the reserved resources are contained in this offer.
-  EXPECT_TRUE(Resources(offer.resources()).contains(
-      dynamicallyReservedWithDefaultPrincipal));
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
 
   // Try to unreserve with no principal in `ReservationInfo` and no
-  // authentication headers. This will fail because currently, unreserve
-  // operations without either an authenticated principal or a principal in
-  // `ReservationInfo` are invalidated.
-  //
-  // TODO(greggomann): Update this request for 0.28.0. This request should
-  // succeed, but since `ReservationInfo.principal` is being migrated to
-  // `optional`, the request is currently invalidated.
+  // authentication headers.
   response = process::http::post(
       master.get(),
       "unreserve",
       None(),
       createRequestBody(slaveId.get(), dynamicallyReservedWithNoPrincipal));
 
-  AWAIT_EXPECT_RESPONSE_STATUS_EQ(BadRequest().status, response);
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
 
   Shutdown();
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/efbdef8d/src/tests/reservation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reservation_tests.cpp b/src/tests/reservation_tests.cpp
index d2ef159..b8878d5 100644
--- a/src/tests/reservation_tests.cpp
+++ b/src/tests/reservation_tests.cpp
@@ -52,6 +52,7 @@ using mesos::internal::master::Master;
 
 using mesos::internal::slave::Slave;
 
+using process::Clock;
 using process::Future;
 using process::PID;
 
@@ -933,9 +934,9 @@ TEST_F(ReservationTest, MasterFailover)
   AWAIT_READY(checkpointResources);
 
   // This is to make sure CheckpointResourcesMessage is processed.
-  process::Clock::pause();
-  process::Clock::settle();
-  process::Clock::resume();
+  Clock::pause();
+  Clock::settle();
+  Clock::resume();
 
   EXPECT_CALL(sched, disconnected(&driver));
 
@@ -1650,7 +1651,7 @@ TEST_F(ReservationTest, ACLMultipleOperations)
 {
   // Pause the clock and control it manually in order to
   // control the timing of the offer cycle.
-  process::Clock::pause();
+  Clock::pause();
 
   ACLs acls;
 
@@ -1715,8 +1716,8 @@ TEST_F(ReservationTest, ACLMultipleOperations)
   driver.start();
 
   // Advance the clock to generate an offer.
-  process::Clock::settle();
-  process::Clock::advance(masterFlags.allocation_interval);
+  Clock::settle();
+  Clock::advance(masterFlags.allocation_interval);
 
   // In the first offer, expect an offer with unreserved resources.
   AWAIT_READY(offers);
@@ -1734,8 +1735,8 @@ TEST_F(ReservationTest, ACLMultipleOperations)
   // Reserve the first set of resources.
   driver.acceptOffers({offer.id()}, {RESERVE(dynamicallyReserved1)}, filters);
 
-  process::Clock::settle();
-  process::Clock::advance(masterFlags.allocation_interval);
+  Clock::settle();
+  Clock::advance(masterFlags.allocation_interval);
 
   // In the next offer, expect an offer with reserved resources.
   AWAIT_READY(offers);
@@ -1780,8 +1781,8 @@ TEST_F(ReservationTest, ACLMultipleOperations)
   AWAIT_READY(statusUpdateAcknowledgement);
   EXPECT_EQ(TASK_FINISHED, statusUpdateAcknowledgement.get().state());
 
-  process::Clock::settle();
-  process::Clock::advance(masterFlags.allocation_interval);
+  Clock::settle();
+  Clock::advance(masterFlags.allocation_interval);
 
   // In the next offer, expect to find both sets of reserved
   // resources, since the Unreserve operation should fail.
@@ -1819,8 +1820,8 @@ TEST_F(ReservationTest, ACLMultipleOperations)
        LAUNCH({taskInfo2})},
       filters);
 
-  process::Clock::settle();
-  process::Clock::advance(masterFlags.allocation_interval);
+  Clock::settle();
+  Clock::advance(masterFlags.allocation_interval);
 
   // In the next offer, expect to find the reserved resources.
   AWAIT_READY(offers);
@@ -1847,16 +1848,13 @@ TEST_F(ReservationTest, ACLMultipleOperations)
 }
 
 
-// This tests that reserve operations containing `ReservationInfo`
-// with no principal set will be invalidated correctly.
-//
-// TODO(greggomann): Change this test once dynamic reservation without a
-// principal is permitted in 0.28.
-TEST_F(ReservationTest, NoAuthentication)
+// Confirms that reserve and unreserve operations work without authentication
+// when a framework has no principal.
+TEST_F(ReservationTest, WithoutAuthenticationWithoutPrincipal)
 {
   // Pause the clock and control it manually in order to
   // control the timing of the offer cycle.
-  process::Clock::pause();
+  Clock::pause();
 
   // Create a framework without a principal.
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
@@ -1865,7 +1863,6 @@ TEST_F(ReservationTest, NoAuthentication)
 
   // Create a master with no framework authentication.
   master::Flags masterFlags = CreateMasterFlags();
-  masterFlags.roles = frameworkInfo.role();
   masterFlags.authenticate_frameworks = false;
 
   Try<PID<Master>> master = StartMaster(masterFlags);
@@ -1897,7 +1894,7 @@ TEST_F(ReservationTest, NoAuthentication)
 
   EXPECT_CALL(sched, registered(&driver, _, _));
 
-  // The expectation for the first offer.
+  // An expectation for an offer with unreserved resources.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
 
@@ -1911,24 +1908,41 @@ TEST_F(ReservationTest, NoAuthentication)
 
   EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
 
-  // The expectation for the next offer.
+  // The expectation for the offer with reserved resources.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
 
-  // Attempt to reserve the resources. This should fail because currently,
-  // reserve operations with no principal are invalid.
+  // Attempt to reserve the resources.
   driver.acceptOffers({offer.id()}, {RESERVE(dynamicallyReserved)}, filters);
 
-  process::Clock::settle();
-  process::Clock::advance(masterFlags.allocation_interval);
+  Clock::settle();
+  Clock::advance(masterFlags.allocation_interval);
+
+  AWAIT_READY(offers);
+
+  ASSERT_EQ(1u, offers.get().size());
+  offer = offers.get()[0];
+
+  // Make sure that the reservation succeeded.
+  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+
+  // An expectation for an offer with unreserved resources.
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  // Unreserve the resources.
+  driver.acceptOffers(
+      {offer.id()}, {UNRESERVE(dynamicallyReserved)}, filters);
 
+  Clock::settle();
+  Clock::advance(masterFlags.allocation_interval);
+
+  // In the next offer, expect an offer with unreserved resources.
   AWAIT_READY(offers);
 
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  // Make sure that the reservation did not succeed and the correct unreserved
-  // resources are still there.
   EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
 
   driver.stop();
@@ -1937,6 +1951,108 @@ TEST_F(ReservationTest, NoAuthentication)
   Shutdown();
 }
 
+
+// Confirms that reserve and unreserve operations work without authentication
+// when a framework has a principal.
+TEST_F(ReservationTest, WithoutAuthenticationWithPrincipal)
+{
+  // Pause the clock and control it manually in order to
+  // control the timing of the offer cycle.
+  Clock::pause();
+
+  // Create a framework with a principal.
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.set_role("role");
+
+  // Create a master with no framework authentication.
+  master::Flags masterFlags = CreateMasterFlags();
+  masterFlags.authenticate_frameworks = false;
+
+  Try<PID<Master>> master = StartMaster(masterFlags);
+  ASSERT_SOME(master);
+
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.resources = "cpus:1;mem:512";
+
+  Try<PID<Slave>> slave = StartSlave(slaveFlags);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(&sched, frameworkInfo, master.get());
+
+  // We use the filter explicitly here so that the resources will not
+  // be filtered for 5 seconds (the default).
+  Filters filters;
+  filters.set_refuse_seconds(0);
+
+  Resources unreserved = Resources::parse("cpus:1;mem:512").get();
+
+  // Create dynamically reserved resources whose `ReservationInfo` contains a
+  // principal.
+  Resources dynamicallyReserved = unreserved.flatten(
+      frameworkInfo.role(), createReservationInfo(frameworkInfo.principal()));
+
+  // We use this to capture offers from `resourceOffers`.
+  Future<vector<Offer>> offers;
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  // An expectation for an offer with unreserved resources.
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  driver.start();
+
+  // In the first offer, expect an offer with unreserved resources.
+  AWAIT_READY(offers);
+
+  ASSERT_EQ(1u, offers.get().size());
+  Offer offer = offers.get()[0];
+
+  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+
+  // The expectation for the offer with reserved resources.
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  // Attempt to reserve the resources.
+  driver.acceptOffers({offer.id()}, {RESERVE(dynamicallyReserved)}, filters);
+
+  Clock::settle();
+  Clock::advance(masterFlags.allocation_interval);
+
+  AWAIT_READY(offers);
+
+  ASSERT_EQ(1u, offers.get().size());
+  offer = offers.get()[0];
+
+  // Make sure that the reservation succeeded.
+  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+
+  // An expectation for an offer with unreserved resources.
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  // Unreserve the resources.
+  driver.acceptOffers(
+      {offer.id()}, {UNRESERVE(dynamicallyReserved)}, filters);
+
+  Clock::settle();
+  Clock::advance(masterFlags.allocation_interval);
+
+  // In the next offer, expect an offer with unreserved resources.
+  AWAIT_READY(offers);
+
+  ASSERT_EQ(1u, offers.get().size());
+  offer = offers.get()[0];
+
+  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
 }  // namespace tests {
 }  // namespace internal {
 }  // namespace mesos {


[3/3] mesos git commit: Updated comments/docs for using reservation endpoints without principal.

Posted by mp...@apache.org.
Updated comments/docs for using reservation endpoints without principal.

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


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

Branch: refs/heads/master
Commit: 866c52c51366da5998db879bbf55ecc278d2c6f2
Parents: a6d27b1
Author: Greg Mann <gr...@mesosphere.io>
Authored: Mon Feb 22 17:21:38 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Mon Feb 22 19:06:05 2016 -0800

----------------------------------------------------------------------
 docs/reservation.md          | 4 +---
 include/mesos/mesos.proto    | 6 +-----
 include/mesos/v1/mesos.proto | 6 +-----
 3 files changed, 3 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/866c52c5/docs/reservation.md
----------------------------------------------------------------------
diff --git a/docs/reservation.md b/docs/reservation.md
index 41321d4..b98ebe6 100644
--- a/docs/reservation.md
+++ b/docs/reservation.md
@@ -58,9 +58,7 @@ see the [authorization documentation](authorization.md).
 * `/reserve` and `/unreserve` HTTP endpoints allow __operators__ to manage
   dynamic reservations through the master. Operators may currently reserve
   resources for any role, although this
-  [will change](https://issues.apache.org/jira/browse/MESOS-4591). NOTE: As of
-  0.27.0, these endpoints cannot be used when HTTP authentication is disabled
-  due to the current implementation. This will change in version 0.28.0.
+  [will change](https://issues.apache.org/jira/browse/MESOS-4591).
 
 In the following sections, we will walk through examples of each of the
 interfaces described above.

http://git-wip-us.apache.org/repos/asf/mesos/blob/866c52c5/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index 11a71cb..33f6b08 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -586,16 +586,12 @@ message Resource {
     // NOTE: We currently do not allow frameworks with role "*" to
     // make dynamic reservations.
 
-    // This field indicates the principal of the framework or operator
+    // Indicates the principal, if any, of the framework or operator
     // that reserved this resource. If reserved by a framework, the
     // field should match the `FrameworkInfo.principal`. It is used in
     // conjunction with the `UnreserveResources` ACL to determine
     // whether the entity attempting to unreserve this resource is
     // permitted to do so.
-    //
-    // NOTE: Currently, a principal is required in order to reserve
-    // resources, so the master disallows any reserve operations
-    // without a principal. This constraint will be removed in 0.28.
     optional string principal = 1;
 
     // Labels are free-form key value pairs that can be used to

http://git-wip-us.apache.org/repos/asf/mesos/blob/866c52c5/include/mesos/v1/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/mesos.proto b/include/mesos/v1/mesos.proto
index 84e933e..46dbec6 100644
--- a/include/mesos/v1/mesos.proto
+++ b/include/mesos/v1/mesos.proto
@@ -583,16 +583,12 @@ message Resource {
     // NOTE: We currently do not allow frameworks with role "*" to
     // make dynamic reservations.
 
-    // This field indicates the principal of the framework or operator
+    // Indicates the principal, if any, of the framework or operator
     // that reserved this resource. If reserved by a framework, the
     // field should match the `FrameworkInfo.principal`. It is used in
     // conjunction with the `UnreserveResources` ACL to determine
     // whether the entity attempting to unreserve this resource is
     // permitted to do so.
-    //
-    // NOTE: Currently, a principal is required in order to reserve
-    // resources, so the master disallows any reserve operations
-    // without a principal. This constraint will be removed in 0.28.
     optional string principal = 1;
 
     // Labels are free-form key value pairs that can be used to


[2/3] mesos git commit: Removed unnecessary parameter from validation function.

Posted by mp...@apache.org.
Removed unnecessary parameter from validation function.

Since unreserve operations are now possible without a principal,
the `bool hasPrincipal` parameter to the Unreserve operation validation
function is no longer necessary.

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


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

Branch: refs/heads/master
Commit: a6d27b138c140239edb4650a744c73dab79d0c06
Parents: efbdef8
Author: Greg Mann <gr...@mesosphere.io>
Authored: Mon Feb 22 17:20:16 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Mon Feb 22 19:06:05 2016 -0800

----------------------------------------------------------------------
 src/master/http.cpp                   |  3 +--
 src/master/master.cpp                 |  2 +-
 src/master/validation.cpp             |  4 +---
 src/master/validation.hpp             |  4 +---
 src/tests/master_validation_tests.cpp | 10 +++++-----
 5 files changed, 9 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a6d27b13/src/master/http.cpp
----------------------------------------------------------------------
diff --git a/src/master/http.cpp b/src/master/http.cpp
index ae6bc78..939fab2 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -2549,8 +2549,7 @@ Future<Response> Master::Http::unreserve(
   operation.set_type(Offer::Operation::UNRESERVE);
   operation.mutable_unreserve()->mutable_resources()->CopyFrom(resources);
 
-  Option<Error> error = validation::operation::validate(
-      operation.unreserve(), principal.isSome());
+  Option<Error> error = validation::operation::validate(operation.unreserve());
 
   if (error.isSome()) {
     return BadRequest(

http://git-wip-us.apache.org/repos/asf/mesos/blob/a6d27b13/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index b453bc7..8d6d3c6 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -3398,7 +3398,7 @@ void Master::_accept(
 
         // Make sure this unreserve operation is valid.
         Option<Error> error = validation::operation::validate(
-            operation.unreserve(), framework->info.has_principal());
+            operation.unreserve());
 
         if (error.isSome()) {
           drop(framework, operation, error.get().message);

http://git-wip-us.apache.org/repos/asf/mesos/blob/a6d27b13/src/master/validation.cpp
----------------------------------------------------------------------
diff --git a/src/master/validation.cpp b/src/master/validation.cpp
index 3d1b65c..b0cc7f7 100644
--- a/src/master/validation.cpp
+++ b/src/master/validation.cpp
@@ -708,9 +708,7 @@ Option<Error> validate(
 }
 
 
-Option<Error> validate(
-    const Offer::Operation::Unreserve& unreserve,
-    bool hasPrincipal)
+Option<Error> validate(const Offer::Operation::Unreserve& unreserve)
 {
   Option<Error> error = resource::validate(unreserve.resources());
   if (error.isSome()) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/a6d27b13/src/master/validation.hpp
----------------------------------------------------------------------
diff --git a/src/master/validation.hpp b/src/master/validation.hpp
index 380b402..6ec883e 100644
--- a/src/master/validation.hpp
+++ b/src/master/validation.hpp
@@ -110,9 +110,7 @@ Option<Error> validate(
 
 
 // Validates the UNRESERVE operation.
-Option<Error> validate(
-    const Offer::Operation::Unreserve& unreserve,
-    bool hasPrincipal);
+Option<Error> validate(const Offer::Operation::Unreserve& unreserve);
 
 
 // Validates the CREATE operation. We need slave's checkpointed

http://git-wip-us.apache.org/repos/asf/mesos/blob/a6d27b13/src/tests/master_validation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_validation_tests.cpp b/src/tests/master_validation_tests.cpp
index 0037f73..ab2df22 100644
--- a/src/tests/master_validation_tests.cpp
+++ b/src/tests/master_validation_tests.cpp
@@ -377,7 +377,7 @@ TEST_F(UnreserveOperationValidationTest, WithoutACL)
   Offer::Operation::Unreserve unreserve;
   unreserve.add_resources()->CopyFrom(resource);
 
-  EXPECT_NONE(operation::validate(unreserve, true));
+  EXPECT_NONE(operation::validate(unreserve));
 }
 
 
@@ -391,7 +391,7 @@ TEST_F(UnreserveOperationValidationTest, FrameworkMissingPrincipal)
   Offer::Operation::Unreserve unreserve;
   unreserve.add_resources()->CopyFrom(resource);
 
-  EXPECT_NONE(operation::validate(unreserve, false));
+  EXPECT_NONE(operation::validate(unreserve));
 }
 
 
@@ -404,7 +404,7 @@ TEST_F(UnreserveOperationValidationTest, StaticReservation)
   Offer::Operation::Unreserve unreserve;
   unreserve.add_resources()->CopyFrom(staticallyReserved);
 
-  EXPECT_SOME(operation::validate(unreserve, true));
+  EXPECT_SOME(operation::validate(unreserve));
 }
 
 
@@ -418,7 +418,7 @@ TEST_F(UnreserveOperationValidationTest, NoPersistentVolumes)
   Offer::Operation::Unreserve unreserve;
   unreserve.add_resources()->CopyFrom(reserved);
 
-  EXPECT_NONE(operation::validate(unreserve, true));
+  EXPECT_NONE(operation::validate(unreserve));
 }
 
 
@@ -436,7 +436,7 @@ TEST_F(UnreserveOperationValidationTest, PersistentVolumes)
   unreserve.add_resources()->CopyFrom(reserved);
   unreserve.add_resources()->CopyFrom(volume);
 
-  EXPECT_SOME(operation::validate(unreserve, true));
+  EXPECT_SOME(operation::validate(unreserve));
 }