You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by bm...@apache.org on 2017/02/04 03:02:11 UTC

[9/9] mesos git commit: Update the tests to handle MULTI_ROLE support.

Update the tests to handle MULTI_ROLE support.

A number of tests and example frameworks assume that allocated
resources did not look different from unallocated resources.
This updates the tests to reflect the presence of
`Resource.AllocationInfo`.

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


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

Branch: refs/heads/master
Commit: c20744a9976b5e83698e9c6062218abb4d2e6b25
Parents: 7e97653
Author: Benjamin Mahler <bm...@apache.org>
Authored: Wed Jan 25 17:14:36 2017 -0800
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Fri Feb 3 19:00:39 2017 -0800

----------------------------------------------------------------------
 src/examples/disk_full_framework.cpp            |  22 ++-
 src/examples/dynamic_reservation_framework.cpp  |  27 +--
 src/examples/no_executor_framework.cpp          |  30 +--
 src/examples/persistent_volume_framework.cpp    |   8 +-
 src/examples/test_framework.cpp                 |   7 +-
 src/examples/test_http_framework.cpp            |   7 +-
 src/tests/api_tests.cpp                         |  23 ++-
 .../containerizer/cgroups_isolator_tests.cpp    |   4 +-
 src/tests/hook_tests.cpp                        |  22 ++-
 src/tests/master_allocator_tests.cpp            |  44 +++--
 src/tests/master_tests.cpp                      |  14 +-
 src/tests/mesos.hpp                             |   7 +-
 src/tests/oversubscription_tests.cpp            |  35 ++--
 src/tests/partition_tests.cpp                   |   1 +
 src/tests/persistent_volume_endpoints_tests.cpp |  79 ++++++--
 src/tests/persistent_volume_tests.cpp           |  86 ++++++---
 src/tests/reservation_endpoints_tests.cpp       |  76 +++++---
 src/tests/reservation_tests.cpp                 | 186 ++++++++++++-------
 src/tests/resource_offers_tests.cpp             |   3 +-
 src/tests/role_tests.cpp                        |  25 ++-
 src/tests/slave_recovery_tests.cpp              |  11 +-
 21 files changed, 476 insertions(+), 241 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/examples/disk_full_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/disk_full_framework.cpp b/src/examples/disk_full_framework.cpp
