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/12/02 00:22:43 UTC

[1/3] mesos git commit: Stripped AllocationInfo from offers to non-MULTI_ROLE schedulers.

Repository: mesos
Updated Branches:
  refs/heads/master 8c2f972b5 -> 5044610ca


Stripped AllocationInfo from offers to non-MULTI_ROLE schedulers.

Per MESOS-8237, it is problematic to show `Resource.allocation_info`
for "old" schedulers. The example that was presented was the
"mesos-go" 3rd party library that provides resource comparison logic.
The library has both MULTI-ROLE and non-MULTI_ROLE schedulers as
clients, these clients use the comparison logic supplied by the
library to see if the offer contains the resources they need.
The library author does not want to have to do any stripping of the
allocation info to preserve the equality.

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


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

Branch: refs/heads/master
Commit: d321b1df397431566c2b92097e1de4d8953a8836
Parents: 8c2f972
Author: Benjamin Mahler <bm...@apache.org>
Authored: Wed Nov 29 17:37:36 2017 -0800
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Fri Dec 1 16:02:33 2017 -0800

----------------------------------------------------------------------
 src/master/master.cpp | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d321b1df/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 398373f..309cd10 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -8402,6 +8402,23 @@ void Master::offer(
         }
       }
 
+      // Per MESOS-8237, it is problematic to show the
+      // `Resource.allocation_info` for pre-MULTI_ROLE schedulers.
+      // Pre-MULTI_ROLE schedulers are not `AllocationInfo` aware,
+      // and since they may be performing operations that
+      // implicitly uses all of Resource's state (e.g. equality
+      // comparison), we strip the `AllocationInfo` from `Resource`,
+      // as well as Offer. The idea here is that since the
+      // information doesn't provide any value to a pre-MULTI_ROLE
+      // scheduler, we preserve the old `Offer` format for them.
+      if (!framework->capabilities.multiRole) {
+        offer_.clear_allocation_info();
+
+        foreach (Resource& resource, *offer_.mutable_resources()) {
+          resource.clear_allocation_info();
+        }
+      }
+
       if (!framework->capabilities.reservationRefinement) {
         convertResourceFormat(
             offer_.mutable_resources(), PRE_RESERVATION_REFINEMENT);


[2/3] mesos git commit: Updated the tests to use MULTI_ROLE frameworks by default.

Posted by bm...@apache.org.
http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/persistent_volume_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/persistent_volume_tests.cpp b/src/tests/persistent_volume_tests.cpp
index 4aa3c2e..f255382 100644
--- a/src/tests/persistent_volume_tests.cpp
+++ b/src/tests/persistent_volume_tests.cpp
@@ -15,6 +15,7 @@
 // limitations under the License.
 
 #include <list>
+#include <set>
 #include <string>
 #include <vector>
 
@@ -69,6 +70,7 @@ using mesos::master::detector::MasterDetector;
 using mesos::master::detector::StandaloneMasterDetector;
 
 using std::list;
+using std::set;
 using std::string;
 using std::vector;
 
@@ -148,14 +150,16 @@ protected:
     master::Flags flags = CreateMasterFlags();
 
     ACLs acls;
-    hashset<string> roles;
+    set<string> roles;
 
     foreach (const FrameworkInfo& framework, frameworks) {
       mesos::ACL::RegisterFramework* acl = acls.add_register_frameworks();
       acl->mutable_principals()->add_values(framework.principal());
-      acl->mutable_roles()->add_values(framework.role());
 
-      roles.insert(framework.role());
+      foreach (const string& role, protobuf::framework::getRoles(framework)) {
+        acl->mutable_roles()->add_values(role);
+        roles.insert(role);
+      }
     }
 
     flags.acls = acls;
@@ -350,11 +354,11 @@ INSTANTIATE_TEST_CASE_P(
 TEST_P(PersistentVolumeTest, CreateAndDestroyPersistentVolumes)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   // Create a master.
   master::Flags masterFlags = CreateMasterFlags();
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -467,9 +471,9 @@ TEST_P(PersistentVolumeTest, CreateAndDestroyPersistentVolumes)
 
   // Expect that the offer contains the persistent volumes we created.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume1, frameworkInfo.role())));
+      allocatedResources(volume1, frameworkInfo.roles(0))));
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume2, frameworkInfo.role())));
+      allocatedResources(volume2, frameworkInfo.roles(0))));
 
   // Destroy `volume1`.
   driver.acceptOffers(
@@ -531,7 +535,7 @@ TEST_P(PersistentVolumeTest, ResourcesCheckpointing)
   AWAIT_READY(slaveReady);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -606,7 +610,7 @@ TEST_P(PersistentVolumeTest, PreparePersistentVolume)
   AWAIT_READY(slaveReady);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -678,7 +682,7 @@ TEST_P(PersistentVolumeTest, MasterFailover)
   AWAIT_READY(slaveReady1);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   MockScheduler sched;
   TestingMesosSchedulerDriver driver(&sched, &detector, frameworkInfo);
@@ -756,7 +760,7 @@ TEST_P(PersistentVolumeTest, MasterFailover)
   Offer offer2 = offers2.get()[0];
 
   EXPECT_TRUE(Resources(offer2.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -794,7 +798,7 @@ TEST_P(PersistentVolumeTest, IncompatibleCheckpointedResources)
   AWAIT_READY(slaveReady);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -891,7 +895,7 @@ TEST_P(PersistentVolumeTest, AccessPersistentVolume)
   AWAIT_READY(slaveReady);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
   frameworkInfo.add_capabilities()->set_type(
       FrameworkInfo::Capability::SHARED_RESOURCES);
 
@@ -1010,7 +1014,7 @@ TEST_P(PersistentVolumeTest, AccessPersistentVolume)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   Future<Resources> message = getOperationMessage(slave.get()->pid);
 
@@ -1069,7 +1073,7 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeMultipleTasks)
   AWAIT_READY(slaveReady);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
   frameworkInfo.add_capabilities()->set_type(
       FrameworkInfo::Capability::SHARED_RESOURCES);
 
@@ -1195,7 +1199,7 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeRescindOnDestroy)
 
   // 1. Create framework1 so that all resources are offered to this framework.
   FrameworkInfo frameworkInfo1 = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo1.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo1.set_roles(0, DEFAULT_TEST_ROLE);
   frameworkInfo1.add_capabilities()->set_type(
       FrameworkInfo::Capability::SHARED_RESOURCES);
 
@@ -1278,7 +1282,7 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeRescindOnDestroy)
   // 3. Create framework2 of the same role. It would be offered resources
   //    recovered from the framework1 call.
   FrameworkInfo frameworkInfo2 = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo2.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo2.set_roles(0, DEFAULT_TEST_ROLE);
   frameworkInfo2.add_capabilities()->set_type(
       FrameworkInfo::Capability::SHARED_RESOURCES);
 
@@ -1300,9 +1304,9 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeRescindOnDestroy)
   Offer offer2 = offers2.get()[0];
 
   EXPECT_TRUE(Resources(offer2.resources()).contains(
-      allocatedResources(volume1, frameworkInfo2.role())));
+      allocatedResources(volume1, frameworkInfo2.roles(0))));
   EXPECT_TRUE(Resources(offer2.resources()).contains(
-      allocatedResources(volume2, frameworkInfo2.role())));
+      allocatedResources(volume2, frameworkInfo2.roles(0))));
 
   // 4. framework1 kills the task which results in an offer to framework1
   //    with the shared volumes. At this point, both frameworks will have
@@ -1322,9 +1326,9 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeRescindOnDestroy)
   offer1 = offers1.get()[0];
 
   EXPECT_TRUE(Resources(offer1.resources()).contains(
-      allocatedResources(volume1, frameworkInfo1.role())));
+      allocatedResources(volume1, frameworkInfo1.roles(0))));
   EXPECT_TRUE(Resources(offer1.resources()).contains(
-      allocatedResources(volume2, frameworkInfo1.role())));
+      allocatedResources(volume2, frameworkInfo1.roles(0))));
 
   // 5. DESTROY both the shared volumes via framework2 which would result
   //    in framework1 being rescinded the offer.
@@ -1375,7 +1379,7 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeMultipleFrameworks)
 
   // 1. Create framework1 so that all resources are offered to this framework.
   FrameworkInfo frameworkInfo1 = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo1.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo1.set_roles(0, DEFAULT_TEST_ROLE);
   frameworkInfo1.add_capabilities()->set_type(
       FrameworkInfo::Capability::SHARED_RESOURCES);
 
@@ -1461,7 +1465,7 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeMultipleFrameworks)
   // 3. Create framework2 of the same role. It would be offered resources
   //    recovered from the framework1 call.
   FrameworkInfo frameworkInfo2 = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo2.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo2.set_roles(0, DEFAULT_TEST_ROLE);
   frameworkInfo2.add_capabilities()->set_type(
       FrameworkInfo::Capability::SHARED_RESOURCES);
 
@@ -1484,7 +1488,7 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeMultipleFrameworks)
   Offer offer2 = offers2.get()[0];
 
   EXPECT_TRUE(Resources(offer2.resources()).contains(
-      allocatedResources(volume, frameworkInfo2.role())));
+      allocatedResources(volume, frameworkInfo2.roles(0))));
 
   // 4. framework2 LAUNCHes a task with a subset of resources from the offer.
 
@@ -1565,7 +1569,7 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeMasterFailover)
 
   // Create the framework with SHARED_RESOURCES capability.
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
   frameworkInfo.add_capabilities()->set_type(
       FrameworkInfo::Capability::SHARED_RESOURCES);
 
@@ -1676,9 +1680,9 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeMasterFailover)
   EXPECT_TRUE(Resources(offer2.resources()).contains(
       allocatedResources(
           Resources::parse("cpus:1;mem:1024").get(),
-          frameworkInfo.role())));
+          frameworkInfo.roles(0))));
   EXPECT_TRUE(Resources(offer2.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -1700,7 +1704,7 @@ TEST_P(PersistentVolumeTest, DestroyPersistentVolumeMultipleTasks)
   filters.set_refuse_seconds(0);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
   frameworkInfo.add_capabilities()->set_type(
       FrameworkInfo::Capability::SHARED_RESOURCES);
 
@@ -1946,7 +1950,7 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeMultipleIterations)
 
   // 1. Create framework so that all resources are offered to this framework.
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
   frameworkInfo.add_capabilities()->set_type(
       FrameworkInfo::Capability::SHARED_RESOURCES);
 
@@ -2088,7 +2092,7 @@ TEST_P(PersistentVolumeTest, SlaveRecovery)
   AWAIT_READY(slaveReady);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
   frameworkInfo.set_checkpoint(true);
 
   MockScheduler sched;
@@ -2234,12 +2238,12 @@ TEST_P(PersistentVolumeTest, GoodACLCreateThenDestroy)
   filters.set_refuse_seconds(0);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   // Create a master.
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -2317,7 +2321,7 @@ TEST_P(PersistentVolumeTest, GoodACLCreateThenDestroy)
 
   // Check that the persistent volume was created successfully.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
   EXPECT_TRUE(os::exists(slave::paths::getPersistentVolumePath(
       slaveFlags.work_dir,
       volume)));
@@ -2351,7 +2355,7 @@ TEST_P(PersistentVolumeTest, GoodACLCreateThenDestroy)
 
   // Check that the persistent volume is not in the offer.
   EXPECT_FALSE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -2386,18 +2390,15 @@ TEST_P(PersistentVolumeTest, GoodACLNoPrincipal)
   filters.set_refuse_seconds(0);
 
   // Create a `FrameworkInfo` with no principal.
-  FrameworkInfo frameworkInfo;
-  frameworkInfo.set_name("no-principal");
-  frameworkInfo.set_user(os::user().get());
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
-  frameworkInfo.add_capabilities()->set_type(
-      FrameworkInfo::Capability::RESERVATION_REFINEMENT);
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.clear_principal();
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   // Create a master. Since the framework has no
   // principal, we don't authenticate frameworks.
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
   masterFlags.authenticate_frameworks = false;
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
@@ -2476,7 +2477,7 @@ TEST_P(PersistentVolumeTest, GoodACLNoPrincipal)
 
   // Check that the persistent volume was successfully created.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
   EXPECT_TRUE(os::exists(slave::paths::getPersistentVolumePath(
       slaveFlags.work_dir,
       volume)));
@@ -2505,7 +2506,7 @@ TEST_P(PersistentVolumeTest, GoodACLNoPrincipal)
 
   // Check that the persistent volume was not created.
   EXPECT_FALSE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
   if (::testing::get<1>(GetParam()) == ENABLED) {
     EXPECT_TRUE(message2->contains(volume));
   } else {
@@ -2551,22 +2552,19 @@ TEST_P(PersistentVolumeTest, BadACLNoPrincipal)
   filters.set_refuse_seconds(0);
 
   // Create a `FrameworkInfo` with no principal.
-  FrameworkInfo frameworkInfo1;
-  frameworkInfo1.set_name("no-principal");
-  frameworkInfo1.set_user(os::user().get());
-  frameworkInfo1.set_role(DEFAULT_TEST_ROLE);
-  frameworkInfo1.add_capabilities()->set_type(
-      FrameworkInfo::Capability::RESERVATION_REFINEMENT);
+  FrameworkInfo frameworkInfo1 = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo1.clear_principal();
+  frameworkInfo1.set_roles(0, DEFAULT_TEST_ROLE);
 
   // Create a `FrameworkInfo` with a principal.
   FrameworkInfo frameworkInfo2 = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo2.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo2.set_roles(0, DEFAULT_TEST_ROLE);
 
   // Create a master. Since one framework has no
   // principal, we don't authenticate frameworks.
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
-  masterFlags.roles = frameworkInfo1.role();
+  masterFlags.roles = frameworkInfo1.roles(0);
   masterFlags.authenticate_frameworks = false;
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
@@ -2636,7 +2634,7 @@ TEST_P(PersistentVolumeTest, BadACLNoPrincipal)
 
     // Check that the persistent volume is not contained in this offer.
     EXPECT_FALSE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo1.role())));
+      allocatedResources(volume, frameworkInfo1.roles(0))));
   }
 
   // Decline the offer and suppress so the second
@@ -2692,7 +2690,7 @@ TEST_P(PersistentVolumeTest, BadACLNoPrincipal)
 
   // Check that the persistent volume is contained in this offer.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo2.role())));
+      allocatedResources(volume, frameworkInfo2.roles(0))));
 
   // Decline and suppress offers to `driver2` so that
   // `driver1` can receive an offer.
@@ -2738,7 +2736,7 @@ TEST_P(PersistentVolumeTest, BadACLNoPrincipal)
   // the offer, we should also confirm that the Destroy operation failed for the
   // correct reason. See MESOS-5470.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo1.role())));
+      allocatedResources(volume, frameworkInfo1.roles(0))));
 
   driver1.stop();
   driver1.join();
@@ -2782,21 +2780,19 @@ TEST_P(PersistentVolumeTest, BadACLDropCreateAndDestroy)
 
   // Create a `FrameworkInfo` that cannot create or destroy volumes.
   FrameworkInfo frameworkInfo1 = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo1.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo1.set_roles(0, DEFAULT_TEST_ROLE);
 
   // Create a `FrameworkInfo` that can create volumes.
-  FrameworkInfo frameworkInfo2;
+  FrameworkInfo frameworkInfo2 = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo2.set_name("creator-framework");
   frameworkInfo2.set_user(os::user().get());
-  frameworkInfo2.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo2.set_roles(0, DEFAULT_TEST_ROLE);
   frameworkInfo2.set_principal("creator-principal");
