You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by kl...@apache.org on 2017/05/31 21:56:39 UTC

mesos git commit: Added '--filter_gpu_resources' flag to the mesos master.

Repository: mesos
Updated Branches:
  refs/heads/master 3e4a12cca -> 0e176c305


Added '--filter_gpu_resources' flag to the mesos master.

When set to true, this flag will cause the mesos master to filter
offers containing GPU resources by only sending them to frameworks
that opt into the 'GPU_RESOURCES' framework capability. When set to
false, this flag will cause the master to not filter offers containing
GPU resources, and indiscriminately send them to all frameworks
whether they set the 'GPU_RESOURCES' capability or not.  This flag is
meant as a temporary workaround towards the eventual deprecation of
the 'GPU_RESOURCES' capability.

Please see the following link for more information:
https://www.mail-archive.com/dev@mesos.apache.org/msg37571.html

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


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

Branch: refs/heads/master
Commit: 0e176c305f6d43c62a665f9c53c00a34e1c80c49
Parents: 3e4a12c
Author: Kevin Klues <kl...@gmail.com>
Authored: Wed May 31 14:40:07 2017 -0700
Committer: Kevin Klues <kl...@gmail.com>
Committed: Wed May 31 14:55:53 2017 -0700

----------------------------------------------------------------------
 docs/configuration.md                           | 17 ++++++++
 include/mesos/allocator/allocator.hpp           |  3 +-
 src/master/allocator/mesos/allocator.hpp        | 12 ++++--
 src/master/allocator/mesos/hierarchical.cpp     | 10 +++--
 src/master/allocator/mesos/hierarchical.hpp     |  6 ++-
 src/master/flags.cpp                            | 16 ++++++++
 src/master/flags.hpp                            |  1 +
 src/master/master.cpp                           |  3 +-
 src/tests/allocator.hpp                         | 11 ++---
 src/tests/api_tests.cpp                         |  4 +-
 src/tests/master_allocator_tests.cpp            | 36 ++++++++---------
 src/tests/master_quota_tests.cpp                | 20 +++++-----
 src/tests/persistent_volume_endpoints_tests.cpp | 42 ++++++++++----------
 src/tests/reservation_endpoints_tests.cpp       | 40 +++++++++----------
 src/tests/reservation_tests.cpp                 |  8 ++--
 src/tests/resource_offers_tests.cpp             |  2 +-
 src/tests/slave_recovery_tests.cpp              |  2 +-
 17 files changed, 141 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index bb4c5a1..6703b9e 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -631,6 +631,23 @@ sharing implementation currently has limitations. E.g. See the problem of
 </tr>
 <tr>
   <td>
+    --[no_]filter_gpu_resources
+  </td>
+  <td>
+When set to true, this flag will cause the mesos master to filter all offers
+from agents with GPU resources by only sending them to frameworks that opt into
+the 'GPU_RESOURCES' framework capability. When set to false, this flag will
+cause the master to not filter offers from agents with GPU resources, and
+indiscriminately send them to all frameworks whether they set the
+'GPU_RESOURCES' capability or not.  This flag is meant as a temporary workaround
+towards the eventual deprecation of the 'GPU_RESOURCES' capability. Please see
+the following for more information:
+    <a href="https://www.mail-archive.com/dev@mesos.apache.org/msg37571.html">msg37571</a>
+    <a href="https://issues.apache.org/jira/browse/MESOS-7576">MESOS-7576</a>
+  </td>
+</tr>
+<tr>
+  <td>
     --framework_sorter=VALUE
   </td>
   <td>

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/include/mesos/allocator/allocator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/allocator/allocator.hpp b/include/mesos/allocator/allocator.hpp
index dc34a1b..b31fbcf 100644
--- a/include/mesos/allocator/allocator.hpp
+++ b/include/mesos/allocator/allocator.hpp
@@ -93,7 +93,8 @@ public:
                const hashmap<SlaveID, UnavailableResources>&)>&
         inverseOfferCallback,
       const Option<std::set<std::string>>&