index e13d4c8..a73e6cf 100644
--- a/src/examples/disk_full_framework.cpp
+++ b/src/examples/disk_full_framework.cpp
@@ -111,8 +111,11 @@ class DiskFullSchedulerProcess
   : public process::Process<DiskFullSchedulerProcess>
 {
 public:
-  DiskFullSchedulerProcess (const Flags& _flags)
+  DiskFullSchedulerProcess (
+      const Flags& _flags,
+      const FrameworkInfo& _frameworkInfo)
     : flags(_flags),
+      frameworkInfo(_frameworkInfo),
       tasksLaunched(0),
       taskActive(false),
       isRegistered(false),
@@ -135,10 +138,11 @@ public:
       SchedulerDriver* driver,
       const std::vector<Offer>& offers)
   {
-    static const Resources TASK_RESOURCES = Resources::parse(
+    Resources taskResources = Resources::parse(
         "cpus:" + stringify(CPUS_PER_TASK) +
         ";mem:" + stringify(MEMORY_PER_TASK) +
         ";disk:" + stringify(DISK_PER_TASK.megabytes())).get();
+    taskResources.allocate(frameworkInfo.role());
 
     foreach (const Offer& offer, offers) {
       LOG(INFO) << "Received offer " << offer.id() << " from agent "
@@ -149,7 +153,7 @@ public:
 
       // If we've already launched the task, or if the offer is not
       // big enough, reject the offer.
-      if (taskActive || !resources.flatten().contains(TASK_RESOURCES)) {
+      if (taskActive || !resources.flatten().contains(taskResources)) {
         Filters filters;
         filters.set_refuse_seconds(600);
 
@@ -173,7 +177,7 @@ public:
       task.set_name("Disk full framework task");
       task.mutable_task_id()->set_value(stringify(taskId));
       task.mutable_slave_id()->MergeFrom(offer.slave_id());
-      task.mutable_resources()->CopyFrom(TASK_RESOURCES);
+      task.mutable_resources()->CopyFrom(taskResources);
       task.mutable_command()->set_shell(true);
       task.mutable_command()->set_value(command);
 
@@ -252,6 +256,8 @@ public:
 
 private:
   const Flags flags;
+  const FrameworkInfo frameworkInfo;
+
   int tasksLaunched;
   bool taskActive;
 
@@ -313,8 +319,8 @@ private:
 class DiskFullScheduler : public Scheduler
 {
 public:
-  DiskFullScheduler(const Flags& _flags)
-    : process(_flags)
+  DiskFullScheduler(const Flags& _flags, const FrameworkInfo& _frameworkInfo)
+    : process(_flags, _frameworkInfo)
   {
     process::spawn(process);
   }
@@ -426,13 +432,13 @@ int main(int argc, char** argv)
     EXIT(EXIT_FAILURE) << flags.usage(load.error());
   }
 
-  DiskFullScheduler scheduler(flags);
-
   FrameworkInfo framework;
   framework.set_user(""); // Have Mesos fill the current user.
   framework.set_name("Disk Full Framework (C++)");
   framework.set_checkpoint(true);
 
+  DiskFullScheduler scheduler(flags, framework);
+
   MesosSchedulerDriver* driver;
 
   // TODO(hartem): Refactor these into a common set of flags.

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/examples/dynamic_reservation_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/dynamic_reservation_framework.cpp b/src/examples/dynamic_reservation_framework.cpp
index 4d3db96..c1650dc 100644
--- a/src/examples/dynamic_reservation_framework.cpp
+++ b/src/examples/dynamic_reservation_framework.cpp
@@ -62,7 +62,14 @@ public:
   {
     reservationInfo.set_principal(principal);
 
-    Try<Resources> flattened = TASK_RESOURCES.flatten(role, reservationInfo);
+    taskResources = Resources::parse(
+        "cpus:" + stringify(CPUS_PER_TASK) +
+        ";mem:" + stringify(MEM_PER_TASK)).get();
+
+    taskResources.allocate(role);
+
+    // The task will run on reserved resources.
+    Try<Resources> flattened = taskResources.flatten(role, reservationInfo);
     CHECK_SOME(flattened);
     taskResources = flattened.get();
   }
@@ -111,13 +118,16 @@ public:
           // the task'll be dispatched when reserved resources are re-offered
           // to this framework.
           Resources resources = offer.resources();
-          Resources unreserved = resources.unreserved();
-          if (!unreserved.contains(TASK_RESOURCES)) {
+          Offer::Operation reserve = RESERVE(taskResources);
+
+          Try<Resources> apply = resources.apply(reserve);
+          if (apply.isError()) {
             LOG(INFO) << "Failed to reserve resources for task in offer "
-                      << stringify(offer.id());
+                      << stringify(offer.id()) << ": " << apply.error();
             break;
           }
-          driver->acceptOffers({offer.id()}, {RESERVE(taskResources)}, filters);
+
+          driver->acceptOffers({offer.id()}, {reserve}, filters);
           states[offer.slave_id()] = State::RESERVING;
           break;
         }
@@ -293,8 +303,6 @@ private:
   Resource::ReservationInfo reservationInfo;
   Resources taskResources;
 
-  static const Resources TASK_RESOURCES;
-
   Offer::Operation RESERVE(Resources resources)
   {
     Offer::Operation operation;
@@ -313,11 +321,6 @@ private:
 };
 
 
-const Resources DynamicReservationScheduler::TASK_RESOURCES = Resources::parse(
-    "cpus:" + stringify(CPUS_PER_TASK) +
-    ";mem:" + stringify(MEM_PER_TASK)).get();
-
-
 class Flags : public virtual flags::FlagsBase
 {
 public:

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/examples/no_executor_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/no_executor_framework.cpp b/src/examples/no_executor_framework.cpp
index e82ae9a..77b7408 100644
--- a/src/examples/no_executor_framework.cpp
+++ b/src/examples/no_executor_framework.cpp
@@ -325,6 +325,20 @@ int main(int argc, char** argv)
                      " to enable authentication");
   }
 
+  FrameworkInfo framework;
+  framework.set_user(""); // Have Mesos fill in the current user.
+  framework.set_name("No Executor Framework");
+  framework.set_checkpoint(flags.checkpoint);
+
+  if (flags.task_revocable_resources.isSome()) {
+    framework.add_capabilities()->set_type(
+        FrameworkInfo::Capability::REVOCABLE_RESOURCES);
+  }
+
+  if (flags.principal.isSome()) {
+    framework.set_principal(flags.principal.get());
+  }
+
   Try<Resources> resources =
     Resources::parse(flags.task_resources);
 
@@ -352,21 +366,9 @@ int main(int argc, char** argv)
     }
   }
 
-  logging::initialize(argv[0], flags, true); // Catch signals.
+  taskResources.allocate(framework.role());
 
-  FrameworkInfo framework;
-  framework.set_user(""); // Have Mesos fill in the current user.
-  framework.set_name("No Executor Framework");
-  framework.set_checkpoint(flags.checkpoint);
-
-  if (flags.task_revocable_resources.isSome()) {
-    framework.add_capabilities()->set_type(
-        FrameworkInfo::Capability::REVOCABLE_RESOURCES);
-  }
-
-  if (flags.principal.isSome()) {
-    framework.set_principal(flags.principal.get());
-  }
+  logging::initialize(argv[0], flags, true); // Catch signals.
 
   NoExecutorScheduler scheduler(
       framework,

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/examples/persistent_volume_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/persistent_volume_framework.cpp b/src/examples/persistent_volume_framework.cpp
index 222018e..702dba3 100644
--- a/src/examples/persistent_volume_framework.cpp
+++ b/src/examples/persistent_volume_framework.cpp
@@ -57,7 +57,12 @@ using std::vector;
 // reserved resources.
 static Resources SHARD_INITIAL_RESOURCES(const string& role)
 {
-  return Resources::parse("cpus:0.1;mem:32;disk:16", role).get();
+  Resources allocation =
+    Resources::parse("cpus:0.1;mem:32;disk:16", role).get();
+
+  allocation.allocate(role);
+
+  return allocation;
 }
 
 
@@ -79,6 +84,7 @@ static Resource SHARD_PERSISTENT_VOLUME(
 
   Resource resource = Resources::parse("disk", "8", role).get();
   resource.mutable_disk()->CopyFrom(info);
+  resource.mutable_allocation_info()->set_role(role);
 
   if (isShared) {
     resource.mutable_shared();

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/examples/test_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_framework.cpp b/src/examples/test_framework.cpp
index 068b85d..05ddc89 100644
--- a/src/examples/test_framework.cpp
+++ b/src/examples/test_framework.cpp
@@ -85,16 +85,17 @@ public:
       cout << "Received offer " << offer.id() << " with " << offer.resources()
            << endl;
 
-      static const Resources TASK_RESOURCES = Resources::parse(
+      Resources taskResources = Resources::parse(
           "cpus:" + stringify(CPUS_PER_TASK) +
           ";mem:" + stringify(MEM_PER_TASK)).get();
+      taskResources.allocate(role);
 
       Resources remaining = offer.resources();
 
       // Launch tasks.
       vector<TaskInfo> tasks;
       while (tasksLaunched < totalTasks &&
-             remaining.flatten().contains(TASK_RESOURCES)) {
+             remaining.flatten().contains(taskResources)) {
         int taskId = tasksLaunched++;
 
         cout << "Launching task " << taskId << " using offer "
@@ -106,7 +107,7 @@ public:
         task.mutable_slave_id()->MergeFrom(offer.slave_id());
         task.mutable_executor()->MergeFrom(executor);
 
-        Try<Resources> flattened = TASK_RESOURCES.flatten(role);
+        Try<Resources> flattened = taskResources.flatten(role);
         CHECK_SOME(flattened);
         Option<Resources> resources = remaining.find(flattened.get());
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/examples/test_http_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_http_framework.cpp b/src/examples/test_http_framework.cpp
index 258cb51..471835c 100644
--- a/src/examples/test_http_framework.cpp
+++ b/src/examples/test_http_framework.cpp
@@ -222,16 +222,17 @@ private:
            << Resources(offer.resources())
            << endl;
 
-      static const Resources TASK_RESOURCES = Resources::parse(
+      Resources taskResources = Resources::parse(
           "cpus:" + stringify(CPUS_PER_TASK) +
           ";mem:" + stringify(MEM_PER_TASK)).get();
+      taskResources.allocate(framework.role());
 
       Resources remaining = offer.resources();
 
       // Launch tasks.
       vector<TaskInfo> tasks;
       while (tasksLaunched < totalTasks &&
-             remaining.flatten().contains(TASK_RESOURCES)) {
+             remaining.flatten().contains(taskResources)) {
         int taskId = tasksLaunched++;
 
         cout << "Launching task " << taskId << " using offer "
@@ -244,7 +245,7 @@ private:
         task.mutable_agent_id()->MergeFrom(offer.agent_id());
         task.mutable_executor()->MergeFrom(executor);
 
-        Try<Resources> flattened = TASK_RESOURCES.flatten(framework.role());
+        Try<Resources> flattened = taskResources.flatten(framework.role());
         CHECK_SOME(flattened);
         Option<Resources> resources = remaining.find(flattened.get());
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 4f43194..277fbe3 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -57,6 +57,7 @@
 #include "tests/allocator.hpp"
 #include "tests/containerizer.hpp"
 #include "tests/mesos.hpp"
+#include "tests/resources_utils.hpp"
 
 #include "tests/containerizer/mock_containerizer.hpp"
 
@@ -894,8 +895,11 @@ TEST_P(MasterAPITest, GetRoles)
   ASSERT_EQ(2, v1Response->get_roles().roles().size());
   EXPECT_EQ("role1", v1Response->get_roles().roles(1).name());
   EXPECT_EQ(2.5, v1Response->get_roles().roles(1).weight());
-  ASSERT_EQ(v1::Resources::parse(slaveFlags.resources.get()).get(),
-            v1Response->get_roles().roles(1).resources());
+  ASSERT_EQ(
+      allocatedResources(
+          devolve(v1::Resources::parse(slaveFlags.resources.get()).get()),
+          "role1"),
+      devolve(v1Response->get_roles().roles(1).resources()));
 
   driver.stop();
   driver.join();
@@ -973,7 +977,8 @@ TEST_P(MasterAPITest, ReserveResources)
   ASSERT_EQ(1u, offers->size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
@@ -1006,7 +1011,8 @@ TEST_P(MasterAPITest, ReserveResources)
   ASSERT_EQ(1u, offers->size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -1083,7 +1089,8 @@ TEST_P(MasterAPITest, UnreserveResources)
   ASSERT_EQ(1u, offers->size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
@@ -1116,7 +1123,8 @@ TEST_P(MasterAPITest, UnreserveResources)
   offer = offers.get()[0];
 
   // Verifies if the resources are unreserved.
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -2022,7 +2030,8 @@ TEST_P(MasterAPITest, CreateAndDestroyVolumes)
   EXPECT_NE(0u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   Resources taskResources = Resources::parse(
       "disk:256",

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/tests/containerizer/cgroups_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/cgroups_isolator_tests.cpp b/src/tests/containerizer/cgroups_isolator_tests.cpp
index ba268b6..b86716d 100644
--- a/src/tests/containerizer/cgroups_isolator_tests.cpp
+++ b/src/tests/containerizer/cgroups_isolator_tests.cpp
@@ -28,6 +28,7 @@
 
 #include "tests/mesos.hpp"
 #include "tests/mock_slave.hpp"
+#include "tests/resources_utils.hpp"
 #include "tests/script.hpp"
 
 #include "tests/containerizer/docker_archive.hpp"
@@ -304,7 +305,8 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_RevocableCpu)
   // Now the framework will get revocable resources.
   AWAIT_READY(offers2);
   EXPECT_NE(0u, offers2.get().size());
-  EXPECT_EQ(cpus, Resources(offers2.get()[0].resources()));
+  EXPECT_EQ(allocatedResources(cpus, frameworkInfo.role()),
+            Resources(offers2.get()[0].resources()));
 
   TaskInfo task = createTask(
       offers2.get()[0].slave_id(),

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/tests/hook_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hook_tests.cpp b/src/tests/hook_tests.cpp
index f4ef629..237df81 100644
--- a/src/tests/hook_tests.cpp
+++ b/src/tests/hook_tests.cpp
@@ -51,6 +51,7 @@
 #include "tests/flags.hpp"
 #include "tests/mesos.hpp"
 #include "tests/mock_docker.hpp"
+#include "tests/resources_utils.hpp"
 
 using namespace mesos::modules;
 
@@ -1106,12 +1107,21 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 
   Resources resources = offers.get()[0].resources();
 
-  // The test hook sets "cpus" to 4.
-  EXPECT_EQ(4, resources.cpus().get());
-
-  // The test hook adds a resource named "foo" of type set with values "bar"
-  // and "baz".
-  EXPECT_EQ(Resources::parse("foo:{bar,baz}").get(), resources.get("foo"));
+  // The test hook sets "cpus" to 4 and adds a resource named
+  // "foo" of type set with values "bar" and "baz".
+  //
+  // TODO(bmahler): Avoid the need for non-local reasoning here
+  // about the test hook. E.g. Expose the resources from the test
+  // hook and use them here.
+  const size_t TEST_HOOK_CPUS = 4;
+  const Resources TEST_HOOK_ADDITIONAL_RESOURCES =
+    Resources::parse("foo:{bar,baz}").get();
+
+  EXPECT_EQ(TEST_HOOK_CPUS, resources.cpus().get());
+
+  const string allocationRole = DEFAULT_FRAMEWORK_INFO.role();
+  EXPECT_TRUE(resources.contains(
+      allocatedResources(TEST_HOOK_ADDITIONAL_RESOURCES, allocationRole)));
 
   // The test hook does not modify "mem", the default value must still be
   // present.

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/tests/master_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_allocator_tests.cpp b/src/tests/master_allocator_tests.cpp
index d22862d..25c67d3 100644
--- a/src/tests/master_allocator_tests.cpp
+++ b/src/tests/master_allocator_tests.cpp
@@ -50,6 +50,7 @@
 #include "tests/containerizer.hpp"
 #include "tests/mesos.hpp"
 #include "tests/module.hpp"
+#include "tests/resources_utils.hpp"
 
 using google::protobuf::RepeatedPtrField;
 
@@ -817,7 +818,9 @@ TYPED_TEST(MasterAllocatorTest, SlaveLost)
   AWAIT_READY(resourceOffers);
 
   EXPECT_EQ(Resources(resourceOffers.get()[0].resources()),
-            Resources::parse(flags2.resources.get()).get());
+            allocatedResources(
+                Resources::parse(flags2.resources.get()).get(),
+                DEFAULT_FRAMEWORK_INFO.role()));
 
   // Shut everything down.
   EXPECT_CALL(allocator, recoverResources(_, _, _, _))
@@ -1604,8 +1607,9 @@ TYPED_TEST(MasterAllocatorTest, RebalancedForUpdatedWeights)
   vector<Owned<cluster::Slave>> slaves;
 
   // Register three agents with the same resources.
-  string agentResources = "cpus:2;gpus:0;mem:1024;"
-                          "disk:4096;ports:[31000-32000]";
+  const Resources agentResources = Resources::parse(
+      "cpus:2;gpus:0;mem:1024;disk:4096;ports:[31000-32000]").get();
+
   for (int i = 0; i < 3; i++) {
     Future<Nothing> addSlave;
     EXPECT_CALL(allocator, addSlave(_, _, _, _, _))
@@ -1613,7 +1617,7 @@ TYPED_TEST(MasterAllocatorTest, RebalancedForUpdatedWeights)
                       FutureSatisfy(&addSlave)));
 
     slave::Flags flags = this->CreateSlaveFlags();
-    flags.resources = agentResources;
+    flags.resources = stringify(agentResources);
 
     Try<Owned<cluster::Slave>> slave = this->StartSlave(detector.get(), flags);
     ASSERT_SOME(slave);
@@ -1656,7 +1660,7 @@ TYPED_TEST(MasterAllocatorTest, RebalancedForUpdatedWeights)
   ASSERT_EQ(3u, framework1offers1.get().size());
   for (int i = 0; i < 3; i++) {
     EXPECT_EQ(Resources(framework1offers1.get()[i].resources()),
-              Resources::parse(agentResources).get());
+              allocatedResources(agentResources, "role1"));
   }
 
   // Framework2 registers with 'role2' which also uses the default weight.
@@ -1718,15 +1722,29 @@ TYPED_TEST(MasterAllocatorTest, RebalancedForUpdatedWeights)
   // 'updateWeights' will rescind all outstanding offers and the rescinded
   // offer resources will only be available to the updated weights once
   // another allocation is invoked.
+  //
+  // TODO(bmahler): This lambda is copied in several places
+  // in the code, consider how best to pull this out.
+  auto unallocated = [](const Resources& resources) {
+    Resources result = resources;
+    result.unallocate();
+    return result;
+  };
+
   AWAIT_READY(recoverResources1);
-  EXPECT_EQ(recoverResources1.get(),
-            Resources::parse(agentResources).get());
+  EXPECT_EQ(agentResources, unallocated(recoverResources1.get()));
+
   AWAIT_READY(recoverResources2);
-  EXPECT_EQ(recoverResources2.get(),
-            Resources::parse(agentResources).get());
+  EXPECT_EQ(agentResources, unallocated(recoverResources2.get()));
+
   AWAIT_READY(recoverResources3);
-  EXPECT_EQ(recoverResources3.get(),
-            Resources::parse(agentResources).get());
+  EXPECT_EQ(agentResources, unallocated(recoverResources3.get()));
+
+  Resources totalRecovered =
+    recoverResources1.get() + recoverResources2.get() + recoverResources3.get();
+  totalRecovered.unallocate();
+
+  EXPECT_EQ(agentResources + agentResources + agentResources, totalRecovered);
 
   // Trigger a batch allocation to make sure all resources are
   // offered out again.
@@ -1743,7 +1761,7 @@ TYPED_TEST(MasterAllocatorTest, RebalancedForUpdatedWeights)
   AWAIT_READY(framework1offers2);
   ASSERT_EQ(1u, framework1offers2.get().size());
   EXPECT_EQ(Resources(framework1offers2.get()[0].resources()),
-            Resources::parse(agentResources).get());
+            allocatedResources(agentResources, "role1"));
 
   ASSERT_EQ(2u, framework2offers.size());
   for (int i = 0; i < 2; i++) {
@@ -1752,7 +1770,7 @@ TYPED_TEST(MasterAllocatorTest, RebalancedForUpdatedWeights)
     // All offers for framework2 are enqueued by now.
     AWAIT_READY(offer);
     EXPECT_EQ(Resources(offer->resources()),
-              Resources::parse(agentResources).get());
+              allocatedResources(agentResources, "role2"));
   }
 
   driver1.stop();

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index da7094d..3b4123b 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -70,6 +70,7 @@
 #include "tests/containerizer.hpp"
 #include "tests/limiter.hpp"
 #include "tests/mesos.hpp"
+#include "tests/resources_utils.hpp"
 #include "tests/utils.hpp"
 
 using mesos::internal::master::Master;
@@ -842,8 +843,9 @@ TEST_F(MasterTest, RecoverResources)
   ExecutorInfo executorInfo;
   executorInfo.MergeFrom(DEFAULT_EXECUTOR_INFO);
 
-  Resources executorResources =
-    Resources::parse("cpus:0.3;mem:200;ports:[5-8, 23-25]").get();
+  Resources executorResources = allocatedResources(
+      Resources::parse("cpus:0.3;mem:200;ports:[5-8, 23-25]").get(),
+      DEFAULT_FRAMEWORK_INFO.role());
   executorInfo.mutable_resources()->MergeFrom(executorResources);
 
   TaskID taskId;
@@ -924,8 +926,10 @@ TEST_F(MasterTest, RecoverResources)
 
   AWAIT_READY(offers);
   EXPECT_NE(0u, offers.get().size());
+
   Resources slaveResources = Resources::parse(flags.resources.get()).get();
-  EXPECT_EQ(slaveResources, offers.get()[0].resources());
+  EXPECT_EQ(allocatedResources(slaveResources, DEFAULT_FRAMEWORK_INFO.role()),
+            offers.get()[0].resources());
 
   driver.stop();
   driver.join();
@@ -3371,7 +3375,9 @@ TEST_F(MasterTest, IgnoreEphemeralPortsResource)
 
   EXPECT_EQ(
       Resources(offers.get()[0].resources()),
-      Resources::parse(resourcesWithoutEphemeralPorts).get());
+      allocatedResources(
+          Resources::parse(resourcesWithoutEphemeralPorts).get(),
+          DEFAULT_FRAMEWORK_INFO.role()));
 
   driver.stop();
   driver.join();

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index ff83a9c..b450a04 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -1558,14 +1558,15 @@ ACTION_P5(LaunchTasks, executor, tasks, cpus, mem, role)
   for (size_t i = 0; i < offers.size(); i++) {
     const Offer& offer = offers[i];
 
-    const Resources TASK_RESOURCES = Resources::parse(
+    Resources taskResources = Resources::parse(
         "cpus:" + stringify(cpus) + ";mem:" + stringify(mem)).get();
+    taskResources.allocate(role);
 
     int nextTaskId = 0;
     std::vector<TaskInfo> tasks;
     Resources remaining = offer.resources();
 
-    while (remaining.flatten().contains(TASK_RESOURCES) &&
+    while (remaining.flatten().contains(taskResources) &&
            launched < numTasks) {
       TaskInfo task;
       task.set_name("TestTask");
@@ -1574,7 +1575,7 @@ ACTION_P5(LaunchTasks, executor, tasks, cpus, mem, role)
       task.mutable_executor()->MergeFrom(executor);
 
       Option<Resources> resources =
-        remaining.find(TASK_RESOURCES.flatten(role).get());
+        remaining.find(taskResources.flatten(role).get());
 
       CHECK_SOME(resources);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/tests/oversubscription_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/oversubscription_tests.cpp b/src/tests/oversubscription_tests.cpp
index 167beaf..e57fcc6 100644
--- a/src/tests/oversubscription_tests.cpp
+++ b/src/tests/oversubscription_tests.cpp
@@ -50,6 +50,7 @@
 #include "tests/containerizer.hpp"
 #include "tests/mesos.hpp"
 #include "tests/mock_slave.hpp"
+#include "tests/resources_utils.hpp"
 #include "tests/utils.hpp"
 
 using namespace process;
@@ -398,9 +399,13 @@ TEST_F(OversubscriptionTest, RevocableOffer)
     .WillRepeatedly(Return()); // Ignore subsequent offers.
 
   // Inject an estimation of oversubscribable cpu resources.
+  estimations.put(createRevocableResources("cpus", "2"));
+
   Resources taskResources = createRevocableResources("cpus", "1");
+  taskResources.allocate(framework.role());
+
   Resources executorResources = createRevocableResources("cpus", "1");
-  estimations.put(taskResources + executorResources);
+  executorResources.allocate(framework.role());
 
   // Now the framework will get revocable resources.
   AWAIT_READY(offers2);
@@ -506,7 +511,8 @@ TEST_F(OversubscriptionTest, RescindRevocableOfferWithIncreasedRevocable)
   EXPECT_EQ(1u, offers.size());
   Future<Offer> offer = offers.get();
   AWAIT_READY(offer);
-  EXPECT_EQ(resources1, Resources(offer->resources()));
+  EXPECT_EQ(allocatedResources(resources1, framework.role()),
+            Resources(offer->resources()));
 
   Future<OfferID> offerId;
   EXPECT_CALL(sched, offerRescinded(&driver, _))
@@ -543,7 +549,7 @@ TEST_F(OversubscriptionTest, RescindRevocableOfferWithIncreasedRevocable)
   }
 
   // The offered resources should match the resource estimate.
-  EXPECT_EQ(resources2, resources3);
+  EXPECT_EQ(allocatedResources(resources2, framework.role()), resources3);
 
   driver.stop();
   driver.join();
@@ -619,7 +625,8 @@ TEST_F(OversubscriptionTest, RescindRevocableOfferWithDecreasedRevocable)
   // Now the framework will get revocable resources.
   AWAIT_READY(offers2);
   EXPECT_NE(0u, offers2->size());
-  EXPECT_EQ(resources1, Resources(offers2.get()[0].resources()));
+  EXPECT_EQ(allocatedResources(resources1, framework.role()),
+            Resources(offers2.get()[0].resources()));
 
   Future<OfferID> offerId;
   EXPECT_CALL(sched, offerRescinded(&driver, _))
@@ -649,7 +656,8 @@ TEST_F(OversubscriptionTest, RescindRevocableOfferWithDecreasedRevocable)
   // The new offer should include the latest oversubscribed resources.
   AWAIT_READY(offers3);
   EXPECT_NE(0u, offers3->size());
-  EXPECT_EQ(resources2, Resources(offers3.get()[0].resources()));
+  EXPECT_EQ(allocatedResources(resources2, framework.role()),
+            Resources(offers3.get()[0].resources()));
 
   driver.stop();
   driver.join();
@@ -1274,13 +1282,12 @@ TEST_F(OversubscriptionTest, UpdateAllocatorOnSchedulerFailover)
   EXPECT_CALL(sched2, resourceOffers(&driver2, _))
     .WillOnce(FutureArg<1>(&offers2));
 
-  Resources taskResources = createRevocableResources("cpus", "1");
-  Resources executorResources = createRevocableResources("cpus", "1");
-  estimations.put(taskResources + executorResources);
+  Resources revocable = createRevocableResources("cpus", "2");
+  estimations.put(revocable);
 
   AWAIT_READY(offers2);
   EXPECT_NE(0u, offers2.get().size());
-  EXPECT_EQ(taskResources + executorResources,
+  EXPECT_EQ(allocatedResources(revocable, framework2.role()),
             Resources(offers2.get()[0].resources()));
 
   EXPECT_EQ(DRIVER_STOPPED, driver2.stop());
@@ -1347,16 +1354,14 @@ TEST_F(OversubscriptionTest, RemoveCapabilitiesOnSchedulerFailover)
     .WillRepeatedly(Return()); // Ignore subsequent offers.
 
   // Inject an estimation of oversubscribable cpu resources.
-  Resources taskResources = createRevocableResources("cpus", "1");
-  Resources executorResources = createRevocableResources("cpus", "1");
-  estimations.put(taskResources + executorResources);
+  Resources revocable = createRevocableResources("cpus", "2");
+  estimations.put(revocable);
 
   // Now the framework will get revocable resources.
   AWAIT_READY(offers2);
   EXPECT_NE(0u, offers2.get().size());
-  EXPECT_EQ(
-      taskResources + executorResources,
-      Resources(offers2.get()[0].resources()));
+  EXPECT_EQ(allocatedResources(revocable, framework1.role()),
+            Resources(offers2.get()[0].resources()));
 
   // Reregister the framework with removal of revocable resources capability.
   FrameworkInfo framework2 = DEFAULT_FRAMEWORK_INFO;

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/tests/partition_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/partition_tests.cpp b/src/tests/partition_tests.cpp
index f03c5bd..105157d 100644
--- a/src/tests/partition_tests.cpp
+++ b/src/tests/partition_tests.cpp
@@ -869,6 +869,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   Offer offer = offers.get()[0];
 
   Resources taskResources = Resources::parse("cpus:1;mem:512").get();
+  taskResources.allocate(DEFAULT_FRAMEWORK_INFO.role());
 
   EXPECT_TRUE(Resources(offer.resources()).contains(taskResources));
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/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 695029f..ec8df33 100644
--- a/src/tests/persistent_volume_endpoints_tests.cpp
+++ b/src/tests/persistent_volume_endpoints_tests.cpp
@@ -38,6 +38,7 @@
 
 #include "tests/allocator.hpp"
 #include "tests/mesos.hpp"
+#include "tests/resources_utils.hpp"
 #include "tests/utils.hpp"
 
 using std::string;
@@ -169,7 +170,8 @@ TEST_F(PersistentVolumeEndpointsTest, StaticReservation)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   Future<OfferID> rescindedOfferId;
 
@@ -196,7 +198,8 @@ TEST_F(PersistentVolumeEndpointsTest, StaticReservation)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_FALSE(Resources(offer.resources()).contains(volume));
+  EXPECT_FALSE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -262,7 +265,8 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservation)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   Future<OfferID> rescindedOfferId;
 
@@ -298,7 +302,8 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservation)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   EXPECT_CALL(sched, offerRescinded(&driver, _))
     .WillOnce(FutureArg<1>(&rescindedOfferId));
@@ -379,7 +384,8 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservationRoleMismatch)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   ASSERT_NE(frameworkInfo.role(), "role2");
   Resources volume = createPersistentVolume(
@@ -896,7 +902,8 @@ TEST_F(PersistentVolumeEndpointsTest, GoodCreateAndDestroyACL)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   Future<OfferID> rescindedOfferId;
 
@@ -926,7 +933,8 @@ TEST_F(PersistentVolumeEndpointsTest, GoodCreateAndDestroyACL)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_FALSE(Resources(offer.resources()).contains(volume));
+  EXPECT_FALSE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -1124,7 +1132,8 @@ TEST_F(PersistentVolumeEndpointsTest, BadCreateAndDestroyACL)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   // The failed destruction attempt.
   Future<Response> destroyResponse = process::http::post(
@@ -1331,7 +1340,8 @@ TEST_F(PersistentVolumeEndpointsTest, GoodCreateAndDestroyACLBadCredential)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   // The failed destruction attempt.
   Future<Response> destroyResponse = process::http::post(
@@ -1603,7 +1613,8 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   // Create a 1MB persistent volume.
   Resources volume = createPersistentVolume(
@@ -1627,7 +1638,8 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   Future<OfferID> rescindedOfferId;
 
@@ -1655,7 +1667,8 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   EXPECT_CALL(sched, offerRescinded(&driver, _))
     .WillOnce(FutureArg<1>(&rescindedOfferId));
@@ -1681,7 +1694,8 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -1765,7 +1779,8 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   // We use the filter explicitly here so that the resources will not
   // be filtered for 5 seconds (the default).
@@ -1786,7 +1801,8 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
@@ -1800,7 +1816,8 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -2018,7 +2035,8 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   Resources taskUnreserved = Resources::parse("cpus:1;mem:256").get();
   Resources taskResources = taskUnreserved.flatten(
@@ -2140,6 +2158,9 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
       R"~(
       [
         {
+          "allocation_info": {
+            "role": "role1"
+          },
           "name": "cpus",
           "reservation": {
             "principal": "test-principal"
@@ -2151,6 +2172,9 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
           "type": "SCALAR"
         },
         {
+          "allocation_info": {
+            "role": "role1"
+          },
           "name": "mem",
           "reservation": {
             "principal": "test-principal"
@@ -2169,6 +2193,9 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
       R"~(
       [
         {
+          "allocation_info": {
+            "role": "role1"
+          },
           "disk": {
             "persistence": {
               "id": "id1",
@@ -2190,6 +2217,9 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
           "type": "SCALAR"
         },
         {
+          "allocation_info": {
+            "role": "role1"
+          },
           "name": "mem",
           "reservation": {
             "principal": "test-principal"
@@ -2201,6 +2231,9 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
           "type": "SCALAR"
         },
         {
+          "allocation_info": {
+            "role": "role1"
+          },
           "name": "disk",
           "reservation": {
             "principal": "test-principal"
@@ -2212,6 +2245,9 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
           "type": "SCALAR"
         },
         {
+          "allocation_info": {
+            "role": "role1"
+          },
           "name": "cpus",
           "role": "*",
           "scalar": {
@@ -2220,6 +2256,9 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
           "type": "SCALAR"
         },
         {
+          "allocation_info": {
+            "role": "role1"
+          },
           "name": "mem",
           "role": "*",
           "scalar": {
@@ -2228,6 +2267,9 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
           "type": "SCALAR"
         },
         {
+          "allocation_info": {
+            "role": "role1"
+          },
           "name": "disk",
           "role": "*",
           "scalar": {
@@ -2236,6 +2278,9 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
           "type": "SCALAR"
         },
         {
+          "allocation_info": {
+            "role": "role1"
+          },
           "name": "ports",
           "ranges": {
             "range": [

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/tests/persistent_volume_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/persistent_volume_tests.cpp b/src/tests/persistent_volume_tests.cpp
index 842e113..7ac8286 100644
--- a/src/tests/persistent_volume_tests.cpp
+++ b/src/tests/persistent_volume_tests.cpp
@@ -54,6 +54,7 @@
 #include "tests/environment.hpp"
 #include "tests/mesos.hpp"
 #include "tests/mock_slave.hpp"
+#include "tests/resources_utils.hpp"
 
 using namespace process;
 
@@ -373,8 +374,10 @@ TEST_P(PersistentVolumeTest, CreateAndDestroyPersistentVolumes)
   offer = offers.get()[0];
 
   // Expect that the offer contains the persistent volumes we created.
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume1));
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume2));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume1, frameworkInfo.role())));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume2, frameworkInfo.role())));
 
   // Destroy `volume1`.
   driver.acceptOffers(
@@ -639,7 +642,8 @@ TEST_P(PersistentVolumeTest, MasterFailover)
 
   Offer offer2 = offers2.get()[0];
 
-  EXPECT_TRUE(Resources(offer2.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer2.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -864,7 +868,8 @@ TEST_P(PersistentVolumeTest, AccessPersistentVolume)
 
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   Future<CheckpointResourcesMessage> checkpointMessage =
     FUTURE_PROTOBUF(CheckpointResourcesMessage(), _, _);
@@ -1119,7 +1124,8 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeRescindOnDestroy)
 
   Offer offer2 = offers2.get()[0];
 
-  EXPECT_TRUE(Resources(offer2.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer2.resources()).contains(
+      allocatedResources(volume, frameworkInfo2.role())));
 
   // 4. framework1 kills the task which results in an offer to framework1
   //    with the shared volume. At this point, both frameworks will have
@@ -1141,7 +1147,8 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeRescindOnDestroy)
 
   offer1 = offers1.get()[0];
 
-  EXPECT_TRUE(Resources(offer1.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer1.resources()).contains(
+      allocatedResources(volume, frameworkInfo1.role())));
 
   // 5. DESTROY the shared volume via framework2 which would result in
   //    framework1 being rescinded the offer.
@@ -1282,8 +1289,11 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeMasterFailover)
 
   // Verify the offer from the failed over master.
   EXPECT_TRUE(Resources(offer2.resources()).contains(
-      Resources::parse("cpus:1;mem:1024").get()));
-  EXPECT_TRUE(Resources(offer2.resources()).contains(volume));
+      allocatedResources(
+          Resources::parse("cpus:1;mem:1024").get(),
+          frameworkInfo.role())));
+  EXPECT_TRUE(Resources(offer2.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -1434,11 +1444,19 @@ TEST_P(PersistentVolumeTest, DestroyPersistentVolumeMultipleTasks)
 
   offer = offers.get()[0];
 
+  // TODO(bmahler): This lambda is copied in several places
+  // in the code, consider how best to pull this out.
+  auto unallocated = [](const Resources& resources) {
+    Resources result = resources;
+    result.unallocate();
+    return result;
+  };
+
   // Check that the persistent volumes are offered back. The shared volume
   // is offered since it can be used in multiple tasks; the non-shared
   // volume is offered since there are no tasks using it.
-  EXPECT_TRUE(Resources(offer.resources()).contains(sharedVolume));
-  EXPECT_TRUE(Resources(offer.resources()).contains(nonSharedVolume));
+  EXPECT_TRUE(unallocated(offer.resources()).contains(sharedVolume));
+  EXPECT_TRUE(unallocated(offer.resources()).contains(nonSharedVolume));
 
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
@@ -1459,8 +1477,8 @@ TEST_P(PersistentVolumeTest, DestroyPersistentVolumeMultipleTasks)
 
   // Check that the shared persistent volume is in the offer, but the
   // non-shared volume is not in the offer.
-  EXPECT_TRUE(Resources(offer.resources()).contains(sharedVolume));
-  EXPECT_FALSE(Resources(offer.resources()).contains(nonSharedVolume));
+  EXPECT_TRUE(unallocated(offer.resources()).contains(sharedVolume));
+  EXPECT_FALSE(unallocated(offer.resources()).contains(nonSharedVolume));
 
   // We kill the long-lived task and wait for TASK_KILLED, so we can
   // DESTROY the persistent volume once the task terminates.
@@ -1586,7 +1604,15 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeMultipleIterations)
 
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume1));
+  // TODO(bmahler): This lambda is copied in several places
+  // in the code, consider how best to pull this out.
+  auto unallocated = [](const Resources& resources) {
+    Resources result = resources;
+    result.unallocate();
+    return result;
+  };
+
+  EXPECT_TRUE(unallocated(offer.resources()).contains(volume1));
 
   // 3. The framework CREATEs the 2nd shared volume, and LAUNCHes a task
   //    using this shared volume.
@@ -1622,8 +1648,8 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeMultipleIterations)
 
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume1));
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume2));
+  EXPECT_TRUE(unallocated(offer.resources()).contains(volume1));
+  EXPECT_TRUE(unallocated(offer.resources()).contains(volume2));
 
   driver.stop();
   driver.join();
@@ -1863,7 +1889,8 @@ TEST_P(PersistentVolumeTest, GoodACLCreateThenDestroy)
   offer = offers.get()[0];
 
   // Check that the persistent volume was created successfully.
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
   EXPECT_TRUE(os::exists(slave::paths::getPersistentVolumePath(
       slaveFlags.work_dir,
       volume)));
@@ -1894,7 +1921,8 @@ TEST_P(PersistentVolumeTest, GoodACLCreateThenDestroy)
   offer = offers.get()[0];
 
   // Check that the persistent volume is not in the offer.
-  EXPECT_FALSE(Resources(offer.resources()).contains(volume));
+  EXPECT_FALSE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -2011,7 +2039,8 @@ TEST_P(PersistentVolumeTest, GoodACLNoPrincipal)
   offer = offers.get()[0];
 
   // Check that the persistent volume was successfully created.
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
   EXPECT_TRUE(os::exists(slave::paths::getPersistentVolumePath(
       slaveFlags.work_dir,
       volume)));
@@ -2040,7 +2069,8 @@ TEST_P(PersistentVolumeTest, GoodACLNoPrincipal)
   offer = offers.get()[0];
 
   // Check that the persistent volume was not created
-  EXPECT_FALSE(Resources(offer.resources()).contains(volume));
+  EXPECT_FALSE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
   EXPECT_FALSE(
       Resources(checkpointResources2.get().resources()).contains(volume));
 
@@ -2159,7 +2189,8 @@ TEST_P(PersistentVolumeTest, BadACLNoPrincipal)
     offer = offers.get()[0];
 
     // Check that the persistent volume is not contained in this offer.
-    EXPECT_FALSE(Resources(offer.resources()).contains(volume));
+    EXPECT_FALSE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo1.role())));
   }
 
   // Decline the offer and suppress so the second
@@ -2214,7 +2245,8 @@ TEST_P(PersistentVolumeTest, BadACLNoPrincipal)
   offer = offers.get()[0];
 
   // Check that the persistent volume is contained in this offer.
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo2.role())));
 
   // Decline and suppress offers to `driver2` so that
   // `driver1` can receive an offer.
@@ -2259,7 +2291,8 @@ TEST_P(PersistentVolumeTest, BadACLNoPrincipal)
   // TODO(greggomann): In addition to checking that the volume is contained in
   // the offer, we should also confirm that the Destroy operation failed for the
   // correct reason. See MESOS-5470.
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo1.role())));
 
   driver1.stop();
   driver1.join();
@@ -2378,7 +2411,8 @@ TEST_P(PersistentVolumeTest, BadACLDropCreateAndDestroy)
     offer = offers.get()[0];
 
     // Check that the persistent volume is not contained in this offer.
-    EXPECT_FALSE(Resources(offer.resources()).contains(volume));
+    EXPECT_FALSE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo1.role())));
   }
 
   // Decline the offer and suppress so the second
