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 05:34:31 UTC

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

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/5a87d51e
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/5a87d51e
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/5a87d51e

Branch: refs/heads/1.3.x
Commit: 5a87d51ece14a5b884341a39dff456bb8ec48461
Parents: 8855f0e
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 20:18:15 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    |  32 +--
 src/examples/test_framework.cpp                 |   4 +-
 src/examples/test_http_framework.cpp            |  13 +-
 src/tests/api_tests.cpp                         |  26 +--
 .../containerizer/cgroups_isolator_tests.cpp    |   5 +-
 .../docker_containerizer_tests.cpp              |   6 +-
 .../linux_filesystem_isolator_tests.cpp         |   8 +-
 src/tests/default_executor_tests.cpp            |  16 +-
 src/tests/disk_quota_tests.cpp                  |   4 +-
 src/tests/fault_tolerance_tests.cpp             |   6 +-
 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                      |  35 ++-
 src/tests/master_validation_tests.cpp           |  26 ++-
 src/tests/mesos.hpp                             |   9 +-
 src/tests/oversubscription_tests.cpp            |  28 ++-
 src/tests/partition_tests.cpp                   |   2 +-
 src/tests/persistent_volume_endpoints_tests.cpp |  71 +++---
 src/tests/persistent_volume_tests.cpp           | 110 ++++-----
 src/tests/reservation_endpoints_tests.cpp       | 104 ++++-----
 src/tests/reservation_tests.cpp                 | 222 +++++++++----------
 src/tests/role_tests.cpp                        |  36 ++-
 src/tests/scheduler_http_api_tests.cpp          |   2 +-
 src/tests/slave_authorization_tests.cpp         |   2 +-
 src/tests/slave_recovery_tests.cpp              |   4 +-
 src/tests/slave_tests.cpp                       |  21 +-
 src/tests/upgrade_tests.cpp                     |   9 +-
 34 files changed, 464 insertions(+), 408 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/5a87d51e/src/examples/balloon_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/balloon_framework.cpp b/src/examples/balloon_framework.cpp
index bfaae37..1ef43d6 100644
--- a/src/examples/balloon_framework.cpp
+++ b/src/examples/balloon_framework.cpp
@@ -140,6 +140,7 @@ public:
       const ExecutorInfo& _executor,
       const Flags& _flags)
     : frameworkInfo(_frameworkInfo),
+      role(_frameworkInfo.roles(0)),
       executor(_executor),
       flags(_flags),
       taskActive(false),
@@ -167,10 +168,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());
@@ -265,6 +266,7 @@ public:
 
 private:
   const FrameworkInfo frameworkInfo;
+  const string role;
   const ExecutorInfo executor;
   const Flags flags;
   bool taskActive;
@@ -497,7 +499,9 @@ int main(int argc, char** argv)
   framework.set_user(os::user().get());
   framework.set_name("Balloon Framework (C++)");
   framework.set_checkpoint(flags.checkpoint);
-  framework.set_role("*");
+  framework.add_roles("*");
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
 
   BalloonScheduler scheduler(framework, executor, flags);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a87d51e/src/examples/disk_full_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/disk_full_framework.cpp b/src/examples/disk_full_framework.cpp
index a73e6cf..de1de5f 100644
--- a/src/examples/disk_full_framework.cpp
+++ b/src/examples/disk_full_framework.cpp
@@ -116,6 +116,7 @@ public:
       const FrameworkInfo& _frameworkInfo)
     : flags(_flags),
       frameworkInfo(_frameworkInfo),
+      role(_frameworkInfo.roles(0)),
       tasksLaunched(0),
       taskActive(false),
       isRegistered(false),
@@ -142,7 +143,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 "
@@ -257,6 +258,7 @@ public:
 private:
   const Flags flags;
   const FrameworkInfo frameworkInfo;
+  const string role;
 
   int tasksLaunched;
   bool taskActive;
@@ -436,6 +438,9 @@ int main(int argc, char** argv)
   framework.set_user(""); // Have Mesos fill the current user.
   framework.set_name("Disk Full Framework (C++)");
   framework.set_checkpoint(true);
+  framework.add_roles("*");
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
 
   DiskFullScheduler scheduler(flags, framework);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a87d51e/src/examples/dynamic_reservation_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/dynamic_reservation_framework.cpp b/src/examples/dynamic_reservation_framework.cpp
