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/12 01:18:23 UTC

[1/2] mesos git commit: Added support for labels to resource reservations.

Repository: mesos
Updated Branches:
  refs/heads/master 376b5c2ae -> 3b02b80fa


Added support for labels to resource reservations.

Labels are free-form key-value pairs that can be used to associate
metadata with reserved resources.

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


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

Branch: refs/heads/master
Commit: 77448c0bda4109ceb0c2aadbb5d240faa12b1f3e
Parents: 376b5c2
Author: Neil Conway <ne...@gmail.com>
Authored: Thu Feb 11 15:56:39 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Thu Feb 11 15:56:39 2016 -0800

----------------------------------------------------------------------
 include/mesos/mesos.proto                 |   7 ++
 include/mesos/type_utils.hpp              |   3 +-
 include/mesos/v1/mesos.hpp                |   2 +
 include/mesos/v1/mesos.proto              |   7 ++
 src/common/resources.cpp                  |  46 +++++++-
 src/common/type_utils.cpp                 |   6 +
 src/tests/mesos.hpp                       |   7 +-
 src/tests/reservation_endpoints_tests.cpp | 156 ++++++++++++++++++++++++-
 src/tests/resources_tests.cpp             | 125 +++++++++++++++++---
 src/v1/mesos.cpp                          |   6 +
 src/v1/resources.cpp                      |  46 +++++++-
 11 files changed, 388 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/77448c0b/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index 5d1a20b..6b4fd91 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -585,6 +585,13 @@ message Resource {
     // 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
+    // associate arbitrary metadata with a reserved resource.  For
+    // example, frameworks can use labels to identify the intended
+    // purpose for a portion of the resources the framework has
+    // reserved at a given agent.
+    optional Labels labels = 2;
   }
 
   // If this is set, this resource was dynamically reserved by an

http://git-wip-us.apache.org/repos/asf/mesos/blob/77448c0b/include/mesos/type_utils.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/type_utils.hpp b/include/mesos/type_utils.hpp
index 16f9cda..27fa8c9 100644
--- a/include/mesos/type_utils.hpp
+++ b/include/mesos/type_utils.hpp
@@ -68,9 +68,10 @@ bool operator==(const TaskStatus& left, const TaskStatus& right);
 bool operator==(const URL& left, const URL& right);
 bool operator==(const Volume& left, const Volume& right);
 
-
+bool operator!=(const Labels& left, const Labels& right);
 bool operator!=(const TaskStatus& left, const TaskStatus& right);
 
+
 inline bool operator==(const ContainerID& left, const ContainerID& right)
 {
   return left.value() == right.value();

http://git-wip-us.apache.org/repos/asf/mesos/blob/77448c0b/include/mesos/v1/mesos.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/v1/mesos.hpp b/include/mesos/v1/mesos.hpp
index b294f02..d220966 100644
--- a/include/mesos/v1/mesos.hpp
+++ b/include/mesos/v1/mesos.hpp
@@ -62,8 +62,10 @@ bool operator==(const TaskStatus& left, const TaskStatus& right);
 bool operator==(const URL& left, const URL& right);
 bool operator==(const Volume& left, const Volume& right);
 
+bool operator!=(const Labels& left, const Labels& right);
 bool operator!=(const TaskStatus& left, const TaskStatus& right);
 
+
 inline bool operator==(const ContainerID& left, const ContainerID& right)
 {
   return left.value() == right.value();

http://git-wip-us.apache.org/repos/asf/mesos/blob/77448c0b/include/mesos/v1/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/mesos.proto b/include/mesos/v1/mesos.proto
index 415a20d..8f126ec 100644
--- a/include/mesos/v1/mesos.proto
+++ b/include/mesos/v1/mesos.proto
@@ -584,6 +584,13 @@ message Resource {
     // 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
+    // associate arbitrary metadata with a reserved resource.  For
+    // example, frameworks can use labels to identify the intended
+    // purpose for a portion of the resources the framework has
+    // reserved at a given agent.
+    optional Labels labels = 2;
   }
 
   // If this is set, this resource was dynamically reserved by an

http://git-wip-us.apache.org/repos/asf/mesos/blob/77448c0b/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index 588a279..5d73187 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -63,7 +63,19 @@ bool operator==(
     const Resource::ReservationInfo& left,
     const Resource::ReservationInfo& right)
 {
-  return left.principal() == right.principal();
+  if (left.principal() != right.principal()) {
+    return false;
+  }
+
+  if (left.has_labels() != right.has_labels()) {
+    return false;
+  }
+
+  if (left.has_labels() && left.labels() != right.labels()) {
+    return false;
+  }
+
+  return true;
 }
 
 
@@ -1472,6 +1484,30 @@ ostream& operator<<(ostream& stream, const Resource::DiskInfo& disk)
 }
 
 