@@ -2433,7 +2467,8 @@ TEST_P(PersistentVolumeTest, BadACLDropCreateAndDestroy)
   offer = offers.get()[0];
 
   // Check that the persistent volume is contained in this offer.
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo2.role())));
 
   // Decline and suppress offers to `driver2` so that
   // `driver1` can receive an offer.
@@ -2478,7 +2513,8 @@ TEST_P(PersistentVolumeTest, BadACLDropCreateAndDestroy)
   // TODO(greggomann): In addition to checking that the volume is contained in
   // the offer, we should also confirm that the Destroy operation failed for the
   // correct reason. See MESOS-5470.
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo1.role())));
 
   driver1.stop();
   driver1.join();

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/tests/reservation_endpoints_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reservation_endpoints_tests.cpp b/src/tests/reservation_endpoints_tests.cpp
index 7bc59c2..7432d75 100644
--- a/src/tests/reservation_endpoints_tests.cpp
+++ b/src/tests/reservation_endpoints_tests.cpp
@@ -36,6 +36,7 @@
 
 #include "tests/allocator.hpp"
 #include "tests/mesos.hpp"
+#include "tests/resources_utils.hpp"
 #include "tests/utils.hpp"
 
 using std::string;
@@ -154,7 +155,8 @@ TEST_F(ReservationEndpointsTest, AvailableResources)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   Future<Nothing> recoverResources;
   EXPECT_CALL(allocator, recoverResources(_, _, _, _))