index 4122814..fa90eff 100644
--- a/src/examples/dynamic_reservation_framework.cpp
+++ b/src/examples/dynamic_reservation_framework.cpp
@@ -379,7 +379,9 @@ int main(int argc, char** argv)
   FrameworkInfo framework;
   framework.set_user(""); // Mesos'll fill in the current user.
   framework.set_name("Dynamic Reservation Framework (C++)");
-  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);
 
   DynamicReservationScheduler scheduler(

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a87d51e/src/examples/long_lived_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/long_lived_framework.cpp b/src/examples/long_lived_framework.cpp
index 7ee4f27..4233135 100644
--- a/src/examples/long_lived_framework.cpp
+++ b/src/examples/long_lived_framework.cpp
@@ -105,12 +105,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),
@@ -243,7 +244,7 @@ protected:
 
     const Resources executorResources = [this]() {
       Resources resources(executor.resources());
-      resources.allocate(framework.role());
+      resources.allocate(role);
       return resources;
     }();
 
@@ -402,6 +403,7 @@ private:
 
   const string master;
   FrameworkInfo framework;
+  const string role;
   const ExecutorInfo executor;
   const Resources taskResources;
   string uri;
@@ -583,6 +585,9 @@ int main(int argc, char** argv)
   framework.set_user(os::user().get());
   framework.set_name("Long Lived Framework (C++)");
   framework.set_checkpoint(flags.checkpoint);
+  framework.add_roles("*");
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
 
   Option<Credential> credential = None();
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a87d51e/src/examples/no_executor_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/no_executor_framework.cpp b/src/examples/no_executor_framework.cpp
index 77b7408..f206378 100644
--- a/src/examples/no_executor_framework.cpp
+++ b/src/examples/no_executor_framework.cpp
@@ -329,6 +329,9 @@ int main(int argc, char** argv)
   framework.set_user(""); // Have Mesos fill in the current user.
   framework.set_name("No Executor Framework");
   framework.set_checkpoint(flags.checkpoint);
+  framework.add_roles("*");
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
 
   if (flags.task_revocable_resources.isSome()) {
     framework.add_capabilities()->set_type(
@@ -366,7 +369,7 @@ int main(int argc, char** argv)
     }
   }
 
-  taskResources.allocate(framework.role());
+  taskResources.allocate(framework.roles(0));
 
   logging::initialize(argv[0], flags, true); // Catch signals.
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a87d51e/src/examples/persistent_volume_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/persistent_volume_framework.cpp b/src/examples/persistent_volume_framework.cpp
index ab4597d..df03ad2 100644
--- a/src/examples/persistent_volume_framework.cpp
+++ b/src/examples/persistent_volume_framework.cpp
@@ -140,24 +140,27 @@ 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(),
-          tasksPerShard,
-          false));
+      shards.push_back(
+          Shard(
+              "shard-" + stringify(i),
+              role,
+              tasksPerShard,
+              false));
     }
 
     // Initialize the shards using shared persistent volume.
     for (size_t i = 0; i < numSharedShards; i++) {
-      shards.push_back(Shard(
-          "shared-shard-" + stringify(i),
-          frameworkInfo.role(),
-          tasksPerShard,
-          true));
+      shards.push_back(
+          Shard(
+              "shared-shard-" + stringify(i),
+              role,
+              tasksPerShard,
+              true));
     }
   }
 
@@ -206,7 +209,7 @@ public:
 
             if (offered.contains(shard.resources)) {
               Resource volume = SHARD_PERSISTENT_VOLUME(
-                  frameworkInfo.role(),
+                  role,
                   UUID::random().toString(),
                   "volume",
                   frameworkInfo.principal(),
@@ -491,6 +494,7 @@ private:
   };
 
   FrameworkInfo frameworkInfo;
+  const string role;
   vector<Shard> shards;
 };
 
@@ -574,7 +578,9 @@ int main(int argc, char** argv)
   FrameworkInfo framework;
   framework.set_user(""); // Have Mesos fill in the current user.
   framework.set_name("Persistent Volume Framework (C++)");