+ostream& operator<<(ostream& stream, const Labels& labels)
+{
+  stream << "{";
+
+  for (int i = 0; i < labels.labels().size(); i++) {
+    const Label& label = labels.labels().Get(i);
+
+    stream << label.key();
+
+    if (label.has_value()) {
+      stream << ": " << label.value();
+    }
+
+    if (i + 1 < labels.labels().size()) {
+      stream << ", ";
+    }
+  }
+
+  stream << "}";
+
+  return stream;
+}
+
+
 ostream& operator<<(ostream& stream, const Resource& resource)
 {
   stream << resource.name();
@@ -1479,7 +1515,13 @@ ostream& operator<<(ostream& stream, const Resource& resource)
   stream << "(" << resource.role();
 
   if (resource.has_reservation()) {
-    stream << ", " << resource.reservation().principal();
+    const Resource::ReservationInfo& reservation = resource.reservation();
+
+    stream << ", " << reservation.principal();
+
+    if (reservation.has_labels()) {
+      stream << ", " << reservation.labels();
+    }
   }
 
   stream << ")";

http://git-wip-us.apache.org/repos/asf/mesos/blob/77448c0b/src/common/type_utils.cpp
----------------------------------------------------------------------
diff --git a/src/common/type_utils.cpp b/src/common/type_utils.cpp
index 42e3061..037c433 100644
--- a/src/common/type_utils.cpp
+++ b/src/common/type_utils.cpp
@@ -288,6 +288,12 @@ bool operator==(const Labels& left, const Labels& right)
 }
 
 