@@ -186,7 +188,8 @@ TEST_F(ReservationEndpointsTest, AvailableResources)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   // Ignore subsequent `recoverResources` calls triggered from recovering the
   // resources that this framework is currently holding onto.
@@ -243,7 +246,8 @@ TEST_F(ReservationEndpointsTest, ReserveOfferedResources)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
@@ -264,7 +268,8 @@ TEST_F(ReservationEndpointsTest, ReserveOfferedResources)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -324,7 +329,8 @@ TEST_F(ReservationEndpointsTest, UnreserveOfferedResources)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
@@ -345,7 +351,8 @@ TEST_F(ReservationEndpointsTest, UnreserveOfferedResources)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -412,7 +419,8 @@ TEST_F(ReservationEndpointsTest, ReserveAvailableAndOfferedResources)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(available + offered));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(available + offered, frameworkInfo.role())));
 
   // Launch a task on the 'available' resources portion of the offer, which
   // recovers 'offered' resources portion.
@@ -446,7 +454,8 @@ TEST_F(ReservationEndpointsTest, ReserveAvailableAndOfferedResources)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(offered));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(offered, frameworkInfo.role())));
 
   // Kill the task running on 'available' resources to make it available.
   EXPECT_CALL(sched, statusUpdate(_, _));
@@ -461,7 +470,8 @@ TEST_F(ReservationEndpointsTest, ReserveAvailableAndOfferedResources)
   driver.killTask(taskInfo.task_id());
 
   AWAIT_READY(availableResources);