-  frameworkInfo2.add_capabilities()->set_type(
-      FrameworkInfo::Capability::RESERVATION_REFINEMENT);
 
   // Create a master.
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
-  masterFlags.roles = frameworkInfo1.role();
+  masterFlags.roles = frameworkInfo1.roles(0);
   masterFlags.authenticate_frameworks = false;
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
@@ -2866,7 +2862,7 @@ TEST_P(PersistentVolumeTest, BadACLDropCreateAndDestroy)
 
     // Check that the persistent volume is not contained in this offer.
     EXPECT_FALSE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo1.role())));
+      allocatedResources(volume, frameworkInfo1.roles(0))));
   }
 
   // Decline the offer and suppress so the second
@@ -2922,7 +2918,7 @@ TEST_P(PersistentVolumeTest, BadACLDropCreateAndDestroy)
 
   // Check that the persistent volume is contained in this offer.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo2.role())));
+      allocatedResources(volume, frameworkInfo2.roles(0))));
 
   // Decline and suppress offers to `driver2` so that
   // `driver1` can receive an offer.
@@ -2968,7 +2964,7 @@ TEST_P(PersistentVolumeTest, BadACLDropCreateAndDestroy)
   // the offer, we should also confirm that the Destroy operation failed for the
   // correct reason. See MESOS-5470.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo1.role())));
+      allocatedResources(volume, frameworkInfo1.roles(0))));
 
   driver1.stop();
   driver1.join();

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/reservation_endpoints_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reservation_endpoints_tests.cpp b/src/tests/reservation_endpoints_tests.cpp
index a96cc71..51146ea 100644
--- a/src/tests/reservation_endpoints_tests.cpp
+++ b/src/tests/reservation_endpoints_tests.cpp
@@ -80,7 +80,7 @@ public:
     // `HierarchicalAllocator::updateAvailable()` and periodic allocations.
     master::Flags flags = MesosTest::CreateMasterFlags();
     flags.allocation_interval = Seconds(1000);
-    flags.roles = createFrameworkInfo().role();
+    flags.roles = createFrameworkInfo().roles(0);
     return flags;
   }
 
@@ -88,7 +88,7 @@ public:
   FrameworkInfo createFrameworkInfo()
   {
     FrameworkInfo info = DEFAULT_FRAMEWORK_INFO;
-    info.set_role("role");
+    info.set_roles(0, "role");
     return info;
   }
 
@@ -124,7 +124,7 @@ TEST_F(ReservationEndpointsTest, AvailableResources)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   Future<Response> response = process::http::post(
       master.get()->pid,
@@ -153,7 +153,7 @@ TEST_F(ReservationEndpointsTest, AvailableResources)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   // The filter to decline the offer "forever".
   Filters filtersForever;
@@ -182,7 +182,7 @@ TEST_F(ReservationEndpointsTest, AvailableResources)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -211,7 +211,7 @@ TEST_F(ReservationEndpointsTest, ReserveOfferedResources)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -232,7 +232,7 @@ TEST_F(ReservationEndpointsTest, ReserveOfferedResources)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   // Expect an offer to be rescinded!
   EXPECT_CALL(sched, offerRescinded(_, _));
@@ -257,7 +257,7 @@ TEST_F(ReservationEndpointsTest, ReserveOfferedResources)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -286,7 +286,7 @@ TEST_F(ReservationEndpointsTest, UnreserveOfferedResources)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   Future<Response> response = process::http::post(
       master.get()->pid,
@@ -315,7 +315,7 @@ TEST_F(ReservationEndpointsTest, UnreserveOfferedResources)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   // Expect an offer to be rescinded.
   EXPECT_CALL(sched, offerRescinded(_, _));
@@ -340,7 +340,7 @@ TEST_F(ReservationEndpointsTest, UnreserveOfferedResources)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -372,7 +372,7 @@ TEST_F(ReservationEndpointsTest, ReserveAvailableAndOfferedResources)
   Resources total = available + offered;
   Resources dynamicallyReserved =
     total.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -402,7 +402,7 @@ TEST_F(ReservationEndpointsTest, ReserveAvailableAndOfferedResources)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(available + offered, frameworkInfo.role())));
+      allocatedResources(available + offered, frameworkInfo.roles(0))));
 
   // Launch a task on the 'available' resources portion of the offer, which
   // recovers 'offered' resources portion.
@@ -437,7 +437,7 @@ TEST_F(ReservationEndpointsTest, ReserveAvailableAndOfferedResources)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(offered, frameworkInfo.role())));
+      allocatedResources(offered, frameworkInfo.roles(0))));
 
   // Kill the task running on 'available' resources to make it available.
   Future<TaskStatus> statusUpdate;
@@ -477,7 +477,7 @@ TEST_F(ReservationEndpointsTest, ReserveAvailableAndOfferedResources)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -505,11 +505,11 @@ TEST_F(ReservationEndpointsTest, UnreserveAvailableAndOfferedResources)
 
   Resources available = Resources::parse("cpus:1;mem:128").get();
   available = available.pushReservation(createDynamicReservationInfo(
-      frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+      frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   Resources offered = Resources::parse("mem:384").get();
   offered = offered.pushReservation(createDynamicReservationInfo(
-      frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+      frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   Resources total = available + offered;
   Resources unreserved = total.toUnreserved();
@@ -550,7 +550,7 @@ TEST_F(ReservationEndpointsTest, UnreserveAvailableAndOfferedResources)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(available + offered, frameworkInfo.role())));
+      allocatedResources(available + offered, frameworkInfo.roles(0))));
 
   // Launch a task on the 'available' resources portion of the offer, which
   // recovers 'offered' resources portion.
@@ -584,7 +584,7 @@ TEST_F(ReservationEndpointsTest, UnreserveAvailableAndOfferedResources)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(offered, frameworkInfo.role())));
+      allocatedResources(offered, frameworkInfo.roles(0))));
 
   // Kill the task running on 'available' resources to make it available.
   Future<TaskStatus> statusUpdate;
@@ -624,7 +624,7 @@ TEST_F(ReservationEndpointsTest, UnreserveAvailableAndOfferedResources)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -664,10 +664,10 @@ TEST_F(ReservationEndpointsTest, LabeledResources)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources labeledResources1 =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal(), labels1));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal(), labels1));
   Resources labeledResources2 =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal(), labels2));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal(), labels2));
 
   // Make two resource reservations with different labels.
   Future<Response> response = process::http::post(
@@ -706,9 +706,9 @@ TEST_F(ReservationEndpointsTest, LabeledResources)
 
   Resources offeredResources = Resources(offer.resources());
   EXPECT_TRUE(offeredResources.contains(
-      allocatedResources(labeledResources1, frameworkInfo.role())));
+      allocatedResources(labeledResources1, frameworkInfo.roles(0))));
   EXPECT_TRUE(offeredResources.contains(
-      allocatedResources(labeledResources2, frameworkInfo.role())));
+      allocatedResources(labeledResources2, frameworkInfo.roles(0))));
 
   // Expect an offer to be rescinded.
   EXPECT_CALL(sched, offerRescinded(_, _));
@@ -735,13 +735,13 @@ TEST_F(ReservationEndpointsTest, LabeledResources)
 
   offeredResources = Resources(offer.resources());
   EXPECT_FALSE(offeredResources.contains(
-      allocatedResources(totalSlaveResources, frameworkInfo.role())));
+      allocatedResources(totalSlaveResources, frameworkInfo.roles(0))));
   EXPECT_TRUE(offeredResources.contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
   EXPECT_FALSE(offeredResources.contains(
-      allocatedResources(labeledResources1, frameworkInfo.role())));
+      allocatedResources(labeledResources1, frameworkInfo.roles(0))));
   EXPECT_TRUE(offeredResources.contains(
-      allocatedResources(labeledResources2, frameworkInfo.role())));
+      allocatedResources(labeledResources2, frameworkInfo.roles(0))));
 
   // Now that the first labeled reservation has been unreserved,
   // attempting to unreserve it again should fail.
@@ -779,11 +779,11 @@ TEST_F(ReservationEndpointsTest, LabeledResources)
   offeredResources = Resources(offer.resources());
 
   EXPECT_TRUE(offeredResources.contains(
-      allocatedResources(totalSlaveResources, frameworkInfo.role())));
+      allocatedResources(totalSlaveResources, frameworkInfo.roles(0))));
   EXPECT_FALSE(offeredResources.contains(
-      allocatedResources(labeledResources1, frameworkInfo.role())));
+      allocatedResources(labeledResources1, frameworkInfo.roles(0))));
   EXPECT_FALSE(offeredResources.contains(
-      allocatedResources(labeledResources2, frameworkInfo.role())));
+      allocatedResources(labeledResources2, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -870,7 +870,7 @@ TEST_F(ReservationEndpointsTest, InsufficientResources)
   Resources unreserved = Resources::parse("cpus:4;mem:4096").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   process::http::Headers headers = createBasicAuthHeaders(DEFAULT_CREDENTIAL);
   string body = createRequestBody(slaveId, dynamicallyReserved);
@@ -908,7 +908,7 @@ TEST_F(ReservationEndpointsTest, NoHeader)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   Future<Response> response = process::http::post(
       master.get()->pid,
@@ -1012,7 +1012,7 @@ TEST_F(ReservationEndpointsTest, GoodReserveAndUnreserveACL)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        createFrameworkInfo().role(), DEFAULT_CREDENTIAL.principal()));
+        createFrameworkInfo().roles(0), DEFAULT_CREDENTIAL.principal()));
 
   // Reserve the resources.
   Future<Response> response = process::http::post(
@@ -1126,7 +1126,7 @@ TEST_F(ReservationEndpointsTest, BadReserveACL)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        createFrameworkInfo().role(), DEFAULT_CREDENTIAL.principal()));
+        createFrameworkInfo().roles(0), DEFAULT_CREDENTIAL.principal()));
 
   // Attempt to reserve the resources.
   Future<Response> response = process::http::post(
@@ -1178,7 +1178,7 @@ TEST_F(ReservationEndpointsTest, BadUnreserveACL)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        createFrameworkInfo().role(), DEFAULT_CREDENTIAL.principal()));
+        createFrameworkInfo().roles(0), DEFAULT_CREDENTIAL.principal()));
 
   // Reserve the resources.
   Future<Response> response = process::http::post(
@@ -1393,7 +1393,7 @@ TEST_F(ReservationEndpointsTest, ReserveAndUnreserveNoAuthentication)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
 
   Resources dynamicallyReservedWithNoPrincipal = unreserved.pushReservation(
-      createDynamicReservationInfo(frameworkInfo.role()));
+      createDynamicReservationInfo(frameworkInfo.roles(0)));
 
   // Try a reservation with no principal in `ReservationInfo` and no
   // authentication headers.
@@ -1417,7 +1417,7 @@ TEST_F(ReservationEndpointsTest, ReserveAndUnreserveNoAuthentication)
 
   Resources dynamicallyReservedWithPrincipal =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   // Try a reservation with a principal in `ReservationInfo` and no
   // authentication headers.
@@ -1467,11 +1467,11 @@ TEST_F(ReservationEndpointsTest, DifferentPrincipalsSameRole)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved1 =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   Resources dynamicallyReserved2 =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL_2.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL_2.principal()));
 
   Future<Response> response = process::http::post(
       master.get()->pid,
@@ -1509,9 +1509,9 @@ TEST_F(ReservationEndpointsTest, DifferentPrincipalsSameRole)
   Resources resources = Resources(offer.resources());
 
   EXPECT_TRUE(resources.contains(
-      allocatedResources(dynamicallyReserved1, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved1, frameworkInfo.roles(0))));
   EXPECT_TRUE(resources.contains(
-      allocatedResources(dynamicallyReserved2, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved2, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/reservation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reservation_tests.cpp b/src/tests/reservation_tests.cpp
index f29e9bb..7f1bb39 100644
--- a/src/tests/reservation_tests.cpp
+++ b/src/tests/reservation_tests.cpp
@@ -200,11 +200,11 @@ INSTANTIATE_TEST_CASE_P(
 TEST_P(ReservationTest, ReserveThenUnreserve)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -232,7 +232,7 @@ TEST_P(ReservationTest, ReserveThenUnreserve)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture offers from 'resourceOffers'.
   Future<vector<Offer>> offers;
@@ -252,7 +252,7 @@ TEST_P(ReservationTest, ReserveThenUnreserve)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -268,7 +268,7 @@ TEST_P(ReservationTest, ReserveThenUnreserve)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -284,7 +284,7 @@ TEST_P(ReservationTest, ReserveThenUnreserve)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -301,7 +301,7 @@ TEST_P(ReservationTest, ReserveThenUnreserve)
 TEST_P(ReservationTest, ReserveTwiceWithDoubleValue)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
@@ -332,7 +332,7 @@ TEST_P(ReservationTest, ReserveTwiceWithDoubleValue)
   Resources unreserved = Resources::parse("cpus:0.1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   Future<vector<Offer>> offers;
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -349,7 +349,7 @@ TEST_P(ReservationTest, ReserveTwiceWithDoubleValue)
 
   // In the first offer, expect an offer with unreserved resources.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -365,7 +365,7 @@ TEST_P(ReservationTest, ReserveTwiceWithDoubleValue)
 
   // In the second offer, expect an offer with reserved resources.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -386,10 +386,10 @@ TEST_P(ReservationTest, ReserveTwiceWithDoubleValue)
   Resources reserved = Resources::parse("cpus:0.2;mem:512").get();
   Resources finalReservation =
     reserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(finalReservation, frameworkInfo.role())));
+      allocatedResources(finalReservation, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -407,11 +407,11 @@ TEST_P(ReservationTest, ReserveTwiceWithDoubleValue)
 TEST_P(ReservationTest, ReserveAndLaunchThenUnreserve)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -439,7 +439,7 @@ TEST_P(ReservationTest, ReserveAndLaunchThenUnreserve)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture offers from 'resourceOffers'.
   Future<vector<Offer>> offers;
@@ -459,7 +459,7 @@ TEST_P(ReservationTest, ReserveAndLaunchThenUnreserve)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   // Create a task.
   TaskInfo taskInfo =
@@ -490,7 +490,7 @@ TEST_P(ReservationTest, ReserveAndLaunchThenUnreserve)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -511,7 +511,7 @@ TEST_P(ReservationTest, ReserveAndLaunchThenUnreserve)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   EXPECT_CALL(exec, shutdown(_))
     .Times(AtMost(1));
@@ -533,11 +533,11 @@ TEST_P(ReservationTest, ReserveShareWithinRole)
 
   FrameworkInfo frameworkInfo1 = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo1.set_name("framework1");
-  frameworkInfo1.set_role(role);
+  frameworkInfo1.set_roles(0, role);
 
   FrameworkInfo frameworkInfo2 = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo1.set_name("framework2");
-  frameworkInfo2.set_role(role);
+  frameworkInfo2.set_roles(0, role);
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
@@ -587,7 +587,7 @@ TEST_P(ReservationTest, ReserveShareWithinRole)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo1.role())));
+      allocatedResources(unreserved, frameworkInfo1.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched1, resourceOffers(&driver1, _))
@@ -608,7 +608,7 @@ TEST_P(ReservationTest, ReserveShareWithinRole)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo1.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo1.roles(0))));
 
   // The filter to decline the offer "forever".
   Filters filtersForever;
@@ -634,7 +634,7 @@ TEST_P(ReservationTest, ReserveShareWithinRole)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo1.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo1.roles(0))));
 
   driver1.stop();
   driver1.join();