-  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/5a87d51e/src/examples/test_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_framework.cpp b/src/examples/test_framework.cpp
index 05ddc89..9c4e08a 100644
--- a/src/examples/test_framework.cpp
+++ b/src/examples/test_framework.cpp
@@ -249,7 +249,9 @@ int main(int argc, char** argv)
   FrameworkInfo framework;
   framework.set_user(""); // Have Mesos fill in the current user.
   framework.set_name("Test Framework (C++)");
-  framework.set_role(flags.role);
+  framework.add_roles(flags.role);
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
 
   value = os::getenv("MESOS_CHECKPOINT");
   if (value.isSome()) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a87d51e/src/examples/test_http_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_http_framework.cpp b/src/examples/test_http_framework.cpp
index 471835c..8b31230 100644
--- a/src/examples/test_http_framework.cpp
+++ b/src/examples/test_http_framework.cpp
@@ -72,6 +72,7 @@ public:
                 const ExecutorInfo& _executor,
                 const string& _master)
     : framework(_framework),
+      role(_framework.roles(0)),
       executor(_executor),
       master(_master),
       state(INITIALIZING),
@@ -84,6 +85,7 @@ public:
                 const string& _master,
                 const Credential& credential)
     : framework(_framework),
+      role(_framework.roles(0)),
       executor(_executor),
       master(_master),
       state(INITIALIZING),
@@ -225,7 +227,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();
 
@@ -245,10 +247,10 @@ private:
         task.mutable_agent_id()->MergeFrom(offer.agent_id());
         task.mutable_executor()->MergeFrom(executor);
 
-        Try<Resources> flattened = taskResources.flatten(framework.role());
+        Try<Resources> flattened = taskResources.flatten(role);
         CHECK_SOME(flattened);
-        Option<Resources> resources = remaining.find(flattened.get());
 
+        Option<Resources> resources = remaining.find(flattened.get());
         CHECK_SOME(resources);
 
         task.mutable_resources()->CopyFrom(resources.get());
@@ -352,6 +354,7 @@ private:
   }
 
   FrameworkInfo framework;
+  const string role;
   const ExecutorInfo executor;
   const string master;
   process::Owned<scheduler::Mesos> mesos;
@@ -429,7 +432,9 @@ int main(int argc, char** argv)
 
   FrameworkInfo framework;
   framework.set_name("Event Call Scheduler using libprocess (C++)");
-  framework.set_role(flags.role);
+  framework.add_roles(flags.role);
+  framework.add_capabilities()->set_type(
+      FrameworkInfo::Capability::MULTI_ROLE);
 
   const Result<string> user = os::user();
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a87d51e/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index faf3242..6ec0436 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -241,7 +241,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());
@@ -864,7 +864,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(
@@ -952,11 +952,11 @@ 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.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       createReservationInfo(DEFAULT_CREDENTIAL.principal())).get();
 
   MockScheduler sched;
@@ -978,7 +978,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_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
@@ -1012,7 +1012,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();
@@ -1044,11 +1044,11 @@ 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.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       createReservationInfo(DEFAULT_CREDENTIAL.principal())).get();
 
   v1::master::Call v1Call;
@@ -1090,7 +1090,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_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
@@ -1124,7 +1124,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();
@@ -2010,7 +2010,7 @@ TEST_P(MasterAPITest, CreateAndDestroyVolumes)
   AWAIT_READY(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;
@@ -2031,11 +2031,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/5a87d51e/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 4e1d027..fb5dc5e 100644
--- a/src/tests/containerizer/cgroups_isolator_tests.cpp
+++ b/src/tests/containerizer/cgroups_isolator_tests.cpp
@@ -309,8 +309,9 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_RevocableCpu)
 
   // Now the framework will get revocable resources.
   AWAIT_READY(offers2);