+bool operator!=(const Labels& left, const Labels& right)
+{
+  return !(left == right);
+}
+
+
 bool operator==(const DiscoveryInfo& left, const DiscoveryInfo& right)
 {
   return left.visibility() == right.visibility() &&

http://git-wip-us.apache.org/repos/asf/mesos/blob/77448c0b/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index c0997db..242a116 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -488,7 +488,8 @@ inline TaskInfo createTask(
 
 
 inline Resource::ReservationInfo createReservationInfo(
-    const Option<std::string>& principal = None())
+    const Option<std::string>& principal = None(),
+    const Option<Labels>& labels = None())
 {
   Resource::ReservationInfo info;
 
@@ -496,6 +497,10 @@ inline Resource::ReservationInfo createReservationInfo(
     info.set_principal(principal.get());
   }
 
+  if (labels.isSome()) {
+    info.mutable_labels()->CopyFrom(labels.get());
+  }
+
   return info;
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/77448c0b/src/tests/reservation_endpoints_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reservation_endpoints_tests.cpp b/src/tests/reservation_endpoints_tests.cpp
index 0a8c479..afe81b1 100644
--- a/src/tests/reservation_endpoints_tests.cpp
+++ b/src/tests/reservation_endpoints_tests.cpp
@@ -45,6 +45,7 @@ using std::vector;
 using google::protobuf::RepeatedPtrField;
 
 using mesos::internal::master::Master;
+using mesos::internal::protobuf::createLabel;
 using mesos::internal::slave::Slave;
 
 using process::Clock;
@@ -326,7 +327,7 @@ TEST_F(ReservationEndpointsTest, UnreserveOfferedResources)
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
 
-  // Expect an offer to be rescinded!
+  // Expect an offer to be rescinded.
   EXPECT_CALL(sched, offerRescinded(_, _));
 
   response = process::http::post(
@@ -664,6 +665,159 @@ TEST_F(ReservationEndpointsTest, UnreserveAvailableAndOfferedResources)
 }
 
 
+// This tests that attempts to reserve/unreserve labeled resources
+// behave as expected.
+TEST_F(ReservationEndpointsTest, LabeledResources)
+{
+  TestAllocator<> allocator;
+
+  EXPECT_CALL(allocator, initialize(_, _, _, _));
+
+  Try<PID<Master>> master = StartMaster(&allocator);
+  ASSERT_SOME(master);
+
+  slave::Flags slaveFlags = CreateSlaveFlags();
+  slaveFlags.resources = "cpus:2;mem:1024";
+  Resources totalSlaveResources =
+    Resources::parse(slaveFlags.resources.get()).get();
+
+  Future<SlaveID> slaveId;
+  EXPECT_CALL(allocator, addSlave(_, _, _, _, _))
+    .WillOnce(DoAll(InvokeAddSlave(&allocator),
+                    FutureArg<0>(&slaveId)));
+
+  Try<PID<Slave>> slave = StartSlave(slaveFlags);
+  ASSERT_SOME(slave);
+
+  FrameworkInfo frameworkInfo = createFrameworkInfo();
+
+  Labels labels1;
+  labels1.add_labels()->CopyFrom(createLabel("foo", "bar"));
+
+  Labels labels2;
+  labels2.add_labels()->CopyFrom(createLabel("foo", "baz"));
+
+  Resources unreserved = Resources::parse("cpus:1;mem:512").get();
+  Resources labeledResources1 = unreserved.flatten(
+      frameworkInfo.role(),
+      createReservationInfo(DEFAULT_CREDENTIAL.principal(), labels1));
+  Resources labeledResources2 = unreserved.flatten(
+      frameworkInfo.role(),
+      createReservationInfo(DEFAULT_CREDENTIAL.principal(), labels2));
+
+  // Make two resource reservations with different labels.
+  Future<Response> response = process::http::post(
+      master.get(),
+      "reserve",
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+      createRequestBody(slaveId.get(), labeledResources1));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
+
+  response = process::http::post(
+      master.get(),
+      "reserve",
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+      createRequestBody(slaveId.get(), labeledResources2));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
+
+  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();
+
+  AWAIT_READY(offers);
+
+  ASSERT_EQ(1u, offers.get().size());
+  Offer offer = offers.get()[0];
+
+  Resources offeredResources = Resources(offer.resources());
+  EXPECT_TRUE(offeredResources.contains(labeledResources1));
+  EXPECT_TRUE(offeredResources.contains(labeledResources2));
+
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  // Expect an offer to be rescinded.
+  EXPECT_CALL(sched, offerRescinded(_, _));
+
+  // Unreserve one of the labeled reservations.
+  response = process::http::post(
+      master.get(),
+      "unreserve",
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+      createRequestBody(slaveId.get(), labeledResources1));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
+
+  AWAIT_READY(offers);
+
+  ASSERT_EQ(1u, offers.get().size());
+  offer = offers.get()[0];
+
+  offeredResources = Resources(offer.resources());
+  EXPECT_FALSE(offeredResources.contains(totalSlaveResources));
+  EXPECT_TRUE(offeredResources.contains(unreserved));
+  EXPECT_FALSE(offeredResources.contains(labeledResources1));
+  EXPECT_TRUE(offeredResources.contains(labeledResources2));
+
+  // Now that the first labeled reservation has been unreserved,
+  // attempting to unreserve it again should fail.
+  response = process::http::post(
+      master.get(),
+      "unreserve",
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+      createRequestBody(slaveId.get(), labeledResources1));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(Conflict().status, response);
+
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  // Expect an offer to be rescinded.
+  EXPECT_CALL(sched, offerRescinded(_, _));
+
+  // Unreserve the other labeled reservation.
+  response = process::http::post(
+      master.get(),
+      "unreserve",
+      createBasicAuthHeaders(DEFAULT_CREDENTIAL),
+      createRequestBody(slaveId.get(), labeledResources2));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
+
+  AWAIT_READY(offers);
+
+  ASSERT_EQ(1u, offers.get().size());
+  offer = offers.get()[0];
+
+  offeredResources = Resources(offer.resources());
+
+  EXPECT_TRUE(offeredResources.contains(totalSlaveResources));
+  EXPECT_FALSE(offeredResources.contains(labeledResources1));
+  EXPECT_FALSE(offeredResources.contains(labeledResources2));
+
+  // Ignore subsequent `recoverResources` calls triggered from recovering the
+  // resources that this framework is currently holding onto.
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
+    .WillRepeatedly(DoDefault());
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+}
+
+
 // This tests that an attempt to reserve/unreserve more resources than available
 // results in a 'Conflict' HTTP error.
 TEST_F(ReservationEndpointsTest, InsufficientResources)

http://git-wip-us.apache.org/repos/asf/mesos/blob/77448c0b/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index a190cc3..a0e674d 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -40,6 +40,8 @@ using std::string;
 
 using google::protobuf::RepeatedPtrField;
 
+using mesos::internal::protobuf::createLabel;
+
 namespace mesos {
 namespace internal {
 namespace tests {
@@ -793,7 +795,7 @@ TEST(ResourcesTest, PrintingExtendedAttributes)
   stream << disk;
   EXPECT_EQ(stream.str(), "disk(*):1");
 
-  // Standard resource with role.
+  // Standard resource with role (statically reserved).
   stream.str("");
   disk.set_role("alice");
   stream << disk;
@@ -827,12 +829,20 @@ TEST(ResourcesTest, PrintingExtendedAttributes)
   stream << disk;
   EXPECT_EQ(stream.str(), "disk(alice)[hadoop:/hdfs:/data:rw]:1");
 
-  // Disk resource with host path and reservation.
+  // Disk resource with host path and dynamic reservation without labels.
+  stream.str("");
+  disk.mutable_reservation()->set_principal("hdfs-p");
+  stream << disk;
+  EXPECT_EQ(stream.str(), "disk(alice, hdfs-p)[hadoop:/hdfs:/data:rw]:1");
+
+  // Disk resource with host path and dynamic reservation with labels.
   stream.str("");
-  disk.mutable_reservation()->set_principal("hdfs-1234-4321");
+  Labels* labels = disk.mutable_reservation()->mutable_labels();
+  labels->add_labels()->CopyFrom(createLabel("foo", "bar"));
+  labels->add_labels()->CopyFrom(createLabel("foo"));
   stream << disk;
   EXPECT_EQ(stream.str(),
-            "disk(alice, hdfs-1234-4321)[hadoop:/hdfs:/data:rw]:1");
+            "disk(alice, hdfs-p, {foo: bar, foo})[hadoop:/hdfs:/data:rw]:1");
 }
 
 
@@ -1578,14 +1588,26 @@ TEST(ReservedResourcesTest, Validation)
   EXPECT_SOME(Resources::validate(createReservedResource(
       "cpus", "8", ".", createReservationInfo("principal1"))));
 
-  // Dynamically role reserved.
+  // Dynamically reserved without labels.
   EXPECT_NONE(Resources::validate(createReservedResource(
       "cpus", "8", "role", createReservationInfo("principal2"))));
+
+  // Dynamically reserved with labels.
+  Labels labels;
+  labels.add_labels()->CopyFrom(createLabel("foo", "bar"));
+  EXPECT_NONE(Resources::validate(createReservedResource(
+      "cpus", "8", "role", createReservationInfo("principal2", labels))));
 }
 
 
 TEST(ReservedResourcesTest, Equals)
 {
+  Labels labels1;
+  labels1.add_labels()->CopyFrom(createLabel("foo", "bar"));
+
+  Labels labels2;
+  labels2.add_labels()->CopyFrom(createLabel("foo", "baz"));
+
   std::vector<Resources> unique = {
     // Unreserved.
     createReservedResource(
@@ -1603,7 +1625,12 @@ TEST(ReservedResourcesTest, Equals)
     createReservedResource(
         "cpus", "8", "role2", createReservationInfo("principal1")),
     createReservedResource(
-        "cpus", "8", "role2", createReservationInfo("principal2"))
+        "cpus", "8", "role2", createReservationInfo("principal2")),
+    // Dynamically reserved with labels.
+    createReservedResource(
+        "cpus", "8", "role1", createReservationInfo("principal2", labels1)),
+    createReservedResource(
+        "cpus", "8", "role1", createReservationInfo("principal2", labels2))
   };
 
   // Test that all resources in 'unique' are considered different.
@@ -1628,7 +1655,7 @@ TEST(ReservedResourcesTest, AdditionStaticallyReserved)
 }
 
 
-TEST(ReservedResourcesTest, AdditionDynamicallyReserved)
+TEST(ReservedResourcesTest, AdditionDynamicallyReservedWithoutLabels)
 {
   Resource::ReservationInfo reservationInfo =
     createReservationInfo("principal");
@@ -1644,25 +1671,91 @@ TEST(ReservedResourcesTest, AdditionDynamicallyReserved)
 }
 
 
-TEST(ReservedResourcesTest, Subtraction)
+TEST(ReservedResourcesTest, AdditionDynamicallyReservedWithSameLabels)
 {
+  Labels labels;
+  labels.add_labels()->CopyFrom(createLabel("foo", "bar"));
+
   Resource::ReservationInfo reservationInfo =
-    createReservationInfo("principal");
+    createReservationInfo("principal", labels);
+
+  Resources left =
+    createReservedResource("cpus", "8", "role", reservationInfo);
+  Resources right =
+    createReservedResource("cpus", "4", "role", reservationInfo);
+  Resources expected =
+    createReservedResource("cpus", "12", "role", reservationInfo);
+
+  EXPECT_EQ(expected, left + right);
+}
+
+
+TEST(ReservedResourcesTest, AdditionDynamicallyReservedWithDistinctLabels)
+{
+  Labels labels1;
+  Labels labels2;
+
+  labels1.add_labels()->CopyFrom(createLabel("foo", "bar"));
+  labels2.add_labels()->CopyFrom(createLabel("foo", "baz"));
+
+  Resource::ReservationInfo reservationInfo1 =
+    createReservationInfo("principal", labels1);
+  Resource::ReservationInfo reservationInfo2 =
+    createReservationInfo("principal", labels2);
+
+  Resources r1 = createReservedResource("cpus", "6", "role", reservationInfo1);
+  Resources r2 = createReservedResource("cpus", "6", "role", reservationInfo2);
+  Resources sum = r1 + r2;
+
+  EXPECT_EQ(2, sum.size());
+  EXPECT_FALSE(sum == r1 + r1);
+  EXPECT_FALSE(sum == r2 + r2);
+}
+
+
+TEST(ReservedResourcesTest, Subtraction)
+{
+  Labels labels1;
+  Labels labels2;
+
+  labels1.add_labels()->CopyFrom(createLabel("foo", "bar"));
+  labels2.add_labels()->CopyFrom(createLabel("foo", "baz"));
+
+  Resource::ReservationInfo reservationInfo1 =
+    createReservationInfo("principal", labels1);
+  Resource::ReservationInfo reservationInfo2 =
+    createReservationInfo("principal", labels2);
 
   Resources r1 = createReservedResource("cpus", "8", "role", None());
-  Resources r2 = createReservedResource("cpus", "8", "role", reservationInfo);
+  Resources r2 = createReservedResource("cpus", "8", "role", reservationInfo1);
+
+  EXPECT_TRUE((r1 - r1).empty());
+  EXPECT_TRUE((r2 - r2).empty());
+  EXPECT_FALSE((r2 - r1).empty());
+  EXPECT_FALSE((r1 - r2).empty());
+  EXPECT_EQ(r1, r1 - r2);
+  EXPECT_EQ(r2, r2 - r1);
 
   Resources total = r1 + r2;
 
-  Resources r4 = createReservedResource("cpus", "6", "role", None());
-  Resources r5 = createReservedResource("cpus", "4", "role", reservationInfo);
+  Resources r3 = createReservedResource("cpus", "6", "role", None());
+  Resources r4 = createReservedResource("cpus", "4", "role", reservationInfo1);
+
+  Resources expected = r3 + r4;
 
-  Resources expected = r4 + r5;
+  Resources r5 = createReservedResource("cpus", "2", "role", None());
+  Resources r6 = createReservedResource("cpus", "4", "role", reservationInfo1);
 
-  Resources r7 = createReservedResource("cpus", "2", "role", None());
-  Resources r8 = createReservedResource("cpus", "4", "role", reservationInfo);
+  EXPECT_EQ(expected, total - r5 - r6);
 
-  EXPECT_EQ(expected, total - r7 - r8);
+  // Distinct labels
+  Resources r7 = createReservedResource("cpus", "8", "role", reservationInfo1);
+  Resources r8 = createReservedResource("cpus", "8", "role", reservationInfo2);
+
+  EXPECT_FALSE((r2 - r1).empty());
+  EXPECT_FALSE((r1 - r2).empty());
+  EXPECT_EQ(r1, r1 - r2);
+  EXPECT_EQ(r2, r2 - r1);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/77448c0b/src/v1/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/v1/mesos.cpp b/src/v1/mesos.cpp
index 21b14cf..30788dc 100644
--- a/src/v1/mesos.cpp
+++ b/src/v1/mesos.cpp
@@ -286,6 +286,12 @@ bool operator==(const Labels& left, const Labels& right)
 }
 
 
+bool operator!=(const Labels& left, const Labels& right)
+{
+  return !(left == right);
+}
+
+
 bool operator==(const DiscoveryInfo& left, const DiscoveryInfo& right)
 {
   return left.visibility() == right.visibility() &&

http://git-wip-us.apache.org/repos/asf/mesos/blob/77448c0b/src/v1/resources.cpp
----------------------------------------------------------------------
diff --git a/src/v1/resources.cpp b/src/v1/resources.cpp
index be4a5d1..207eb61 100644
--- a/src/v1/resources.cpp
+++ b/src/v1/resources.cpp
@@ -65,7 +65,19 @@ bool operator==(
     const Resource::ReservationInfo& left,
     const Resource::ReservationInfo& right)
 {
-  return left.principal() == right.principal();
+  if (left.principal() != right.principal()) {
+    return false;
+  }
+
+  if (left.has_labels() != right.has_labels()) {
+    return false;
+  }
+
+  if (left.has_labels() && left.labels() != right.labels()) {
+    return false;
+  }
+
+  return true;
 }
 
 
@@ -1475,6 +1487,30 @@ ostream& operator<<(ostream& stream, const Resource::DiskInfo& disk)
 }
 
 
+ostream& operator<<(ostream& stream, const Labels& labels)
+{
+  stream << "{";
+
+  for (int i = 0; i < labels.labels().size(); i++) {
+    const Label& label = labels.labels().Get(i);
+
+    stream << label.key();
+
+    if (label.has_value()) {
+      stream << ": " << label.value();
+    }
+
+    if (i + 1 < labels.labels().size()) {
+      stream << ", ";
+    }
+  }
+
+  stream << "}";
+
+  return stream;
+}
+
+
 ostream& operator<<(ostream& stream, const Resource& resource)
 {
   stream << resource.name();
@@ -1482,7 +1518,13 @@ ostream& operator<<(ostream& stream, const Resource& resource)
   stream << "(" << resource.role();
 
   if (resource.has_reservation()) {
-    stream << ", " << resource.reservation().principal();
+    const Resource::ReservationInfo& reservation = resource.reservation();
+
+    stream << ", " << reservation.principal();
+
+    if (reservation.has_labels()) {
+      stream << ", " << reservation.labels();
+    }
   }
 
   stream << ")";


Re: [2/2] mesos git commit: Added documentation for labeled reserved resources.

Posted by Benjamin Mahler <bm...@apache.org>.
Got it, this is what I figured would be the answer :)

Just want to make sure this information is somewhat discoverable to the
user (e.g. a note on reservation info that references MESOS-4476).

On Sat, Feb 13, 2016 at 2:59 AM, Guangya Liu <gy...@gmail.com> wrote:

> Neil, I think what you want to show for setting reservationInfo for static
> reservation is https://issues.apache.org/jira/browse/MESOS-4476
>
> On Sat, Feb 13, 2016 at 2:16 AM, Neil Conway <ne...@gmail.com>
> wrote:
>
> > Hi Ben,
> >
> > On Fri, Feb 12, 2016 at 2:34 AM, Benjamin Mahler <bm...@apache.org>
> > wrote:
> > > Any plans to support labels for static reservations?
> > >
> > > Are we intentionally not supporting ReservationInfo for static
> > > reservations? Or is this just outside of the initial scope?
> >
> > Labels for static reservations are not currently supported because
> > `labels` is part of `ReservationInfo`, and the latter is not set for
> > static reservations.
> >
> > Setting ReservationInfo for static reservations is
> > https://issues.apache.org/jira/browse/MESOS-3486 . I didn't take this
> > on right now, because there are some backward compatibility concerns
> > with making this change. It is also unclear if we want to continue
> > adding features to static reservations vs. continuing to enhance
> > dynamic reservations to the point at which they can replace static
> > reservations for most use cases.
> >
> > Neil
> >
>

Re: [2/2] mesos git commit: Added documentation for labeled reserved resources.

Posted by Benjamin Mahler <bm...@apache.org>.
Got it, this is what I figured would be the answer :)