@@ -652,11 +652,11 @@ TEST_P(ReservationTest, DropReserveTooLarge)
   TestAllocator<> allocator;
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
 
@@ -684,7 +684,7 @@ TEST_P(ReservationTest, DropReserveTooLarge)
   Resources unreservedTooLarge = Resources::parse("cpus:1;mem:1024").get();
   Resources dynamicallyReservedTooLarge =
     unreservedTooLarge.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture the offers from 'resourceOffers'.
   Future<vector<Offer>> offers;
@@ -706,7 +706,7 @@ TEST_P(ReservationTest, DropReserveTooLarge)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -735,7 +735,7 @@ TEST_P(ReservationTest, DropReserveTooLarge)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -748,11 +748,11 @@ TEST_P(ReservationTest, DropReserveTooLarge)
 TEST_P(ReservationTest, ResourcesCheckpointing)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -775,7 +775,7 @@ TEST_P(ReservationTest, ResourcesCheckpointing)
 
   Resources unreserved = Resources::parse("cpus:8;mem:2048").get();
   Resources reserved = unreserved.pushReservation(createDynamicReservationInfo(
-      frameworkInfo.role(), frameworkInfo.principal()));
+      frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture offers from 'resourceOffers'.
   Future<vector<Offer>> offers;
@@ -796,7 +796,7 @@ TEST_P(ReservationTest, ResourcesCheckpointing)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   Future<Resources> message = getOperationMessage(slave.get()->pid);
 
@@ -847,11 +847,11 @@ TEST_P(ReservationTest, ResourcesCheckpointing)
 TEST_P(ReservationTest, MasterFailover)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master1 = StartMaster(masterFlags);
   ASSERT_SOME(master1);
@@ -873,7 +873,7 @@ TEST_P(ReservationTest, MasterFailover)
 
   Resources unreserved = Resources::parse("cpus:8;mem:2048").get();
   Resources reserved = unreserved.pushReservation(createDynamicReservationInfo(
-      frameworkInfo.role(), frameworkInfo.principal()));
+      frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture offers from 'resourceOffers'.
   Future<vector<Offer>> offers;
@@ -953,7 +953,7 @@ TEST_P(ReservationTest, MasterFailover)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(reserved, frameworkInfo.role())));
+      allocatedResources(reserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -975,11 +975,11 @@ TEST_P(ReservationTest, MasterFailover)
 TEST_P(ReservationTest, CompatibleCheckpointedResources)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -1013,7 +1013,7 @@ TEST_P(ReservationTest, CompatibleCheckpointedResources)
 
   Resources unreserved = Resources::parse("cpus:8;mem:2048").get();
   Resources reserved = unreserved.pushReservation(createDynamicReservationInfo(
-      frameworkInfo.role(), frameworkInfo.principal()));
+      frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture offers from 'resourceOffers'.
   Future<vector<Offer>> offers;
@@ -1034,7 +1034,7 @@ TEST_P(ReservationTest, CompatibleCheckpointedResources)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   Future<Resources> message = getOperationMessage(_);
 
@@ -1098,11 +1098,11 @@ TEST_P(ReservationTest, CompatibleCheckpointedResources)
 TEST_P(ReservationTest, CompatibleCheckpointedResourcesWithPersistentVolumes)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -1136,12 +1136,12 @@ TEST_P(ReservationTest, CompatibleCheckpointedResourcesWithPersistentVolumes)
 
   Resources unreserved = Resources::parse("cpus:8;mem:2048").get();
   Resources reserved = unreserved.pushReservation(createDynamicReservationInfo(
-      frameworkInfo.role(), frameworkInfo.principal()));
+      frameworkInfo.roles(0), frameworkInfo.principal()));
 
   Resource unreservedDisk = Resources::parse("disk", "1024", "*").get();
   Resource reservedDisk = unreservedDisk;
   reservedDisk.add_reservations()->CopyFrom(createDynamicReservationInfo(
-      frameworkInfo.role(), frameworkInfo.principal()));
+      frameworkInfo.roles(0), frameworkInfo.principal()));
 
   Resource volume = reservedDisk;
   volume.mutable_disk()->CopyFrom(createDiskInfo(
@@ -1168,7 +1168,7 @@ TEST_P(ReservationTest, CompatibleCheckpointedResourcesWithPersistentVolumes)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved + unreservedDisk, frameworkInfo.role())));
+      allocatedResources(unreserved + unreservedDisk, frameworkInfo.roles(0))));
 
   Future<Resources> message2 = getOperationMessage(_);
   Future<Resources> message1 = getOperationMessage(_);
@@ -1203,7 +1203,7 @@ TEST_P(ReservationTest, CompatibleCheckpointedResourcesWithPersistentVolumes)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(reserved + volume, frameworkInfo.role())));
+      allocatedResources(reserved + volume, frameworkInfo.roles(0))));
 
   Future<OfferID> rescindedOfferId;
 
@@ -1261,11 +1261,11 @@ TEST_P(ReservationTest, CompatibleCheckpointedResourcesWithPersistentVolumes)
 TEST_P(ReservationTest, IncompatibleCheckpointedResources)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -1299,7 +1299,7 @@ TEST_P(ReservationTest, IncompatibleCheckpointedResources)
 
   Resources unreserved = Resources::parse("cpus:8;mem:2048").get();
   Resources reserved = unreserved.pushReservation(createDynamicReservationInfo(
-      frameworkInfo.role(), frameworkInfo.principal()));
+      frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture offers from 'resourceOffers'.
   Future<vector<Offer>> offers;
@@ -1319,7 +1319,7 @@ TEST_P(ReservationTest, IncompatibleCheckpointedResources)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   Future<Resources> message = getOperationMessage(_);
 
@@ -1395,13 +1395,13 @@ TEST_P(ReservationTest, GoodACLReserveThenUnreserve)
       DEFAULT_CREDENTIAL.principal());
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   // Create a master.
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -1431,7 +1431,7 @@ TEST_P(ReservationTest, GoodACLReserveThenUnreserve)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture offers from 'resourceOffers'.
   Future<vector<Offer>> offers;
@@ -1451,7 +1451,7 @@ TEST_P(ReservationTest, GoodACLReserveThenUnreserve)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1467,7 +1467,7 @@ TEST_P(ReservationTest, GoodACLReserveThenUnreserve)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1483,7 +1483,7 @@ TEST_P(ReservationTest, GoodACLReserveThenUnreserve)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -1502,13 +1502,13 @@ TEST_P(ReservationTest, BadACLDropReserve)
   reserve->mutable_roles()->set_type(mesos::ACL::Entity::ANY);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   // Create a master.
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -1538,7 +1538,7 @@ TEST_P(ReservationTest, BadACLDropReserve)
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture offers from 'resourceOffers'.
   Future<vector<Offer>> offers;
@@ -1558,7 +1558,7 @@ TEST_P(ReservationTest, BadACLDropReserve)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1575,7 +1575,7 @@ TEST_P(ReservationTest, BadACLDropReserve)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -1599,13 +1599,13 @@ TEST_P(ReservationTest, BadACLDropUnreserve)
   unreserve->mutable_reserver_principals()->set_type(mesos::ACL::Entity::NONE);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   // Create a master.
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -1636,12 +1636,12 @@ TEST_P(ReservationTest, BadACLDropUnreserve)
   Resources unreserved1 = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved1 =
     unreserved1.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   Resources unreserved2 = Resources::parse("cpus:0.5;mem:256").get();
   Resources dynamicallyReserved2 =
     unreserved2.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture offers from 'resourceOffers'.
   Future<vector<Offer>> offers;
@@ -1662,7 +1662,7 @@ TEST_P(ReservationTest, BadACLDropUnreserve)
 
   // The slave's total resources are twice those defined by `unreserved1`.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved1 + unreserved1, frameworkInfo.role())));
+      allocatedResources(unreserved1 + unreserved1, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1682,7 +1682,7 @@ TEST_P(ReservationTest, BadACLDropUnreserve)
   EXPECT_TRUE(Resources(offer.resources()).contains(
       allocatedResources(
           dynamicallyReserved1 + unreserved1,
-          frameworkInfo.role())));
+          frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1704,7 +1704,7 @@ TEST_P(ReservationTest, BadACLDropUnreserve)
   EXPECT_TRUE(Resources(offer.resources()).contains(
       allocatedResources(
           dynamicallyReserved1 + dynamicallyReserved2 + unreserved2,
-          frameworkInfo.role())));
+          frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -1732,13 +1732,13 @@ TEST_P(ReservationTest, ACLMultipleOperations)
   unreserve->mutable_reserver_principals()->set_type(mesos::ACL::Entity::NONE);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   // Create a master.
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -1776,12 +1776,12 @@ TEST_P(ReservationTest, ACLMultipleOperations)
   Resources unreserved1 = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved1 =
     unreserved1.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   Resources unreserved2 = Resources::parse("cpus:0.5;mem:256").get();
   Resources dynamicallyReserved2 =
     unreserved2.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture offers from 'resourceOffers'.
   Future<vector<Offer>> offers;
@@ -1808,7 +1808,7 @@ TEST_P(ReservationTest, ACLMultipleOperations)
 
   // The slave's total resources are twice those defined by `unreserved1`.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved1 + unreserved1, frameworkInfo.role())));
+      allocatedResources(unreserved1 + unreserved1, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1831,7 +1831,7 @@ TEST_P(ReservationTest, ACLMultipleOperations)
   EXPECT_TRUE(Resources(offer.resources()).contains(
       allocatedResources(
           dynamicallyReserved1 + unreserved1,
-          frameworkInfo.role())));
+          frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1876,7 +1876,7 @@ TEST_P(ReservationTest, ACLMultipleOperations)
   EXPECT_TRUE(Resources(offer.resources()).contains(
       allocatedResources(
           dynamicallyReserved1 + dynamicallyReserved2 + unreserved2,
-          frameworkInfo.role())));
+          frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -1913,7 +1913,7 @@ TEST_P(ReservationTest, ACLMultipleOperations)
   EXPECT_TRUE(Resources(offer.resources()).contains(
       allocatedResources(
           dynamicallyReserved1 + dynamicallyReserved2 + unreserved2,
-          frameworkInfo.role())));
+          frameworkInfo.roles(0))));
 
   // Check that the task launched as expected.
   EXPECT_EQ(TASK_FINISHED, failedTaskStatus->state());
@@ -1936,7 +1936,7 @@ TEST_P(ReservationTest, WithoutAuthenticationWithoutPrincipal)
 
   // Create a framework without a principal.
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
   frameworkInfo.clear_principal();
 
   // Create a master with no framework authentication.
@@ -1972,7 +1972,7 @@ TEST_P(ReservationTest, WithoutAuthenticationWithoutPrincipal)
   // Create dynamically reserved resources whose `ReservationInfo` does not
   // contain a principal.
   Resources dynamicallyReserved = unreserved.pushReservation(
-      createDynamicReservationInfo(frameworkInfo.role()));
+      createDynamicReservationInfo(frameworkInfo.roles(0)));
 
   // We use this to capture offers from `resourceOffers`.
   Future<vector<Offer>> offers;
@@ -1994,7 +1994,7 @@ TEST_P(ReservationTest, WithoutAuthenticationWithoutPrincipal)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   // The expectation for the offer with reserved resources.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -2013,7 +2013,7 @@ TEST_P(ReservationTest, WithoutAuthenticationWithoutPrincipal)
 
   // Make sure that the reservation succeeded.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   // An expectation for an offer with unreserved resources.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -2033,7 +2033,7 @@ TEST_P(ReservationTest, WithoutAuthenticationWithoutPrincipal)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -2050,7 +2050,7 @@ TEST_P(ReservationTest, WithoutAuthenticationWithPrincipal)
 
   // Create a framework with a principal.
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   // Create a master with no framework authentication.
   master::Flags masterFlags = CreateMasterFlags();
@@ -2086,7 +2086,7 @@ TEST_P(ReservationTest, WithoutAuthenticationWithPrincipal)
   // principal.
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture offers from `resourceOffers`.
   Future<vector<Offer>> offers;
@@ -2108,7 +2108,7 @@ TEST_P(ReservationTest, WithoutAuthenticationWithPrincipal)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   // The expectation for the offer with reserved resources.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -2127,7 +2127,7 @@ TEST_P(ReservationTest, WithoutAuthenticationWithPrincipal)
 
   // Make sure that the reservation succeeded.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   // An expectation for an offer with unreserved resources.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -2147,7 +2147,7 @@ TEST_P(ReservationTest, WithoutAuthenticationWithPrincipal)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -2161,7 +2161,7 @@ TEST_P(ReservationTest, DropReserveWithDifferentRole)
   const string frameworkRole = "role";
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(frameworkRole);
+  frameworkInfo.set_roles(0, frameworkRole);
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
@@ -2210,7 +2210,7 @@ TEST_P(ReservationTest, DropReserveWithDifferentRole)
 
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -2243,7 +2243,7 @@ TEST_P(ReservationTest, DropReserveWithDifferentRole)
   offer = offers->front();
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -2257,12 +2257,12 @@ TEST_P(ReservationTest, PreventUnreservingAlienResources)
   const string frameworkRole1 = "role1";
   FrameworkInfo frameworkInfo1 = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo1.set_name("framework1");
-  frameworkInfo1.set_role(frameworkRole1);
+  frameworkInfo1.set_roles(0, frameworkRole1);
 
   const string frameworkRole2 = "role2";
   FrameworkInfo frameworkInfo2 = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo2.set_name("framework2");
-  frameworkInfo2.set_role(frameworkRole2);
+  frameworkInfo2.set_roles(0, frameworkRole2);
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
@@ -2312,7 +2312,7 @@ TEST_P(ReservationTest, PreventUnreservingAlienResources)
   const Resources unreserved = Resources::parse("cpus:1;mem:512").get();
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo1.role())));
+      allocatedResources(unreserved, frameworkInfo1.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched1, resourceOffers(&driver1, _))
@@ -2337,9 +2337,9 @@ TEST_P(ReservationTest, PreventUnreservingAlienResources)
   offer = offers->front();
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo1.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo1.roles(0))));
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(halfMemory, frameworkInfo1.role())));
+      allocatedResources(halfMemory, frameworkInfo1.roles(0))));
 
   // The filter to decline the offer "forever".
   Filters filtersForever;
@@ -2364,9 +2364,9 @@ TEST_P(ReservationTest, PreventUnreservingAlienResources)
   offer = offers->front();
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(halfMemory, frameworkInfo2.role())));
+      allocatedResources(halfMemory, frameworkInfo2.roles(0))));
   EXPECT_FALSE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo2.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo2.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched2, resourceOffers(&driver2, _))
@@ -2384,9 +2384,9 @@ TEST_P(ReservationTest, PreventUnreservingAlienResources)
   AWAIT_READY(offers);
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(halfMemory, frameworkInfo2.role())));
+      allocatedResources(halfMemory, frameworkInfo2.roles(0))));
   EXPECT_FALSE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo2.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo2.roles(0))));
 
   // Decline the offer "forever" in order to force `framework1` to
   // receive the remaining resources.
@@ -2410,9 +2410,9 @@ TEST_P(ReservationTest, PreventUnreservingAlienResources)
 
   // Make sure that the reservation is still in place.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(halfMemory, frameworkInfo1.role())));
+      allocatedResources(halfMemory, frameworkInfo1.roles(0))));
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo1.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo1.roles(0))));
 
   driver1.stop();
   driver1.join();