-  EXPECT_TRUE(availableResources.get().contains(available));
+  EXPECT_TRUE(availableResources.get().contains(
+      allocatedResources(available, frameworkInfo.role())));
 
   // At this point, we have 'available' resources in the allocator, and
   // 'offered' resources offered to the framework.
@@ -490,7 +500,8 @@ TEST_F(ReservationEndpointsTest, ReserveAvailableAndOfferedResources)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   // Ignore subsequent `recoverResources` calls triggered from recovering the
   // resources that this framework is currently holding onto.
@@ -575,7 +586,8 @@ TEST_F(ReservationEndpointsTest, UnreserveAvailableAndOfferedResources)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(available + offered));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(available + offered, frameworkInfo.role())));
 
   // Launch a task on the 'available' resources portion of the offer, which
   // recovers 'offered' resources portion.
@@ -609,7 +621,8 @@ TEST_F(ReservationEndpointsTest, UnreserveAvailableAndOfferedResources)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(offered));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(offered, frameworkInfo.role())));
 
   // Kill the task running on 'available' resources to make it available.
   EXPECT_CALL(sched, statusUpdate(_, _));
@@ -624,7 +637,8 @@ TEST_F(ReservationEndpointsTest, UnreserveAvailableAndOfferedResources)
   driver.killTask(taskInfo.task_id());
 
   AWAIT_READY(availableResources);