Just want to make sure this information is somewhat discoverable to the
user (e.g. a note on reservation info that references MESOS-4476).

On Sat, Feb 13, 2016 at 2:59 AM, Guangya Liu <gy...@gmail.com> wrote:

> Neil, I think what you want to show for setting reservationInfo for static
> reservation is https://issues.apache.org/jira/browse/MESOS-4476
>
> On Sat, Feb 13, 2016 at 2:16 AM, Neil Conway <ne...@gmail.com>
> wrote:
>
> > Hi Ben,
> >
> > On Fri, Feb 12, 2016 at 2:34 AM, Benjamin Mahler <bm...@apache.org>
> > wrote:
> > > Any plans to support labels for static reservations?
> > >
> > > Are we intentionally not supporting ReservationInfo for static
> > > reservations? Or is this just outside of the initial scope?
> >
> > Labels for static reservations are not currently supported because
> > `labels` is part of `ReservationInfo`, and the latter is not set for
> > static reservations.
> >
> > Setting ReservationInfo for static reservations is
> > https://issues.apache.org/jira/browse/MESOS-3486 . I didn't take this
> > on right now, because there are some backward compatibility concerns
> > with making this change. It is also unclear if we want to continue
> > adding features to static reservations vs. continuing to enhance
> > dynamic reservations to the point at which they can replace static
> > reservations for most use cases.
> >
> > Neil
> >
>