-  EXPECT_NE(0u, offers2->size());
-  EXPECT_EQ(allocatedResources(cpus, frameworkInfo.role()),
+
+  ASSERT_FALSE(offers2->empty());
+  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/5a87d51e/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 4eef399..5635b07 100644
--- a/src/tests/containerizer/docker_containerizer_tests.cpp
+++ b/src/tests/containerizer/docker_containerizer_tests.cpp
@@ -1590,7 +1590,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(
@@ -1745,7 +1745,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;
@@ -1908,7 +1908,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/5a87d51e/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 70a0dce..9c7f34f 100644
--- a/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
+++ b/src/tests/containerizer/linux_filesystem_isolator_tests.cpp
@@ -1075,7 +1075,7 @@ TEST_F(LinuxFilesystemIsolatorMesosTest,
 
   MockScheduler sched;
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
 
   MesosSchedulerDriver driver(
       &sched,
@@ -1211,7 +1211,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(
@@ -1431,7 +1431,7 @@ TEST_F(LinuxFilesystemIsolatorMesosTest,
 
   MockScheduler sched;
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role1");
+  frameworkInfo.set_roles(0, "role1");
 
   MesosSchedulerDriver driver(
       &sched,
@@ -1523,7 +1523,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/5a87d51e/src/tests/default_executor_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/default_executor_tests.cpp b/src/tests/default_executor_tests.cpp
index 22af7e9..a8c7a97 100644
--- a/src/tests/default_executor_tests.cpp
+++ b/src/tests/default_executor_tests.cpp
@@ -1293,7 +1293,7 @@ TEST_P(DefaultExecutorTest, ReservedResources)
   AWAIT_READY(connected);
 
   v1::FrameworkInfo frameworkInfo = v1::DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role("role");
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   Future<v1::scheduler::Event::Subscribed> subscribed;
   EXPECT_CALL(*scheduler, subscribed(_, _))
@@ -1323,7 +1323,7 @@ TEST_P(DefaultExecutorTest, ReservedResources)
 
   // Launch the executor using reserved resources.
   v1::Resources reserved = unreserved.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       v1::createReservationInfo(frameworkInfo.principal())).get();
 
   v1::ExecutorInfo executorInfo;
@@ -1443,7 +1443,7 @@ TEST_P(PersistentVolumeDefaultExecutor, ROOT_PersistentResources)
   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);
 
   Future<Nothing> connected;
   EXPECT_CALL(*scheduler, connected(_))
@@ -1476,12 +1476,12 @@ TEST_P(PersistentVolumeDefaultExecutor, ROOT_PersistentResources)
     v1::Resources::parse("cpus:0.1;mem:32;disk:32").get();
 
   v1::Resources reserved = unreserved.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       v1::createReservationInfo(frameworkInfo.principal())).get();
 
   v1::Resource volume = v1::createPersistentVolume(
       Megabytes(1),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "executor_volume_path",
       frameworkInfo.principal(),
@@ -1585,7 +1585,7 @@ TEST_P(PersistentVolumeDefaultExecutor, ROOT_TaskSandboxPersistentVolume)
   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);
 
   Future<Nothing> connected;
   EXPECT_CALL(*scheduler, connected(_))
@@ -1633,7 +1633,7 @@ TEST_P(PersistentVolumeDefaultExecutor, ROOT_TaskSandboxPersistentVolume)
 
   v1::Resource volume = v1::createPersistentVolume(
       Megabytes(1),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "task_volume_path",
       frameworkInfo.principal(),
@@ -1641,7 +1641,7 @@ TEST_P(PersistentVolumeDefaultExecutor, ROOT_TaskSandboxPersistentVolume)
       frameworkInfo.principal());
 
   v1::Resources reserved = unreserved.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       v1::createReservationInfo(frameworkInfo.principal())).get();
 
   // Launch a task that expects the persistent volume to be

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a87d51e/src/tests/disk_quota_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/disk_quota_tests.cpp b/src/tests/disk_quota_tests.cpp
index 9fe2c86..c361883 100644
--- a/src/tests/disk_quota_tests.cpp
+++ b/src/tests/disk_quota_tests.cpp
@@ -245,7 +245,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();
 
@@ -472,7 +472,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/5a87d51e/src/tests/fault_tolerance_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fault_tolerance_tests.cpp b/src/tests/fault_tolerance_tests.cpp
index ee2130c..e551ba4 100644
--- a/src/tests/fault_tolerance_tests.cpp
+++ b/src/tests/fault_tolerance_tests.cpp
@@ -2039,6 +2039,8 @@ TEST_F(FaultToleranceTest, UpdateFrameworkInfoOnSchedulerFailover)
   // scheduler with updated information.
 
   FrameworkInfo finfo1 = DEFAULT_FRAMEWORK_INFO;
+  finfo1.clear_capabilities();
+  finfo1.clear_roles();
   finfo1.set_name("Framework 1");
   finfo1.set_failover_timeout(1000);
   finfo1.mutable_labels()->add_labels()->CopyFrom(createLabel("foo", "bar"));
@@ -2063,7 +2065,8 @@ TEST_F(FaultToleranceTest, UpdateFrameworkInfoOnSchedulerFailover)
   // updated FrameworkInfo and wait until it gets a registered
   // callback.
 
-  FrameworkInfo finfo2 = DEFAULT_FRAMEWORK_INFO;
+  FrameworkInfo finfo2 = finfo1;
+
   finfo2.mutable_id()->MergeFrom(frameworkId.get());
   auto capabilityType = FrameworkInfo::Capability::REVOCABLE_RESOURCES;
   finfo2.add_capabilities()->set_type(capabilityType);
@@ -2071,6 +2074,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/5a87d51e/src/tests/hook_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hook_tests.cpp b/src/tests/hook_tests.cpp
index 02d8f80..5c8f601 100644
--- a/src/tests/hook_tests.cpp
+++ b/src/tests/hook_tests.cpp
@@ -1119,7 +1119,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/5a87d51e/src/tests/master_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_allocator_tests.cpp b/src/tests/master_allocator_tests.cpp
index d05ee44..c0593de 100644
--- a/src/tests/master_allocator_tests.cpp
+++ b/src/tests/master_allocator_tests.cpp
@@ -820,7 +820,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(_, _, _, _))
@@ -1268,7 +1268,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(
@@ -1287,7 +1287,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(
@@ -1637,7 +1637,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);
@@ -1667,7 +1667,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);
@@ -1818,7 +1818,7 @@ TYPED_TEST(MasterAllocatorTest, DISABLED_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(
@@ -1848,7 +1848,7 @@ TYPED_TEST(MasterAllocatorTest, DISABLED_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(
@@ -1867,7 +1867,7 @@ TYPED_TEST(MasterAllocatorTest, DISABLED_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/5a87d51e/src/tests/master_authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_authorization_tests.cpp b/src/tests/master_authorization_tests.cpp
index 5809bd9..d41e72e 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;
@@ -1501,7 +1501,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".
@@ -1690,7 +1690,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".
@@ -1881,7 +1881,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/5a87d51e/src/tests/master_quota_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_quota_tests.cpp b/src/tests/master_quota_tests.cpp
index 2c7ec5a..190de3e 100644
--- a/src/tests/master_quota_tests.cpp
+++ b/src/tests/master_quota_tests.cpp
@@ -112,11 +112,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/5a87d51e/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index b0678ad..c4320a4 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -833,7 +833,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;
@@ -915,7 +915,7 @@ TEST_F(MasterTest, RecoverResources)
   EXPECT_NE(0u, offers->size());
 
   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();
@@ -1925,11 +1925,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(
@@ -3538,7 +3535,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();
@@ -4050,13 +4047,16 @@ TEST_F(MasterTest, StateEndpointFrameworkInfo)
   AWAIT_READY(slaveRegisteredMessage);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfo.clear_capabilities();
+
   frameworkInfo.set_webui_url("http://localhost:8080/");
 
   vector<FrameworkInfo::Capability::Type> capabilities = {
     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,
   };
 
   foreach (FrameworkInfo::Capability::Type capability, capabilities) {
@@ -4132,7 +4132,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);
 
   EXPECT_EQ(1u, offer.values.count("allocation_info"));
   EXPECT_EQ(allocationInfo, offer.values.at("allocation_info"));
@@ -4304,7 +4304,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(
@@ -4374,10 +4374,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) } } }
           }
         }
       }
@@ -5451,7 +5451,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(
@@ -6677,10 +6677,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(
@@ -6822,12 +6820,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(

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a87d51e/src/tests/master_validation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_validation_tests.cpp b/src/tests/master_validation_tests.cpp
index 3308803..5972c32 100644
--- a/src/tests/master_validation_tests.cpp
+++ b/src/tests/master_validation_tests.cpp
@@ -759,7 +759,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(), frameworkInfo);
 
@@ -781,14 +781,14 @@ TEST_F(CreateOperationValidationTest, SharedVolumeBasedOnCapability)
 TEST_F(CreateOperationValidationTest, InsufficientDiskResource)
 {
   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";
@@ -3333,7 +3333,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(
@@ -3356,7 +3357,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);
@@ -3386,7 +3387,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);
@@ -3419,6 +3420,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.
@@ -3485,7 +3488,10 @@ TEST_F(FrameworkInfoValidationTest, DowngradeFromMultipleRoles)
   ASSERT_SOME(master);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.add_roles("role1");
+
+  frameworkInfo.clear_capabilities();
+
+  frameworkInfo.set_roles(0, "role1");
   frameworkInfo.add_roles("role2");
   frameworkInfo.add_capabilities()->set_type(
       FrameworkInfo::Capability::MULTI_ROLE);
@@ -3555,9 +3561,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());
@@ -3627,6 +3631,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/5a87d51e/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 3c57f25..3b1d58e 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -439,6 +439,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);
 
     return framework;
   }
@@ -1584,7 +1587,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/5a87d51e/src/tests/oversubscription_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/oversubscription_tests.cpp b/src/tests/oversubscription_tests.cpp
index 25e2cdf..18bc49f 100644
--- a/src/tests/oversubscription_tests.cpp
+++ b/src/tests/oversubscription_tests.cpp
@@ -402,10 +402,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);
@@ -511,7 +511,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;
@@ -549,7 +549,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();
@@ -624,8 +624,9 @@ TEST_F(OversubscriptionTest, RescindRevocableOfferWithDecreasedRevocable)
 
   // Now the framework will get revocable resources.
   AWAIT_READY(offers2);