-  EXPECT_TRUE(availableResources.get().contains(available));
+  EXPECT_TRUE(availableResources.get().contains(
+      allocatedResources(available, frameworkInfo.role())));
 
   // At this point, we have 'available' resources in the allocator, and
   // 'offered' resources offered to the framework.
@@ -653,7 +667,8 @@ TEST_F(ReservationEndpointsTest, UnreserveAvailableAndOfferedResources)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   // Ignore subsequent `recoverResources` calls triggered from recovering the
   // resources that this framework is currently holding onto.
@@ -742,8 +757,10 @@ TEST_F(ReservationEndpointsTest, LabeledResources)
   Offer offer = offers.get()[0];
 
   Resources offeredResources = Resources(offer.resources());
-  EXPECT_TRUE(offeredResources.contains(labeledResources1));
-  EXPECT_TRUE(offeredResources.contains(labeledResources2));
+  EXPECT_TRUE(offeredResources.contains(
+      allocatedResources(labeledResources1, frameworkInfo.role())));
+  EXPECT_TRUE(offeredResources.contains(
+      allocatedResources(labeledResources2, frameworkInfo.role())));
 
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
@@ -766,10 +783,14 @@ TEST_F(ReservationEndpointsTest, LabeledResources)
   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));
+  EXPECT_FALSE(offeredResources.contains(
+      allocatedResources(totalSlaveResources, frameworkInfo.role())));
+  EXPECT_TRUE(offeredResources.contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
+  EXPECT_FALSE(offeredResources.contains(
+      allocatedResources(labeledResources1, frameworkInfo.role())));
+  EXPECT_TRUE(offeredResources.contains(
+      allocatedResources(labeledResources2, frameworkInfo.role())));
 
   // Now that the first labeled reservation has been unreserved,
   // attempting to unreserve it again should fail.
@@ -803,9 +824,12 @@ TEST_F(ReservationEndpointsTest, LabeledResources)
 
   offeredResources = Resources(offer.resources());
 
-  EXPECT_TRUE(offeredResources.contains(totalSlaveResources));
-  EXPECT_FALSE(offeredResources.contains(labeledResources1));
-  EXPECT_FALSE(offeredResources.contains(labeledResources2));
+  EXPECT_TRUE(offeredResources.contains(
+      allocatedResources(totalSlaveResources, frameworkInfo.role())));
+  EXPECT_FALSE(offeredResources.contains(
+      allocatedResources(labeledResources1, frameworkInfo.role())));
+  EXPECT_FALSE(offeredResources.contains(
+      allocatedResources(labeledResources2, frameworkInfo.role())));
 
   // Ignore subsequent `recoverResources` calls triggered from recovering the
   // resources that this framework is currently holding onto.
@@ -1598,8 +1622,10 @@ TEST_F(ReservationEndpointsTest, DifferentPrincipalsSameRole)
   Offer offer = offers.get()[0];
   Resources resources = Resources(offer.resources());
 
-  EXPECT_TRUE(resources.contains(dynamicallyReserved1));
-  EXPECT_TRUE(resources.contains(dynamicallyReserved2));
+  EXPECT_TRUE(resources.contains(
+      allocatedResources(dynamicallyReserved1, frameworkInfo.role())));
+  EXPECT_TRUE(resources.contains(
+      allocatedResources(dynamicallyReserved2, frameworkInfo.role())));
 
   driver.stop();
   driver.join();

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/tests/reservation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reservation_tests.cpp b/src/tests/reservation_tests.cpp
index b7061de..309ce8b 100644
--- a/src/tests/reservation_tests.cpp
+++ b/src/tests/reservation_tests.cpp
@@ -48,6 +48,7 @@
 #include "tests/containerizer.hpp"
 #include "tests/mesos.hpp"
 #include "tests/mock_slave.hpp"
+#include "tests/resources_utils.hpp"
 
 using mesos::internal::master::allocator::HierarchicalDRFAllocator;
 
@@ -135,7 +136,8 @@ TEST_F(ReservationTest, ReserveThenUnreserve)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -150,7 +152,8 @@ TEST_F(ReservationTest, ReserveThenUnreserve)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -165,7 +168,8 @@ TEST_F(ReservationTest, ReserveThenUnreserve)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -226,7 +230,8 @@ TEST_F(ReservationTest, ReserveTwiceWithDoubleValue)
   Offer offer = offers.get()[0];
 
   // In the first offer, expect an offer with unreserved resources.
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -241,7 +246,8 @@ TEST_F(ReservationTest, ReserveTwiceWithDoubleValue)
   offer = offers.get()[0];
 
   // In the second offer, expect an offer with reserved resources.
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -265,7 +271,8 @@ TEST_F(ReservationTest, ReserveTwiceWithDoubleValue)
         frameworkInfo.role(),
         createReservationInfo(frameworkInfo.principal())).get();
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(finalReservation));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(finalReservation, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -330,7 +337,8 @@ TEST_F(ReservationTest, ReserveAndLaunchThenUnreserve)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   // Create a task.
   TaskInfo taskInfo =
@@ -360,7 +368,8 @@ TEST_F(ReservationTest, ReserveAndLaunchThenUnreserve)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -380,7 +389,8 @@ TEST_F(ReservationTest, ReserveAndLaunchThenUnreserve)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   EXPECT_CALL(exec, shutdown(_))
     .Times(AtMost(1));
@@ -451,7 +461,8 @@ TEST_F(ReservationTest, ReserveShareWithinRole)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo1.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched1, resourceOffers(&driver1, _))
@@ -471,7 +482,8 @@ TEST_F(ReservationTest, ReserveShareWithinRole)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo1.role())));
 
   // The filter to decline the offer "forever".
   Filters filtersForever;