Re: [2/2] mesos git commit: Added documentation for labeled reserved resources.

Posted by Guangya Liu <gy...@gmail.com>.
Neil, I think what you want to show for setting reservationInfo for static
reservation is https://issues.apache.org/jira/browse/MESOS-4476

On Sat, Feb 13, 2016 at 2:16 AM, Neil Conway <ne...@gmail.com> wrote:

> Hi Ben,
>
> On Fri, Feb 12, 2016 at 2:34 AM, Benjamin Mahler <bm...@apache.org>
> wrote:
> > Any plans to support labels for static reservations?
> >
> > Are we intentionally not supporting ReservationInfo for static
> > reservations? Or is this just outside of the initial scope?
>
> Labels for static reservations are not currently supported because
> `labels` is part of `ReservationInfo`, and the latter is not set for
> static reservations.
>
> Setting ReservationInfo for static reservations is
> https://issues.apache.org/jira/browse/MESOS-3486 . I didn't take this
> on right now, because there are some backward compatibility concerns
> with making this change. It is also unclear if we want to continue
> adding features to static reservations vs. continuing to enhance
> dynamic reservations to the point at which they can replace static
> reservations for most use cases.
>
> Neil
>

Re: [2/2] mesos git commit: Added documentation for labeled reserved resources.