-  EXPECT_NE(0u, offers2->size());
-  EXPECT_EQ(allocatedResources(resources1, framework.role()),
+
+  ASSERT_FALSE(offers2->empty());
+  EXPECT_EQ(allocatedResources(resources1, framework.roles(0)),
             Resources(offers2.get()[0].resources()));
 
   Future<OfferID> offerId;
@@ -655,8 +656,9 @@ TEST_F(OversubscriptionTest, RescindRevocableOfferWithDecreasedRevocable)
 
   // The new offer should include the latest oversubscribed resources.
   AWAIT_READY(offers3);
-  EXPECT_NE(0u, offers3->size());
-  EXPECT_EQ(allocatedResources(resources2, framework.role()),
+
+  ASSERT_FALSE(offers3->empty());
+  EXPECT_EQ(allocatedResources(resources2, framework.roles(0)),
             Resources(offers3.get()[0].resources()));
 
   driver.stop();
@@ -1286,8 +1288,9 @@ TEST_F(OversubscriptionTest, UpdateAllocatorOnSchedulerFailover)
   estimations.put(revocable);
 
   AWAIT_READY(offers2);
-  EXPECT_NE(0u, offers2->size());
-  EXPECT_EQ(allocatedResources(revocable, framework2.role()),
+
+  ASSERT_FALSE(offers2->empty());
+  EXPECT_EQ(allocatedResources(revocable, framework2.roles(0)),
             Resources(offers2.get()[0].resources()));
 
   EXPECT_EQ(DRIVER_STOPPED, driver2.stop());
@@ -1359,8 +1362,9 @@ TEST_F(OversubscriptionTest, RemoveCapabilitiesOnSchedulerFailover)
 
   // Now the framework will get revocable resources.
   AWAIT_READY(offers2);
-  EXPECT_NE(0u, offers2->size());
-  EXPECT_EQ(allocatedResources(revocable, framework1.role()),
+
+  ASSERT_FALSE(offers2->empty());
+  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/5a87d51e/src/tests/partition_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/partition_tests.cpp b/src/tests/partition_tests.cpp
index 4ff4285..0b8f3b1 100644
--- a/src/tests/partition_tests.cpp
+++ b/src/tests/partition_tests.cpp
@@ -867,7 +867,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/5a87d51e/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 1237d08..e9e579e 100644
--- a/src/tests/persistent_volume_endpoints_tests.cpp
+++ b/src/tests/persistent_volume_endpoints_tests.cpp
@@ -84,7 +84,7 @@ public:
   {
     master::Flags flags = MesosTest::CreateMasterFlags();
     flags.allocation_interval = Milliseconds(50);
-    flags.roles = createFrameworkInfo().role();
+    flags.roles = createFrameworkInfo().roles(0);
     return flags;
   }
 
@@ -92,7 +92,7 @@ public:
   FrameworkInfo createFrameworkInfo()
   {
     FrameworkInfo info = DEFAULT_FRAMEWORK_INFO;
-    info.set_role("role1");
+    info.set_roles(0, "role1");
     return info;
   }
 
@@ -171,7 +171,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;
 
@@ -199,7 +199,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();
@@ -233,7 +233,7 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservation)
 
   Resources unreserved = Resources::parse("disk:1024").get();
   Resources dynamicallyReserved = unreserved.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       createReservationInfo(DEFAULT_CREDENTIAL.principal())).get();
 
   Future<Response> response = process::http::post(
@@ -266,7 +266,7 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservation)
   Offer offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   Future<OfferID> rescindedOfferId;
 
@@ -278,7 +278,7 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservation)
 
   Resources volume = createPersistentVolume(
       Megabytes(64),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "path1",
       DEFAULT_CREDENTIAL.principal(),
@@ -303,7 +303,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));
@@ -355,7 +355,7 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservationRoleMismatch)
 
   Resources unreserved = Resources::parse("disk:1024").get();
   Resources dynamicallyReserved = unreserved.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       createReservationInfo(DEFAULT_CREDENTIAL.principal())).get();
 
   Future<Response> response = process::http::post(
@@ -385,9 +385,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",
@@ -437,7 +437,7 @@ TEST_F(PersistentVolumeEndpointsTest, UnreserveVolumeResources)
 
   Resources unreserved = Resources::parse("disk:1024").get();
   Resources dynamicallyReserved = unreserved.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       createReservationInfo(DEFAULT_CREDENTIAL.principal())).get();
 
   Future<Response> response = process::http::post(
@@ -450,7 +450,7 @@ TEST_F(PersistentVolumeEndpointsTest, UnreserveVolumeResources)
 
   Resources volume = createPersistentVolume(
       Megabytes(64),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "path1",
       DEFAULT_CREDENTIAL.principal(),
@@ -728,7 +728,7 @@ TEST_F(PersistentVolumeEndpointsTest, NoHeader)
 
   Resources volume = createPersistentVolume(
       Megabytes(64),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "path1",
       None(),
@@ -903,7 +903,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;
 
@@ -934,7 +934,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();
@@ -1133,7 +1133,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(
@@ -1341,7 +1341,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(
@@ -1575,11 +1575,10 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
   // Make a dynamic reservation for 512MB of disk.
   Resources unreserved = Resources::parse("disk:512").get();
   Resources dynamicallyReserved = unreserved.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       createReservationInfo(DEFAULT_CREDENTIAL.principal())).get();
 
   EXPECT_CALL(allocator, addFramework(_, _, _, _));
-
   EXPECT_CALL(sched, registered(_, _, _));
 
   Future<vector<Offer>> offers;
@@ -1614,12 +1613,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(),
@@ -1639,7 +1638,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;
 
@@ -1668,7 +1667,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));
@@ -1695,7 +1694,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();
@@ -1731,7 +1730,7 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
   // Make a dynamic reservation for 512MB of disk.
   Resources unreserved = Resources::parse("disk:512").get();
   Resources dynamicallyReserved = unreserved.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       createReservationInfo(DEFAULT_CREDENTIAL.principal())).get();
 
   Future<Response> response = process::http::post(
@@ -1745,7 +1744,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(),
@@ -1780,7 +1779,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))));
 
   // We use the filter explicitly here so that the resources will not
   // be filtered for 5 seconds (the default).
