You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ne...@apache.org on 2017/03/09 16:53:32 UTC

[5/5] mesos git commit: Simplified interface for setting weights in allocator.

Simplified interface for setting weights in allocator.

Now that weights can change dynamically, passing a list of weights to
the allocator at initialization-time is unnecessary and confusing (e.g.,
the initial weights might be wrong, if a different set of weight values
are recovered from the registry).

Instead, require that weights are communicated to the allocator via the
existing `updateWeights` method.

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


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

Branch: refs/heads/master
Commit: eb08f089574c8e57499872f9c16e8fd828cd355f
Parents: 2d7f284
Author: Neil Conway <ne...@gmail.com>
Authored: Thu Mar 9 11:47:27 2017 -0500
Committer: Neil Conway <ne...@gmail.com>
Committed: Thu Mar 9 11:48:11 2017 -0500

----------------------------------------------------------------------
 include/mesos/allocator/allocator.hpp           |  8 ++-
 src/master/allocator/mesos/allocator.hpp        |  4 --
 src/master/allocator/mesos/hierarchical.cpp     |  2 -
 src/master/allocator/mesos/hierarchical.hpp     |  1 -
 src/master/master.cpp                           | 56 +++++++-------------
 src/tests/allocator.hpp                         |  9 ++--
 src/tests/api_tests.cpp                         |  4 +-
 src/tests/hierarchical_allocator_tests.cpp      |  1 -
 src/tests/master_allocator_tests.cpp            | 34 ++++++------
 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 +-
 15 files changed, 103 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/eb08f089/include/mesos/allocator/allocator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/allocator/allocator.hpp b/include/mesos/allocator/allocator.hpp
index 25ed5f3..6eda1b8 100644
--- a/include/mesos/allocator/allocator.hpp
+++ b/include/mesos/allocator/allocator.hpp
@@ -81,8 +81,6 @@ public:
    *     to the frameworks.
    * @param inverseOfferCallback A callback the allocator uses to send reclaim
    *     allocations from the frameworks.
-   * @param weights Configured per-role weights. Any roles that do not
-   *     appear in this map will be assigned the default weight of 1.
    */
   virtual void initialize(
       const Duration& allocationInterval,
@@ -94,7 +92,6 @@ public:
           void(const FrameworkID&,
                const hashmap<SlaveID, UnavailableResources>&)>&
         inverseOfferCallback,
-      const hashmap<std::string, double>& weights,
       const Option<std::set<std::string>>&
         fairnessExcludeResourceNames = None()) = 0;
 
@@ -415,8 +412,9 @@ public:
       const std::string& role) = 0;
 
   /**
-   * Updates the weight of each provided role.
-   * Subsequent allocation calculations will use these updated weights.
+   * Updates the weight associated with one or more roles. If a role
+   * was previously configured to have a weight and that role is
+   * omitted from this list, it keeps its old weight.
    */
   virtual void updateWeights(
       const std::vector<WeightInfo>& weightInfos) = 0;

http://git-wip-us.apache.org/repos/asf/mesos/blob/eb08f089/src/master/allocator/mesos/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/allocator.hpp b/src/master/allocator/mesos/allocator.hpp
index 1defb59..57b54b8 100644
--- a/src/master/allocator/mesos/allocator.hpp
+++ b/src/master/allocator/mesos/allocator.hpp
@@ -56,7 +56,6 @@ public:
           void(const FrameworkID&,
                const hashmap<SlaveID, UnavailableResources>&)>&
         inverseOfferCallback,
-      const hashmap<std::string, double>& weights,
       const Option<std::set<std::string>>&
         fairnessExcludeResourceNames = None());
 
@@ -193,7 +192,6 @@ public:
           void(const FrameworkID&,
                const hashmap<SlaveID, UnavailableResources>&)>&
         inverseOfferCallback,
-      const hashmap<std::string, double>& weights,
       const Option<std::set<std::string>>&
         fairnessExcludeResourceNames = None()) = 0;
 