-        fairnessExcludeResourceNames = None()) = 0;
+        fairnessExcludeResourceNames = None(),
+      bool filterGpuResources = true) = 0;
 
   /**
    * Informs the allocator of the recovered state from the master.

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/master/allocator/mesos/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/allocator.hpp b/src/master/allocator/mesos/allocator.hpp
index 119b461..b2dcb56 100644
--- a/src/master/allocator/mesos/allocator.hpp
+++ b/src/master/allocator/mesos/allocator.hpp
@@ -57,7 +57,8 @@ public:
                const hashmap<SlaveID, UnavailableResources>&)>&
         inverseOfferCallback,
       const Option<std::set<std::string>>&
-        fairnessExcludeResourceNames = None());
+        fairnessExcludeResourceNames = None(),
+      bool filterGpuResources = true);
 
   void recover(
       const int expectedAgentCount,
@@ -193,7 +194,8 @@ public:
                const hashmap<SlaveID, UnavailableResources>&)>&
         inverseOfferCallback,
       const Option<std::set<std::string>>&
-        fairnessExcludeResourceNames = None()) = 0;
+        fairnessExcludeResourceNames = None(),
+      bool filterGpuResources = true) = 0;
 
   virtual void recover(
       const int expectedAgentCount,
@@ -337,7 +339,8 @@ inline void MesosAllocator<AllocatorProcess>::initialize(
         void(const FrameworkID&,
               const hashmap<SlaveID, UnavailableResources>&)>&
       inverseOfferCallback,
-    const Option<std::set<std::string>>& fairnessExcludeResourceNames)
+    const Option<std::set<std::string>>& fairnessExcludeResourceNames,
+    bool filterGpuResources)
 {
   process::dispatch(
       process,
@@ -345,7 +348,8 @@ inline void MesosAllocator<AllocatorProcess>::initialize(
       allocationInterval,
       offerCallback,
       inverseOfferCallback,
-      fairnessExcludeResourceNames);
+      fairnessExcludeResourceNames,
+      filterGpuResources);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/master/allocator/mesos/hierarchical.cpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.cpp b/src/master/allocator/mesos/hierarchical.cpp
index ca368bb..6679d47 100644
--- a/src/master/allocator/mesos/hierarchical.cpp
+++ b/src/master/allocator/mesos/hierarchical.cpp
@@ -147,12 +147,14 @@ void HierarchicalAllocatorProcess::initialize(
         void(const FrameworkID&,
              const hashmap<SlaveID, UnavailableResources>&)>&
       _inverseOfferCallback,
-    const Option<set<string>>& _fairnessExcludeResourceNames)
+    const Option<set<string>>& _fairnessExcludeResourceNames,
+    bool _filterGpuResources)
 {
   allocationInterval = _allocationInterval;
   offerCallback = _offerCallback;
   inverseOfferCallback = _inverseOfferCallback;
   fairnessExcludeResourceNames = _fairnessExcludeResourceNames;
+  filterGpuResources = _filterGpuResources;
   initialized = true;
   paused = false;
 
@@ -1576,7 +1578,8 @@ void HierarchicalAllocatorProcess::__allocate()
         // Only offer resources from slaves that have GPUs to
         // frameworks that are capable of receiving GPUs.
         // See MESOS-5634.
-        if (!framework.capabilities.gpuResources &&
+        if (filterGpuResources &&
+            !framework.capabilities.gpuResources &&
             slave.total.gpus().getOrElse(0) > 0) {
           continue;
         }
@@ -1741,7 +1744,8 @@ void HierarchicalAllocatorProcess::__allocate()
         // Only offer resources from slaves that have GPUs to
         // frameworks that are capable of receiving GPUs.
         // See MESOS-5634.
-        if (!framework.capabilities.gpuResources &&
+        if (filterGpuResources &&
+            !framework.capabilities.gpuResources &&
             slave.total.gpus().getOrElse(0) > 0) {
           continue;
         }

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/master/allocator/mesos/hierarchical.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.hpp b/src/master/allocator/mesos/hierarchical.hpp
index 82fea40..5e7c306 100644
--- a/src/master/allocator/mesos/hierarchical.hpp
+++ b/src/master/allocator/mesos/hierarchical.hpp
@@ -102,7 +102,8 @@ public:
                const hashmap<SlaveID, UnavailableResources>&)>&
         inverseOfferCallback,
       const Option<std::set<std::string>>&
-        fairnessExcludeResourceNames = None());
+        fairnessExcludeResourceNames = None(),
+      bool filterGpuResources = true);
 
   void recover(
       const int _expectedAgentCount,
@@ -438,6 +439,9 @@ protected:
   // Resources (by name) that will be excluded from a role's fair share.
   Option<std::set<std::string>> fairnessExcludeResourceNames;
 
+  // Filter GPU resources based on the `GPU_RESOURCES` framework capability.
+  bool filterGpuResources;
+
   // There are two stages of allocation. During the first stage resources
   // are allocated only to frameworks in roles with quota set. During the
   // second stage remaining resources that would not be required to satisfy

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/master/flags.cpp
----------------------------------------------------------------------
diff --git a/src/master/flags.cpp b/src/master/flags.cpp
index 750b7b9..eee5ec7 100644
--- a/src/master/flags.cpp
+++ b/src/master/flags.cpp
@@ -460,6 +460,22 @@ mesos::internal::master::Flags::Flags()
       "  http://www.mail-archive.com/dev@mesos.apache.org/msg35631.html\n"
       "  https://issues.apache.org/jira/browse/MESOS-5377");
 
+  add(&Flags::filter_gpu_resources,
+      "filter_gpu_resources",
+      "When set to true, this flag will cause the mesos master to\n"
+      "filter all offers from agents with GPU resources by only sending\n"
+      "them to frameworks that opt into the `GPU_RESOURCES` framework\n"
+      "capability. When set to false, this flag will cause the master\n"
+      "to not filter offers from agents with GPU resources, and\n"
+      "indiscriminately send them to all frameworks whether they set\n"
+      "the `GPU_RESOURCES` capability or not. This flag is meant as a\n"
+      "temporary workaround towards the eventual deprecation of the\n"
+      "`GPU_RESOURCES` capability. Please see the following for more\n"
+      "information:\n"
+      "  https://www.mail-archive.com/dev@mesos.apache.org/msg37571.html\n"
+      "  https://issues.apache.org/jira/browse/MESOS-7576",
+      true);
+
   add(&Flags::hooks,
       "hooks",
       "A comma-separated list of hook modules to be\n"

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/master/flags.hpp
----------------------------------------------------------------------
diff --git a/src/master/flags.hpp b/src/master/flags.hpp
index 93ca9b9..8a9bd2d 100644
--- a/src/master/flags.hpp
+++ b/src/master/flags.hpp
@@ -81,6 +81,7 @@ public:
   std::string authenticators;
   std::string allocator;
   Option<std::set<std::string>> fair_sharing_excluded_resource_names;
+  bool filter_gpu_resources;
   Option<std::string> hooks;
   Duration agent_ping_timeout;
   size_t max_agent_ping_timeouts;

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index a5376f9..c66907b 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -761,7 +761,8 @@ void Master::initialize()
       flags.allocation_interval,
       defer(self(), &Master::offer, lambda::_1, lambda::_2),
       defer(self(), &Master::inverseOffer, lambda::_1, lambda::_2),
-      flags.fair_sharing_excluded_resource_names);
+      flags.fair_sharing_excluded_resource_names,
+      flags.filter_gpu_resources);
 
   // Parse the whitelist. Passing Allocator::updateWhitelist()
   // callback is safe because we shut down the whitelistWatcher in

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/tests/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/tests/allocator.hpp b/src/tests/allocator.hpp
index 4ea7224..0b9fdea 100644
--- a/src/tests/allocator.hpp
+++ b/src/tests/allocator.hpp
@@ -45,7 +45,7 @@ namespace tests {
 
 ACTION_P(InvokeInitialize, allocator)
 {
-  allocator->real->initialize(arg0, arg1, arg2, arg3);
+  allocator->real->initialize(arg0, arg1, arg2, arg3, arg4);
 }
 
 
@@ -229,9 +229,9 @@ public:
     // to get the best of both worlds: the ability to use 'DoDefault'
     // and no warnings when expectations are not explicit.
 
-    ON_CALL(*this, initialize(_, _, _, _))
+    ON_CALL(*this, initialize(_, _, _, _, _))
       .WillByDefault(InvokeInitialize(this));
-    EXPECT_CALL(*this, initialize(_, _, _, _))
+    EXPECT_CALL(*this, initialize(_, _, _, _, _))
       .WillRepeatedly(DoDefault());
 
     ON_CALL(*this, recover(_, _))
@@ -357,7 +357,7 @@ public:
 
   virtual ~TestAllocator() {}
 
-  MOCK_METHOD4(initialize, void(
+  MOCK_METHOD5(initialize, void(
       const Duration&,
       const lambda::function<
           void(const FrameworkID&,
@@ -365,7 +365,8 @@ public:
       const lambda::function<
           void(const FrameworkID&,
                const hashmap<SlaveID, UnavailableResources>&)>&,
-      const Option<std::set<std::string>>&));
+      const Option<std::set<std::string>>&,
+      bool));
 
   MOCK_METHOD2(recover, void(
       const int expectedAgentCount,

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 97a8cc9..91b3473 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -933,7 +933,7 @@ TEST_P(MasterAPITest, ReserveResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   // Set a low allocation interval to speed up this test.
   master::Flags flags = MesosTest::CreateMasterFlags();
@@ -1025,7 +1025,7 @@ TEST_P(MasterAPITest, UnreserveResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   // Set a low allocation interval to speed up this test.
   master::Flags flags = MesosTest::CreateMasterFlags();

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/tests/master_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_allocator_tests.cpp b/src/tests/master_allocator_tests.cpp
index b9e0422..0082045 100644
--- a/src/tests/master_allocator_tests.cpp
+++ b/src/tests/master_allocator_tests.cpp
@@ -164,7 +164,7 @@ TYPED_TEST(MasterAllocatorTest, SingleFramework)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -212,7 +212,7 @@ TYPED_TEST(MasterAllocatorTest, ResourcesUnused)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -321,7 +321,7 @@ TYPED_TEST(MasterAllocatorTest, OutOfOrderDispatch)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -450,7 +450,7 @@ TYPED_TEST(MasterAllocatorTest, SchedulerFailover)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -576,7 +576,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkExited)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -736,7 +736,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveLost)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -860,7 +860,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveAdded)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -956,7 +956,7 @@ TYPED_TEST(MasterAllocatorTest, TaskFinished)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -1060,7 +1060,7 @@ TYPED_TEST(MasterAllocatorTest, CpusOnlyOfferedAndTaskLaunched)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -1141,7 +1141,7 @@ TYPED_TEST(MasterAllocatorTest, MemoryOnlyOfferedAndTaskLaunched)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -1235,7 +1235,7 @@ TYPED_TEST(MasterAllocatorTest, Whitelist)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Future<Nothing> updateWhitelist1;
   EXPECT_CALL(allocator, updateWhitelist(Option<hashset<string>>(hosts)))
@@ -1274,7 +1274,7 @@ TYPED_TEST(MasterAllocatorTest, RoleTest)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.roles = Some("role2");
@@ -1367,7 +1367,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkReregistersFirst)
   {
     TestAllocator<TypeParam> allocator;
 
-    EXPECT_CALL(allocator, initialize(_, _, _, _));
+    EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
     Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
     ASSERT_SOME(master);
@@ -1424,7 +1424,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkReregistersFirst)
   {
     TestAllocator<TypeParam> allocator2;
 
-    EXPECT_CALL(allocator2, initialize(_, _, _, _));
+    EXPECT_CALL(allocator2, initialize(_, _, _, _, _));
 
     Future<Nothing> addFramework;
     EXPECT_CALL(allocator2, addFramework(_, _, _, _))
@@ -1491,7 +1491,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveReregistersFirst)
   {
     TestAllocator<TypeParam> allocator;
 
-    EXPECT_CALL(allocator, initialize(_, _, _, _));
+    EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
     Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
     ASSERT_SOME(master);
@@ -1547,7 +1547,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveReregistersFirst)
   {
     TestAllocator<TypeParam> allocator2;
 
-    EXPECT_CALL(allocator2, initialize(_, _, _, _));
+    EXPECT_CALL(allocator2, initialize(_, _, _, _, _));
 
     Future<Nothing> addSlave;
     EXPECT_CALL(allocator2, addSlave(_, _, _, _, _, _))
@@ -1614,7 +1614,7 @@ TYPED_TEST(MasterAllocatorTest, RebalancedForUpdatedWeights)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   // Start Mesos master.
   master::Flags masterFlags = this->CreateMasterFlags();
@@ -1810,7 +1810,7 @@ TYPED_TEST(MasterAllocatorTest, DISABLED_NestedRoles)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   Try<Owned<cluster::Master>> master =

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/tests/master_quota_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_quota_tests.cpp b/src/tests/master_quota_tests.cpp
index 2c7ec5a..6ca4570 100644
--- a/src/tests/master_quota_tests.cpp
+++ b/src/tests/master_quota_tests.cpp
@@ -405,7 +405,7 @@ TEST_F(MasterQuotaTest, SetExistingQuota)
 TEST_F(MasterQuotaTest, RemoveSingleQuota)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -574,7 +574,7 @@ TEST_F(MasterQuotaTest, Status)
 TEST_F(MasterQuotaTest, InsufficientResourcesSingleAgent)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -636,7 +636,7 @@ TEST_F(MasterQuotaTest, InsufficientResourcesSingleAgent)
 TEST_F(MasterQuotaTest, InsufficientResourcesMultipleAgents)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -713,7 +713,7 @@ TEST_F(MasterQuotaTest, InsufficientResourcesMultipleAgents)
 TEST_F(MasterQuotaTest, AvailableResourcesSingleAgent)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -763,7 +763,7 @@ TEST_F(MasterQuotaTest, AvailableResourcesSingleAgent)
 TEST_F(MasterQuotaTest, AvailableResourcesMultipleAgents)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -832,7 +832,7 @@ TEST_F(MasterQuotaTest, AvailableResourcesMultipleAgents)
 TEST_F(MasterQuotaTest, AvailableResourcesAfterRescinding)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -1068,7 +1068,7 @@ TEST_F(MasterQuotaTest, RecoverQuotaEmptyCluster)
   }
 
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   // Restart the master; configured quota should be recovered from the registry.
   master->reset();
@@ -1102,7 +1102,7 @@ TEST_F(MasterQuotaTest, RecoverQuotaEmptyCluster)
 TEST_F(MasterQuotaTest, NoAuthenticationNoAuthorization)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   // Disable http_readwrite authentication and authorization.
   // TODO(alexr): Setting master `--acls` flag to `ACLs()` or `None()` seems
@@ -1212,7 +1212,7 @@ TEST_F(MasterQuotaTest, UnauthenticatedQuotaRequest)
 TEST_F(MasterQuotaTest, AuthorizeGetUpdateQuotaRequests)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   // Setup ACLs so that only the default principal can modify quotas
   // for `ROLE1` and read status.
@@ -1776,7 +1776,7 @@ TEST_F(MasterQuotaTest, DISABLED_ChildRoleDeleteParentQuota)
 TEST_F(MasterQuotaTest, DISABLED_ClusterCapacityWithNestedRoles)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/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 21136b2..9b820b8 100644
--- a/src/tests/persistent_volume_endpoints_tests.cpp
+++ b/src/tests/persistent_volume_endpoints_tests.cpp
@@ -114,7 +114,7 @@ TEST_F(PersistentVolumeEndpointsTest, StaticReservation)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -210,7 +210,7 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservation)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -332,7 +332,7 @@ TEST_F(PersistentVolumeEndpointsTest, DynamicReservationRoleMismatch)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -414,7 +414,7 @@ TEST_F(PersistentVolumeEndpointsTest, UnreserveVolumeResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -479,7 +479,7 @@ TEST_F(PersistentVolumeEndpointsTest, InvalidVolume)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -547,7 +547,7 @@ TEST_F(PersistentVolumeEndpointsTest, VolumeExceedsReservedSize)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -589,7 +589,7 @@ TEST_F(PersistentVolumeEndpointsTest, DeleteNonExistentVolume)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -705,7 +705,7 @@ TEST_F(PersistentVolumeEndpointsTest, NoHeader)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -761,7 +761,7 @@ TEST_F(PersistentVolumeEndpointsTest, BadCredentials)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -837,7 +837,7 @@ TEST_F(PersistentVolumeEndpointsTest, GoodCreateAndDestroyACL)
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -960,7 +960,7 @@ TEST_F(PersistentVolumeEndpointsTest, GoodCreateACLMultipleRoles)
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -1045,7 +1045,7 @@ TEST_F(PersistentVolumeEndpointsTest, BadCreateAndDestroyACL)
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -1174,7 +1174,7 @@ TEST_F(PersistentVolumeEndpointsTest, BadCreateACLMultipleRoles)
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -1264,7 +1264,7 @@ TEST_F(PersistentVolumeEndpointsTest, GoodCreateAndDestroyACLBadCredential)
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -1372,7 +1372,7 @@ TEST_F(PersistentVolumeEndpointsTest, NoAuthentication)
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.authenticate_http_readwrite = false;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -1431,7 +1431,7 @@ TEST_F(PersistentVolumeEndpointsTest, NoSlaveId)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -1490,7 +1490,7 @@ TEST_F(PersistentVolumeEndpointsTest, NoVolumes)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -1547,7 +1547,7 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -1707,7 +1707,7 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -1829,7 +1829,7 @@ TEST_F(PersistentVolumeEndpointsTest, ReserveAndSlaveRemoval)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -1964,7 +1964,7 @@ TEST_F(PersistentVolumeEndpointsTest, SlavesEndpointFullResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/tests/reservation_endpoints_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reservation_endpoints_tests.cpp b/src/tests/reservation_endpoints_tests.cpp
index 8c195eb..505c542 100644
--- a/src/tests/reservation_endpoints_tests.cpp
+++ b/src/tests/reservation_endpoints_tests.cpp
@@ -108,7 +108,7 @@ TEST_F(ReservationEndpointsTest, AvailableResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -207,7 +207,7 @@ TEST_F(ReservationEndpointsTest, ReserveOfferedResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -282,7 +282,7 @@ TEST_F(ReservationEndpointsTest, UnreserveOfferedResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -365,7 +365,7 @@ TEST_F(ReservationEndpointsTest, ReserveAvailableAndOfferedResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   master::Flags masterFlags = CreateMasterFlags();
   // Turn off allocation. We're doing it manually.
@@ -523,7 +523,7 @@ TEST_F(ReservationEndpointsTest, UnreserveAvailableAndOfferedResources)
   // Turn off allocation. We're doing it manually.
   masterFlags.allocation_interval = Seconds(1000);
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -686,7 +686,7 @@ TEST_F(ReservationEndpointsTest, LabeledResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -847,7 +847,7 @@ TEST_F(ReservationEndpointsTest, InvalidResource)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -906,7 +906,7 @@ TEST_F(ReservationEndpointsTest, InsufficientResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -947,7 +947,7 @@ TEST_F(ReservationEndpointsTest, NoHeader)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -996,7 +996,7 @@ TEST_F(ReservationEndpointsTest, BadCredentials)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -1065,7 +1065,7 @@ TEST_F(ReservationEndpointsTest, GoodReserveAndUnreserveACL)
   masterFlags.allocation_interval = Milliseconds(50);
   masterFlags.roles = frameworkInfo.role();
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -1127,7 +1127,7 @@ TEST_F(ReservationEndpointsTest, GoodReserveACLMultipleRoles)
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -1188,7 +1188,7 @@ TEST_F(ReservationEndpointsTest, BadReserveACL)
   masterFlags.allocation_interval = Milliseconds(50);
   masterFlags.roles = frameworkInfo.role();
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -1247,7 +1247,7 @@ TEST_F(ReservationEndpointsTest, BadUnreserveACL)
   masterFlags.allocation_interval = Milliseconds(50);
   masterFlags.roles = frameworkInfo.role();
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -1320,7 +1320,7 @@ TEST_F(ReservationEndpointsTest, BadReserveACLMultipleRoles)
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.acls = acls;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -1397,7 +1397,7 @@ TEST_F(ReservationEndpointsTest, NoResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -1431,7 +1431,7 @@ TEST_F(ReservationEndpointsTest, NonMatchingPrincipal)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -1477,7 +1477,7 @@ TEST_F(ReservationEndpointsTest, ReserveAndUnreserveNoAuthentication)
   masterFlags.authenticate_frameworks = false;
   masterFlags.authenticate_http_readwrite = false;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -1559,7 +1559,7 @@ TEST_F(ReservationEndpointsTest, DifferentPrincipalsSameRole)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -1640,7 +1640,7 @@ TEST_F(ReservationEndpointsTest, AgentStateEndpointResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/tests/reservation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reservation_tests.cpp b/src/tests/reservation_tests.cpp
index 47ccf7f..34628ee 100644
--- a/src/tests/reservation_tests.cpp
+++ b/src/tests/reservation_tests.cpp
@@ -533,7 +533,7 @@ TEST_F(ReservationTest, DropReserveTooLarge)
   masterFlags.allocation_interval = Milliseconds(5);
   masterFlags.roles = frameworkInfo.role();
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -626,7 +626,7 @@ TEST_F(ReservationTest, DropReserveStaticReservation)
   masterFlags.allocation_interval = Milliseconds(5);
   masterFlags.roles = frameworkInfo.role();
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -2148,7 +2148,7 @@ TEST_F(ReservationTest, DropReserveWithInvalidRole)
   masterFlags.allocation_interval = Milliseconds(5);
 
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -2246,7 +2246,7 @@ TEST_F(ReservationTest, PreventUnreservingAlienResources)
   masterFlags.allocation_interval = Milliseconds(5);
 
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/tests/resource_offers_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resource_offers_tests.cpp b/src/tests/resource_offers_tests.cpp
index f0bca1d..7ad037e 100644
--- a/src/tests/resource_offers_tests.cpp
+++ b/src/tests/resource_offers_tests.cpp
@@ -285,7 +285,7 @@ TEST_F(ResourceOffersTest, Request)
 {
   TestAllocator<master::allocator::HierarchicalDRFAllocator> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e176c30/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index df0c5c8..e140f4d 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -3228,7 +3228,7 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileTasksMissingFromSlave)
 {
   TestAllocator<master::allocator::HierarchicalDRFAllocator> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);