@@ -2429,11 +2429,11 @@ class ReservationCheckpointingTest : public MesosTest {};
 TEST_F(ReservationCheckpointingTest, SendingCheckpointResourcesMessage)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -2452,12 +2452,12 @@ TEST_F(ReservationCheckpointingTest, SendingCheckpointResourcesMessage)
   Resources unreserved1 = Resources::parse("cpus:8").get();
   Resources reserved1 =
     unreserved1.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   Resources unreserved2 = Resources::parse("mem:2048").get();
   Resources reserved2 =
     unreserved2.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture offers from 'resourceOffers'.
   Future<vector<Offer>> offers;
@@ -2479,7 +2479,7 @@ TEST_F(ReservationCheckpointingTest, SendingCheckpointResourcesMessage)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved1 + unreserved2, frameworkInfo.role())));
+      allocatedResources(unreserved1 + unreserved2, frameworkInfo.roles(0))));
 
   Future<CheckpointResourcesMessage> message3 =
     FUTURE_PROTOBUF(CheckpointResourcesMessage(), _, _);

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/resource_provider_manager_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resource_provider_manager_tests.cpp b/src/tests/resource_provider_manager_tests.cpp
index 5764102..feede62 100644
--- a/src/tests/resource_provider_manager_tests.cpp
+++ b/src/tests/resource_provider_manager_tests.cpp
@@ -696,7 +696,7 @@ TEST_P(ResourceProviderManagerHttpApiTest, ConvertResources)
   Option<mesos::v1::Offer> offer1;
 
   v1::FrameworkInfo frameworkInfo = v1::DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   {
     Future<v1::scheduler::Event::Subscribed> subscribed;
@@ -738,7 +738,7 @@ TEST_P(ResourceProviderManagerHttpApiTest, ConvertResources)
   v1::Resource reserved = *(resources.begin());
   reserved.add_reservations()->CopyFrom(
       v1::createDynamicReservationInfo(
-          frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+          frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   Future<v1::scheduler::Event::Offers> offers;
   EXPECT_CALL(*scheduler, offers(_, _))

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/role_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/role_tests.cpp b/src/tests/role_tests.cpp
index 084555a..a609ed2 100644
--- a/src/tests/role_tests.cpp
+++ b/src/tests/role_tests.cpp
@@ -70,7 +70,7 @@ class RoleTest : public MesosTest {};
 TEST_F(RoleTest, BadRegister)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("invalid");
+  frameworkInfo.set_roles(0, "invalid");
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.roles = "foo,bar";
@@ -115,7 +115,7 @@ TEST_F(RoleTest, ImplicitRoleRegister)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("new-role-name");
+  frameworkInfo.set_roles(0, "new-role-name");
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -129,7 +129,7 @@ TEST_F(RoleTest, ImplicitRoleRegister)
   Resources unreserved = Resources::parse("disk:1024").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // We use this to capture offers from `resourceOffers`.
   Future<vector<Offer>> offers;
@@ -149,9 +149,9 @@ TEST_F(RoleTest, ImplicitRoleRegister)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
   EXPECT_FALSE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -167,13 +167,13 @@ TEST_F(RoleTest, ImplicitRoleRegister)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
   EXPECT_FALSE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   Resources volume = createPersistentVolume(
       Megabytes(64),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "path1",
       frameworkInfo.principal(),
@@ -193,11 +193,11 @@ TEST_F(RoleTest, ImplicitRoleRegister)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
   EXPECT_FALSE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
   EXPECT_FALSE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -227,7 +227,7 @@ TEST_F(RoleTest, ImplicitRoleStaticReservation)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -258,7 +258,7 @@ TEST_F(RoleTest, ImplicitRoleStaticReservation)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(staticallyReserved, frameworkInfo.role())));
+      allocatedResources(staticallyReserved, frameworkInfo.roles(0))));
 
   // Create a task to launch with the resources of `staticallyReserved`.
   TaskInfo taskInfo =
@@ -463,7 +463,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(RoleTest, EndpointImplicitRolesWeights)
   ASSERT_SOME(master);
 
   FrameworkInfo frameworkInfo1 = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo1.set_role("roleX");
+  frameworkInfo1.set_roles(0, "roleX");
 
   MockScheduler sched1;
   MesosSchedulerDriver driver1(
@@ -476,7 +476,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(RoleTest, EndpointImplicitRolesWeights)
   driver1.start();
 
   FrameworkInfo frameworkInfo2 = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo2.set_role("roleZ");
+  frameworkInfo2.set_roles(0, "roleZ");
 
   MockScheduler sched2;
   MesosSchedulerDriver driver2(
@@ -668,10 +668,8 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
   ASSERT_SOME(master);
 
   FrameworkInfo framework = DEFAULT_FRAMEWORK_INFO;
-  framework.add_roles("role1");
+  framework.set_roles(0, "role1");
   framework.add_roles("role2");
-  framework.add_capabilities()->set_type(
-      FrameworkInfo::Capability::MULTI_ROLE);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -927,7 +925,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(RoleTest, VolumesInOverlappingHierarchies)
   auto runTask = [&master, &PATH](
       const string& role, const string& id) {
     FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-    frameworkInfo.set_role(role);
+    frameworkInfo.set_roles(0, role);
 
     MockScheduler sched;
     MesosSchedulerDriver driver(

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/scheduler_http_api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/scheduler_http_api_tests.cpp b/src/tests/scheduler_http_api_tests.cpp
index 80e52fb..67782ed 100644
--- a/src/tests/scheduler_http_api_tests.cpp
+++ b/src/tests/scheduler_http_api_tests.cpp
@@ -324,7 +324,7 @@ TEST_P(SchedulerHttpApiTest, RejectFrameworkWithInvalidRole)
   Call::Subscribe* subscribe = call.mutable_subscribe();
   v1::FrameworkInfo framework = v1::DEFAULT_FRAMEWORK_INFO;
   // Set invalid role.
-  framework.set_role("/test/test1");
+  framework.set_roles(0, "/test/test1");
   subscribe->mutable_framework_info()->CopyFrom(framework);
 
   // Retrieve the parameter passed as content type to this test.

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/scheduler_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/scheduler_tests.cpp b/src/tests/scheduler_tests.cpp
index 29cab82..ecfce4d 100644
--- a/src/tests/scheduler_tests.cpp
+++ b/src/tests/scheduler_tests.cpp
@@ -1509,7 +1509,7 @@ TEST_P(SchedulerTest, NoOffersWithAllRolesSuppressed)
 
     Call::Subscribe* subscribe = call.mutable_subscribe();
     subscribe->mutable_framework_info()->CopyFrom(frameworkInfo);
-    subscribe->add_suppressed_roles(frameworkInfo.role());
+    subscribe->add_suppressed_roles(frameworkInfo.roles(0));
 
     mesos.send(call);
   }
@@ -1541,7 +1541,7 @@ TEST_P(SchedulerTest, NoOffersWithAllRolesSuppressed)
     v1::FrameworkInfo frameworkInfo = v1::DEFAULT_FRAMEWORK_INFO;
 
     Call::Revive* revive = call.mutable_revive();
-    revive->add_roles(frameworkInfo.role());
+    revive->add_roles(frameworkInfo.roles(0));
 
     mesos.send(call);
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/slave_authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_authorization_tests.cpp b/src/tests/slave_authorization_tests.cpp
index 6362fdf..b7e8d3d 100644
--- a/src/tests/slave_authorization_tests.cpp
+++ b/src/tests/slave_authorization_tests.cpp
@@ -196,7 +196,7 @@ TYPED_TEST(SlaveAuthorizerTest, FilterStateEndpoint)
   // Register framework with user "bar" and role "superhero".
   FrameworkInfo frameworkSuperhero = DEFAULT_FRAMEWORK_INFO;
   frameworkSuperhero.set_name("framework-" + roleSuperhero);
-  frameworkSuperhero.set_role(roleSuperhero);
+  frameworkSuperhero.set_roles(0, roleSuperhero);
   frameworkSuperhero.set_user("bar");
 
   // Create an executor with user "bar".
@@ -209,7 +209,7 @@ TYPED_TEST(SlaveAuthorizerTest, FilterStateEndpoint)
   FrameworkInfo frameworkMuggle = DEFAULT_FRAMEWORK_INFO;
   frameworkMuggle.set_name("framework-" + roleMuggle);
   frameworkMuggle.set_principal(DEFAULT_CREDENTIAL_2.principal());
-  frameworkMuggle.set_role(roleMuggle);
+  frameworkMuggle.set_roles(0, roleMuggle);
   frameworkMuggle.set_user("foo");
 
   // Create an executor with user "foo".

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index f14c6ef..7674e60 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -2178,13 +2178,13 @@ TYPED_TEST(SlaveRecoveryTest, RemoveNonCheckpointingFramework)
 
   Offer offer1 = offer;
   Resources resources1 = allocatedResources(
-      Resources::parse("cpus:1;mem:512").get(), frameworkInfo.role());
+      Resources::parse("cpus:1;mem:512").get(), frameworkInfo.roles(0));
   offer1.mutable_resources()->CopyFrom(resources1);
   tasks.push_back(createTask(offer1, "sleep 1000")); // Long-running task.
 
   Offer offer2 = offer;
   Resources resources2 = allocatedResources(
-      Resources::parse("cpus:1;mem:512").get(), frameworkInfo.role());
+      Resources::parse("cpus:1;mem:512").get(), frameworkInfo.roles(0));
   offer2.mutable_resources()->CopyFrom(resources2);
   tasks.push_back(createTask(offer2, "sleep 1000")); // Long-running task,
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/slave_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_tests.cpp b/src/tests/slave_tests.cpp
index 8ab63ac..a7f6658 100644
--- a/src/tests/slave_tests.cpp
+++ b/src/tests/slave_tests.cpp
@@ -824,12 +824,12 @@ TEST_F(SlaveTest, GetExecutorInfo)
   FrameworkID frameworkId;
   frameworkId.set_value("20141010-221431-251662764-60288-32120-0000");
 
-  FrameworkInfo frameworkInfo;
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo.mutable_id()->CopyFrom(frameworkId);
 
   // Launch a task with the command executor.
   Resources taskResources = Resources::parse("cpus:0.1;mem:32").get();
-  taskResources.allocate(frameworkInfo.role());
+  taskResources.allocate(frameworkInfo.roles(0));
 
   TaskInfo task;
   task.set_name("task");
@@ -883,14 +883,14 @@ TEST_F(SlaveTest, GetExecutorInfoForTaskWithContainer)
   ASSERT_SOME(slave);
   ASSERT_NE(nullptr, slave.get()->mock());
 
-  FrameworkInfo frameworkInfo;
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo.mutable_id()->set_value(
       "20141010-221431-251662764-60288-12345-0000");
 
   // Launch a task with the command executor and ContainerInfo with
   // NetworkInfo.
   Resources taskResources = Resources::parse("cpus:0.1;mem:32").get();
-  taskResources.allocate(frameworkInfo.role());
+  taskResources.allocate(frameworkInfo.roles(0));
 
   TaskInfo task;
   task.set_name("task");
@@ -962,12 +962,12 @@ TEST_F(SlaveTest, ROOT_LaunchTaskInfoWithContainerInfo)
   ASSERT_SOME(slave);
   ASSERT_NE(nullptr, slave.get()->mock());
 
-  FrameworkInfo frameworkInfo;
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo.mutable_id()->set_value(
       "20141010-221431-251662764-60288-12345-0000");
 
   Resources taskResources = Resources::parse("cpus:0.1;mem:32").get();
-  taskResources.allocate(frameworkInfo.role());
+  taskResources.allocate(frameworkInfo.roles(0));
 
   // Launch a task with the command executor and ContainerInfo with
   // NetworkInfo.
@@ -1700,9 +1700,14 @@ TEST_F(SlaveTest, StateEndpoint)
   EXPECT_EQ(1u, frameworks.values.size());
 
   ASSERT_TRUE(frameworks.values[0].is<JSON::Object>());
-  JSON::Object framework = frameworks.values[0].as<JSON::Object>();
 
-  EXPECT_EQ("*", framework.values["role"]);
+  JSON::Object roles = {
+    { "roles", JSON::Array { DEFAULT_FRAMEWORK_INFO.roles(0) } }
+  };
+
+  EXPECT_TRUE(frameworks.values[0].contains(roles));
+
+  JSON::Object framework = frameworks.values[0].as<JSON::Object>();
   EXPECT_EQ("default", framework.values["name"]);
   EXPECT_EQ(model(resources.get()), state.values["resources"]);
 
@@ -8928,7 +8933,7 @@ TEST_F(SlaveTest, ResourceProviderReconciliation)
     .WillRepeatedly(Return()); // Ignore subsequent offers;
 
   v1::FrameworkInfo frameworkInfo = v1::DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("foo");
+  frameworkInfo.set_roles(0, "foo");
 
   // Subscribe the framework.
   {
@@ -8963,7 +8968,7 @@ TEST_F(SlaveTest, ResourceProviderReconciliation)
     reserved = reserved.filter(
         [](const v1::Resource& r) { return r.has_provider_id(); });
     reserved = reserved.pushReservation(v1::createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
     Call call =
       v1::createCallAccept(frameworkId, offer, {v1::RESERVE(reserved)});

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/upgrade_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/upgrade_tests.cpp b/src/tests/upgrade_tests.cpp
index 7f434db..0efaa58 100644
--- a/src/tests/upgrade_tests.cpp
+++ b/src/tests/upgrade_tests.cpp
@@ -120,8 +120,15 @@ TEST_F(UpgradeTest, ReregisterOldAgentWithMultiRoleMaster)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.clear_capabilities();
+  frameworkInfo.clear_roles();
   frameworkInfo.set_role("foo");
 
+  // TODO(bmahler): Introduce an easier way to strip just one
+  // of the capabilities without having to add back the others.
+  frameworkInfo.add_capabilities()->set_type(
+      FrameworkInfo::Capability::RESERVATION_REFINEMENT);
+
   MockScheduler sched;
   TestingMesosSchedulerDriver driver(&sched, &detector, frameworkInfo);
 
@@ -335,10 +342,7 @@ TEST_F(UpgradeTest, UpgradeSlaveIntoMultiRole)
   AWAIT_READY(slaveRegisteredMessage);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.clear_role();
-  frameworkInfo.add_roles("foo");
-  frameworkInfo.add_capabilities()->set_type(
-      FrameworkInfo::Capability::MULTI_ROLE);
+  frameworkInfo.set_roles(0, "foo");
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -408,6 +412,8 @@ TEST_F(UpgradeTest, MultiRoleSchedulerUpgrade)
   ASSERT_SOME(agent);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.clear_capabilities();
+  frameworkInfo.clear_roles();
   frameworkInfo.set_role("foo");
 
   MockScheduler sched1;
@@ -602,7 +608,7 @@ TEST_F(UpgradeTest, UpgradeAgentIntoHierarchicalRoleForNonHierarchicalRole)
   AWAIT_READY(slaveRegisteredMessage);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("foo");
+  frameworkInfo.set_roles(0, "foo");
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -729,7 +735,7 @@ TEST_F(UpgradeTest, UpgradeAgentIntoHierarchicalRoleForHierarchicalRole)
   AWAIT_READY(slaveRegisteredMessage);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("foo/bar");
+  frameworkInfo.set_roles(0, "foo/bar");
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -832,7 +838,7 @@ TEST_F(UpgradeTest, RefineResourceOnOldAgent)
   AWAIT_READY(slaveRegisteredMessage);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1/xyz");
+  frameworkInfo.set_roles(0, "role1/xyz");
 
   // Check that `DEFAULT_FRAMEWORK_INFO` includes the
   // RESERVATION_REFINEMENT framework capability.
@@ -859,13 +865,13 @@ TEST_F(UpgradeTest, RefineResourceOnOldAgent)
   Resources baseReservation = Resources::parse("disk(role1):1024").get();
   Resources refinedReservation = baseReservation.pushReservation(
       createDynamicReservationInfo(
-          frameworkInfo.role(), frameworkInfo.principal()));
+          frameworkInfo.roles(0), frameworkInfo.principal()));
 
   Offer offer = offers->front();
 
   // Expect a resource offer containing a static reservation for `role1`.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(baseReservation, frameworkInfo.role())));
+      allocatedResources(baseReservation, frameworkInfo.roles(0))));
 
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
@@ -895,7 +901,7 @@ TEST_F(UpgradeTest, RefineResourceOnOldAgent)
 
   // Expect another resource offer with the same static reservation.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(baseReservation, frameworkInfo.role())));