@@ -339,7 +337,6 @@ inline void MesosAllocator<AllocatorProcess>::initialize(
         void(const FrameworkID&,
               const hashmap<SlaveID, UnavailableResources>&)>&
       inverseOfferCallback,
-    const hashmap<std::string, double>& weights,
     const Option<std::set<std::string>>& fairnessExcludeResourceNames)
 {
   process::dispatch(
@@ -348,7 +345,6 @@ inline void MesosAllocator<AllocatorProcess>::initialize(
       allocationInterval,
       offerCallback,
       inverseOfferCallback,
-      weights,
       fairnessExcludeResourceNames);
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/eb08f089/src/master/allocator/mesos/hierarchical.cpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.cpp b/src/master/allocator/mesos/hierarchical.cpp
index 0059cce..37fb7a6 100644
--- a/src/master/allocator/mesos/hierarchical.cpp
+++ b/src/master/allocator/mesos/hierarchical.cpp
@@ -141,13 +141,11 @@ void HierarchicalAllocatorProcess::initialize(
         void(const FrameworkID&,
              const hashmap<SlaveID, UnavailableResources>&)>&
       _inverseOfferCallback,
-    const hashmap<string, double>& _weights,
     const Option<set<string>>& _fairnessExcludeResourceNames)
 {
   allocationInterval = _allocationInterval;
   offerCallback = _offerCallback;
   inverseOfferCallback = _inverseOfferCallback;
-  weights = _weights;
   fairnessExcludeResourceNames = _fairnessExcludeResourceNames;
   initialized = true;
   paused = false;

http://git-wip-us.apache.org/repos/asf/mesos/blob/eb08f089/src/master/allocator/mesos/hierarchical.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.hpp b/src/master/allocator/mesos/hierarchical.hpp
index 646f66e..f84b057 100644
--- a/src/master/allocator/mesos/hierarchical.hpp
+++ b/src/master/allocator/mesos/hierarchical.hpp
@@ -101,7 +101,6 @@ public:
           void(const FrameworkID&,
                const hashmap<SlaveID, UnavailableResources>&)>&
         inverseOfferCallback,
-      const hashmap<std::string, double>& weights,
       const Option<std::set<std::string>>&
         fairnessExcludeResourceNames = None());
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/eb08f089/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index dd1e4cd..d43350d 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -762,7 +762,6 @@ void Master::initialize()
       flags.allocation_interval,
       defer(self(), &Master::offer, lambda::_1, lambda::_2),
       defer(self(), &Master::inverseOffer, lambda::_1, lambda::_2),
-      weights,
       flags.fair_sharing_excluded_resource_names);
 
   // Parse the whitelist. Passing Allocator::updateWhitelist()
@@ -1761,51 +1760,34 @@ Future<Nothing> Master::_recover(const Registry& registry)
   // satisfiable given all recovering agents reregister. We may want
   // to notify operators early if total quota cannot be met.
 
-  if (registry.weights_size() != 0) {
-    vector<WeightInfo> weightInfos;
-    hashmap<string, double> registry_weights;
-
-    // Save the weights.
-    foreach (const Registry::Weight& weight, registry.weights()) {
-      registry_weights[weight.info().role()] = weight.info().weight();
-      WeightInfo weightInfo;
-      weightInfo.set_role(weight.info().role());
-      weightInfo.set_weight(weight.info().weight());
-      weightInfos.push_back(weightInfo);
-    }
+  // Recover weights, and update the allocator accordingly. If we
+  // recovered weights from the registry, any weights specified on the
+  // command-line are ignored. If no weights were recovered from the
+  // registry, any weights specified on the command-line are used and
+  // then stored in the registry.
+  vector<WeightInfo> weightInfos;
 
+  if (registry.weights_size() != 0) {
     // TODO(Yongqiao Wang): After the Mesos master quorum is achieved,
-    // operator can send an update weights request to do a batch configuration
-    // for weights, so the `--weights` flag can be deprecated and this check
-    // can eventually be removed.
+    // operator can send an update weights request to do a batch
+    // configuration for weights, so the `--weights` flag can be
+    // deprecated and this check can eventually be removed.
     if (!weights.empty()) {
       LOG(WARNING) << "Ignoring --weights flag '" << flags.weights.get()
                    << "' and recovering the weights from registry";
 
-      // Before recovering weights from the registry, the allocator was already
-      // initialized with `--weights`, so here we need to reset (to 1.0)
-      // weights in the allocator that are not overridden by the registry.
-      foreachkey (const string& role, weights) {
-        if (!registry_weights.contains(role)) {
-          WeightInfo weightInfo;
-          weightInfo.set_role(role);
-          weightInfo.set_weight(1.0);
-          weightInfos.push_back(weightInfo);
-        }
-      }
+      weights.clear();
     }
 
-    // Recover `weights` with `registry_weights`.
-    weights = registry_weights;
+    foreach (const Registry::Weight& weight, registry.weights()) {
+      WeightInfo weightInfo;
+      weightInfo.set_role(weight.info().role());
+      weightInfo.set_weight(weight.info().weight());
+      weightInfos.push_back(weightInfo);
 
-    // Update allocator.
-    allocator->updateWeights(weightInfos);
+      weights[weight.info().role()] = weight.info().weight();
+    }
   } else if (!weights.empty()) {
-    // The allocator was already updated with the `--weights` flag values
-    // on startup.
-    // Initialize the registry with `--weights` flag when bootstrapping
-    // the cluster.
-    vector<WeightInfo> weightInfos;
     foreachpair (const string& role, double weight, weights) {
       WeightInfo weightInfo;
       weightInfo.set_role(role);
@@ -1815,6 +1797,8 @@ Future<Nothing> Master::_recover(const Registry& registry)
     registrar->apply(Owned<Operation>(new weights::UpdateWeights(weightInfos)));
   }
 
+  allocator->updateWeights(weightInfos);
+
   // Recovery is now complete!
   LOG(INFO) << "Recovered " << registry.slaves().slaves().size() << " agents"
             << " from the registry (" << Bytes(registry.ByteSize()) << ")"

http://git-wip-us.apache.org/repos/asf/mesos/blob/eb08f089/src/tests/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/tests/allocator.hpp b/src/tests/allocator.hpp
index b6b0022..6b71c57 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, arg4);
+  allocator->real->initialize(arg0, arg1, arg2, arg3);
 }
 
 
@@ -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_METHOD5(initialize, void(
+  MOCK_METHOD4(initialize, void(
       const Duration&,
       const lambda::function<
           void(const FrameworkID&,
@@ -365,7 +365,6 @@ public:
       const lambda::function<
           void(const FrameworkID&,
                const hashmap<SlaveID, UnavailableResources>&)>&,
-      const hashmap<std::string, double>&,
       const Option<std::set<std::string>>&));
 
   MOCK_METHOD2(recover, void(

http://git-wip-us.apache.org/repos/asf/mesos/blob/eb08f089/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index fae24eb..08ad02e 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/eb08f089/src/tests/hierarchical_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hierarchical_allocator_tests.cpp b/src/tests/hierarchical_allocator_tests.cpp
index cdf1f15..dce619e 100644
--- a/src/tests/hierarchical_allocator_tests.cpp
+++ b/src/tests/hierarchical_allocator_tests.cpp
@@ -180,7 +180,6 @@ protected:
         flags.allocation_interval,
         offerCallback.get(),
         inverseOfferCallback.get(),
-        {},
         flags.fair_sharing_excluded_resource_names);
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/eb08f089/src/tests/master_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_allocator_tests.cpp b/src/tests/master_allocator_tests.cpp
index 7b0b786..0fb66e4 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);
@@ -210,7 +210,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);
@@ -317,7 +317,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);
@@ -445,7 +445,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);
@@ -569,7 +569,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);
@@ -717,7 +717,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);
@@ -841,7 +841,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);
@@ -937,7 +937,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);
@@ -1041,7 +1041,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);
@@ -1122,7 +1122,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);
@@ -1216,7 +1216,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)))
@@ -1255,7 +1255,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");
@@ -1348,7 +1348,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);
@@ -1405,7 +1405,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkReregistersFirst)
   {
     TestAllocator<TypeParam> allocator2;
 
-    EXPECT_CALL(allocator2, initialize(_, _, _, _, _));
+    EXPECT_CALL(allocator2, initialize(_, _, _, _));
 
     Future<Nothing> addFramework;
     EXPECT_CALL(allocator2, addFramework(_, _, _, _))
@@ -1472,7 +1472,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);
@@ -1528,7 +1528,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveReregistersFirst)
   {
     TestAllocator<TypeParam> allocator2;
 
-    EXPECT_CALL(allocator2, initialize(_, _, _, _, _));
+    EXPECT_CALL(allocator2, initialize(_, _, _, _));
 
     Future<Nothing> addSlave;
     EXPECT_CALL(allocator2, addSlave(_, _, _, _, _, _))
@@ -1595,7 +1595,7 @@ TYPED_TEST(MasterAllocatorTest, RebalancedForUpdatedWeights)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _));
 
   // Start Mesos master.
   master::Flags masterFlags = this->CreateMasterFlags();