@@ -496,7 +508,8 @@ TEST_F(ReservationTest, ReserveShareWithinRole)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo1.role())));
 
   driver1.stop();
   driver1.join();
@@ -563,7 +576,8 @@ TEST_F(ReservationTest, DropReserveTooLarge)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -591,7 +605,8 @@ TEST_F(ReservationTest, DropReserveTooLarge)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -655,7 +670,8 @@ TEST_F(ReservationTest, DropReserveStaticReservation)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(staticallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(staticallyReserved, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -680,7 +696,8 @@ TEST_F(ReservationTest, DropReserveStaticReservation)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(staticallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(staticallyReserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -743,7 +760,8 @@ TEST_F(ReservationTest, SendingCheckpointResourcesMessage)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved1 + unreserved2));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved1 + unreserved2, frameworkInfo.role())));
 
   Future<CheckpointResourcesMessage> message3 =
     FUTURE_PROTOBUF(CheckpointResourcesMessage(), _, _);
@@ -839,7 +857,8 @@ TEST_F(ReservationTest, ResourcesCheckpointing)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   Future<CheckpointResourcesMessage> checkpointResources =
     FUTURE_PROTOBUF(CheckpointResourcesMessage(), _, slave.get()->pid);
@@ -985,7 +1004,8 @@ TEST_F(ReservationTest, MasterFailover)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(reserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(reserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -1054,7 +1074,8 @@ TEST_F(ReservationTest, CompatibleCheckpointedResources)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   Future<CheckpointResourcesMessage> checkpointResources =
     FUTURE_PROTOBUF(CheckpointResourcesMessage(), _, _);
@@ -1173,8 +1194,8 @@ TEST_F(ReservationTest, CompatibleCheckpointedResourcesWithPersistentVolumes)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(
-      Resources(offer.resources()).contains(unreserved + unreservedDisk));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved + unreservedDisk, frameworkInfo.role())));
 
   Future<CheckpointResourcesMessage> message2 =
     FUTURE_PROTOBUF(CheckpointResourcesMessage(), _, _);
@@ -1211,7 +1232,8 @@ TEST_F(ReservationTest, CompatibleCheckpointedResourcesWithPersistentVolumes)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(reserved + volume));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(reserved + volume, frameworkInfo.role())));
 
   Future<OfferID> rescindedOfferId;
 
@@ -1310,7 +1332,8 @@ TEST_F(ReservationTest, IncompatibleCheckpointedResources)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   Future<CheckpointResourcesMessage> checkpointResources =
     FUTURE_PROTOBUF(CheckpointResourcesMessage(), _, _);
@@ -1433,7 +1456,8 @@ TEST_F(ReservationTest, GoodACLReserveThenUnreserve)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1448,7 +1472,8 @@ TEST_F(ReservationTest, GoodACLReserveThenUnreserve)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1463,7 +1488,8 @@ TEST_F(ReservationTest, GoodACLReserveThenUnreserve)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -1533,7 +1559,8 @@ TEST_F(ReservationTest, BadACLDropReserve)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1549,7 +1576,8 @@ TEST_F(ReservationTest, BadACLDropReserve)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -1631,7 +1659,8 @@ TEST_F(ReservationTest, BadACLDropUnreserve)
   Offer offer = offers.get()[0];
 
   // The slave's total resources are twice those defined by `unreserved1`.
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved1 + unreserved1));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved1 + unreserved1, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1648,10 +1677,10 @@ TEST_F(ReservationTest, BadACLDropUnreserve)
 
   // The reserved resources and an equal portion of
   // unreserved resources should be present.