+      allocatedResources(baseReservation, frameworkInfo.roles(0))));
 
   TaskInfo taskInfo =
     createTask(offer.slave_id(), refinedReservation, "sleep 100");
@@ -927,7 +933,7 @@ TEST_F(UpgradeTest, RefineResourceOnOldAgent)
 
   // Expect another resource offer with the same static reservation.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(baseReservation, frameworkInfo.role())));
+      allocatedResources(baseReservation, frameworkInfo.roles(0))));
 
   // Make sure that any in-flight messages are delivered.
   Clock::settle();


[3/3] mesos git commit: Updated the tests to use MULTI_ROLE frameworks by default.

Posted by bm...@apache.org.
Updated the tests to use MULTI_ROLE frameworks by default.

Now that we strip the `Resource.allocation_info` for non-MULTI_ROLE
schedulers, it's simpler to default the tests to use the MULTI_ROLE
capability, since we've already updated the majority of the tests
to be aware of `Resource.allocation_info`.

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


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

Branch: refs/heads/master
Commit: 5044610ca405fe11fc507aaaa8d29e2f80bd1301
Parents: d321b1d
Author: Benjamin Mahler <bm...@apache.org>
Authored: Wed Nov 29 17:41:11 2017 -0800
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Fri Dec 1 16:20:33 2017 -0800

----------------------------------------------------------------------
 src/examples/balloon_framework.cpp              |  10 +-
 src/examples/disk_full_framework.cpp            |   7 +-
 src/examples/dynamic_reservation_framework.cpp  |   4 +-
 src/examples/long_lived_framework.cpp           |  11 +-
 src/examples/no_executor_framework.cpp          |   5 +-
 src/examples/persistent_volume_framework.cpp    |  14 +-
 src/examples/test_framework.cpp                 |   4 +-
 src/examples/test_http_framework.cpp            |  13 +-
 src/tests/api_tests.cpp                         |  26 +--
 .../containerizer/cgroups_isolator_tests.cpp    |   2 +-
 .../docker_containerizer_tests.cpp              |   6 +-
 .../linux_filesystem_isolator_tests.cpp         |   8 +-
 src/tests/default_executor_tests.cpp            |  38 ++--
 src/tests/disk_quota_tests.cpp                  |   4 +-
 src/tests/fault_tolerance_tests.cpp             |  11 +-
 src/tests/hook_tests.cpp                        |   2 +-
 src/tests/master_allocator_tests.cpp            |  16 +-
 src/tests/master_authorization_tests.cpp        |  12 +-
 src/tests/master_quota_tests.cpp                |   4 +-
 src/tests/master_tests.cpp                      |  71 ++++---
 src/tests/master_validation_tests.cpp           |  23 +-
 src/tests/mesos.hpp                             |   9 +-
 src/tests/oversubscription_tests.cpp            |  16 +-
 src/tests/partition_tests.cpp                   |   2 +-
 src/tests/persistent_volume_endpoints_tests.cpp |  72 +++----
 src/tests/persistent_volume_tests.cpp           | 122 ++++++-----
 src/tests/reservation_endpoints_tests.cpp       |  84 ++++----
 src/tests/reservation_tests.cpp                 | 210 +++++++++----------
 src/tests/resource_provider_manager_tests.cpp   |   4 +-
 src/tests/role_tests.cpp                        |  36 ++--
 src/tests/scheduler_http_api_tests.cpp          |   2 +-
 src/tests/scheduler_tests.cpp                   |   4 +-
 src/tests/slave_authorization_tests.cpp         |   4 +-
 src/tests/slave_recovery_tests.cpp              |   4 +-
 src/tests/slave_tests.cpp                       |  25 ++-
 src/tests/upgrade_tests.cpp                     |  28 ++-
 36 files changed, 476 insertions(+), 437 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/examples/balloon_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/balloon_framework.cpp b/src/examples/balloon_framework.cpp
index 2895253..a36e04b 100644
--- a/src/examples/balloon_framework.cpp
+++ b/src/examples/balloon_framework.cpp
@@ -184,6 +184,7 @@ public:
       const ExecutorInfo& _executor,
       const Flags& _flags)
     : frameworkInfo(_frameworkInfo),
+      role(_frameworkInfo.roles(0)),
       executor(_executor),
       flags(_flags),
       taskActive(false),
@@ -211,10 +212,10 @@ public:
     Resources taskResources = Resources::parse(
         "cpus:" + stringify(CPUS_PER_TASK) +
         ";mem:" + stringify(flags.task_memory.megabytes())).get();
-    taskResources.allocate(frameworkInfo.role());
+    taskResources.allocate(role);
 
     Resources executorResources = Resources(executor.resources());
-    executorResources.allocate(frameworkInfo.role());
+    executorResources.allocate(role);
 
     foreach (const Offer& offer, offers) {
       Resources resources(offer.resources());
@@ -312,6 +313,7 @@ public:
 
 private:
   const FrameworkInfo frameworkInfo;
+  const string role;
   const ExecutorInfo executor;
   const Flags flags;
   bool taskActive;
@@ -568,7 +570,9 @@ int main(int argc, char** argv)
   framework.set_user(os::user().get());
   framework.set_name(flags.name);
   framework.set_checkpoint(flags.checkpoint);
-  framework.set_role("*");
+  framework.add_roles("*");
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
   framework.add_capabilities()->set_type(
       FrameworkInfo::Capability::RESERVATION_REFINEMENT);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/examples/disk_full_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/disk_full_framework.cpp b/src/examples/disk_full_framework.cpp
index 28f71c3..d9d2d35 100644
--- a/src/examples/disk_full_framework.cpp
+++ b/src/examples/disk_full_framework.cpp
@@ -123,6 +123,7 @@ public:
       const FrameworkInfo& _frameworkInfo)
     : flags(_flags),
       frameworkInfo(_frameworkInfo),
+      role(_frameworkInfo.roles(0)),
       tasksLaunched(0),
       taskActive(false),
       isRegistered(false),
@@ -149,7 +150,7 @@ public:
         "cpus:" + stringify(CPUS_PER_TASK) +
         ";mem:" + stringify(MEMORY_PER_TASK) +
         ";disk:" + stringify(DISK_PER_TASK.megabytes())).get();
-    taskResources.allocate(frameworkInfo.role());
+    taskResources.allocate(role);
 
     foreach (const Offer& offer, offers) {
       LOG(INFO) << "Received offer " << offer.id() << " from agent "
@@ -272,6 +273,7 @@ public:
 private:
   const Flags flags;
   const FrameworkInfo frameworkInfo;
+  const string role;
 
   int tasksLaunched;
   bool taskActive;
@@ -465,6 +467,9 @@ int main(int argc, char** argv)
   framework.set_user(""); // Have Mesos fill the current user.
   framework.set_name(flags.name);
   framework.set_checkpoint(true);
+  framework.add_roles("*");
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
   framework.add_capabilities()->set_type(
       FrameworkInfo::Capability::RESERVATION_REFINEMENT);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/examples/dynamic_reservation_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/dynamic_reservation_framework.cpp b/src/examples/dynamic_reservation_framework.cpp
index 5ee3867..538fbe8 100644
--- a/src/examples/dynamic_reservation_framework.cpp
+++ b/src/examples/dynamic_reservation_framework.cpp
@@ -399,7 +399,9 @@ int main(int argc, char** argv)
   FrameworkInfo framework;
   framework.set_user(""); // Mesos'll fill in the current user.
   framework.set_name(FRAMEWORK_NAME);
-  framework.set_role(flags.role.get());
+  framework.add_roles(flags.role.get());
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
   framework.set_principal(flags.principal);
   framework.add_capabilities()->set_type(
       FrameworkInfo::Capability::RESERVATION_REFINEMENT);

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/examples/long_lived_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/long_lived_framework.cpp b/src/examples/long_lived_framework.cpp
index a8516a7..bb9fc2d 100644
--- a/src/examples/long_lived_framework.cpp
+++ b/src/examples/long_lived_framework.cpp
@@ -118,12 +118,13 @@ public:
     : state(DISCONNECTED),
       master(_master),
       framework(_framework),
+      role(_framework.roles(0)),
       executor(_executor),
-      taskResources([&_framework]() {
+      taskResources([this]() {
         Resources resources = Resources::parse(
             "cpus:" + stringify(CPUS_PER_TASK) +
             ";mem:" + stringify(MEM_PER_TASK)).get();
-        resources.allocate(_framework.role());
+        resources.allocate(this->role);
         return resources;
       }()),
       tasksLaunched(0),
@@ -260,7 +261,7 @@ protected:
 
     const Resources executorResources = [this]() {
       Resources resources(executor.resources());
-      resources.allocate(framework.role());
+      resources.allocate(role);
       return resources;
     }();
 
@@ -425,6 +426,7 @@ private:
 
   const string master;
   FrameworkInfo framework;
+  const string role;
   const ExecutorInfo executor;
   const Resources taskResources;
   string uri;
@@ -662,6 +664,9 @@ int main(int argc, char** argv)
   framework.set_user(os::user().get());
   framework.set_name(FRAMEWORK_NAME);
   framework.set_checkpoint(flags.checkpoint);
+  framework.add_roles("*");
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
   framework.add_capabilities()->set_type(
       FrameworkInfo::Capability::RESERVATION_REFINEMENT);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/examples/no_executor_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/no_executor_framework.cpp b/src/examples/no_executor_framework.cpp
index fd920f5..972ef77 100644
--- a/src/examples/no_executor_framework.cpp
+++ b/src/examples/no_executor_framework.cpp
@@ -328,6 +328,9 @@ int main(int argc, char** argv)
   framework.set_user(""); // Have Mesos fill in the current user.
   framework.set_name(FRAMEWORK_NAME);
   framework.set_checkpoint(flags.checkpoint);
+  framework.add_roles("*");
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
   framework.add_capabilities()->set_type(
       FrameworkInfo::Capability::RESERVATION_REFINEMENT);
 
@@ -364,7 +367,7 @@ int main(int argc, char** argv)
     }
   }
 
-  taskResources.allocate(framework.role());
+  taskResources.allocate(framework.roles(0));
 
   NoExecutorScheduler scheduler(
       framework,

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/examples/persistent_volume_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/persistent_volume_framework.cpp b/src/examples/persistent_volume_framework.cpp
index 674d58a..9ccc7eb 100644
--- a/src/examples/persistent_volume_framework.cpp
+++ b/src/examples/persistent_volume_framework.cpp
@@ -142,14 +142,15 @@ public:
       size_t numShards,
       size_t numSharedShards,
       size_t tasksPerShard)
-    : frameworkInfo(_frameworkInfo)
+    : frameworkInfo(_frameworkInfo),
+      role(_frameworkInfo.roles(0))
   {
     // Initialize the shards using regular persistent volume.
     for (size_t i = 0; i < numShards; i++) {
       shards.push_back(
           Shard(
               "shard-" + stringify(i),
-              frameworkInfo.role(),
+              role,
               tasksPerShard,
               false));
     }
@@ -159,7 +160,7 @@ public:
       shards.push_back(
           Shard(
               "shared-shard-" + stringify(i),
-              frameworkInfo.role(),
+              role,
               tasksPerShard,
               true));
     }
@@ -209,7 +210,7 @@ public:
 
             if (offered.contains(shard.resources)) {
               Resource volume = SHARD_PERSISTENT_VOLUME(
-                  frameworkInfo.role(),
+                  role,
                   UUID::random().toString(),
                   "volume",
                   frameworkInfo.principal(),
@@ -484,6 +485,7 @@ private:
   };
 
   FrameworkInfo frameworkInfo;
+  const string role;
   vector<Shard> shards;
 };
 
@@ -566,7 +568,9 @@ int main(int argc, char** argv)
   FrameworkInfo framework;
   framework.set_user(""); // Have Mesos fill in the current user.
   framework.set_name(FRAMEWORK_NAME);
-  framework.set_role(flags.role);
+  framework.add_roles(flags.role);
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
   framework.set_checkpoint(true);
   framework.set_principal(flags.principal);
   framework.add_capabilities()->set_type(

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/examples/test_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_framework.cpp b/src/examples/test_framework.cpp
index d60adff..acf1faf 100644
--- a/src/examples/test_framework.cpp
+++ b/src/examples/test_framework.cpp
@@ -271,7 +271,9 @@ int main(int argc, char** argv)
   FrameworkInfo framework;
   framework.set_user(""); // Have Mesos fill in the current user.
   framework.set_name(FRAMEWORK_NAME);
-  framework.set_role(flags.role);
+  framework.add_roles(flags.role);
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
   framework.add_capabilities()->set_type(
       FrameworkInfo::Capability::RESERVATION_REFINEMENT);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/examples/test_http_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_http_framework.cpp b/src/examples/test_http_framework.cpp
index 76b64fe..e83aded 100644
--- a/src/examples/test_http_framework.cpp
+++ b/src/examples/test_http_framework.cpp
@@ -78,6 +78,7 @@ public:
                 const ExecutorInfo& _executor,
                 const string& _master)
     : framework(_framework),
+      role(_framework.roles(0)),
       executor(_executor),
       master(_master),
       state(INITIALIZING),
@@ -90,6 +91,7 @@ public:
                 const string& _master,
                 const Credential& credential)
     : framework(_framework),
+      role(_framework.roles(0)),
       executor(_executor),
       master(_master),
       state(INITIALIZING),
@@ -235,7 +237,7 @@ private:
       Resources taskResources = Resources::parse(
           "cpus:" + stringify(CPUS_PER_TASK) +
           ";mem:" + stringify(MEM_PER_TASK)).get();
-      taskResources.allocate(framework.role());
+      taskResources.allocate(role);
 
       Resources remaining = offer.resources();
 