Posted by Neil Conway <ne...@gmail.com>.
Hi Ben,

On Fri, Feb 12, 2016 at 2:34 AM, Benjamin Mahler <bm...@apache.org> wrote:
> Any plans to support labels for static reservations?
>
> Are we intentionally not supporting ReservationInfo for static
> reservations? Or is this just outside of the initial scope?

Labels for static reservations are not currently supported because
`labels` is part of `ReservationInfo`, and the latter is not set for
static reservations.

Setting ReservationInfo for static reservations is
https://issues.apache.org/jira/browse/MESOS-3486 . I didn't take this
on right now, because there are some backward compatibility concerns
with making this change. It is also unclear if we want to continue
adding features to static reservations vs. continuing to enhance
dynamic reservations to the point at which they can replace static
reservations for most use cases.

Neil

Re: [2/2] mesos git commit: Added documentation for labeled reserved resources.

Posted by Benjamin Mahler <bm...@apache.org>.
Any plans to support labels for static reservations?

Are we intentionally not supporting ReservationInfo for static
reservations? Or is this just outside of the initial scope?

On Fri, Feb 12, 2016 at 1:18 AM, <mp...@apache.org> wrote:

> Added documentation for labeled reserved resources.
>
> Review: https://reviews.apache.org/r/42755/
>
>
> Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
> Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/3b02b80f
> Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/3b02b80f
> Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/3b02b80f
>
> Branch: refs/heads/master
> Commit: 3b02b80fae886caccd242f5fc205e91a42723861
> Parents: 77448c0
> Author: Neil Conway <ne...@gmail.com>
> Authored: Thu Feb 11 16:07:05 2016 -0800
> Committer: Michael Park <mp...@apache.org>
> Committed: Thu Feb 11 16:07:05 2016 -0800
>
> ----------------------------------------------------------------------
>  docs/reservation.md | 27 ++++++++++++++++++++-------
>  1 file changed, 20 insertions(+), 7 deletions(-)
> ----------------------------------------------------------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/mesos/blob/3b02b80f/docs/reservation.md
> ----------------------------------------------------------------------
> diff --git a/docs/reservation.md b/docs/reservation.md
> index 51fa7fc..d3ae445 100644
> --- a/docs/reservation.md
> +++ b/docs/reservation.md
> @@ -65,13 +65,26 @@ see the [authorization documentation](authorization.md
> ).
>  In the following sections, we will walk through examples of each of the
>  interfaces described above.
>
> -Note that if two dynamic reservations are made for resources at a single
> slave,
> -the reservations will be combined by adding together the resources
> reserved by
> -each request. Similarly, "partial" unreserve operations are allowed: an
> -unreserve operation can release only some of the resources at a slave
> that have
> -been reserved for a given role. In this case, the unreserved resources
> will be
> -subtracted from the previous reservation, and any remaining resources
> will still
> -be reserved.
> +If two dynamic reservations are made for the same role at a single slave
> (using
> +the same labels, if any; see below), the reservations will be combined by
> adding
> +together the resources reserved by each request. This will result in a
> single
> +reserved resource at the slave. Similarly, "partial" unreserve operations
> are
> +allowed: an unreserve operation can release some but not all of the
> resources at
> +a slave that have been reserved for a role. In this case, the unreserved
> +resources will be subtracted from the previous reservation and any
> remaining
> +resources will still be reserved.
> +
> +
> +### Labeled Reservations
> +
> +Dynamic reservations can optionally include a list of _labels_, which are
> +arbitrary key-value pairs. Labels can be used to associate arbitrary
> metadata
> +with a resource reservation. For example, frameworks can use labels to
> identify
> +the intended purpose for a portion of the resources that have been
> reserved at a
> +given slave. Note that two reservations with different labels will not be
> +combined together into a single reservation, even if the reservations are
> at the
> +same slave and use the same role.
> +
>
>  ### Framework Scheduler API
>
>
>