@@ -1802,7 +1801,7 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
   offer = offers.get()[0];
 
   EXPECT_TRUE(Resources(offer.resources()).contains(
-      allocatedResources(dynamicallyReserved, frameworkInfo.role())));
+      allocatedResources(dynamicallyReserved, frameworkInfo.roles(0))));
 
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
@@ -1817,7 +1816,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();
@@ -1868,7 +1867,7 @@ TEST_F(PersistentVolumeEndpointsTest, ReserveAndSlaveRemoval)
   // Reserve all CPUs on `slave1` via HTTP endpoint.
   Resources slave1Unreserved = Resources::parse("cpus:4").get();
   Resources slave1Reserved = slave1Unreserved.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       createReservationInfo(DEFAULT_CREDENTIAL.principal())).get();
 
   Future<Response> response = process::http::post(
@@ -1907,7 +1906,7 @@ TEST_F(PersistentVolumeEndpointsTest, ReserveAndSlaveRemoval)
   // Use the offers API to reserve all CPUs on `slave2`.
   Resources slave2Unreserved = Resources::parse("cpus:3").get();
   Resources slave2Reserved = slave2Unreserved.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       createReservationInfo(DEFAULT_CREDENTIAL.principal())).get();
 
   for (size_t i = 0; i < offers->size(); i++) {
@@ -1987,7 +1986,7 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
 
   Resources unreserved = Resources::parse("cpus:1;mem:512;disk:1024").get();
   Resources dynamicallyReserved = unreserved.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       createReservationInfo(DEFAULT_CREDENTIAL.principal())).get();
 
   Future<Response> response = process::http::post(
@@ -2000,7 +1999,7 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
 
   Resources volume = createPersistentVolume(
       Megabytes(64),
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       "id1",
       "path1",
       DEFAULT_CREDENTIAL.principal(),
@@ -2036,11 +2035,11 @@ 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.flatten(
-      frameworkInfo.role(),
+      frameworkInfo.roles(0),
       createReservationInfo(DEFAULT_CREDENTIAL.principal())).get();
 
   TaskInfo taskInfo = createTask(offer.slave_id(), taskResources, "sleep 1000");

http://git-wip-us.apache.org/repos/asf/mesos/blob/5a87d51e/src/tests/persistent_volume_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/persistent_volume_tests.cpp b/src/tests/persistent_volume_tests.cpp
index 8f8bb26..3718841 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>
 
@@ -68,6 +69,7 @@ using mesos::master::detector::MasterDetector;
 using mesos::master::detector::StandaloneMasterDetector;
 
 using std::list;
+using std::set;
 using std::string;
 using std::vector;
 
@@ -138,14 +140,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;
@@ -255,11 +259,11 @@ TEST_P(PersistentVolumeTest, CreateAndDestroyPersistentVolumes)
   Clock::pause();
 
   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);
@@ -375,9 +379,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(
@@ -432,7 +436,7 @@ TEST_P(PersistentVolumeTest, ResourcesCheckpointing)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -498,7 +502,7 @@ TEST_P(PersistentVolumeTest, PreparePersistentVolume)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -567,7 +571,7 @@ TEST_P(PersistentVolumeTest, MasterFailover)
   ASSERT_SOME(slave);
 
   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);
@@ -643,7 +647,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();
@@ -669,7 +673,7 @@ TEST_P(PersistentVolumeTest, IncompatibleCheckpointedResources)
   spawn(slave1);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
@@ -756,7 +760,7 @@ TEST_P(PersistentVolumeTest, AccessPersistentVolume)
   ASSERT_SOME(slave);
 
   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);
 