@@ -255,12 +257,12 @@ private:
         task.mutable_executor()->MergeFrom(executor);
 
         Option<Resources> resources = [&]() {
-          if (framework.role() == "*") {
+          if (role == "*") {
             return remaining.find(taskResources);
           } else {
             Resource::ReservationInfo reservation;
             reservation.set_type(Resource::ReservationInfo::STATIC);
-            reservation.set_role(framework.role());
+            reservation.set_role(role);
 
             return remaining.find(taskResources.pushReservation(reservation));
           }
@@ -367,6 +369,7 @@ private:
   }
 
   FrameworkInfo framework;
+  const string role;
   const ExecutorInfo executor;
   const string master;
   process::Owned<scheduler::Mesos> mesos;
@@ -447,7 +450,9 @@ int main(int argc, char** argv)
 
   FrameworkInfo framework;
   framework.set_name(FRAMEWORK_NAME);
-  framework.set_role(flags.role);
+  framework.add_roles(flags.role);
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
   framework.add_capabilities()->set_type(
       FrameworkInfo::Capability::RESERVATION_REFINEMENT);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 66cb059..53c705e 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -259,7 +259,7 @@ TEST_P(MasterAPITest, GetFrameworks)
 
   ASSERT_EQ(1, frameworks.frameworks_size());
   ASSERT_EQ("default", frameworks.frameworks(0).framework_info().name());
-  ASSERT_EQ("*", frameworks.frameworks(0).framework_info().role());
+  ASSERT_EQ("*", frameworks.frameworks(0).framework_info().roles(0));
   ASSERT_FALSE(frameworks.frameworks(0).framework_info().checkpoint());
   ASSERT_TRUE(frameworks.frameworks(0).active());
   ASSERT_TRUE(frameworks.frameworks(0).connected());
@@ -904,7 +904,7 @@ TEST_P(MasterAPITest, GetRoles)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -995,12 +995,12 @@ TEST_P(MasterAPITest, ReserveResources)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -1021,7 +1021,7 @@ TEST_P(MasterAPITest, ReserveResources)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   // Expect an offer to be rescinded!
   EXPECT_CALL(sched, offerRescinded(_, _));
@@ -1058,7 +1058,7 @@ TEST_P(MasterAPITest, ReserveResources)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -1086,12 +1086,12 @@ TEST_P(MasterAPITest, UnreserveResources)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
 
   Resources unreserved = Resources::parse("cpus:1;mem:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   v1::master::Call v1Call;
   v1Call.set_type(v1::master::Call::RESERVE_RESOURCES);
@@ -1132,7 +1132,7 @@ TEST_P(MasterAPITest, UnreserveResources)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   // Expect an offer to be rescinded!
   EXPECT_CALL(sched, offerRescinded(_, _));
@@ -1169,7 +1169,7 @@ TEST_P(MasterAPITest, UnreserveResources)
 
   // Verifies if the resources are unreserved.
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -3016,7 +3016,7 @@ TEST_P(MasterAPITest, CreateAndDestroyVolumes)
       v1CreateVolumesResponse);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
 
   // Start a framework and launch a task on the persistent volume.
   MockScheduler sched;
@@ -3037,11 +3037,11 @@ TEST_P(MasterAPITest, CreateAndDestroyVolumes)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   Resources taskResources = Resources::parse(
       "disk:256",
-      frameworkInfo.role()).get();
+      frameworkInfo.roles(0)).get();
 
   TaskInfo taskInfo = createTask(
       offer.slave_id(),

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/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 421a72f..54404ee 100644
--- a/src/tests/containerizer/cgroups_isolator_tests.cpp
+++ b/src/tests/containerizer/cgroups_isolator_tests.cpp
@@ -318,7 +318,7 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_RevocableCpu)
   // Now the framework will get revocable resources.
   AWAIT_READY(offers2);
   ASSERT_FALSE(offers2->empty());
-  EXPECT_EQ(allocatedResources(cpus, frameworkInfo.role()),
+  EXPECT_EQ(allocatedResources(cpus, frameworkInfo.roles(0)),
             Resources(offers2.get()[0].resources()));
 
   TaskInfo task = createTask(

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/containerizer/docker_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/docker_containerizer_tests.cpp b/src/tests/containerizer/docker_containerizer_tests.cpp
index 7a42bb9..1f6ff52 100644
--- a/src/tests/containerizer/docker_containerizer_tests.cpp
+++ b/src/tests/containerizer/docker_containerizer_tests.cpp
@@ -1607,7 +1607,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_LaunchWithPersistentVolumes)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -1767,7 +1767,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_RecoverPersistentVolumes)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
   frameworkInfo.set_checkpoint(true);
 
   MockScheduler sched;