Re: [2/2] mesos git commit: Added documentation for labeled reserved resources.

Posted by Benjamin Mahler <bm...@apache.org>.
Any plans to support labels for static reservations?

Are we intentionally not supporting ReservationInfo for static
reservations? Or is this just outside of the initial scope?

On Fri, Feb 12, 2016 at 1:18 AM, <mp...@apache.org> wrote:

> Added documentation for labeled reserved resources.
>
> Review: https://reviews.apache.org/r/42755/
>
>
> Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
> Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/3b02b80f
> Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/3b02b80f
> Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/3b02b80f
>
> Branch: refs/heads/master
> Commit: 3b02b80fae886caccd242f5fc205e91a42723861
> Parents: 77448c0
> Author: Neil Conway <ne...@gmail.com>
> Authored: Thu Feb 11 16:07:05 2016 -0800
> Committer: Michael Park <mp...@apache.org>
> Committed: Thu Feb 11 16:07:05 2016 -0800
>
> ----------------------------------------------------------------------
>  docs/reservation.md | 27 ++++++++++++++++++++-------
>  1 file changed, 20 insertions(+), 7 deletions(-)
> ----------------------------------------------------------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/mesos/blob/3b02b80f/docs/reservation.md
> ----------------------------------------------------------------------
> diff --git a/docs/reservation.md b/docs/reservation.md
> index 51fa7fc..d3ae445 100644
> --- a/docs/reservation.md
> +++ b/docs/reservation.md
> @@ -65,13 +65,26 @@ see the [authorization documentation](authorization.md
> ).
>  In the following sections, we will walk through examples of each of the
>  interfaces described above.
>
> -Note that if two dynamic reservations are made for resources at a single
> slave,
> -the reservations will be combined by adding together the resources
> reserved by
> -each request. Similarly, "partial" unreserve operations are allowed: an
> -unreserve operation can release only some of the resources at a slave
> that have
> -been reserved for a given role. In this case, the unreserved resources
> will be
> -subtracted from the previous reservation, and any remaining resources
> will still
> -be reserved.
> +If two dynamic reservations are made for the same role at a single slave
> (using
> +the same labels, if any; see below), the reservations will be combined by
> adding
> +together the resources reserved by each request. This will result in a
> single
> +reserved resource at the slave. Similarly, "partial" unreserve operations
> are
> +allowed: an unreserve operation can release some but not all of the
> resources at
> +a slave that have been reserved for a role. In this case, the unreserved
> +resources will be subtracted from the previous reservation and any
> remaining
> +resources will still be reserved.
> +
> +
> +### Labeled Reservations
> +
> +Dynamic reservations can optionally include a list of _labels_, which are
> +arbitrary key-value pairs. Labels can be used to associate arbitrary
> metadata
> +with a resource reservation. For example, frameworks can use labels to
> identify
> +the intended purpose for a portion of the resources that have been
> reserved at a
> +given slave. Note that two reservations with different labels will not be
> +combined together into a single reservation, even if the reservations are
> at the
> +same slave and use the same role.
> +
>
>  ### Framework Scheduler API
>
>
>