@@ -869,7 +873,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<CheckpointResourcesMessage> checkpointMessage =
     FUTURE_PROTOBUF(CheckpointResourcesMessage(), _, _);
@@ -923,7 +927,7 @@ TEST_P(PersistentVolumeTest, SharedPersistentVolumeMultipleTasks)
   ASSERT_SOME(slave);
 
   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);
 
@@ -1037,7 +1041,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);
 
@@ -1115,7 +1119,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);
 
@@ -1137,9 +1141,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
@@ -1162,9 +1166,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.
@@ -1205,7 +1209,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);
 
@@ -1307,9 +1311,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();
@@ -1331,7 +1335,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);
 
@@ -1556,7 +1560,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);
 
@@ -1694,7 +1698,7 @@ TEST_P(PersistentVolumeTest, SlaveRecovery)
   ASSERT_SOME(slave);
 
   FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
-  frameworkInfo.set_role(DEFAULT_TEST_ROLE);
+  frameworkInfo.set_roles(0, DEFAULT_TEST_ROLE);
   frameworkInfo.set_checkpoint(true);
 
   MockScheduler sched;
@@ -1828,12 +1832,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);
@@ -1906,7 +1910,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)));
@@ -1938,7 +1942,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();
@@ -1973,16 +1977,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 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);
@@ -2056,7 +2059,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)));
@@ -2086,7 +2089,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))));
   EXPECT_FALSE(
       Resources(checkpointResources2->resources()).contains(volume));
 
@@ -2129,20 +2132,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);
+  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);
@@ -2206,7 +2208,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
@@ -2262,7 +2264,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.
@@ -2308,7 +2310,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();
@@ -2352,19 +2354,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");
 
   // 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);
@@ -2428,7 +2430,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
@@ -2484,7 +2486,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.
@@ -2530,7 +2532,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();