http://git-wip-us.apache.org/repos/asf/mesos/blob/eb08f089/src/tests/master_quota_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_quota_tests.cpp b/src/tests/master_quota_tests.cpp
index e109656..e418f22 100644
--- a/src/tests/master_quota_tests.cpp
+++ b/src/tests/master_quota_tests.cpp
@@ -402,7 +402,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);
@@ -571,7 +571,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);
@@ -633,7 +633,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);
@@ -710,7 +710,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);
@@ -760,7 +760,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);
@@ -829,7 +829,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);
@@ -1065,7 +1065,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();
@@ -1099,7 +1099,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
@@ -1209,7 +1209,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.
@@ -1392,7 +1392,7 @@ TEST_F(MasterQuotaTest, AuthorizeGetUpdateQuotaRequests)
 TEST_F(MasterQuotaTest, AuthorizeSetAndRemoveQuotaRequests)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _));
 
   // Setup ACLs so that only the default principal can set and see
   // quotas for `ROLE1` and can remove its own quotas.

http://git-wip-us.apache.org/repos/asf/mesos/blob/eb08f089/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 34d569c..741d62e 100644
--- a/src/tests/persistent_volume_endpoints_tests.cpp
+++ b/src/tests/persistent_volume_endpoints_tests.cpp
@@ -116,7 +116,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);
@@ -212,7 +212,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);
@@ -334,7 +334,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);
@@ -416,7 +416,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);
@@ -481,7 +481,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);
@@ -549,7 +549,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);
@@ -591,7 +591,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);
@@ -707,7 +707,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);
@@ -763,7 +763,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);
@@ -839,7 +839,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);
@@ -962,7 +962,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);
@@ -1047,7 +1047,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);
@@ -1176,7 +1176,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);
@@ -1266,7 +1266,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);
@@ -1374,7 +1374,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);
@@ -1433,7 +1433,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);
@@ -1492,7 +1492,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);
@@ -1549,7 +1549,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);
@@ -1709,7 +1709,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);
@@ -1831,7 +1831,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);
@@ -1966,7 +1966,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/eb08f089/src/tests/reservation_endpoints_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reservation_endpoints_tests.cpp b/src/tests/reservation_endpoints_tests.cpp
index 345f045..ddf279b 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/eb08f089/src/tests/reservation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reservation_tests.cpp b/src/tests/reservation_tests.cpp
index 5c0d014..95cc911 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/eb08f089/src/tests/resource_offers_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resource_offers_tests.cpp b/src/tests/resource_offers_tests.cpp
index 74dacf1..02fb248 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/eb08f089/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index a836101..dc82db9 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -3115,7 +3115,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);