[2/2] mesos git commit: Added documentation for labeled reserved resources.

Posted by mp...@apache.org.
Added documentation for labeled reserved resources.

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


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

Branch: refs/heads/master
Commit: 3b02b80fae886caccd242f5fc205e91a42723861
Parents: 77448c0
Author: Neil Conway <ne...@gmail.com>
Authored: Thu Feb 11 16:07:05 2016 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Thu Feb 11 16:07:05 2016 -0800

----------------------------------------------------------------------
 docs/reservation.md | 27 ++++++++++++++++++++-------
 1 file changed, 20 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3b02b80f/docs/reservation.md
----------------------------------------------------------------------
diff --git a/docs/reservation.md b/docs/reservation.md
index 51fa7fc..d3ae445 100644
--- a/docs/reservation.md
+++ b/docs/reservation.md
@@ -65,13 +65,26 @@ see the [authorization documentation](authorization.md).
 In the following sections, we will walk through examples of each of the
 interfaces described above.
 
-Note that if two dynamic reservations are made for resources at a single slave,
-the reservations will be combined by adding together the resources reserved by
-each request. Similarly, "partial" unreserve operations are allowed: an
-unreserve operation can release only some of the resources at a slave that have
-been reserved for a given role. In this case, the unreserved resources will be
-subtracted from the previous reservation, and any remaining resources will still
-be reserved.
+If two dynamic reservations are made for the same role at a single slave (using
+the same labels, if any; see below), the reservations will be combined by adding
+together the resources reserved by each request. This will result in a single
+reserved resource at the slave. Similarly, "partial" unreserve operations are
+allowed: an unreserve operation can release some but not all of the resources at
+a slave that have been reserved for a role. In this case, the unreserved
+resources will be subtracted from the previous reservation and any remaining
+resources will still be reserved.
+
+
+### Labeled Reservations
+
+Dynamic reservations can optionally include a list of _labels_, which are
+arbitrary key-value pairs. Labels can be used to associate arbitrary metadata
+with a resource reservation. For example, frameworks can use labels to identify
+the intended purpose for a portion of the resources that have been reserved at a
+given slave. Note that two reservations with different labels will not be
+combined together into a single reservation, even if the reservations are at the
+same slave and use the same role.
+
 
 ### Framework Scheduler API