@@ -1935,7 +1935,7 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_RecoverOrphanedPersistentVolumes)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
   frameworkInfo.set_checkpoint(true);
 
   MockScheduler sched;

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/linux_filesystem_isolator_tests.cpp b/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
index 655f9f8..4d796a8 100644
--- a/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
+++ b/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
@@ -755,7 +755,7 @@ TEST_F(LinuxFilesystemIsolatorMesosTest,
 
   MockScheduler sched;
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
 
   MesosSchedulerDriver driver(
       &sched,
@@ -898,7 +898,7 @@ TEST_F(LinuxFilesystemIsolatorMesosTest,
 
   MockScheduler sched;
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
   frameworkInfo.set_checkpoint(true);
 
   MesosSchedulerDriver driver(
@@ -1128,7 +1128,7 @@ TEST_F(LinuxFilesystemIsolatorMesosTest,
 
   MockScheduler sched;
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
 
   MesosSchedulerDriver driver(
       &sched,
@@ -1226,7 +1226,7 @@ TEST_F(LinuxFilesystemIsolatorMesosTest,
 
   MockScheduler sched;
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
   frameworkInfo.add_capabilities()->set_type(
       FrameworkInfo::Capability::SHARED_RESOURCES);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/default_executor_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/default_executor_tests.cpp b/src/tests/default_executor_tests.cpp
index 0b80d4a..d3e3ef7 100644
--- a/src/tests/default_executor_tests.cpp
+++ b/src/tests/default_executor_tests.cpp
@@ -1322,7 +1322,7 @@ TEST_P(DefaultExecutorTest, ReservedResources)
   auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
 
   v1::FrameworkInfo frameworkInfo = v1::DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   EXPECT_CALL(*scheduler, connected(_))
     .WillOnce(v1::scheduler::SendSubscribe(frameworkInfo));
@@ -1359,7 +1359,7 @@ TEST_P(DefaultExecutorTest, ReservedResources)
   // Launch the executor using reserved resources.
   v1::Resources reserved =
     unreserved.pushReservation(v1::createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   v1::ExecutorInfo executorInfo = v1::createExecutorInfo(
       v1::DEFAULT_EXECUTOR_ID,
@@ -2300,7 +2300,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
 
   v1::FrameworkInfo frameworkInfo = v1::DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   EXPECT_CALL(*scheduler, connected(_))
     .WillOnce(v1::scheduler::SendSubscribe(frameworkInfo));
@@ -2330,11 +2330,11 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
 
   v1::Resources reserved =
     unreserved.pushReservation(v1::createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   v1::Resource volume = v1::createPersistentVolume(
       Megabytes(1),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "executor_volume_path",
       frameworkInfo.principal(),
@@ -2446,7 +2446,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
 
   v1::FrameworkInfo frameworkInfo = v1::DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   EXPECT_CALL(*scheduler, connected(_))
     .WillOnce(v1::scheduler::SendSubscribe(frameworkInfo));
@@ -2488,7 +2488,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
 
   v1::Resource volume = v1::createPersistentVolume(
       Megabytes(1),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "task_volume_path",
       frameworkInfo.principal(),
@@ -2497,7 +2497,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
 
   v1::Resources reserved =
     unreserved.pushReservation(v1::createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // Launch a task that expects the persistent volume to be
   // mounted in its sandbox.
@@ -2573,7 +2573,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
 
   v1::FrameworkInfo frameworkInfo = v1::DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   EXPECT_CALL(*scheduler, connected(_))
     .WillOnce(v1::scheduler::SendSubscribe(frameworkInfo));
@@ -2602,17 +2602,17 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
     v1::Resources::parse("cpus:0.1;mem:32;disk:32").get()
       .pushReservation(
           v1::createDynamicReservationInfo(
-              frameworkInfo.role(), frameworkInfo.principal()));
+              frameworkInfo.roles(0), frameworkInfo.principal()));
 
   v1::Resources totalResources =
     v1::Resources::parse("cpus:0.3;mem:96;disk:96").get()
       .pushReservation(
           v1::createDynamicReservationInfo(
-              frameworkInfo.role(), frameworkInfo.principal()));
+              frameworkInfo.roles(0), frameworkInfo.principal()));
 
   v1::Resource executorVolume = v1::createPersistentVolume(
       Megabytes(1),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "executor",
       "executor_volume_path",
       frameworkInfo.principal(),
@@ -2790,7 +2790,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
 
   v1::FrameworkInfo frameworkInfo = v1::DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   EXPECT_CALL(*scheduler, connected(_))
     .WillOnce(v1::scheduler::SendSubscribe(frameworkInfo));
@@ -2819,17 +2819,17 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
     v1::Resources::parse("cpus:0.1;mem:32;disk:32").get()
       .pushReservation(
           v1::createDynamicReservationInfo(
-              frameworkInfo.role(), frameworkInfo.principal()));
+              frameworkInfo.roles(0), frameworkInfo.principal()));
 
   v1::Resources totalResources =
     v1::Resources::parse("cpus:0.3;mem:96;disk:96").get()
       .pushReservation(
           v1::createDynamicReservationInfo(
-              frameworkInfo.role(), frameworkInfo.principal()));
+              frameworkInfo.roles(0), frameworkInfo.principal()));
 
   v1::Resource executorVolume = v1::createPersistentVolume(
       Megabytes(1),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "executor",
       "executor_volume_path",
       frameworkInfo.principal(),
@@ -3022,7 +3022,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
   auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
 
   v1::FrameworkInfo frameworkInfo = v1::DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   EXPECT_CALL(*scheduler, connected(_))
     .WillOnce(v1::scheduler::SendSubscribe(frameworkInfo));
@@ -3064,7 +3064,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
 
   v1::Resource volume = v1::createPersistentVolume(
       Megabytes(1),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "task_volume_path",
       frameworkInfo.principal(),
@@ -3073,7 +3073,7 @@ TEST_P_TEMP_DISABLED_ON_WINDOWS(
 
   v1::Resources reserved =
     unreserved.pushReservation(v1::createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   // Launch a task that expects the persistent volume to be
   // mounted in its sandbox.

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/disk_quota_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/disk_quota_tests.cpp b/src/tests/disk_quota_tests.cpp
index fc29799..4e25e32 100644
--- a/src/tests/disk_quota_tests.cpp
+++ b/src/tests/disk_quota_tests.cpp
@@ -251,7 +251,7 @@ TEST_F(DiskQuotaTest, DiskUsageExceedsQuota)
 TEST_F(DiskQuotaTest, VolumeUsageExceedsQuota)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
 
   master::Flags masterFlags = CreateMasterFlags();
 
@@ -490,7 +490,7 @@ TEST_F(DiskQuotaTest, ResourceStatistics)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   MockScheduler sched;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/fault_tolerance_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fault_tolerance_tests.cpp b/src/tests/fault_tolerance_tests.cpp
index 33a2220..6f69b6f 100644
--- a/src/tests/fault_tolerance_tests.cpp
+++ b/src/tests/fault_tolerance_tests.cpp
@@ -2040,10 +2040,8 @@ TEST_F(FaultToleranceTest, UpdateFrameworkInfoOnSchedulerFailover)
   // scheduler with updated information.
 
   FrameworkInfo finfo1 = DEFAULT_FRAMEWORK_INFO;
-
-  // TODO(mpark): Remove this once `RESERVATION_REFINEMENT`
-  // is removed from `DEFAULT_FRAMEWORK_INFO`.
   finfo1.clear_capabilities();
+  finfo1.clear_roles();
 
   finfo1.set_name("Framework 1");
   finfo1.set_failover_timeout(1000);
@@ -2069,11 +2067,7 @@ TEST_F(FaultToleranceTest, UpdateFrameworkInfoOnSchedulerFailover)
   // updated FrameworkInfo and wait until it gets a registered
   // callback.
 
-  FrameworkInfo finfo2 = DEFAULT_FRAMEWORK_INFO;
-
-  // TODO(mpark): Remove this once `RESERVATION_REFINEMENT`
-  // is removed from `DEFAULT_FRAMEWORK_INFO`.
-  finfo2.clear_capabilities();
+  FrameworkInfo finfo2 = finfo1;
 
   finfo2.mutable_id()->MergeFrom(frameworkId.get());
   auto capabilityType = FrameworkInfo::Capability::REVOCABLE_RESOURCES;
@@ -2082,6 +2076,7 @@ TEST_F(FaultToleranceTest, UpdateFrameworkInfoOnSchedulerFailover)
   finfo2.set_webui_url("http://localhost:8080/");
   finfo2.set_failover_timeout(100);
   finfo2.set_hostname("myHostname");
+  finfo2.clear_labels();
   finfo2.mutable_labels()->add_labels()->CopyFrom(createLabel("baz", "qux"));
 
   MockScheduler sched2;

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/hook_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hook_tests.cpp b/src/tests/hook_tests.cpp
index 2e58d11..26394d8 100644
--- a/src/tests/hook_tests.cpp
+++ b/src/tests/hook_tests.cpp
@@ -1132,7 +1132,7 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(
 
   EXPECT_EQ(TEST_HOOK_CPUS, resources.cpus().get());
 
-  const string allocationRole = DEFAULT_FRAMEWORK_INFO.role();
+  const string allocationRole = DEFAULT_FRAMEWORK_INFO.roles(0);
   EXPECT_TRUE(resources.contains(
       allocatedResources(TEST_HOOK_ADDITIONAL_RESOURCES, allocationRole)));
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/master_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_allocator_tests.cpp b/src/tests/master_allocator_tests.cpp
index 3400d70..10de6f0 100644
--- a/src/tests/master_allocator_tests.cpp
+++ b/src/tests/master_allocator_tests.cpp
@@ -839,7 +839,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveLost)
   EXPECT_EQ(Resources(resourceOffers.get()[0].resources()),
             allocatedResources(
                 Resources::parse(flags2.resources.get()).get(),
-                DEFAULT_FRAMEWORK_INFO.role()));
+                DEFAULT_FRAMEWORK_INFO.roles(0)));
 
   // Shut everything down.
   EXPECT_CALL(allocator, recoverResources(_, _, _, _))
@@ -1287,7 +1287,7 @@ TYPED_TEST(MasterAllocatorTest, RoleTest)
   FrameworkInfo frameworkInfo1 = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo1.set_name("framework1");
   frameworkInfo1.set_user("user1");
-  frameworkInfo1.set_role("role1");
+  frameworkInfo1.set_roles(0, "role1");
 
   MockScheduler sched1;
   MesosSchedulerDriver driver1(
@@ -1306,7 +1306,7 @@ TYPED_TEST(MasterAllocatorTest, RoleTest)
   FrameworkInfo frameworkInfo2 = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo2.set_name("framework2");
   frameworkInfo2.set_user("user2");
-  frameworkInfo2.set_role("role2");
+  frameworkInfo2.set_roles(0, "role2");
 
   MockScheduler sched2;
   MesosSchedulerDriver driver2(
@@ -1656,7 +1656,7 @@ TYPED_TEST(MasterAllocatorTest, RebalancedForUpdatedWeights)
   // and all resources will be offered to this framework since it is the only
   // framework running so far.
   FrameworkInfo frameworkInfo1 = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo1.set_role("role1");
+  frameworkInfo1.set_roles(0, "role1");
   MockScheduler sched1;
   MesosSchedulerDriver driver1(
       &sched1, frameworkInfo1, master.get()->pid, DEFAULT_CREDENTIAL);
@@ -1686,7 +1686,7 @@ TYPED_TEST(MasterAllocatorTest, RebalancedForUpdatedWeights)
   // It will not get any offers due to all resources having outstanding offers
   // to framework1 when it registered.
   FrameworkInfo frameworkInfo2 = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo2.set_role("role2");
+  frameworkInfo2.set_roles(0, "role2");
   MockScheduler sched2;
   MesosSchedulerDriver driver2(
       &sched2, frameworkInfo2, master.get()->pid, DEFAULT_CREDENTIAL);
@@ -1834,7 +1834,7 @@ TYPED_TEST(MasterAllocatorTest, NestedRoles)
   // Register a framework in the "a/b" role and launch a single task,
   // consuming all the resources on `slave1`.
   FrameworkInfo frameworkInfo1 = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo1.set_role("a/b");
+  frameworkInfo1.set_roles(0, "a/b");
 
   MockScheduler sched1;
   MesosSchedulerDriver driver1(
@@ -1864,7 +1864,7 @@ TYPED_TEST(MasterAllocatorTest, NestedRoles)
   // Register a framework in the "a/c" role. It should not get any
   // offers, because there are no unused resources.
   FrameworkInfo frameworkInfo2 = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo2.set_role("a/c");
+  frameworkInfo2.set_roles(0, "a/c");
 
   MockScheduler sched2;
   MesosSchedulerDriver driver2(
@@ -1883,7 +1883,7 @@ TYPED_TEST(MasterAllocatorTest, NestedRoles)
   // Register a framework in the "b/x" role. It should not get any
   // offers, because there are no unused resources.
   FrameworkInfo frameworkInfo3 = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo3.set_role("b/x");
+  frameworkInfo3.set_roles(0, "b/x");
 
   MockScheduler sched3;
   MesosSchedulerDriver driver3(

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/master_authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_authorization_tests.cpp b/src/tests/master_authorization_tests.cpp
index eff97f1..676543a 100644
--- a/src/tests/master_authorization_tests.cpp
+++ b/src/tests/master_authorization_tests.cpp
@@ -993,7 +993,7 @@ TEST_F(MasterAuthorizationTest, AuthorizedRole)
   ASSERT_SOME(master);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("foo");
+  frameworkInfo.set_roles(0, "foo");
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -1031,7 +1031,7 @@ TEST_F(MasterAuthorizationTest, UnauthorizedRole)
   ASSERT_SOME(master);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("foo");
+  frameworkInfo.set_roles(0, "foo");
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -1369,7 +1369,7 @@ TYPED_TEST(MasterAuthorizerTest, FilterStateSummaryEndpoint)
 
   // Start framwork with user "bar".
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
   frameworkInfo.set_user(user);
 
   MockScheduler sched;
@@ -1499,7 +1499,7 @@ TYPED_TEST(MasterAuthorizerTest, FilterStateEndpoint)
 
   // Start framwork with user "bar".
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
   frameworkInfo.set_user(user);
 
   // Create an executor with user "bar".
@@ -1686,7 +1686,7 @@ TYPED_TEST(MasterAuthorizerTest, FilterFrameworksEndpoint)
 
   // Start framwork with user "bar".
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
   frameworkInfo.set_user("bar");
 
   // Create an executor with user "bar".
@@ -1875,7 +1875,7 @@ TYPED_TEST(MasterAuthorizerTest, FilterTasksEndpoint)
 
   // Start framwork with user "bar".
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, "role");
   frameworkInfo.set_user(user);
 
   // Create an executor with user "bar".

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/master_quota_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_quota_tests.cpp b/src/tests/master_quota_tests.cpp
index 058f6d2..79c0739 100644
--- a/src/tests/master_quota_tests.cpp
+++ b/src/tests/master_quota_tests.cpp
@@ -115,11 +115,11 @@ protected:
   // Creates a FrameworkInfo with the specified role.
   FrameworkInfo createFrameworkInfo(const string& role)
   {
-    FrameworkInfo info;
+    FrameworkInfo info = DEFAULT_FRAMEWORK_INFO;
     info.set_user("user");
     info.set_name("framework" + process::ID::generate());
     info.mutable_id()->set_value(info.name());
-    info.set_role(role);
+    info.set_roles(0, role);
 
     return info;
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index 57eae32..08742ec 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -976,7 +976,7 @@ TEST_F(MasterTest, RecoverResources)
 
   Resources executorResources = allocatedResources(
       Resources::parse("cpus:0.3;mem:200;ports:[5-8, 23-25]").get(),
-      DEFAULT_FRAMEWORK_INFO.role());
+      DEFAULT_FRAMEWORK_INFO.roles(0));
   executorInfo.mutable_resources()->MergeFrom(executorResources);
 
   TaskID taskId;
@@ -1058,7 +1058,7 @@ TEST_F(MasterTest, RecoverResources)
   ASSERT_FALSE(offers->empty());
 
   Resources slaveResources = Resources::parse(flags.resources.get()).get();
-  EXPECT_EQ(allocatedResources(slaveResources, DEFAULT_FRAMEWORK_INFO.role()),
+  EXPECT_EQ(allocatedResources(slaveResources, DEFAULT_FRAMEWORK_INFO.roles(0)),
             offers.get()[0].resources());
 
   driver.stop();
@@ -2072,11 +2072,8 @@ TEST_F(MasterTest, LaunchDifferentRoleLost)
   ASSERT_SOME(slave);
 
   FrameworkInfo framework = DEFAULT_FRAMEWORK_INFO;
-  framework.clear_role();
-  framework.add_roles("role1");
+  framework.set_roles(0, "role1");
   framework.add_roles("role2");
-  framework.add_capabilities()->set_type(
-      FrameworkInfo::Capability::MULTI_ROLE);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -4032,7 +4029,7 @@ TEST_F(MasterTest, IgnoreEphemeralPortsResource)
       Resources(offers.get()[0].resources()),
       allocatedResources(
           Resources::parse(resourcesWithoutEphemeralPorts).get(),
-          DEFAULT_FRAMEWORK_INFO.role()));
+          DEFAULT_FRAMEWORK_INFO.roles(0)));
 
   driver.stop();
   driver.join();
@@ -4547,9 +4544,6 @@ TEST_F(MasterTest, StateEndpointFrameworkInfo)
   AWAIT_READY(slaveRegisteredMessage);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-
-  // TODO(mpark): Remove this once `RESERVATION_REFINEMENT`
-  // is removed from `DEFAULT_FRAMEWORK_INFO`.
   frameworkInfo.clear_capabilities();
 
   frameworkInfo.set_webui_url("http://localhost:8080/");
@@ -4558,7 +4552,9 @@ TEST_F(MasterTest, StateEndpointFrameworkInfo)
     FrameworkInfo::Capability::REVOCABLE_RESOURCES,
     FrameworkInfo::Capability::TASK_KILLING_STATE,
     FrameworkInfo::Capability::GPU_RESOURCES,
-    FrameworkInfo::Capability::PARTITION_AWARE
+    FrameworkInfo::Capability::PARTITION_AWARE,
+    FrameworkInfo::Capability::MULTI_ROLE,
+    FrameworkInfo::Capability::RESERVATION_REFINEMENT,
   };
 
   foreach (FrameworkInfo::Capability::Type capability, capabilities) {
@@ -4634,7 +4630,7 @@ TEST_F(MasterTest, StateEndpointFrameworkInfo)
     .as<JSON::Array>().values[0].as<JSON::Object>();
 
   JSON::Object allocationInfo;
-  allocationInfo.values["role"] = frameworkInfo.role();
+  allocationInfo.values["role"] = frameworkInfo.roles(0);
 
   ASSERT_EQ(1u, offer.values.count("allocation_info"));
   EXPECT_EQ(allocationInfo, offer.values.at("allocation_info"));
@@ -4809,7 +4805,7 @@ TEST_F(MasterTest, StateEndpointAllocationRole)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("foo");
+  frameworkInfo.set_roles(0, "foo");
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -4879,10 +4875,10 @@ TEST_F(MasterTest, StateEndpointAllocationRole)
       JSON::Array {
         JSON::Object {
           { "executors", JSON::Array {
-            JSON::Object { { "role", frameworkInfo.role() } } }
+            JSON::Object { { "role", frameworkInfo.roles(0) } } }
           },
           { "tasks", JSON::Array {
-            JSON::Object { { "role", frameworkInfo.role() } } }
+            JSON::Object { { "role", frameworkInfo.roles(0) } } }
           }
         }
       }
@@ -5954,7 +5950,7 @@ TEST_F(MasterTest, RejectFrameworkWithInvalidRole)
   FrameworkInfo framework = DEFAULT_FRAMEWORK_INFO;
 
   // Add invalid role to the FrameworkInfo.
-  framework.set_role("/test/test1");
+  framework.set_roles(0, "/test/test1");
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -7366,10 +7362,8 @@ TEST_F(MasterTest, MultiRoleFrameworkReceivesOffers)
   ASSERT_SOME(slave1);
 
   FrameworkInfo framework = DEFAULT_FRAMEWORK_INFO;
-  framework.add_roles("role1");
+  framework.set_roles(0, "role1");
   framework.add_roles("role2");
-  framework.add_capabilities()->set_type(
-      FrameworkInfo::Capability::MULTI_ROLE);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -7517,12 +7511,9 @@ TEST_F(MasterTest, MultiRoleSchedulerUnsubscribeFromRole)
   // capability, so we expect its tasks to continue running when the
   // partitioned agent reregisters.
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.clear_role();
-  frameworkInfo.add_roles("foo");
+  frameworkInfo.set_roles(0, "foo");
   frameworkInfo.add_capabilities()->set_type(
       FrameworkInfo::Capability::PARTITION_AWARE);
-  frameworkInfo.add_capabilities()->set_type(
-      FrameworkInfo::Capability::MULTI_ROLE);
 
   MockScheduler sched1;
   MesosSchedulerDriver driver1(
@@ -8299,11 +8290,13 @@ INSTANTIATE_TEST_CASE_P(
 TEST_P(MasterTestPrePostReservationRefinement, LaunchTask)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   // TODO(mpark): Remove this once `RESERVATION_REFINEMENT`
   // is removed from `DEFAULT_FRAMEWORK_INFO`.
   frameworkInfo.clear_capabilities();
+  frameworkInfo.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
 
   if (GetParam()) {
     frameworkInfo.add_capabilities()->set_type(
@@ -8379,11 +8372,13 @@ TEST_P(MasterTestPrePostReservationRefinement, LaunchTask)
 TEST_P(MasterTestPrePostReservationRefinement, LaunchGroup)
 {
   v1::FrameworkInfo frameworkInfo = v1::DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   // TODO(mpark): Remove this once `RESERVATION_REFINEMENT`
   // is removed from `DEFAULT_FRAMEWORK_INFO`.
   frameworkInfo.clear_capabilities();
+  frameworkInfo.add_capabilities()->set_type(
+      v1::FrameworkInfo::Capability::MULTI_ROLE);
 
   if (GetParam()) {
     frameworkInfo.add_capabilities()->set_type(
@@ -8544,11 +8539,13 @@ TEST_P(MasterTestPrePostReservationRefinement,
        ReserveCreateLaunchDestroyUnreserve)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   // TODO(mpark): Remove this once `RESERVATION_REFINEMENT`
   // is removed from `DEFAULT_FRAMEWORK_INFO`.
   frameworkInfo.clear_capabilities();
+  frameworkInfo.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
 
   if (GetParam()) {
     frameworkInfo.add_capabilities()->set_type(
@@ -8557,7 +8554,7 @@ TEST_P(MasterTestPrePostReservationRefinement,
 
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(5);
-  masterFlags.roles = frameworkInfo.role();
+  masterFlags.roles = frameworkInfo.roles(0);
 
   Try<Owned<cluster::Master>> master = StartMaster(masterFlags);
   ASSERT_SOME(master);
@@ -8583,11 +8580,11 @@ TEST_P(MasterTestPrePostReservationRefinement,
 
   Resources reservedCpus =
     unreservedCpus.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   Resources reservedDisk =
     unreservedDisk.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), frameworkInfo.principal()));
+        frameworkInfo.roles(0), frameworkInfo.principal()));
 
   Resources volume = createPersistentVolume(
       createDiskResource("512", DEFAULT_TEST_ROLE, None(), None()),
@@ -8615,7 +8612,7 @@ TEST_P(MasterTestPrePostReservationRefinement,
 
   EXPECT_TRUE(inboundResources(offer.resources())
                 .contains(allocatedResources(
-                    unreservedCpus + unreservedDisk, frameworkInfo.role())));
+                    unreservedCpus + unreservedDisk, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -8640,7 +8637,7 @@ TEST_P(MasterTestPrePostReservationRefinement,
 
   EXPECT_TRUE(inboundResources(offer.resources())
                 .contains(allocatedResources(
-                    reservedCpus + reservedDisk, frameworkInfo.role())));
+                    reservedCpus + reservedDisk, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -8665,7 +8662,7 @@ TEST_P(MasterTestPrePostReservationRefinement,
 
   EXPECT_TRUE(inboundResources(offer.resources())
                 .contains(allocatedResources(
-                    reservedCpus + volume, frameworkInfo.role())));
+                    reservedCpus + volume, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -8690,7 +8687,7 @@ TEST_P(MasterTestPrePostReservationRefinement,
 
   EXPECT_TRUE(inboundResources(offer.resources())
                 .contains(allocatedResources(
-                    reservedCpus + reservedDisk, frameworkInfo.role())));
+                    reservedCpus + reservedDisk, frameworkInfo.roles(0))));
 
   // The expectation for the next offer.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -8715,7 +8712,7 @@ TEST_P(MasterTestPrePostReservationRefinement,
 
   EXPECT_TRUE(inboundResources(offer.resources())
                 .contains(allocatedResources(
-                    unreservedCpus + unreservedDisk, frameworkInfo.role())));
+                    unreservedCpus + unreservedDisk, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -8727,11 +8724,13 @@ TEST_P(MasterTestPrePostReservationRefinement,
 TEST_P(MasterTestPrePostReservationRefinement, StateEndpointPendingTasks)
 {
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   // TODO(mpark): Remove this once `RESERVATION_REFINEMENT`
   // is removed from `DEFAULT_FRAMEWORK_INFO`.
   frameworkInfo.clear_capabilities();
+  frameworkInfo.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
 
   if (GetParam()) {
     frameworkInfo.add_capabilities()->set_type(
@@ -8806,7 +8805,7 @@ TEST_P(MasterTestPrePostReservationRefinement, StateEndpointPendingTasks)
             JSON::Array {
               JSON::Object {
                 { "id", "1" },
-                { "role", frameworkInfo.role() },
+                { "role", frameworkInfo.roles(0) },
                 { "state", "TASK_STAGING" }
               }
             }

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/master_validation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_validation_tests.cpp b/src/tests/master_validation_tests.cpp
index 0e1c8b4..6398f16 100644
--- a/src/tests/master_validation_tests.cpp
+++ b/src/tests/master_validation_tests.cpp
@@ -917,7 +917,7 @@ TEST_F(CreateOperationValidationTest, SharedVolumeBasedOnCapability)
   // When a FrameworkInfo with no SHARED_RESOURCES capability is
   // specified, the validation should fail.
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
 
   error = operation::validate(
       create, Resources(), None(), capabilities, frameworkInfo);
@@ -943,14 +943,14 @@ TEST_F(CreateOperationValidationTest, InsufficientDiskResource)
   protobuf::slave::Capabilities capabilities;
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
 
   master::Flags masterFlags = CreateMasterFlags();
 
   ACLs acls;
   mesos::ACL::RegisterFramework* acl = acls.add_register_frameworks();
   acl->mutable_principals()->add_values(frameworkInfo.principal());
-  acl->mutable_roles()->add_values(frameworkInfo.role());
+  acl->mutable_roles()->add_values(frameworkInfo.roles(0));
 
   masterFlags.acls = acls;
   masterFlags.roles = "role1";
@@ -3703,7 +3703,8 @@ TEST_F(FrameworkInfoValidationTest, MissingMultiRoleCapability)
   ASSERT_SOME(master);
 
   FrameworkInfo framework = DEFAULT_FRAMEWORK_INFO;
-  framework.add_roles("role");
+  framework.clear_capabilities();
+  framework.set_roles(0, "role");
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -3726,7 +3727,7 @@ TEST_F(FrameworkInfoValidationTest, AcceptMultiRoleFramework)
   ASSERT_SOME(master);
 
   FrameworkInfo framework = DEFAULT_FRAMEWORK_INFO;
-  framework.add_roles("role1");
+  framework.set_roles(0, "role1");
   framework.add_roles("role2");
   framework.add_capabilities()->set_type(
       FrameworkInfo::Capability::MULTI_ROLE);
@@ -3756,7 +3757,7 @@ TEST_F(FrameworkInfoValidationTest, MultiRoleWhitelist)
   ASSERT_SOME(master);
 
   FrameworkInfo framework = DEFAULT_FRAMEWORK_INFO;
-  framework.add_roles("role1");
+  framework.set_roles(0, "role1");
   framework.add_roles("role2");
   framework.add_capabilities()->set_type(
       FrameworkInfo::Capability::MULTI_ROLE);
@@ -3789,6 +3790,8 @@ TEST_F(FrameworkInfoValidationTest, UpgradeToMultiRole)
   ASSERT_SOME(master);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.clear_capabilities();
+  frameworkInfo.clear_roles();
   frameworkInfo.set_role("role");
 
   // Set a long failover timeout so the framework isn't immediately removed.
@@ -3860,7 +3863,7 @@ TEST_F(FrameworkInfoValidationTest, DowngradeFromMultipleRoles)
   // is removed from `DEFAULT_FRAMEWORK_INFO`.
   frameworkInfo.clear_capabilities();
 
-  frameworkInfo.add_roles("role1");
+  frameworkInfo.set_roles(0, "role1");
   frameworkInfo.add_roles("role2");
   frameworkInfo.add_capabilities()->set_type(
       FrameworkInfo::Capability::MULTI_ROLE);
@@ -3930,9 +3933,7 @@ TEST_F(FrameworkInfoValidationTest, RoleChangeWithMultiRole)
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
 
   ASSERT_FALSE(frameworkInfo.has_role());
-  frameworkInfo.add_roles("role1");
-  frameworkInfo.add_capabilities()->set_type(
-      FrameworkInfo::Capability::MULTI_ROLE);
+  frameworkInfo.set_roles(0, "role1");
 
   // Set a long failover timeout so the framework isn't immediately removed.
   frameworkInfo.set_failover_timeout(Weeks(1).secs());
@@ -4002,6 +4003,8 @@ TEST_F(FrameworkInfoValidationTest, RoleChangeWithMultiRoleMasterFailover)
   // immediately cleaned up.
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
   frameworkInfo.set_failover_timeout(Weeks(1).secs());
+  frameworkInfo.clear_capabilities();
+  frameworkInfo.clear_roles();
   frameworkInfo.set_role("role1");
 
   Future<FrameworkID> frameworkId;

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index aa2571f..3a9b1fb 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -489,6 +489,9 @@ struct DefaultFrameworkInfo
     framework.set_user(os::user().get());
     framework.set_principal(
         DefaultCredential<TCredential>::create().principal());
+    framework.add_roles("*");
+    framework.add_capabilities()->set_type(
+        TFrameworkInfo::Capability::MULTI_ROLE);
     framework.add_capabilities()->set_type(
         TFrameworkInfo::Capability::RESERVATION_REFINEMENT);
 
@@ -2265,7 +2268,11 @@ ACTION_P5(LaunchTasks, executor, tasks, cpus, mem, role)
 
     Resources taskResources = Resources::parse(
         "cpus:" + stringify(cpus) + ";mem:" + stringify(mem)).get();
-    taskResources.allocate(role);
+
+    if (offer.resources_size() > 0 &&
+        offer.resources(0).has_allocation_info()) {
+      taskResources.allocate(role);
+    }
 
     int nextTaskId = 0;
     std::vector<TaskInfo> tasks;

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/oversubscription_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/oversubscription_tests.cpp b/src/tests/oversubscription_tests.cpp
index 57f42e0..3f57ce1 100644
--- a/src/tests/oversubscription_tests.cpp
+++ b/src/tests/oversubscription_tests.cpp
@@ -405,10 +405,10 @@ TEST_F(OversubscriptionTest, RevocableOffer)
   estimations.put(createRevocableResources("cpus", "2"));
 
   Resources taskResources = createRevocableResources("cpus", "1");
-  taskResources.allocate(framework.role());
+  taskResources.allocate(framework.roles(0));
 
   Resources executorResources = createRevocableResources("cpus", "1");
-  executorResources.allocate(framework.role());
+  executorResources.allocate(framework.roles(0));
 
   // Now the framework will get revocable resources.
   AWAIT_READY(offers2);
@@ -514,7 +514,7 @@ TEST_F(OversubscriptionTest, RescindRevocableOfferWithIncreasedRevocable)
   EXPECT_EQ(1u, offers.size());
   Future<Offer> offer = offers.get();
   AWAIT_READY(offer);
-  EXPECT_EQ(allocatedResources(resources1, framework.role()),
+  EXPECT_EQ(allocatedResources(resources1, framework.roles(0)),
             Resources(offer->resources()));
 
   Future<OfferID> offerId;
@@ -552,7 +552,7 @@ TEST_F(OversubscriptionTest, RescindRevocableOfferWithIncreasedRevocable)
   }
 
   // The offered resources should match the resource estimate.
-  EXPECT_EQ(allocatedResources(resources2, framework.role()), resources3);
+  EXPECT_EQ(allocatedResources(resources2, framework.roles(0)), resources3);
 
   driver.stop();
   driver.join();
@@ -628,7 +628,7 @@ TEST_F(OversubscriptionTest, RescindRevocableOfferWithDecreasedRevocable)
   // Now the framework will get revocable resources.
   AWAIT_READY(offers2);
   ASSERT_FALSE(offers2->empty());
-  EXPECT_EQ(allocatedResources(resources1, framework.role()),
+  EXPECT_EQ(allocatedResources(resources1, framework.roles(0)),
             Resources(offers2.get()[0].resources()));
 
   Future<OfferID> offerId;
@@ -659,7 +659,7 @@ TEST_F(OversubscriptionTest, RescindRevocableOfferWithDecreasedRevocable)
   // The new offer should include the latest oversubscribed resources.
   AWAIT_READY(offers3);
   ASSERT_FALSE(offers3->empty());
-  EXPECT_EQ(allocatedResources(resources2, framework.role()),
+  EXPECT_EQ(allocatedResources(resources2, framework.roles(0)),
             Resources(offers3.get()[0].resources()));
 
   driver.stop();
@@ -1314,7 +1314,7 @@ TEST_F(OversubscriptionTest, UpdateAllocatorOnSchedulerFailover)
 
   AWAIT_READY(offers2);
   ASSERT_FALSE(offers2->empty());
-  EXPECT_EQ(allocatedResources(revocable, framework2.role()),
+  EXPECT_EQ(allocatedResources(revocable, framework2.roles(0)),
             Resources(offers2.get()[0].resources()));
 
   EXPECT_EQ(DRIVER_STOPPED, driver2.stop());
@@ -1388,7 +1388,7 @@ TEST_F(OversubscriptionTest, RemoveCapabilitiesOnSchedulerFailover)
   // Now the framework will get revocable resources.
   AWAIT_READY(offers2);
   ASSERT_FALSE(offers2->empty());
-  EXPECT_EQ(allocatedResources(revocable, framework1.role()),
+  EXPECT_EQ(allocatedResources(revocable, framework1.roles(0)),
             Resources(offers2.get()[0].resources()));
 
   // Reregister the framework with removal of revocable resources capability.

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/src/tests/partition_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/partition_tests.cpp b/src/tests/partition_tests.cpp
index 31ebfe1..54ccf78 100644
--- a/src/tests/partition_tests.cpp
+++ b/src/tests/partition_tests.cpp
@@ -873,7 +873,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());
+  taskResources.allocate(DEFAULT_FRAMEWORK_INFO.roles(0));
 
   EXPECT_TRUE(Resources(offer.resources()).contains(taskResources));
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5044610c/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 883192d..94b8caa 100644
--- a/src/tests/persistent_volume_endpoints_tests.cpp
+++ b/src/tests/persistent_volume_endpoints_tests.cpp
@@ -82,7 +82,7 @@ public:
     // `HierarchicalAllocator::updateAvailable()` and periodic allocations.
     master::Flags flags = MesosTest::CreateMasterFlags();
     flags.allocation_interval = Seconds(1000);
-    flags.roles = createFrameworkInfo().role();
+    flags.roles = createFrameworkInfo().roles(0);
     return flags;
   }
 
@@ -90,7 +90,7 @@ public:
   FrameworkInfo createFrameworkInfo()
   {
     FrameworkInfo info = DEFAULT_FRAMEWORK_INFO;
-    info.set_role("role1");
+    info.set_roles(0, "role1");
     return info;
   }
 
@@ -166,7 +166,7 @@ TEST_F(PersistentVolumeEndpointsTest, StaticReservation)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   Future<OfferID> rescindedOfferId;
 
@@ -197,7 +197,7 @@ TEST_F(PersistentVolumeEndpointsTest, StaticReservation)
   offer = offers.get()[0];
 
   EXPECT_FALSE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -229,7 +229,7 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservation)
   Resources unreserved = Resources::parse("disk:1024").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   Future<Response> response = process::http::post(
       master.get()->pid,
@@ -262,7 +262,7 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservation)
 
   EXPECT_TRUE(Resources(offer.resources())
                 .contains(allocatedResources(
-                    dynamicallyReserved, frameworkInfo.role())));
+                    dynamicallyReserved, frameworkInfo.roles(0))));
 
   Future<OfferID> rescindedOfferId;
 
@@ -271,7 +271,7 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservation)
 
   Resources volume = createPersistentVolume(
       Megabytes(64),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "path1",
       DEFAULT_CREDENTIAL.principal(),
@@ -302,7 +302,7 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservation)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   EXPECT_CALL(sched, offerRescinded(&driver, _))
     .WillOnce(FutureArg<1>(&rescindedOfferId));
@@ -352,7 +352,7 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservationRoleMismatch)
   Resources unreserved = Resources::parse("disk:1024").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   Future<Response> response = process::http::post(
       master.get()->pid,
@@ -381,9 +381,9 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservationRoleMismatch)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
-  ASSERT_NE(frameworkInfo.role(), "role2");
+  ASSERT_NE(frameworkInfo.roles(0), "role2");
   Resources volume = createPersistentVolume(
       Megabytes(64),
       "role2",
@@ -431,7 +431,7 @@ TEST_F(PersistentVolumeEndpointsTest, UnreserveVolumeResources)
   Resources unreserved = Resources::parse("disk:1024").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   Future<Response> response = process::http::post(
       master.get()->pid,
@@ -443,7 +443,7 @@ TEST_F(PersistentVolumeEndpointsTest, UnreserveVolumeResources)
 
   Resources volume = createPersistentVolume(
       Megabytes(64),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "path1",
       DEFAULT_CREDENTIAL.principal(),
@@ -711,7 +711,7 @@ TEST_F(PersistentVolumeEndpointsTest, NoHeader)
 
   Resources volume = createPersistentVolume(
       Megabytes(64),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "path1",
       None(),
@@ -868,7 +868,7 @@ TEST_F(PersistentVolumeEndpointsTest, GoodCreateAndDestroyACL)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   Future<OfferID> rescindedOfferId;
 
@@ -899,7 +899,7 @@ TEST_F(PersistentVolumeEndpointsTest, GoodCreateAndDestroyACL)
   offer = offers.get()[0];
 
   EXPECT_FALSE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -1089,7 +1089,7 @@ TEST_F(PersistentVolumeEndpointsTest, BadCreateAndDestroyACL)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   // The failed destruction attempt.
   Future<Response> destroyResponse = process::http::post(
@@ -1286,7 +1286,7 @@ TEST_F(PersistentVolumeEndpointsTest, GoodCreateAndDestroyACLBadCredential)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   // The failed destruction attempt.
   Future<Response> destroyResponse = process::http::post(
@@ -1506,7 +1506,7 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
   Resources unreserved = Resources::parse("disk:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
@@ -1547,12 +1547,12 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   // Create a 1MB persistent volume.
   Resources volume = createPersistentVolume(
       Megabytes(1),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "volume_path",
       frameworkInfo.principal(),
@@ -1580,7 +1580,7 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   Future<OfferID> rescindedOfferId;
 
@@ -1612,7 +1612,7 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   EXPECT_CALL(sched, offerRescinded(&driver, _))
     .WillOnce(FutureArg<1>(&rescindedOfferId));
@@ -1642,7 +1642,7 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -1676,7 +1676,7 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
   Resources unreserved = Resources::parse("disk:512").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   Future<Response> response = process::http::post(
       master.get()->pid,
@@ -1689,7 +1689,7 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
   // Create a 1MB persistent volume.
   Resources volume = createPersistentVolume(
       Megabytes(1),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "volume_path",
       frameworkInfo.principal(),
@@ -1724,7 +1724,7 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   Future<CheckpointResourcesMessage> checkpointResources =
     FUTURE_PROTOBUF(CheckpointResourcesMessage(), _, _);
@@ -1748,9 +1748,9 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
 
   EXPECT_TRUE(Resources(offer.resources())
                 .contains(allocatedResources(
-                    dynamicallyReserved, frameworkInfo.role())))
+                    dynamicallyReserved, frameworkInfo.roles(0))))
     << Resources(offer.resources()) << " vs "
-    << allocatedResources(dynamicallyReserved, frameworkInfo.role());
+    << allocatedResources(dynamicallyReserved, frameworkInfo.roles(0));
 
   checkpointResources = FUTURE_PROTOBUF(CheckpointResourcesMessage(), _, _);
 
@@ -1773,7 +1773,7 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(unreserved, frameworkInfo.role())));
+      allocatedResources(unreserved, frameworkInfo.roles(0))));
 
   driver.stop();
   driver.join();
@@ -1826,7 +1826,7 @@ TEST_F(PersistentVolumeEndpointsTest, ReserveAndSlaveRemoval)
   Resources slave1Unreserved = Resources::parse("cpus:4").get();
   Resources slave1Reserved =
     slave1Unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   Future<Response> response = process::http::post(
       master.get()->pid,
@@ -1863,7 +1863,7 @@ TEST_F(PersistentVolumeEndpointsTest, ReserveAndSlaveRemoval)
   Resources slave2Unreserved = Resources::parse("cpus:3").get();
   Resources slave2Reserved =
     slave2Unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   for (size_t i = 0; i < offers->size(); i++) {
     const Offer& offer = offers.get()[i];
@@ -1931,7 +1931,7 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
   Resources unreserved = Resources::parse("cpus:1;mem:512;disk:1024").get();
   Resources dynamicallyReserved =
     unreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   Future<Response> response = process::http::post(
       master.get()->pid,
@@ -1943,7 +1943,7 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
 
   Resources volume = createPersistentVolume(
       Megabytes(64),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "path1",
       DEFAULT_CREDENTIAL.principal(),
@@ -1979,12 +1979,12 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(volume, frameworkInfo.role())));
+      allocatedResources(volume, frameworkInfo.roles(0))));
 
   Resources taskUnreserved = Resources::parse("cpus:1;mem:256").get();
   Resources taskResources =
     taskUnreserved.pushReservation(createDynamicReservationInfo(
-        frameworkInfo.role(), DEFAULT_CREDENTIAL.principal()));
+        frameworkInfo.roles(0), DEFAULT_CREDENTIAL.principal()));
 
   TaskInfo taskInfo = createTask(offer.slave_id(), taskResources, "sleep 1000");