-  EXPECT_TRUE(
-      Resources(offer.resources()).contains(
-          dynamicallyReserved1 +
-          unreserved1));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(
+          dynamicallyReserved1 + unreserved1,
+          frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1670,11 +1699,10 @@ TEST_F(ReservationTest, BadACLDropUnreserve)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(
-      Resources(offer.resources()).contains(
-          dynamicallyReserved1 +
-          dynamicallyReserved2 +
-          unreserved2));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(
+          dynamicallyReserved1 + dynamicallyReserved2 + unreserved2,
+          frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -1771,7 +1799,8 @@ TEST_F(ReservationTest, ACLMultipleOperations)
   Offer offer = offers.get()[0];
 
   // The slave's total resources are twice those defined by `unreserved1`.
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved1 + unreserved1));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved1 + unreserved1, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1791,10 +1820,10 @@ TEST_F(ReservationTest, ACLMultipleOperations)
 
   // The reserved resources and an equal portion of
   // unreserved resources should be present.
-  EXPECT_TRUE(
-      Resources(offer.resources()).contains(
-          dynamicallyReserved1 +
-          unreserved1));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(
+          dynamicallyReserved1 + unreserved1,
+          frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1836,11 +1865,10 @@ TEST_F(ReservationTest, ACLMultipleOperations)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(
-      Resources(offer.resources()).contains(
-          dynamicallyReserved1 +
-          dynamicallyReserved2 +
-          unreserved2));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(
+          dynamicallyReserved1 + dynamicallyReserved2 + unreserved2,
+          frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1874,11 +1902,10 @@ TEST_F(ReservationTest, ACLMultipleOperations)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(
-      Resources(offer.resources()).contains(
-          dynamicallyReserved1 +
-          dynamicallyReserved2 +
-          unreserved2));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(
+          dynamicallyReserved1 + dynamicallyReserved2 + unreserved2,
+          frameworkInfo.role())));
 
   // Check that the task launched as expected.
   EXPECT_EQ(TASK_FINISHED, failedTaskStatus.get().state());
@@ -1952,7 +1979,8 @@ TEST_F(ReservationTest, WithoutAuthenticationWithoutPrincipal)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   // The expectation for the offer with reserved resources.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1970,7 +1998,8 @@ TEST_F(ReservationTest, WithoutAuthenticationWithoutPrincipal)
   offer = offers.get()[0];
 
   // Make sure that the reservation succeeded.
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   // An expectation for an offer with unreserved resources.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1989,7 +2018,8 @@ TEST_F(ReservationTest, WithoutAuthenticationWithoutPrincipal)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -2057,7 +2087,8 @@ TEST_F(ReservationTest, WithoutAuthenticationWithPrincipal)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   // The expectation for the offer with reserved resources.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -2075,7 +2106,8 @@ TEST_F(ReservationTest, WithoutAuthenticationWithPrincipal)
   offer = offers.get()[0];
 
   // Make sure that the reservation succeeded.
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   // An expectation for an offer with unreserved resources.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -2094,7 +2126,8 @@ TEST_F(ReservationTest, WithoutAuthenticationWithPrincipal)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -2153,7 +2186,8 @@ TEST_F(ReservationTest, DropReserveWithInvalidRole)
   Offer offer = offers->front();
 
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -2186,7 +2220,8 @@ TEST_F(ReservationTest, DropReserveWithInvalidRole)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers->front();
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -2250,7 +2285,8 @@ TEST_F(ReservationTest, PreventUnreservingAlienResources)
 
   const Resources unreserved = Resources::parse("cpus:1;mem:512").get();
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo1.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched1, resourceOffers(&driver1, _))
@@ -2277,8 +2313,10 @@ TEST_F(ReservationTest, PreventUnreservingAlienResources)
   ASSERT_EQ(1u, offers->size());
   offer = offers->front();
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
-  EXPECT_TRUE(Resources(offer.resources()).contains(halfMemory));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo1.role())));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(halfMemory, frameworkInfo1.role())));
 
   // The filter to decline the offer "forever".
   Filters filtersForever;
@@ -2302,8 +2340,10 @@ TEST_F(ReservationTest, PreventUnreservingAlienResources)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers->front();
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(halfMemory));
-  EXPECT_FALSE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(halfMemory, frameworkInfo2.role())));
+  EXPECT_FALSE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo2.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched2, resourceOffers(&driver2, _))
@@ -2320,8 +2360,10 @@ TEST_F(ReservationTest, PreventUnreservingAlienResources)
   // Expect another offer without the resources reserved by `framework1`.
   AWAIT_READY(offers);
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(halfMemory));
-  EXPECT_FALSE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(halfMemory, frameworkInfo2.role())));
+  EXPECT_FALSE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo2.role())));
 
   // Decline the offer "forever" in order to force `framework1` to
   // receive the remaining resources.
@@ -2344,8 +2386,10 @@ TEST_F(ReservationTest, PreventUnreservingAlienResources)
   offer = offers->front();
 
   // Make sure that the reservation is still in place.
-  EXPECT_TRUE(Resources(offer.resources()).contains(halfMemory));
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(halfMemory, frameworkInfo1.role())));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo1.role())));
 
   driver1.stop();
   driver1.join();

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/tests/resource_offers_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resource_offers_tests.cpp b/src/tests/resource_offers_tests.cpp
index 72ceb86..9d6fa92 100644
--- a/src/tests/resource_offers_tests.cpp
+++ b/src/tests/resource_offers_tests.cpp
@@ -262,7 +262,8 @@ TEST_F(ResourceOffersTest, ResourcesGetReofferedAfterTaskInfoError)
   EXPECT_EQ(TaskStatus::REASON_TASK_INVALID, status.get().reason());
   EXPECT_TRUE(status.get().has_message());
   EXPECT_TRUE(strings::startsWith(
-        status.get().message(), "Task uses invalid resources"));
+        status.get().message(), "Task uses invalid resources"))
+    << status->message();
 
   MockScheduler sched2;
   MesosSchedulerDriver driver2(

http://git-wip-us.apache.org/repos/asf/mesos/blob/c20744a9/src/tests/role_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/role_tests.cpp b/src/tests/role_tests.cpp
index 0d0f1a9..6ef7af7 100644
--- a/src/tests/role_tests.cpp
+++ b/src/tests/role_tests.cpp
@@ -26,6 +26,7 @@
 
 #include "tests/containerizer.hpp"
 #include "tests/mesos.hpp"
+#include "tests/resources_utils.hpp"
 
 using mesos::internal::master::Master;
 using mesos::internal::slave::Slave;
@@ -138,8 +139,10 @@ TEST_F(RoleTest, ImplicitRoleRegister)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(unreserved));
-  EXPECT_FALSE(Resources(offer.resources()).contains(dynamicallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
+  EXPECT_FALSE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -154,8 +157,10 @@ TEST_F(RoleTest, ImplicitRoleRegister)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
-  EXPECT_FALSE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+  EXPECT_FALSE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   Resources volume = createPersistentVolume(
       Megabytes(64),
@@ -178,9 +183,12 @@ TEST_F(RoleTest, ImplicitRoleRegister)
   ASSERT_EQ(1u, offers.get().size());
   offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(volume));
-  EXPECT_FALSE(Resources(offer.resources()).contains(dynamicallyReserved));
-  EXPECT_FALSE(Resources(offer.resources()).contains(unreserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(volume, frameworkInfo.role())));
+  EXPECT_FALSE(Resources(offer.resources()).contains(
+      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+  EXPECT_FALSE(Resources(offer.resources()).contains(
+      allocatedResources(unreserved, frameworkInfo.role())));
 
   driver.stop();
   driver.join();
@@ -240,7 +248,8 @@ TEST_F(RoleTest, ImplicitRoleStaticReservation)
   ASSERT_EQ(1u, offers.get().size());
   Offer offer = offers.get()[0];
 
-  EXPECT_TRUE(Resources(offer.resources()).contains(staticallyReserved));
+  EXPECT_TRUE(Resources(offer.resources()).contains(
+      allocatedResources(staticallyReserved, frameworkInfo.role())));
 
   // Create a task to launch with the resources of `staticallyReserved`.
   TaskInfo taskInfo =