You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by nn...@apache.org on 2015/04/21 21:56:27 UTC

[1/9] mesos git commit: Removed Master::Flags dependency from Allocator.

Repository: mesos
Updated Branches:
  refs/heads/master 01ee1e504 -> da607079b


Removed Master::Flags dependency from Allocator.

On the way to moving allocator to public includes we need to remove the
dependency to internal Master::Flags type.

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


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

Branch: refs/heads/master
Commit: 97dce737c0ea8822e37f6eb13cc195039f77c623
Parents: 01ee1e5
Author: Alexander Rukletsov <ru...@gmail.com>
Authored: Tue Apr 21 12:09:09 2015 -0700
Committer: Niklas Q. Nielsen <ni...@qni.dk>
Committed: Tue Apr 21 12:09:11 2015 -0700

----------------------------------------------------------------------
 src/master/allocator/allocator.hpp          |  7 ++-----
 src/master/allocator/mesos/allocator.hpp    |  8 ++++----
 src/master/allocator/mesos/hierarchical.hpp | 12 ++++++------
 src/master/master.cpp                       |  2 +-
 src/tests/hierarchical_allocator_tests.cpp  |  2 +-
 src/tests/mesos.hpp                         |  2 +-
 6 files changed, 15 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/97dce737/src/master/allocator/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/allocator.hpp b/src/master/allocator/allocator.hpp
index 5e3e613..38bc3e9 100644
--- a/src/master/allocator/allocator.hpp
+++ b/src/master/allocator/allocator.hpp
@@ -24,18 +24,15 @@
 
 #include <mesos/resources.hpp>
 
+#include <stout/duration.hpp>
 #include <stout/hashmap.hpp>
 #include <stout/hashset.hpp>
 #include <stout/lambda.hpp>
 #include <stout/option.hpp>
 
-#include "master/flags.hpp"
-#include "messages/messages.hpp"
-
 namespace mesos {
 namespace internal {
 namespace master {
-
 namespace allocator {
 
 // Basic model of an allocator: resources are allocated to a framework
@@ -56,7 +53,7 @@ public:
   virtual ~Allocator() {}
 
   virtual void initialize(
-      const Flags& flags,
+      const Duration& allocationInterval,
       const lambda::function<
           void(const FrameworkID&,
                const hashmap<SlaveID, Resources>&)>& offerCallback,

http://git-wip-us.apache.org/repos/asf/mesos/blob/97dce737/src/master/allocator/mesos/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/allocator.hpp b/src/master/allocator/mesos/allocator.hpp
index af27a9b..5100aec 100644
--- a/src/master/allocator/mesos/allocator.hpp
+++ b/src/master/allocator/mesos/allocator.hpp
@@ -44,7 +44,7 @@ public:
   ~MesosAllocator();
 
   void initialize(
-      const Flags& flags,
+      const Duration& allocationInterval,
       const lambda::function<
           void(const FrameworkID&,
                const hashmap<SlaveID, Resources>&)>& offerCallback,
@@ -121,7 +121,7 @@ public:
   using process::ProcessBase::initialize;
 
   virtual void initialize(
-      const Flags& flags,
+      const Duration& allocationInterval,
       const lambda::function<
           void(const FrameworkID&,
                const hashmap<SlaveID, Resources>&)>& offerCallback,
@@ -198,7 +198,7 @@ MesosAllocator<AllocatorProcess>::~MesosAllocator()
 
 template <typename AllocatorProcess>
 inline void MesosAllocator<AllocatorProcess>::initialize(
-    const Flags& flags,
+    const Duration& allocationInterval,
     const lambda::function<
         void(const FrameworkID&,
              const hashmap<SlaveID, Resources>&)>& offerCallback,
@@ -207,7 +207,7 @@ inline void MesosAllocator<AllocatorProcess>::initialize(
   process::dispatch(
       process,
       &MesosAllocatorProcess::initialize,
-      flags,
+      allocationInterval,
       offerCallback,
       roles);
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/97dce737/src/master/allocator/mesos/hierarchical.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.hpp b/src/master/allocator/mesos/hierarchical.hpp
index 90ac197..95b7905 100644
--- a/src/master/allocator/mesos/hierarchical.hpp
+++ b/src/master/allocator/mesos/hierarchical.hpp
@@ -72,7 +72,7 @@ public:
   process::PID<HierarchicalAllocatorProcess> self();
 
   void initialize(
-      const Flags& flags,
+      const Duration& allocationInterval,
       const lambda::function<
           void(const FrameworkID&,
                const hashmap<SlaveID, Resources>&)>& offerCallback,
@@ -162,7 +162,7 @@ protected:
 
   bool initialized;
 
-  Flags flags;
+  Duration allocationInterval;
 
   lambda::function<
       void(const FrameworkID&,
@@ -263,13 +263,13 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::self()
 template <class RoleSorter, class FrameworkSorter>
 void
 HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::initialize(
-    const Flags& _flags,
+    const Duration& _allocationInterval,
     const lambda::function<
         void(const FrameworkID&,
              const hashmap<SlaveID, Resources>&)>& _offerCallback,
     const hashmap<std::string, RoleInfo>& _roles)
 {
-  flags = _flags;
+  allocationInterval = _allocationInterval;
   offerCallback = _offerCallback;
   roles = _roles;
   initialized = true;
@@ -286,7 +286,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::initialize(
 
   VLOG(1) << "Initialized hierarchical allocator process";
 
-  delay(flags.allocation_interval, self(), &Self::batch);
+  delay(allocationInterval, self(), &Self::batch);
 }
 
 
@@ -713,7 +713,7 @@ void
 HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::batch()
 {
   allocate();
-  delay(flags.allocation_interval, self(), &Self::batch);
+  delay(allocationInterval, self(), &Self::batch);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/97dce737/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index b1093bb..2c9187d 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -600,7 +600,7 @@ void Master::initialize()
 
   // Initialize the allocator.
   allocator->initialize(
-      flags,
+      flags.allocation_interval,
       defer(self(), &Master::offer, lambda::_1, lambda::_2),
       roleInfos);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/97dce737/src/tests/hierarchical_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hierarchical_allocator_tests.cpp b/src/tests/hierarchical_allocator_tests.cpp
index 0b564a7..cf822a2 100644
--- a/src/tests/hierarchical_allocator_tests.cpp
+++ b/src/tests/hierarchical_allocator_tests.cpp
@@ -95,7 +95,7 @@ protected:
     }
 
     allocator->initialize(
-        flags,
+        flags.allocation_interval,
         lambda::bind(&put, &queue, lambda::_1, lambda::_2),
         roles);
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/97dce737/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 4edb33b..f8c9604 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -846,7 +846,7 @@ public:
   ~TestAllocator() {}
 
   MOCK_METHOD3(initialize, void(
-      const master::Flags&,
+      const Duration&,
       const lambda::function<
           void(const FrameworkID&,
                const hashmap<SlaveID, Resources>&)>&,


[5/9] mesos git commit: Added a test allocator module.

Posted by nn...@apache.org.
Added a test allocator module.

The test allocator module is a built-in HierarchicalDRFAllocator put in
a module.

NOTE: Here we add harness code to load the module, tests will be wired
up later in the patch stack.

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


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

Branch: refs/heads/master
Commit: a5f270fe513ecb50021eb06d429a1a84e260f98a
Parents: 079fc0af
Author: Alexander Rukletsov <ru...@gmail.com>
Authored: Tue Apr 21 12:11:03 2015 -0700
Committer: Niklas Q. Nielsen <ni...@qni.dk>
Committed: Tue Apr 21 12:11:03 2015 -0700

----------------------------------------------------------------------
 src/Makefile.am                        |  6 ++++
 src/examples/test_allocator_module.cpp | 53 +++++++++++++++++++++++++++++
 src/tests/module.cpp                   | 32 +++++++++++++++++
 src/tests/module.hpp                   |  3 +-
 4 files changed, 93 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a5f270fe/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index afca3c8..93c7c8a 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1355,6 +1355,12 @@ libtestauthentication_la_SOURCES = examples/test_authentication_modules.cpp
 libtestauthentication_la_CPPFLAGS = $(MESOS_CPPFLAGS)
 libtestauthentication_la_LDFLAGS = $(MESOS_MODULE_LDFLAGS)
 
+# Library containing the test DRF allocator module.
+noinst_LTLIBRARIES += libtestallocator.la
+libtestallocator_la_SOURCES = examples/test_allocator_module.cpp
+libtestallocator_la_CPPFLAGS = $(MESOS_CPPFLAGS)
+libtestallocator_la_LDFLAGS = $(MESOS_MODULE_LDFLAGS)
+
 # Library containing test Hook module.
 noinst_LTLIBRARIES += libtesthook.la
 libtesthook_la_SOURCES = examples/test_hook_module.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/a5f270fe/src/examples/test_allocator_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_allocator_module.cpp b/src/examples/test_allocator_module.cpp
new file mode 100644
index 0000000..5fe68e3
--- /dev/null
+++ b/src/examples/test_allocator_module.cpp
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <mesos/mesos.hpp>
+#include <mesos/module.hpp>
+
+#include <mesos/master/allocator.hpp>
+
+#include <mesos/module/allocator.hpp>
+
+#include <stout/try.hpp>
+
+#include "master/constants.hpp"
+
+#include "master/allocator/mesos/hierarchical.hpp"
+
+using namespace mesos;
+
+using mesos::master::allocator::Allocator;
+using mesos::internal::master::allocator::HierarchicalDRFAllocator;
+
+
+static Allocator* createDRFAllocator(const Parameters& parameters)
+{
+  return new HierarchicalDRFAllocator();
+}
+
+
+// Declares a DRFAllocator module named
+// 'org_apache_mesos_TestDRFAllocator'.
+mesos::modules::Module<Allocator> org_apache_mesos_TestDRFAllocator(
+    MESOS_MODULE_API_VERSION,
+    MESOS_VERSION,
+    "Apache Mesos",
+    "modules@mesos.apache.org",
+    "Test DRFAllocator module.",
+    NULL,
+    createDRFAllocator);

http://git-wip-us.apache.org/repos/asf/mesos/blob/a5f270fe/src/tests/module.cpp
----------------------------------------------------------------------
diff --git a/src/tests/module.cpp b/src/tests/module.cpp
index b81144f..c4f8119 100644
--- a/src/tests/module.cpp
+++ b/src/tests/module.cpp
@@ -53,6 +53,8 @@ static void addModule(
 // Add available Isolator modules.
 static void addIsolatorModules(Modules* modules)
 {
+  CHECK_NOTNULL(modules);
+
   const string libraryPath = path::join(
       tests::flags.build_dir,
       "src",
@@ -73,6 +75,8 @@ static void addIsolatorModules(Modules* modules)
 // Add available Authentication modules.
 static void addAuthenticationModules(Modules* modules)
 {
+  CHECK_NOTNULL(modules);
+
   const string libraryPath = path::join(
       tests::flags.build_dir,
       "src",
@@ -96,6 +100,8 @@ static void addAuthenticationModules(Modules* modules)
 
 static void addHookModules(Modules* modules)
 {
+  CHECK_NOTNULL(modules);
+
   const string libraryPath = path::join(
       tests::flags.build_dir,
       "src",
@@ -114,6 +120,8 @@ static void addHookModules(Modules* modules)
 
 static void addAnonymousModules(Modules* modules)
 {
+  CHECK_NOTNULL(modules);
+
   const string libraryPath = path::join(
       tests::flags.build_dir,
       "src",
@@ -131,6 +139,27 @@ static void addAnonymousModules(Modules* modules)
 }
 
 
+// Add available Allocator modules.
+static void addAllocatorModules(Modules* modules)
+{
+  CHECK_NOTNULL(modules);
+
+  const string libraryPath = path::join(
+      tests::flags.build_dir,
+      "src",
+      ".libs",
+      os::libraries::expandName("testallocator"));
+
+  // Now add our allocator module.
+  Modules::Library* library = modules->add_libraries();
+  library->set_file(libraryPath);
+
+  // To add a new module from this library, create a new ModuleID enum
+  // and tie it with a module name.
+  addModule(library, TestDRFAllocator, "org_apache_mesos_TestDRFAllocator");
+}
+
+
 Try<Nothing> initModules(const Option<Modules>& modules)
 {
   // First get the user provided modules.
@@ -151,6 +180,9 @@ Try<Nothing> initModules(const Option<Modules>& modules)
   // Add anonymous modules from testanonymous library.
   addAnonymousModules(&mergedModules);
 
+  // Add allocator modules from testallocator library.
+  addAllocatorModules(&mergedModules);
+
   return ModuleManager::load(mergedModules);
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a5f270fe/src/tests/module.hpp
----------------------------------------------------------------------
diff --git a/src/tests/module.hpp b/src/tests/module.hpp
index 65e567f..c379f01 100644
--- a/src/tests/module.hpp
+++ b/src/tests/module.hpp
@@ -46,7 +46,8 @@ enum ModuleID
   TestCRAMMD5Authenticatee,
   TestCRAMMD5Authenticator,
   TestHook,
-  TestAnonymous
+  TestAnonymous,
+  TestDRFAllocator
 };
 
 


[8/9] mesos git commit: Provided a factory for allocator in tests.

Posted by nn...@apache.org.
Provided a factory for allocator in tests.

The factory creates allocator instances in a way identical to how
instances from modules are created. It allows us to use same typed tests
for built-in and modularized allocators.

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


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

Branch: refs/heads/master
Commit: 54b5bdccb43f724353fd3112115e980360e53c9a
Parents: ab48d54
Author: Alexander Rukletsov <ru...@gmail.com>
Authored: Tue Apr 21 12:11:58 2015 -0700
Committer: Niklas Q. Nielsen <ni...@qni.dk>
Committed: Tue Apr 21 12:11:59 2015 -0700

----------------------------------------------------------------------
 src/examples/test_allocator_module.cpp     |  7 ++++++-
 src/local/local.cpp                        | 14 ++++++++++++--
 src/master/allocator/mesos/allocator.hpp   | 13 ++++++++++++-
 src/master/main.cpp                        | 10 ++++++++--
 src/tests/cluster.hpp                      | 10 +++++++++-
 src/tests/hierarchical_allocator_tests.cpp |  4 +++-
 src/tests/mesos.hpp                        | 13 +++++++++++--
 7 files changed, 61 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/54b5bdcc/src/examples/test_allocator_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_allocator_module.cpp b/src/examples/test_allocator_module.cpp
index 5fe68e3..9679fd9 100644
--- a/src/examples/test_allocator_module.cpp
+++ b/src/examples/test_allocator_module.cpp
@@ -37,7 +37,12 @@ using mesos::internal::master::allocator::HierarchicalDRFAllocator;
 
 static Allocator* createDRFAllocator(const Parameters& parameters)
 {
-  return new HierarchicalDRFAllocator();
+  Try<Allocator*> allocator = HierarchicalDRFAllocator::create();
+  if (allocator.isError()) {
+    return NULL;
+  }
+
+  return allocator.get();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/54b5bdcc/src/local/local.cpp
----------------------------------------------------------------------
diff --git a/src/local/local.cpp b/src/local/local.cpp
index f448912..dff55d8 100644
--- a/src/local/local.cpp
+++ b/src/local/local.cpp
@@ -133,8 +133,18 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
   }
 
   if (_allocator == NULL) {
-    // Create default allocator, save it for deleting later.
-    _allocator = allocator = new HierarchicalDRFAllocator();
+    // Create a default allocator.
+    Try<Allocator*> defaultAllocator = HierarchicalDRFAllocator::create();
+    if (defaultAllocator.isError()) {
+      EXIT(1) << "Failed to create an instance of HierarchicalDRFAllocator: "
+              << defaultAllocator.error();
+    }
+
+    // Update caller's instance.
+    _allocator = defaultAllocator.get();
+
+    // Save the instance for deleting later.
+    allocator = defaultAllocator.get();
   } else {
     // TODO(benh): Figure out the behavior of allocator pointer and remove the
     // else block.

http://git-wip-us.apache.org/repos/asf/mesos/blob/54b5bdcc/src/master/allocator/mesos/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/allocator.hpp b/src/master/allocator/mesos/allocator.hpp
index 5360d0a..2681af5 100644
--- a/src/master/allocator/mesos/allocator.hpp
+++ b/src/master/allocator/mesos/allocator.hpp
@@ -24,6 +24,8 @@
 #include <process/dispatch.hpp>
 #include <process/process.hpp>
 
+#include <stout/try.hpp>
+
 namespace mesos {
 namespace internal {
 namespace master {
@@ -39,7 +41,8 @@ template <typename AllocatorProcess>
 class MesosAllocator : public mesos::master::allocator::Allocator
 {
 public:
-  MesosAllocator();
+  // Factory to allow for typed tests.
+  static Try<mesos::master::allocator::Allocator*> create();
 
   ~MesosAllocator();
 
@@ -101,6 +104,7 @@ public:
       const FrameworkID& frameworkId);
 
 private:
+  MesosAllocator();
   MesosAllocator(const MesosAllocator&); // Not copyable.
   MesosAllocator& operator=(const MesosAllocator&); // Not assignable.
 
@@ -180,6 +184,13 @@ public:
 
 
 template <typename AllocatorProcess>
+Try<mesos::master::allocator::Allocator*>
+MesosAllocator<AllocatorProcess>::create()
+{
+  return new MesosAllocator<AllocatorProcess>();
+}
+
+template <typename AllocatorProcess>
 MesosAllocator<AllocatorProcess>::MesosAllocator()
 {
   process = new AllocatorProcess();

http://git-wip-us.apache.org/repos/asf/mesos/blob/54b5bdcc/src/master/main.cpp
----------------------------------------------------------------------
diff --git a/src/master/main.cpp b/src/master/main.cpp
index 5ec1825..88bb39d 100644
--- a/src/master/main.cpp
+++ b/src/master/main.cpp
@@ -200,8 +200,14 @@ int main(int argc, char** argv)
     LOG(INFO) << "Git SHA: " << build::GIT_SHA.get();
   }
 
-  mesos::master::allocator::Allocator* allocator =
-    new allocator::HierarchicalDRFAllocator();
+  // Create an instance of allocator.
+  Try<mesos::master::allocator::Allocator*> allocator_ =
+    allocator::HierarchicalDRFAllocator::create();
+  if (allocator_.isError()) {
+    EXIT(1) << "Failed to create an instance of HierarchicalDRFAllocator: "
+            << allocator_.error();
+  }
+  mesos::master::allocator::Allocator* allocator = allocator_.get();
 
   state::Storage* storage = NULL;
   Log* log = NULL;

http://git-wip-us.apache.org/repos/asf/mesos/blob/54b5bdcc/src/tests/cluster.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cluster.hpp b/src/tests/cluster.hpp
index fc4d5d1..8a176be 100644
--- a/src/tests/cluster.hpp
+++ b/src/tests/cluster.hpp
@@ -271,7 +271,15 @@ inline Try<process::PID<master::Master> > Cluster::Masters::start(
   } else {
     // If allocator is not provided, fall back to the default one,
     // managed by Cluster::Masters.
-    master.allocator = new master::allocator::HierarchicalDRFAllocator();
+    Try<mesos::master::allocator::Allocator*> allocator_ =
+      master::allocator::HierarchicalDRFAllocator::create();
+    if (allocator_.isError()) {
+      return Error(
+          "Failed to create an instance of HierarchicalDRFAllocator: " +
+          allocator_.error());
+    }
+
+    master.allocator = allocator_.get();
     master.createdAllocator = true;
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/54b5bdcc/src/tests/hierarchical_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hierarchical_allocator_tests.cpp b/src/tests/hierarchical_allocator_tests.cpp
index e25b99f..1a43dc7 100644
--- a/src/tests/hierarchical_allocator_tests.cpp
+++ b/src/tests/hierarchical_allocator_tests.cpp
@@ -40,6 +40,8 @@
 
 #include "master/allocator/mesos/hierarchical.hpp"
 
+#include "tests/mesos.hpp"
+
 using mesos::internal::master::MIN_CPUS;
 using mesos::internal::master::MIN_MEM;
 
@@ -71,7 +73,7 @@ class HierarchicalAllocatorTest : public ::testing::Test
 {
 protected:
   HierarchicalAllocatorTest()
-    : allocator(new HierarchicalDRFAllocator),
+    : allocator(createAllocator<HierarchicalDRFAllocator>()),
       nextSlaveId(1),
       nextFrameworkId(1) {}
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/54b5bdcc/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index f23fdcb..4294e28 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -859,13 +859,22 @@ ACTION_P(InvokeReviveOffers, allocator)
 
 
 template <typename T = master::allocator::HierarchicalDRFAllocator>
+mesos::master::allocator::Allocator* createAllocator()
+{
+  // T represents the allocator type. It can be a default built-in
+  // allocator, or one provided by an allocator module.
+  Try<mesos::master::allocator::Allocator*> instance = T::create();
+  CHECK_SOME(instance);
+  return CHECK_NOTNULL(instance.get());
+}
+
+template <typename T = master::allocator::HierarchicalDRFAllocator>
 class TestAllocator : public mesos::master::allocator::Allocator
 {
 public:
   // Actual allocation is done by an instance of real allocator,
   // which is specified by the template parameter.
-  TestAllocator()
-    : real(new master::allocator::HierarchicalDRFAllocator())
+  TestAllocator() : real(createAllocator<T>())
   {
     // We use 'ON_CALL' and 'WillByDefault' here to specify the
     // default actions (call in to the real allocator). This allows


[6/9] mesos git commit: Moved allocator actions before TestAllocator.

Posted by nn...@apache.org.
Moved allocator actions before TestAllocator.

The allocator actions should be defined before they are used in
TestAllocator, despite it compiles now since TestAllocator is a
template.

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


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

Branch: refs/heads/master
Commit: c97c93e3ee5c6f7286c99b5e623cdcff663fd19b
Parents: a5f270f
Author: Alexander Rukletsov <ru...@gmail.com>
Authored: Tue Apr 21 12:11:20 2015 -0700
Committer: Niklas Q. Nielsen <ni...@qni.dk>
Committed: Tue Apr 21 12:11:21 2015 -0700

----------------------------------------------------------------------
 src/tests/mesos.hpp | 198 +++++++++++++++++++++++------------------------
 1 file changed, 99 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c97c93e3/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 8a53430..bab45ce 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -757,6 +757,105 @@ public:
 };
 
 
+// The following actions make up for the fact that DoDefault
+// cannot be used inside a DoAll, for example:
+// EXPECT_CALL(allocator, addFramework(_, _, _))
+//   .WillOnce(DoAll(InvokeAddFramework(&allocator),
+//                   FutureSatisfy(&addFramework)));
+
+ACTION_P(InvokeInitialize, allocator)
+{
+  allocator->real.initialize(arg0, arg1, arg2);
+}
+
+
+ACTION_P(InvokeAddFramework, allocator)
+{
+  allocator->real.addFramework(arg0, arg1, arg2);
+}
+
+
+ACTION_P(InvokeRemoveFramework, allocator)
+{
+  allocator->real.removeFramework(arg0);
+}
+
+
+ACTION_P(InvokeActivateFramework, allocator)
+{
+  allocator->real.activateFramework(arg0);
+}
+
+
+ACTION_P(InvokeDeactivateFramework, allocator)
+{
+  allocator->real.deactivateFramework(arg0);
+}
+
+
+ACTION_P(InvokeAddSlave, allocator)
+{
+  allocator->real.addSlave(arg0, arg1, arg2, arg3);
+}
+
+
+ACTION_P(InvokeRemoveSlave, allocator)
+{
+  allocator->real.removeSlave(arg0);
+}
+
+
+ACTION_P(InvokeActivateSlave, allocator)
+{
+  allocator->real.activateSlave(arg0);
+}
+
+
+ACTION_P(InvokeDeactivateSlave, allocator)
+{
+  allocator->real.deactivateSlave(arg0);
+}
+
+
+ACTION_P(InvokeUpdateWhitelist, allocator)
+{
+  allocator->real.updateWhitelist(arg0);
+}
+
+
+ACTION_P(InvokeRequestResources, allocator)
+{
+  allocator->real.requestResources(arg0, arg1);
+}
+
+
+ACTION_P(InvokeUpdateAllocation, allocator)
+{
+  allocator->real.updateAllocation(arg0, arg1, arg2);
+}
+
+
+ACTION_P(InvokeRecoverResources, allocator)
+{
+  allocator->real.recoverResources(arg0, arg1, arg2, arg3);
+}
+
+
+ACTION_P2(InvokeRecoverResourcesWithFilters, allocator, timeout)
+{
+  Filters filters;
+  filters.set_refuse_seconds(timeout);
+
+  allocator->real.recoverResources(arg0, arg1, arg2, filters);
+}
+
+
+ACTION_P(InvokeReviveOffers, allocator)
+{
+  allocator->real.reviveOffers(arg0);
+}
+
+
 template <typename T = mesos::master::allocator::Allocator>
 class TestAllocator : public mesos::master::allocator::Allocator
 {
@@ -905,105 +1004,6 @@ public:
 };
 
 
-// The following actions make up for the fact that DoDefault
-// cannot be used inside a DoAll, for example:
-// EXPECT_CALL(allocator, addFramework(_, _, _))
-//   .WillOnce(DoAll(InvokeAddFramework(&allocator),
-//                   FutureSatisfy(&addFramework)));
-
-ACTION_P(InvokeInitialize, allocator)
-{
-  allocator->real.initialize(arg0, arg1, arg2);
-}
-
-
-ACTION_P(InvokeAddFramework, allocator)
-{
-  allocator->real.addFramework(arg0, arg1, arg2);
-}
-
-
-ACTION_P(InvokeRemoveFramework, allocator)
-{
-  allocator->real.removeFramework(arg0);
-}
-
-
-ACTION_P(InvokeActivateFramework, allocator)
-{
-  allocator->real.activateFramework(arg0);
-}
-
-
-ACTION_P(InvokeDeactivateFramework, allocator)
-{
-  allocator->real.deactivateFramework(arg0);
-}
-
-
-ACTION_P(InvokeAddSlave, allocator)
-{
-  allocator->real.addSlave(arg0, arg1, arg2, arg3);
-}
-
-
-ACTION_P(InvokeRemoveSlave, allocator)
-{
-  allocator->real.removeSlave(arg0);
-}
-
-
-ACTION_P(InvokeActivateSlave, allocator)
-{
-  allocator->real.activateSlave(arg0);
-}
-
-
-ACTION_P(InvokeDeactivateSlave, allocator)
-{
-  allocator->real.deactivateSlave(arg0);
-}
-
-
-ACTION_P(InvokeUpdateWhitelist, allocator)
-{
-  allocator->real.updateWhitelist(arg0);
-}
-
-
-ACTION_P(InvokeRequestResources, allocator)
-{
-  allocator->real.requestResources(arg0, arg1);
-}
-
-
-ACTION_P(InvokeUpdateAllocation, allocator)
-{
-  allocator->real.updateAllocation(arg0, arg1, arg2);
-}
-
-
-ACTION_P(InvokeRecoverResources, allocator)
-{
-  allocator->real.recoverResources(arg0, arg1, arg2, arg3);
-}
-
-
-ACTION_P2(InvokeRecoverResourcesWithFilters, allocator, timeout)
-{
-  Filters filters;
-  filters.set_refuse_seconds(timeout);
-
-  allocator->real.recoverResources(arg0, arg1, arg2, filters);
-}
-
-
-ACTION_P(InvokeReviveOffers, allocator)
-{
-  allocator->real.reviveOffers(arg0);
-}
-
-
 class OfferEqMatcher
   : public ::testing::MatcherInterface<const std::vector<Offer>& >
 {


[2/9] mesos git commit: Moved allocator to public headers.

Posted by nn...@apache.org.
Moved allocator to public headers.

This is required for out-of-tree allocator modules. RoleInfo protobuf
message has to be extracted into its own public proto file.

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


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

Branch: refs/heads/master
Commit: f1fc3d76de93b0149e2797e3923f4bf939037c98
Parents: 97dce73
Author: Alexander Rukletsov <ru...@gmail.com>
Authored: Tue Apr 21 12:10:07 2015 -0700
Committer: Niklas Q. Nielsen <ni...@qni.dk>
Committed: Tue Apr 21 12:10:11 2015 -0700

----------------------------------------------------------------------
 include/mesos/master/allocator.hpp              | 130 +++++++++++++++++++
 include/mesos/master/allocator.proto            |  27 ++++
 src/Makefile.am                                 |  23 +++-
 src/local/local.cpp                             |   6 +-
 src/local/local.hpp                             |  14 +-
 src/master/allocator/allocator.hpp              | 129 ------------------
 src/master/allocator/mesos/allocator.hpp        |   6 +-
 src/master/main.cpp                             |   6 +-
 src/master/master.cpp                           |   6 +-
 src/master/master.hpp                           |  11 +-
 src/messages/messages.proto                     |   9 --
 src/tests/cluster.hpp                           |   9 +-
 src/tests/fault_tolerance_tests.cpp             |   4 +-
 src/tests/hierarchical_allocator_tests.cpp      |   5 +-
 src/tests/master_allocator_tests.cpp            |   5 +-
 src/tests/master_authorization_tests.cpp        |   4 +-
 src/tests/master_slave_reconciliation_tests.cpp |   4 +-
 src/tests/master_tests.cpp                      |   4 +-
 src/tests/mesos.cpp                             |   2 +-
 src/tests/mesos.hpp                             |  10 +-
 src/tests/rate_limiting_tests.cpp               |   4 +-
 21 files changed, 227 insertions(+), 191 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/include/mesos/master/allocator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/master/allocator.hpp b/include/mesos/master/allocator.hpp
new file mode 100644
index 0000000..bb40b1c
--- /dev/null
+++ b/include/mesos/master/allocator.hpp
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __MESOS_MASTER_ALLOCATOR_HPP__
+#define __MESOS_MASTER_ALLOCATOR_HPP__
+
+#include <string>
+#include <vector>
+
+// ONLY USEFUL AFTER RUNNING PROTOC.
+#include <mesos/master/allocator.pb.h>
+
+#include <mesos/resources.hpp>
+
+#include <stout/duration.hpp>
+#include <stout/hashmap.hpp>
+#include <stout/hashset.hpp>
+#include <stout/lambda.hpp>
+#include <stout/option.hpp>
+
+namespace mesos {
+namespace master {
+namespace allocator {
+
+// Basic model of an allocator: resources are allocated to a framework
+// in the form of offers. A framework can refuse some resources in
+// offers and run tasks in others. Allocated resources can have offer
+// operations applied to them in order for frameworks to alter the
+// resource metadata (e.g. creating persistent volumes). Resources can
+// be recovered from a framework when tasks finish/fail (or are lost
+// due to a slave failure) or when an offer is rescinded.
+//
+// This is the public API for resource allocators.
+// TODO(alexr): Document API calls.
+class Allocator
+{
+public:
+  Allocator() {}
+
+  virtual ~Allocator() {}
+
+  virtual void initialize(
+      const Duration& allocationInterval,
+      const lambda::function<
+          void(const FrameworkID&,
+               const hashmap<SlaveID, Resources>&)>& offerCallback,
+      const hashmap<std::string, internal::RoleInfo>& roles) = 0;
+
+  virtual void addFramework(
+      const FrameworkID& frameworkId,
+      const FrameworkInfo& frameworkInfo,
+      const hashmap<SlaveID, Resources>& used) = 0;
+
+  virtual void removeFramework(
+      const FrameworkID& frameworkId) = 0;
+
+  // Offers are sent only to activated frameworks.
+  virtual void activateFramework(
+      const FrameworkID& frameworkId) = 0;
+
+  virtual void deactivateFramework(
+      const FrameworkID& frameworkId) = 0;
+
+  // Note that the 'total' resources are passed explicitly because it
+  // includes resources that are dynamically "checkpointed" on the
+  // slave (e.g. persistent volumes, dynamic reservations, etc). The
+  // slaveInfo resources, on the other hand, correspond directly to
+  // the static --resources flag value on the slave.
+  virtual void addSlave(
+      const SlaveID& slaveId,
+      const SlaveInfo& slaveInfo,
+      const Resources& total,
+      const hashmap<FrameworkID, Resources>& used) = 0;
+
+  virtual void removeSlave(
+      const SlaveID& slaveId) = 0;
+
+  // Offers are sent only for activated slaves.
+  virtual void activateSlave(
+      const SlaveID& slaveId) = 0;
+
+  virtual void deactivateSlave(
+      const SlaveID& slaveId) = 0;
+
+  virtual void updateWhitelist(
+      const Option<hashset<std::string> >& whitelist) = 0;
+
+  virtual void requestResources(
+      const FrameworkID& frameworkId,
+      const std::vector<Request>& requests) = 0;
+
+  virtual void updateAllocation(
+      const FrameworkID& frameworkId,
+      const SlaveID& slaveId,
+      const std::vector<Offer::Operation>& operations) = 0;
+
+  // Informs the Allocator to recover resources that are considered
+  // used by the framework.
+  virtual void recoverResources(
+      const FrameworkID& frameworkId,
+      const SlaveID& slaveId,
+      const Resources& resources,
+      const Option<Filters>& filters) = 0;
+
+  // Whenever a framework that has filtered resources wants to revive
+  // offers for those resources the master invokes this callback.
+  virtual void reviveOffers(
+      const FrameworkID& frameworkId) = 0;
+};
+
+} // namespace allocator {
+} // namespace master {
+} // namespace mesos {
+
+#endif // __MESOS_MASTER_ALLOCATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/include/mesos/master/allocator.proto
----------------------------------------------------------------------
diff --git a/include/mesos/master/allocator.proto b/include/mesos/master/allocator.proto
new file mode 100644
index 0000000..5566719
--- /dev/null
+++ b/include/mesos/master/allocator.proto
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package mesos.internal;
+
+// Describes a role, which are used to group frameworks for allocation
+// decisions, depending on the allocation policy being used.
+// The weight field can be used to indicate forms of priority.
+message RoleInfo {
+  required string name = 1;
+  optional double weight = 2 [default = 1];
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index d15a373..5751e96 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -140,6 +140,9 @@ endif
 
 MESOS_PROTO = $(top_srcdir)/include/mesos/mesos.proto
 
+ALLOCATOR_PROTO =							\
+  $(top_srcdir)/include/mesos/master/allocator.proto
+
 AUTHENTICATION_PROTO =							\
   $(top_srcdir)/include/mesos/authentication/authentication.proto
 
@@ -164,6 +167,8 @@ CXX_PROTOS =								\
   ../include/mesos/containerizer/containerizer.pb.h			\
   fetcher/fetcher.pb.cc							\
   ../include/mesos/fetcher/fetcher.pb.h					\
+  master/allocator.pb.cc						\
+  ../include/mesos/master/allocator.pb.h				\
   module/module.pb.cc							\
   ../include/mesos/module/module.pb.h					\
   scheduler/scheduler.pb.cc						\
@@ -226,6 +231,12 @@ fetcher/%.pb.cc ../include/mesos/fetcher/%.pb.h: $(FETCHER_PROTO)
 	$(PROTOC) $(PROTOCFLAGS) --cpp_out=../include $^
 	mv ../include/mesos/fetcher/*.pb.cc $(@D)
 
+master/%.pb.cc ../include/mesos/master/%.pb.h: $(ALLOCATOR_PROTO)
+	$(MKDIR_P) $(@D)
+	$(MKDIR_P) ../include/mesos/master
+	$(PROTOC) $(PROTOCFLAGS) --cpp_out=../include $^
+	mv ../include/mesos/master/*.pb.cc $(@D)
+
 module/%.pb.cc ../include/mesos/module/%.pb.h: $(MODULE_PROTO)
 	$(MKDIR_P) $(@D)
 	$(MKDIR_P) ../include/mesos/module
@@ -407,6 +418,14 @@ fetcher_HEADERS =							\
 
 nodist_fetcher_HEADERS = ../include/mesos/fetcher/fetcher.pb.h
 
+masterdir = $(pkgincludedir)/master
+
+master_HEADERS =							\
+  $(top_srcdir)/include/mesos/master/allocator.hpp			\
+  $(top_srcdir)/include/mesos/master/allocator.proto
+
+nodist_master_HEADERS = ../include/mesos/master/allocator.pb.h
+
 moduledir = $(pkgincludedir)/module
 
 module_HEADERS =							\
@@ -526,7 +545,6 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	master/repairer.hpp						\
 	master/registrar.hpp						\
 	master/validation.hpp						\
-	master/allocator/allocator.hpp					\
 	master/allocator/mesos/allocator.hpp				\
 	master/allocator/mesos/hierarchical.hpp				\
 	master/allocator/sorter/drf/sorter.hpp				\
@@ -692,7 +710,8 @@ libmesos_no_3rdparty_la_LIBADD += libstate.la
 lib_LTLIBRARIES += libmesos.la
 
 # Include as part of the distribution.
-libmesos_la_SOURCES = 							\
+libmesos_la_SOURCES =							\
+  $(ALLOCATOR_PROTO)							\
   $(CONTAINERIZER_PROTO)						\
   $(FETCHER_PROTO)							\
   $(MESOS_PROTO)							\

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/local/local.cpp
----------------------------------------------------------------------
diff --git a/src/local/local.cpp b/src/local/local.cpp
index 289b9bc..f448912 100644
--- a/src/local/local.cpp
+++ b/src/local/local.cpp
@@ -22,6 +22,8 @@
 #include <string>
 #include <vector>
 
+#include <mesos/master/allocator.hpp>
+
 #include <mesos/module/anonymous.hpp>
 
 #include <process/limiter.hpp>
@@ -52,7 +54,6 @@
 #include "master/registrar.hpp"
 #include "master/repairer.hpp"
 
-#include "master/allocator/allocator.hpp"
 #include "master/allocator/mesos/hierarchical.hpp"
 #include "master/allocator/sorter/drf/sorter.hpp"
 
@@ -75,7 +76,8 @@ using memory::shared_ptr;
 using namespace mesos::internal;
 using namespace mesos::internal::log;
 
-using mesos::internal::master::allocator::Allocator;
+using mesos::master::allocator::Allocator;
+
 using mesos::internal::master::allocator::HierarchicalDRFAllocator;
 
 using mesos::internal::master::Master;

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/local/local.hpp
----------------------------------------------------------------------
diff --git a/src/local/local.hpp b/src/local/local.hpp
index 0aa50ef..6bb25f1 100644
--- a/src/local/local.hpp
+++ b/src/local/local.hpp
@@ -19,24 +19,18 @@
 #ifndef __MESOS_LOCAL_HPP__
 #define __MESOS_LOCAL_HPP__
 
+#include <mesos/master/allocator.hpp>
+
 #include <process/process.hpp>
 
 #include "local/flags.hpp"
 
+// Forward declarations.
 namespace mesos {
 namespace internal {
 
-// Forward declarations.
 namespace master {
-
 class Master;
-
-namespace allocator {
-
-class Allocator;
-
-} // namespace allocator {
-
 } // namespace master {
 
 class Configuration;
@@ -46,7 +40,7 @@ namespace local {
 // Launch a local cluster with the given flags.
 process::PID<master::Master> launch(
     const Flags& flags,
-    master::allocator::Allocator* _allocator = NULL);
+    mesos::master::allocator::Allocator* _allocator = NULL);
 
 void shutdown();
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/master/allocator/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/allocator.hpp b/src/master/allocator/allocator.hpp
deleted file mode 100644
index 38bc3e9..0000000
--- a/src/master/allocator/allocator.hpp
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#ifndef __MASTER_ALLOCATOR_ALLOCATOR_HPP__
-#define __MASTER_ALLOCATOR_ALLOCATOR_HPP__
-
-#include <string>
-#include <vector>
-
-#include <mesos/resources.hpp>
-
-#include <stout/duration.hpp>
-#include <stout/hashmap.hpp>
-#include <stout/hashset.hpp>
-#include <stout/lambda.hpp>
-#include <stout/option.hpp>
-
-namespace mesos {
-namespace internal {
-namespace master {
-namespace allocator {
-
-// Basic model of an allocator: resources are allocated to a framework
-// in the form of offers. A framework can refuse some resources in
-// offers and run tasks in others. Allocated resources can have offer
-// operations applied to them in order for frameworks to alter the
-// resource metadata (e.g. creating persistent volumes). Resources can
-// be recovered from a framework when tasks finish/fail (or are lost
-// due to a slave failure) or when an offer is rescinded.
-//
-// This is the public API for resource allocators.
-// TODO(alexr): Document API calls.
-class Allocator
-{
-public:
-  Allocator() {}
-
-  virtual ~Allocator() {}
-
-  virtual void initialize(
-      const Duration& allocationInterval,
-      const lambda::function<
-          void(const FrameworkID&,
-               const hashmap<SlaveID, Resources>&)>& offerCallback,
-      const hashmap<std::string, RoleInfo>& roles) = 0;
-
-  virtual void addFramework(
-      const FrameworkID& frameworkId,
-      const FrameworkInfo& frameworkInfo,
-      const hashmap<SlaveID, Resources>& used) = 0;
-
-  virtual void removeFramework(
-      const FrameworkID& frameworkId) = 0;
-
-  // Offers are sent only to activated frameworks.
-  virtual void activateFramework(
-      const FrameworkID& frameworkId) = 0;
-
-  virtual void deactivateFramework(
-      const FrameworkID& frameworkId) = 0;
-
-  // Note that the 'total' resources are passed explicitly because it
-  // includes resources that are dynamically "checkpointed" on the
-  // slave (e.g. persistent volumes, dynamic reservations, etc). The
-  // slaveInfo resources, on the other hand, correspond directly to
-  // the static --resources flag value on the slave.
-  virtual void addSlave(
-      const SlaveID& slaveId,
-      const SlaveInfo& slaveInfo,
-      const Resources& total,
-      const hashmap<FrameworkID, Resources>& used) = 0;
-
-  virtual void removeSlave(
-      const SlaveID& slaveId) = 0;
-
-  // Offers are sent only for activated slaves.
-  virtual void activateSlave(
-      const SlaveID& slaveId) = 0;
-
-  virtual void deactivateSlave(
-      const SlaveID& slaveId) = 0;
-
-  virtual void updateWhitelist(
-      const Option<hashset<std::string> >& whitelist) = 0;
-
-  virtual void requestResources(
-      const FrameworkID& frameworkId,
-      const std::vector<Request>& requests) = 0;
-
-  virtual void updateAllocation(
-      const FrameworkID& frameworkId,
-      const SlaveID& slaveId,
-      const std::vector<Offer::Operation>& operations) = 0;
-
-  // Informs the Allocator to recover resources that are considered
-  // used by the framework.
-  virtual void recoverResources(
-      const FrameworkID& frameworkId,
-      const SlaveID& slaveId,
-      const Resources& resources,
-      const Option<Filters>& filters) = 0;
-
-  // Whenever a framework that has filtered resources wants to revive
-  // offers for those resources the master invokes this callback.
-  virtual void reviveOffers(
-      const FrameworkID& frameworkId) = 0;
-};
-
-} // namespace allocator {
-} // namespace master {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __MASTER_ALLOCATOR_ALLOCATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/master/allocator/mesos/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/allocator.hpp b/src/master/allocator/mesos/allocator.hpp
index 5100aec..4089ac0 100644
--- a/src/master/allocator/mesos/allocator.hpp
+++ b/src/master/allocator/mesos/allocator.hpp
@@ -19,11 +19,11 @@
 #ifndef __MASTER_ALLOCATOR_MESOS_ALLOCATOR_HPP__
 #define __MASTER_ALLOCATOR_MESOS_ALLOCATOR_HPP__
 
+#include <mesos/master/allocator.hpp>
+
 #include <process/dispatch.hpp>
 #include <process/process.hpp>
 
-#include "master/allocator/allocator.hpp"
-
 namespace mesos {
 namespace internal {
 namespace master {
@@ -36,7 +36,7 @@ class MesosAllocatorProcess;
 // lifetime. We ensure the template parameter AllocatorProcess
 // implements MesosAllocatorProcess by storing a pointer to it.
 template <typename AllocatorProcess>
-class MesosAllocator : public Allocator
+class MesosAllocator : public mesos::master::allocator::Allocator
 {
 public:
   MesosAllocator();

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/master/main.cpp
----------------------------------------------------------------------
diff --git a/src/master/main.cpp b/src/master/main.cpp
index 7cce3a0..5ec1825 100644
--- a/src/master/main.cpp
+++ b/src/master/main.cpp
@@ -24,6 +24,8 @@
 
 #include <mesos/mesos.hpp>
 
+#include <mesos/master/allocator.hpp>
+
 #include <mesos/module/anonymous.hpp>
 
 #include <process/limiter.hpp>
@@ -59,7 +61,6 @@
 #include "master/registrar.hpp"
 #include "master/repairer.hpp"
 
-#include "master/allocator/allocator.hpp"
 #include "master/allocator/mesos/hierarchical.hpp"
 
 #include "module/manager.hpp"
@@ -199,7 +200,8 @@ int main(int argc, char** argv)
     LOG(INFO) << "Git SHA: " << build::GIT_SHA.get();
   }
 
-  allocator::Allocator* allocator = new allocator::HierarchicalDRFAllocator();
+  mesos::master::allocator::Allocator* allocator =
+    new allocator::HierarchicalDRFAllocator();
 
   state::Storage* storage = NULL;
   Log* log = NULL;

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 2c9187d..b726838 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -29,6 +29,8 @@
 
 #include <mesos/authentication/authenticator.hpp>
 
+#include <mesos/master/allocator.hpp>
+
 #include <mesos/module/authenticator.hpp>
 
 #include <process/check.hpp>
@@ -77,8 +79,6 @@
 #include "master/flags.hpp"
 #include "master/master.hpp"
 
-#include "master/allocator/allocator.hpp"
-
 #include "module/manager.hpp"
 
 #include "watcher/whitelist_watcher.hpp"
@@ -112,7 +112,7 @@ namespace mesos {
 namespace internal {
 namespace master {
 
-using allocator::Allocator;
+using mesos::master::allocator::Allocator;
 
 
 class SlaveObserver : public Process<SlaveObserver>

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index c10e7c0..d21129b 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -32,6 +32,8 @@
 #include <mesos/scheduler.hpp>
 #include <mesos/type_utils.hpp>
 
+#include <mesos/master/allocator.hpp>
+
 #include <mesos/module/authenticator.hpp>
 
 #include <process/limiter.hpp>
@@ -81,11 +83,6 @@ class WhitelistWatcher;
 
 namespace master {
 
-// Forward declarations.
-namespace allocator {
-class Allocator;
-}
-
 class Repairer;
 class SlaveObserver;
 
@@ -98,7 +95,7 @@ struct Slave;
 class Master : public ProtobufProcess<Master>
 {
 public:
-  Master(allocator::Allocator* allocator,
+  Master(mesos::master::allocator::Allocator* allocator,
          Registrar* registrar,
          Repairer* repairer,
          Files* files,
@@ -538,7 +535,7 @@ private:
 
   Option<MasterInfo> leader; // Current leading master.
 
-  allocator::Allocator* allocator;
+  mesos::master::allocator::Allocator* allocator;
   WhitelistWatcher* whitelistWatcher;
   Registrar* registrar;
   Repairer* repairer;

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/messages/messages.proto
----------------------------------------------------------------------
diff --git a/src/messages/messages.proto b/src/messages/messages.proto
index 2d242dc..bdf474b 100644
--- a/src/messages/messages.proto
+++ b/src/messages/messages.proto
@@ -64,15 +64,6 @@ message Task {
 }
 
 
-// Describes a role, which are used to group frameworks for allocation
-// decisions, depending on the allocation policy being used.
-// The weight field can be used to indicate forms of priority.
-message RoleInfo {
-  required string name = 1;
-  optional double weight = 2 [default = 1];
-}
-
-
 // TODO(vinod): Create a new UUID message type.
 message StatusUpdate {
   required FrameworkID framework_id = 1;

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/tests/cluster.hpp
----------------------------------------------------------------------
diff --git a/src/tests/cluster.hpp b/src/tests/cluster.hpp
index a56b654..fc4d5d1 100644
--- a/src/tests/cluster.hpp
+++ b/src/tests/cluster.hpp
@@ -25,6 +25,8 @@
 
 #include <mesos/mesos.hpp>
 
+#include <mesos/master/allocator.hpp>
+
 #include <process/clock.hpp>
 #include <process/future.hpp>
 #include <process/gmock.hpp>
@@ -64,7 +66,6 @@
 #include "master/registrar.hpp"
 #include "master/repairer.hpp"
 
-#include "master/allocator/allocator.hpp"
 #include "master/allocator/mesos/hierarchical.hpp"
 
 #include "slave/flags.hpp"
@@ -105,7 +106,7 @@ public:
     // Start a new master with the provided flags and injections.
     Try<process::PID<master::Master> > start(
         const master::Flags& flags = master::Flags(),
-        const Option<master::allocator::Allocator*>& allocator = None(),
+        const Option<mesos::master::allocator::Allocator*>& allocator = None(),
         const Option<Authorizer*>& authorizer = None(),
         const Option<memory::shared_ptr<process::RateLimiter> >&
           slaveRemovalLimiter = None());
@@ -129,7 +130,7 @@ public:
     {
       Master() : allocator(NULL), createdAllocator(false), master(NULL) {}
 
-      master::allocator::Allocator* allocator;
+      mesos::master::allocator::Allocator* allocator;
       bool createdAllocator; // Whether we own the allocator.
 
       process::Owned<log::Log> log;
@@ -254,7 +255,7 @@ inline void Cluster::Masters::shutdown()
 
 inline Try<process::PID<master::Master> > Cluster::Masters::start(
     const master::Flags& flags,
-    const Option<master::allocator::Allocator*>& allocator,
+    const Option<mesos::master::allocator::Allocator*>& allocator,
     const Option<Authorizer*>& authorizer,
     const Option<memory::shared_ptr<process::RateLimiter>>& slaveRemovalLimiter)
 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/tests/fault_tolerance_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fault_tolerance_tests.cpp b/src/tests/fault_tolerance_tests.cpp
index a637c32..3a27d82 100644
--- a/src/tests/fault_tolerance_tests.cpp
+++ b/src/tests/fault_tolerance_tests.cpp
@@ -26,6 +26,8 @@
 
 #include <mesos/authentication/authentication.hpp>
 
+#include <mesos/master/allocator.hpp>
+
 #include <process/future.hpp>
 #include <process/gmock.hpp>
 #include <process/http.hpp>
@@ -41,8 +43,6 @@
 
 #include "master/master.hpp"
 
-#include "master/allocator/allocator.hpp"
-
 #include "sched/constants.hpp"
 
 #include "slave/constants.hpp"

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/tests/hierarchical_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hierarchical_allocator_tests.cpp b/src/tests/hierarchical_allocator_tests.cpp
index cf822a2..ca93bd1 100644
--- a/src/tests/hierarchical_allocator_tests.cpp
+++ b/src/tests/hierarchical_allocator_tests.cpp
@@ -22,6 +22,8 @@
 #include <queue>
 #include <vector>
 
+#include <mesos/master/allocator.hpp>
+
 #include <process/clock.hpp>
 #include <process/future.hpp>
 #include <process/gtest.hpp>
@@ -36,13 +38,12 @@
 #include "master/constants.hpp"
 #include "master/flags.hpp"
 
-#include "master/allocator/allocator.hpp"
 #include "master/allocator/mesos/hierarchical.hpp"
 
 using mesos::internal::master::MIN_CPUS;
 using mesos::internal::master::MIN_MEM;
 
-using mesos::internal::master::allocator::Allocator;
+using mesos::master::allocator::Allocator;
 using mesos::internal::master::allocator::HierarchicalDRFAllocator;
 
 using process::Clock;

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/tests/master_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_allocator_tests.cpp b/src/tests/master_allocator_tests.cpp
index 03a1bb8..f29e90e 100644
--- a/src/tests/master_allocator_tests.cpp
+++ b/src/tests/master_allocator_tests.cpp
@@ -25,6 +25,8 @@
 #include <mesos/executor.hpp>
 #include <mesos/scheduler.hpp>
 
+#include <mesos/master/allocator.hpp>
+
 #include <process/clock.hpp>
 #include <process/future.hpp>
 #include <process/gmock.hpp>
@@ -37,13 +39,12 @@
 #include "master/detector.hpp"
 #include "master/master.hpp"
 
-#include "master/allocator/allocator.hpp"
 #include "master/allocator/mesos/hierarchical.hpp"
 
 #include "tests/containerizer.hpp"
 #include "tests/mesos.hpp"
 
-using mesos::internal::master::allocator::Allocator;
+using mesos::master::allocator::Allocator;
 using mesos::internal::master::allocator::HierarchicalDRFAllocator;
 
 using mesos::internal::master::Master;

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/tests/master_authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_authorization_tests.cpp b/src/tests/master_authorization_tests.cpp
index ac79303..5633c82 100644
--- a/src/tests/master_authorization_tests.cpp
+++ b/src/tests/master_authorization_tests.cpp
@@ -23,6 +23,8 @@
 #include <mesos/executor.hpp>
 #include <mesos/scheduler.hpp>
 
+#include <mesos/master/allocator.hpp>
+
 #include <process/clock.hpp>
 #include <process/future.hpp>
 #include <process/pid.hpp>
@@ -33,8 +35,6 @@
 
 #include "master/master.hpp"
 
-#include "master/allocator/allocator.hpp"
-
 #include "messages/messages.hpp"
 
 #include "slave/slave.hpp"

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/tests/master_slave_reconciliation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_slave_reconciliation_tests.cpp b/src/tests/master_slave_reconciliation_tests.cpp
index e60f601..ded3f52 100644
--- a/src/tests/master_slave_reconciliation_tests.cpp
+++ b/src/tests/master_slave_reconciliation_tests.cpp
@@ -24,6 +24,8 @@
 #include <mesos/mesos.hpp>
 #include <mesos/scheduler.hpp>
 
+#include <mesos/master/allocator.hpp>
+
 #include <process/future.hpp>
 #include <process/gmock.hpp>
 #include <process/pid.hpp>
@@ -34,8 +36,6 @@
 
 #include "master/master.hpp"
 
-#include "master/allocator/allocator.hpp"
-
 #include "slave/slave.hpp"
 
 #include "tests/containerizer.hpp"

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index 32b1e9b..8405105 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -26,6 +26,8 @@
 #include <mesos/executor.hpp>
 #include <mesos/scheduler.hpp>
 
+#include <mesos/master/allocator.hpp>
+
 #include <process/clock.hpp>
 #include <process/future.hpp>
 #include <process/gmock.hpp>
@@ -48,8 +50,6 @@
 #include "master/flags.hpp"
 #include "master/master.hpp"
 
-#include "master/allocator/allocator.hpp"
-
 #include "slave/constants.hpp"
 #include "slave/gc.hpp"
 #include "slave/flags.hpp"

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/tests/mesos.cpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.cpp b/src/tests/mesos.cpp
index 42a4015..1dde4fe 100644
--- a/src/tests/mesos.cpp
+++ b/src/tests/mesos.cpp
@@ -189,7 +189,7 @@ Try<PID<master::Master> > MesosTest::StartMaster(
 
 
 Try<PID<master::Master> > MesosTest::StartMaster(
-    master::allocator::Allocator* allocator,
+    mesos::master::allocator::Allocator* allocator,
     const Option<master::Flags>& flags)
 {
   return cluster.masters.start(

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index f8c9604..61eaa28 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -27,6 +27,8 @@
 #include <mesos/executor.hpp>
 #include <mesos/scheduler.hpp>
 
+#include <mesos/master/allocator.hpp>
+
 #include <process/future.hpp>
 #include <process/gmock.hpp>
 #include <process/gtest.hpp>
@@ -51,8 +53,6 @@
 #include "master/detector.hpp"
 #include "master/master.hpp"
 
-#include "master/allocator/allocator.hpp"
-
 #include "slave/slave.hpp"
 
 #include "slave/containerizer/containerizer.hpp"
@@ -99,7 +99,7 @@ protected:
 
   // Starts a master with the specified allocator process and flags.
   virtual Try<process::PID<master::Master> > StartMaster(
-      master::allocator::Allocator* allocator,
+      mesos::master::allocator::Allocator* allocator,
       const Option<master::Flags>& flags = None());
 
   // Starts a master with the specified authorizer and flags.
@@ -757,8 +757,8 @@ public:
 };
 
 
-template <typename T = master::allocator::Allocator>
-class TestAllocator : public master::allocator::Allocator
+template <typename T = mesos::master::allocator::Allocator>
+class TestAllocator : public mesos::master::allocator::Allocator
 {
 public:
   TestAllocator()

http://git-wip-us.apache.org/repos/asf/mesos/blob/f1fc3d76/src/tests/rate_limiting_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/rate_limiting_tests.cpp b/src/tests/rate_limiting_tests.cpp
index d5c00b8..49d907b 100644
--- a/src/tests/rate_limiting_tests.cpp
+++ b/src/tests/rate_limiting_tests.cpp
@@ -18,6 +18,8 @@
 
 #include <gmock/gmock.h>
 
+#include <mesos/master/allocator.hpp>
+
 #include <process/clock.hpp>
 #include <process/future.hpp>
 #include <process/gmock.hpp>
@@ -28,8 +30,6 @@
 #include "master/flags.hpp"
 #include "master/master.hpp"
 
-#include "master/allocator/allocator.hpp"
-
 #include "tests/mesos.hpp"
 #include "tests/utils.hpp"
 


[7/9] mesos git commit: Refactored TestAllocator and allocator text fixture.

Posted by nn...@apache.org.
Refactored TestAllocator and allocator text fixture.

TestAllocator owns a pointer to a real allocator. Each test should
instantiate and destroy Allocator instances explicitly to avoid not
expected calls.

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


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

Branch: refs/heads/master
Commit: ab48d54c0250bffc5542bf68afd648e6a12025ad
Parents: c97c93e
Author: Alexander Rukletsov <ru...@gmail.com>
Authored: Tue Apr 21 12:11:37 2015 -0700
Committer: Niklas Q. Nielsen <ni...@qni.dk>
Committed: Tue Apr 21 12:11:39 2015 -0700

----------------------------------------------------------------------
 src/tests/containerizer.cpp          |   2 +-
 src/tests/master_allocator_tests.cpp | 525 ++++++++++++++++--------------
 src/tests/mesos.hpp                  |  41 ++-
 src/tests/slave_recovery_tests.cpp   |   2 +
 4 files changed, 310 insertions(+), 260 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ab48d54c/src/tests/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer.cpp b/src/tests/containerizer.cpp
index 26b87ac..80b9105 100644
--- a/src/tests/containerizer.cpp
+++ b/src/tests/containerizer.cpp
@@ -224,7 +224,7 @@ void TestContainerizer::setup()
   // verbose.
   //
   // TODO(bmahler): Update this to use the same style as the
-  // TestAllocatorProcess, which allows us to have default actions
+  // TestAllocator, which allows us to have default actions
   // 'DoDefault', without requiring each test to put expectations in
   // place.
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/ab48d54c/src/tests/master_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_allocator_tests.cpp b/src/tests/master_allocator_tests.cpp
index f29e90e..8d1a866 100644
--- a/src/tests/master_allocator_tests.cpp
+++ b/src/tests/master_allocator_tests.cpp
@@ -72,12 +72,7 @@ namespace tests {
 
 
 template <typename T>
-class MasterAllocatorTest : public MesosTest
-{
-protected:
-  TestAllocator<T> allocator;
-};
-
+class MasterAllocatorTest : public MesosTest {};
 
 typedef ::testing::Types<HierarchicalDRFAllocator> AllocatorTypes;
 
@@ -91,24 +86,26 @@ TYPED_TEST_CASE(MasterAllocatorTest, AllocatorTypes);
 // the slave's resources are offered to the framework.
 TYPED_TEST(MasterAllocatorTest, SingleFramework)
 {
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
+  TestAllocator<TypeParam> allocator;
 
-  Try<PID<Master> > master = this->StartMaster(&this->allocator);
+  EXPECT_CALL(allocator, initialize(_, _, _));
+
+  Try<PID<Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
 
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:2;mem:1024;disk:0");
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
-  Try<PID<Slave> > slave = this->StartSlave(flags);
+  Try<PID<Slave>> slave = this->StartSlave(flags);
   ASSERT_SOME(slave);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, addFramework(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
@@ -135,9 +132,11 @@ TYPED_TEST(MasterAllocatorTest, SingleFramework)
 // reoffered appropriately.
 TYPED_TEST(MasterAllocatorTest, ResourcesUnused)
 {
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
+  TestAllocator<TypeParam> allocator;
+
+  EXPECT_CALL(allocator, initialize(_, _, _));
 
-  Try<PID<Master> > master = this->StartMaster(&this->allocator);
+  Try<PID<Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
@@ -145,7 +144,7 @@ TYPED_TEST(MasterAllocatorTest, ResourcesUnused)
   slave::Flags flags1 = this->CreateSlaveFlags();
   flags1.resources = Some("cpus:2;mem:1024");
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave1 = this->StartSlave(&exec, flags1);
   ASSERT_SOME(slave1);
@@ -154,7 +153,7 @@ TYPED_TEST(MasterAllocatorTest, ResourcesUnused)
   MesosSchedulerDriver driver1(
       &sched1, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, addFramework(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched1, registered(_, _, _));
 
@@ -172,8 +171,8 @@ TYPED_TEST(MasterAllocatorTest, ResourcesUnused)
     .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 1, 512, "*"));
 
   Future<Nothing> recoverResources;
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
-    .WillOnce(DoAll(InvokeRecoverResources(&this->allocator),
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
+    .WillOnce(DoAll(InvokeRecoverResources(&allocator),
                     FutureSatisfy(&recoverResources)));
 
   EXPECT_CALL(exec, registered(_, _, _, _));
@@ -200,7 +199,7 @@ TYPED_TEST(MasterAllocatorTest, ResourcesUnused)
   MesosSchedulerDriver driver2(
       &sched2, frameworkInfo2, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, addFramework(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched2, registered(_, _, _));
 
@@ -215,7 +214,7 @@ TYPED_TEST(MasterAllocatorTest, ResourcesUnused)
   AWAIT_READY(resourceOffers);
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
   Future<Nothing> shutdown;
@@ -239,15 +238,17 @@ TYPED_TEST(MasterAllocatorTest, ResourcesUnused)
 // recoverResources is called for an already removed framework.
 TYPED_TEST(MasterAllocatorTest, OutOfOrderDispatch)
 {
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
+  TestAllocator<TypeParam> allocator;
 
-  Try<PID<Master> > master = this->StartMaster(&this->allocator);
+  EXPECT_CALL(allocator, initialize(_, _, _));
+
+  Try<PID<Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
 
   slave::Flags flags1 = this->CreateSlaveFlags();
   flags1.resources = Some("cpus:2;mem:1024");
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave1 = this->StartSlave(flags1);
   ASSERT_SOME(slave1);
@@ -261,8 +262,8 @@ TYPED_TEST(MasterAllocatorTest, OutOfOrderDispatch)
   MesosSchedulerDriver driver1(
       &sched1, frameworkInfo1, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, addFramework(_, Eq(frameworkInfo1), _))
-    .WillOnce(InvokeAddFramework(&this->allocator));
+  EXPECT_CALL(allocator, addFramework(_, Eq(frameworkInfo1), _))
+    .WillOnce(InvokeAddFramework(&allocator));
 
   Future<FrameworkID> frameworkId1;
   EXPECT_CALL(sched1, registered(_, _, _))
@@ -285,18 +286,18 @@ TYPED_TEST(MasterAllocatorTest, OutOfOrderDispatch)
   // framework has terminated or is inactive.
   Future<SlaveID> slaveId;
   Future<Resources> savedResources;
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     // "Catches" the recoverResources call from the master, so
     // that it doesn't get processed until we redispatch it after
     // the removeFramework trigger.
     .WillOnce(DoAll(FutureArg<1>(&slaveId),
                     FutureArg<2>(&savedResources)));
 
-  EXPECT_CALL(this->allocator, deactivateFramework(_));
+  EXPECT_CALL(allocator, deactivateFramework(_));
 
   Future<Nothing> removeFramework;
-  EXPECT_CALL(this->allocator, removeFramework(Eq(frameworkId1.get())))
-    .WillOnce(DoAll(InvokeRemoveFramework(&this->allocator),
+  EXPECT_CALL(allocator, removeFramework(Eq(frameworkId1.get())))
+    .WillOnce(DoAll(InvokeRemoveFramework(&allocator),
                     FutureSatisfy(&removeFramework)));
 
   driver1.stop();
@@ -306,13 +307,13 @@ TYPED_TEST(MasterAllocatorTest, OutOfOrderDispatch)
   AWAIT_READY(slaveId);
   AWAIT_READY(savedResources);
 
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillOnce(DoDefault()); // For the re-dispatch.
 
   // Re-dispatch the recoverResources call which we "caught"
   // earlier now that the framework has been removed, to test
   // that recovering resources from a removed framework works.
-  this->allocator.recoverResources(
+  allocator.recoverResources(
       frameworkId1.get(),
       slaveId.get(),
       savedResources.get(),
@@ -330,8 +331,8 @@ TYPED_TEST(MasterAllocatorTest, OutOfOrderDispatch)
   MesosSchedulerDriver driver2(
       &sched2, frameworkInfo2, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, addFramework(_, Eq(frameworkInfo2), _))
-    .WillOnce(InvokeAddFramework(&this->allocator));
+  EXPECT_CALL(allocator, addFramework(_, Eq(frameworkInfo2), _))
+    .WillOnce(InvokeAddFramework(&allocator));
 
   FrameworkID frameworkId2;
   EXPECT_CALL(sched2, registered(_, _, _))
@@ -347,11 +348,11 @@ TYPED_TEST(MasterAllocatorTest, OutOfOrderDispatch)
   AWAIT_READY(resourceOffers);
 
   // Called when driver2 stops.
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
-  EXPECT_CALL(this->allocator, deactivateFramework(_))
+  EXPECT_CALL(allocator, deactivateFramework(_))
     .WillRepeatedly(DoDefault());
-  EXPECT_CALL(this->allocator, removeFramework(_))
+  EXPECT_CALL(allocator, removeFramework(_))
     .WillRepeatedly(DoDefault());
 
   // Shut everything down.
@@ -367,9 +368,11 @@ TYPED_TEST(MasterAllocatorTest, OutOfOrderDispatch)
 // is running.
 TYPED_TEST(MasterAllocatorTest, SchedulerFailover)
 {
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
+  TestAllocator<TypeParam> allocator;
+
+  EXPECT_CALL(allocator, initialize(_, _, _));
 
-  Try<PID<Master> > master = this->StartMaster(&this->allocator);
+  Try<PID<Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
@@ -377,7 +380,7 @@ TYPED_TEST(MasterAllocatorTest, SchedulerFailover)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:3;mem:1024");
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&exec, flags);
   ASSERT_SOME(slave);
@@ -393,7 +396,7 @@ TYPED_TEST(MasterAllocatorTest, SchedulerFailover)
   MesosSchedulerDriver driver1(
       &sched1, frameworkInfo1, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, addFramework(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   FrameworkID frameworkId;
   EXPECT_CALL(sched1, registered(&driver1, _, _))
@@ -412,10 +415,10 @@ TYPED_TEST(MasterAllocatorTest, SchedulerFailover)
 
   // We don't filter the unused resources to make sure that
   // they get offered to the framework as soon as it fails over.
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
-    .WillOnce(InvokeRecoverResourcesWithFilters(&this->allocator, 0))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
+    .WillOnce(InvokeRecoverResourcesWithFilters(&allocator, 0))
     // For subsequent offers.
-    .WillRepeatedly(InvokeRecoverResourcesWithFilters(&this->allocator, 0));
+    .WillRepeatedly(InvokeRecoverResourcesWithFilters(&allocator, 0));
 
   EXPECT_CALL(exec, registered(_, _, _, _));
 
@@ -435,8 +438,8 @@ TYPED_TEST(MasterAllocatorTest, SchedulerFailover)
   DROP_PROTOBUFS(UnregisterFrameworkMessage(), _, _);
 
   Future<Nothing> deactivateFramework;
-  EXPECT_CALL(this->allocator, deactivateFramework(_))
-    .WillOnce(DoAll(InvokeDeactivateFramework(&this->allocator),
+  EXPECT_CALL(allocator, deactivateFramework(_))
+    .WillOnce(DoAll(InvokeDeactivateFramework(&allocator),
                     FutureSatisfy(&deactivateFramework)));
 
   driver1.stop();
@@ -453,7 +456,7 @@ TYPED_TEST(MasterAllocatorTest, SchedulerFailover)
   MesosSchedulerDriver driver2(
       &sched2, frameworkInfo2, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, activateFramework(_));
+  EXPECT_CALL(allocator, activateFramework(_));
 
   EXPECT_CALL(sched2, registered(_, frameworkId, _));
 
@@ -472,10 +475,10 @@ TYPED_TEST(MasterAllocatorTest, SchedulerFailover)
     .Times(AtMost(1));
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(this->allocator, deactivateFramework(_))
+  EXPECT_CALL(allocator, deactivateFramework(_))
     .Times(AtMost(1));
 
   driver2.stop();
@@ -489,11 +492,13 @@ TYPED_TEST(MasterAllocatorTest, SchedulerFailover)
 // is killed, the tasks resources are returned and reoffered correctly.
 TYPED_TEST(MasterAllocatorTest, FrameworkExited)
 {
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
+  TestAllocator<TypeParam> allocator;
+
+  EXPECT_CALL(allocator, initialize(_, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
-  Try<PID<Master> > master = this->StartMaster(&this->allocator, masterFlags);
+  Try<PID<Master>> master = this->StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
 
   ExecutorInfo executor1; // Bug in gcc 4.1.*, must assign on next line.
@@ -514,7 +519,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkExited)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:3;mem:1024");
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&containerizer, flags);
   ASSERT_SOME(slave);
@@ -523,7 +528,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkExited)
   MesosSchedulerDriver driver1(
       &sched1, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, addFramework(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched1, registered(_, _, _));
 
@@ -541,8 +546,8 @@ TYPED_TEST(MasterAllocatorTest, FrameworkExited)
 
   // The framework does not use all the resources.
   Future<Nothing> recoverResources;
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
-    .WillOnce(DoAll(InvokeRecoverResources(&this->allocator),
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
+    .WillOnce(DoAll(InvokeRecoverResources(&allocator),
                     FutureSatisfy(&recoverResources)));
 
   EXPECT_CALL(exec1, registered(_, _, _, _));
@@ -567,7 +572,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkExited)
   MesosSchedulerDriver driver2(
       &sched2, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, addFramework(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched2, registered(_, _, _));
 
@@ -585,8 +590,8 @@ TYPED_TEST(MasterAllocatorTest, FrameworkExited)
 
   // The framework 2 does not use all the resources.
   Future<Nothing> recoverResources2;
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
-    .WillOnce(DoAll(InvokeRecoverResources(&this->allocator),
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
+    .WillOnce(DoAll(InvokeRecoverResources(&allocator),
                     FutureSatisfy(&recoverResources2)));
 
   EXPECT_CALL(exec2, registered(_, _, _, _));
@@ -602,7 +607,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkExited)
 
   // Shut everything down but check that framework 2 gets the
   // resources from framework 1 after it is shutdown.
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
   // After we stop framework 1, all of it's resources should
@@ -636,9 +641,11 @@ TYPED_TEST(MasterAllocatorTest, FrameworkExited)
 // slave, never offered again.
 TYPED_TEST(MasterAllocatorTest, SlaveLost)
 {
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
+  TestAllocator<TypeParam> allocator;
+
+  EXPECT_CALL(allocator, initialize(_, _, _));
 
-  Try<PID<Master> > master = this->StartMaster(&this->allocator);
+  Try<PID<Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
@@ -646,7 +653,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveLost)
   slave::Flags flags1 = this->CreateSlaveFlags();
   flags1.resources = Some("cpus:2;mem:1024");
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave1 = this->StartSlave(&exec, flags1);
   ASSERT_SOME(slave1);
@@ -655,7 +662,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveLost)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, addFramework(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
@@ -664,8 +671,8 @@ TYPED_TEST(MasterAllocatorTest, SlaveLost)
     .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 2, 512, "*"));
 
   Future<Nothing> recoverResources;
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
-    .WillOnce(DoAll(InvokeRecoverResources(&this->allocator),
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
+    .WillOnce(DoAll(InvokeRecoverResources(&allocator),
                     FutureSatisfy(&recoverResources)));
 
   EXPECT_CALL(exec, registered(_, _, _, _));
@@ -691,12 +698,12 @@ TYPED_TEST(MasterAllocatorTest, SlaveLost)
 
   // 'recoverResources' should be called twice, once for the task
   // and once for the executor.
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .Times(2);
 
   Future<Nothing> removeSlave;
-  EXPECT_CALL(this->allocator, removeSlave(_))
-    .WillOnce(DoAll(InvokeRemoveSlave(&this->allocator),
+  EXPECT_CALL(allocator, removeSlave(_))
+    .WillOnce(DoAll(InvokeRemoveSlave(&allocator),
                     FutureSatisfy(&removeSlave)));
 
   EXPECT_CALL(exec, shutdown(_))
@@ -713,7 +720,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveLost)
   slave::Flags flags2 = this->CreateSlaveFlags();
   flags2.resources = string("cpus:3;mem:256;disk:1024;ports:[31000-32000]");
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   // Eventually after slave2 is launched, we should get
   // an offer that contains all of slave2's resources
@@ -731,13 +738,13 @@ TYPED_TEST(MasterAllocatorTest, SlaveLost)
             Resources::parse(flags2.resources.get()).get());
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
   driver.stop();
   driver.join();
 
-  EXPECT_CALL(this->allocator, removeSlave(_))
+  EXPECT_CALL(allocator, removeSlave(_))
     .Times(AtMost(1));
 
   this->Shutdown();
@@ -749,11 +756,13 @@ TYPED_TEST(MasterAllocatorTest, SlaveLost)
 // resources and offered appropriately.
 TYPED_TEST(MasterAllocatorTest, SlaveAdded)
 {
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
+  TestAllocator<TypeParam> allocator;
+
+  EXPECT_CALL(allocator, initialize(_, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
-  Try<PID<Master> > master = this->StartMaster(&this->allocator, masterFlags);
+  Try<PID<Master>> master = this->StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
@@ -761,7 +770,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveAdded)
   slave::Flags flags1 = this->CreateSlaveFlags();
   flags1.resources = Some("cpus:3;mem:1024");
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave1 = this->StartSlave(&exec, flags1);
   ASSERT_SOME(slave1);
@@ -770,7 +779,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveAdded)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, addFramework(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
@@ -789,9 +798,9 @@ TYPED_TEST(MasterAllocatorTest, SlaveAdded)
   // on slave1 from the task launch won't get reoffered
   // immediately and will get combined with slave2's
   // resources for a single offer.
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
-    .WillOnce(InvokeRecoverResourcesWithFilters(&this->allocator, 0.1))
-    .WillRepeatedly(InvokeRecoverResourcesWithFilters(&this->allocator, 0));
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
+    .WillOnce(InvokeRecoverResourcesWithFilters(&allocator, 0.1))
+    .WillRepeatedly(InvokeRecoverResourcesWithFilters(&allocator, 0));
 
   EXPECT_CALL(exec, registered(_, _, _, _));
 
@@ -810,7 +819,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveAdded)
   slave::Flags flags2 = this->CreateSlaveFlags();
   flags2.resources = Some("cpus:4;mem:2048");
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   // After slave2 launches, all of its resources are combined with the
   // resources on slave1 that the task isn't using.
@@ -827,7 +836,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveAdded)
     .Times(AtMost(1));
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
   driver.stop();
@@ -841,11 +850,13 @@ TYPED_TEST(MasterAllocatorTest, SlaveAdded)
 // resources are recovered and reoffered correctly.
 TYPED_TEST(MasterAllocatorTest, TaskFinished)
 {
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
+  TestAllocator<TypeParam> allocator;
+
+  EXPECT_CALL(allocator, initialize(_, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
-  Try<PID<Master> > master = this->StartMaster(&this->allocator, masterFlags);
+  Try<PID<Master>> master = this->StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
@@ -853,7 +864,7 @@ TYPED_TEST(MasterAllocatorTest, TaskFinished)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:3;mem:1024");
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&exec, flags);
   ASSERT_SOME(slave);
@@ -862,7 +873,7 @@ TYPED_TEST(MasterAllocatorTest, TaskFinished)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, addFramework(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
@@ -882,8 +893,8 @@ TYPED_TEST(MasterAllocatorTest, TaskFinished)
   // allocator knows about the unused resources so that it can
   // aggregate them with the resources from the finished task.
   Future<Nothing> recoverResources;
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
-    .WillRepeatedly(DoAll(InvokeRecoverResources(&this->allocator),
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
+    .WillRepeatedly(DoAll(InvokeRecoverResources(&allocator),
                           FutureSatisfy(&recoverResources)));
 
   EXPECT_CALL(exec, registered(_, _, _, _));
@@ -911,7 +922,7 @@ TYPED_TEST(MasterAllocatorTest, TaskFinished)
   status.mutable_task_id()->MergeFrom(taskInfo.task_id());
   status.set_state(TASK_FINISHED);
 
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _));
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _));
 
   // After the first task gets killed.
   Future<Nothing> resourceOffers;
@@ -926,7 +937,7 @@ TYPED_TEST(MasterAllocatorTest, TaskFinished)
     .Times(AtMost(1));
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
   driver.stop();
@@ -940,11 +951,13 @@ TYPED_TEST(MasterAllocatorTest, TaskFinished)
 // and tasks using only cpus are launched.
 TYPED_TEST(MasterAllocatorTest, CpusOnlyOfferedAndTaskLaunched)
 {
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
+  TestAllocator<TypeParam> allocator;
+
+  EXPECT_CALL(allocator, initialize(_, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
-  Try<PID<Master> > master = this->StartMaster(&this->allocator, masterFlags);
+  Try<PID<Master>> master = this->StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
@@ -953,7 +966,7 @@ TYPED_TEST(MasterAllocatorTest, CpusOnlyOfferedAndTaskLaunched)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:2;mem:0");
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&exec, flags);
   ASSERT_SOME(slave);
@@ -962,7 +975,7 @@ TYPED_TEST(MasterAllocatorTest, CpusOnlyOfferedAndTaskLaunched)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, addFramework(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
@@ -1016,11 +1029,13 @@ TYPED_TEST(MasterAllocatorTest, CpusOnlyOfferedAndTaskLaunched)
 // and tasks using only memory are launched.
 TYPED_TEST(MasterAllocatorTest, MemoryOnlyOfferedAndTaskLaunched)
 {
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
+  TestAllocator<TypeParam> allocator;
+
+  EXPECT_CALL(allocator, initialize(_, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
-  Try<PID<Master> > master = this->StartMaster(&this->allocator, masterFlags);
+  Try<PID<Master>> master = this->StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
@@ -1029,7 +1044,7 @@ TYPED_TEST(MasterAllocatorTest, MemoryOnlyOfferedAndTaskLaunched)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:0;mem:200");
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&exec, flags);
   ASSERT_SOME(slave);
@@ -1038,7 +1053,7 @@ TYPED_TEST(MasterAllocatorTest, MemoryOnlyOfferedAndTaskLaunched)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, addFramework(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
@@ -1105,14 +1120,16 @@ TYPED_TEST(MasterAllocatorTest, Whitelist)
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.whitelist = path;
 
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
+  TestAllocator<TypeParam> allocator;
+
+  EXPECT_CALL(allocator, initialize(_, _, _));
 
   Future<Nothing> updateWhitelist1;
-  EXPECT_CALL(this->allocator, updateWhitelist(Option<hashset<string>>(hosts)))
-    .WillOnce(DoAll(InvokeUpdateWhitelist(&this->allocator),
+  EXPECT_CALL(allocator, updateWhitelist(Option<hashset<string>>(hosts)))
+    .WillOnce(DoAll(InvokeUpdateWhitelist(&allocator),
                     FutureSatisfy(&updateWhitelist1)));
 
-  Try<PID<Master> > master = this->StartMaster(&this->allocator, masterFlags);
+  Try<PID<Master>> master = this->StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
 
   // Make sure the allocator has been given the initial whitelist.
@@ -1123,8 +1140,8 @@ TYPED_TEST(MasterAllocatorTest, Whitelist)
   hosts.insert("dummy-slave2");
 
   Future<Nothing> updateWhitelist2;
-  EXPECT_CALL(this->allocator, updateWhitelist(Option<hashset<string>>(hosts)))
-    .WillOnce(DoAll(InvokeUpdateWhitelist(&this->allocator),
+  EXPECT_CALL(allocator, updateWhitelist(Option<hashset<string>>(hosts)))
+    .WillOnce(DoAll(InvokeUpdateWhitelist(&allocator),
                     FutureSatisfy(&updateWhitelist2)));
 
   ASSERT_SOME(os::write(path, strings::join("\n", hosts)));
@@ -1143,11 +1160,13 @@ TYPED_TEST(MasterAllocatorTest, Whitelist)
 // master's command line flags.
 TYPED_TEST(MasterAllocatorTest, RoleTest)
 {
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
+  TestAllocator<TypeParam> allocator;
+
+  EXPECT_CALL(allocator, initialize(_, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.roles = Some("role2");
-  Try<PID<Master> > master = this->StartMaster(&this->allocator, masterFlags);
+  Try<PID<Master>> master = this->StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
 
   // Launch a framework with a role that doesn't exist to see that it
@@ -1187,7 +1206,7 @@ TYPED_TEST(MasterAllocatorTest, RoleTest)
     .WillOnce(FutureSatisfy(&registered2));
 
   Future<Nothing> addFramework;
-  EXPECT_CALL(this->allocator, addFramework(_, _, _))
+  EXPECT_CALL(allocator, addFramework(_, _, _))
     .WillOnce(FutureSatisfy(&addFramework));
 
   driver2.start();
@@ -1197,11 +1216,11 @@ TYPED_TEST(MasterAllocatorTest, RoleTest)
 
   // Shut everything down.
   Future<Nothing> deactivateFramework;
-  EXPECT_CALL(this->allocator, deactivateFramework(_))
+  EXPECT_CALL(allocator, deactivateFramework(_))
     .WillOnce(FutureSatisfy(&deactivateFramework));
 
   Future<Nothing> removeFramework;
-  EXPECT_CALL(this->allocator, removeFramework(_))
+  EXPECT_CALL(allocator, removeFramework(_))
     .WillOnce(FutureSatisfy(&removeFramework));
 
   driver2.stop();
@@ -1223,110 +1242,122 @@ TYPED_TEST(MasterAllocatorTest, RoleTest)
 // accounted for correctly.
 TYPED_TEST(MasterAllocatorTest, FrameworkReregistersFirst)
 {
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
-
-  Try<PID<Master> > master = this->StartMaster(&this->allocator);
-  ASSERT_SOME(master);
-
+  // Objects that persist after the election of a new master.
+  StandaloneMasterDetector slaveDetector;
+  StandaloneMasterDetector schedulerDetector;
+  MockScheduler sched;
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
+  TestingMesosSchedulerDriver driver(&sched, &schedulerDetector);
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  // Explicit scope is to ensure all object associated with the
+  // leading master (e.g. allocator) are destroyed once the master
+  // is shut down. Otherwise subsequent (not expected) calls into
+  // Allocator::recoverResources() are possibly rendering tests flaky.
+  {
+    TestAllocator<TypeParam> allocator;
 
-  StandaloneMasterDetector slaveDetector(master.get());
+    EXPECT_CALL(allocator, initialize(_, _, _));
 
-  slave::Flags flags = this->CreateSlaveFlags();
-  flags.resources = Some("cpus:2;mem:1024");
+    Try<PID<Master>> master = this->StartMaster(&allocator);
+    ASSERT_SOME(master);
+    slaveDetector.appoint(master.get());
+    schedulerDetector.appoint(master.get());
 
-  Try<PID<Slave> > slave = this->StartSlave(&exec, &slaveDetector, flags);
-  ASSERT_SOME(slave);
+    EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
-  EXPECT_CALL(this->allocator, addFramework(_, _, _));
+    slave::Flags flags = this->CreateSlaveFlags();
+    flags.resources = Some("cpus:2;mem:1024");
 
-  MockScheduler sched;
-  StandaloneMasterDetector schedulerDetector(master.get());
-  TestingMesosSchedulerDriver driver(&sched, &schedulerDetector);
+    Try<PID<Slave>> slave = this->StartSlave(&exec, &slaveDetector, flags);
+    ASSERT_SOME(slave);
 
-  EXPECT_CALL(sched, registered(&driver, _, _));
+    EXPECT_CALL(allocator, addFramework(_, _, _));
 
-  // The framework should be offered all of the resources on the slave
-  // since it is the only framework running.
-  EXPECT_CALL(sched, resourceOffers(&driver, OfferEq(2, 1024)))
-    .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 1, 500, "*"))
-    .WillRepeatedly(DeclineOffers());
+    EXPECT_CALL(sched, registered(&driver, _, _));
 
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _));
+    // The framework should be offered all of the resources on the
+    // slave since it is the only framework running.
+    EXPECT_CALL(sched, resourceOffers(&driver, OfferEq(2, 1024)))
+      .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 1, 500, "*"))
+      .WillRepeatedly(DeclineOffers());
 
-  EXPECT_CALL(exec, registered(_, _, _, _));
+    EXPECT_CALL(allocator, recoverResources(_, _, _, _));
 
-  EXPECT_CALL(exec, launchTask(_, _))
-    .WillOnce(SendStatusUpdateFromTask(TASK_RUNNING));
+    EXPECT_CALL(exec, registered(_, _, _, _));
 
-  Future<TaskStatus> status;
-  EXPECT_CALL(sched, statusUpdate(&driver, _))
-    .WillOnce(FutureArg<1>(&status));
+    EXPECT_CALL(exec, launchTask(_, _))
+      .WillOnce(SendStatusUpdateFromTask(TASK_RUNNING));
 
-  Future<Nothing> _statusUpdateAcknowledgement =
-    FUTURE_DISPATCH(_, &Slave::_statusUpdateAcknowledgement);
+    Future<TaskStatus> status;
+    EXPECT_CALL(sched, statusUpdate(&driver, _))
+      .WillOnce(FutureArg<1>(&status));
 
-  driver.start();
+    Future<Nothing> _statusUpdateAcknowledgement =
+      FUTURE_DISPATCH(_, &Slave::_statusUpdateAcknowledgement);
 
-  AWAIT_READY(status);
+    driver.start();
 
-  EXPECT_EQ(TASK_RUNNING, status.get().state());
+    AWAIT_READY(status);
 
-  // Make sure the slave handles status update acknowledgement so that
-  // it doesn't try to retry the update after master failover.
-  AWAIT_READY(_statusUpdateAcknowledgement);
+    EXPECT_EQ(TASK_RUNNING, status.get().state());
 
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
-    .WillRepeatedly(DoDefault());
+    // Make sure the slave handles status update acknowledgement so
+    // that it doesn't try to retry the update after master failover.
+    AWAIT_READY(_statusUpdateAcknowledgement);
 
-  this->ShutdownMasters();
+    EXPECT_CALL(allocator, recoverResources(_, _, _, _))
+      .WillRepeatedly(DoDefault());
 
-  TestAllocator<TypeParam> allocator2;
+    this->ShutdownMasters();
+  }
 
-  EXPECT_CALL(allocator2, initialize(_, _, _));
+  {
+    TestAllocator<TypeParam> allocator2;
 
-  Future<Nothing> addFramework;
-  EXPECT_CALL(allocator2, addFramework(_, _, _))
-    .WillOnce(DoAll(InvokeAddFramework(&allocator2),
-                    FutureSatisfy(&addFramework)));
+    EXPECT_CALL(allocator2, initialize(_, _, _));
 
-  EXPECT_CALL(sched, registered(&driver, _, _));
+    Future<Nothing> addFramework;
+    EXPECT_CALL(allocator2, addFramework(_, _, _))
+      .WillOnce(DoAll(InvokeAddFramework(&allocator2),
+                      FutureSatisfy(&addFramework)));
 
-  Try<PID<Master> > master2 = this->StartMaster(&allocator2);
-  ASSERT_SOME(master2);
+    EXPECT_CALL(sched, registered(&driver, _, _));
 
-  EXPECT_CALL(sched, disconnected(_));
+    Try<PID<Master>> master2 = this->StartMaster(&allocator2);
+    ASSERT_SOME(master2);
 
-  // Inform the scheduler about the new master.
-  schedulerDetector.appoint(master2.get());
+    EXPECT_CALL(sched, disconnected(_));
 
-  AWAIT_READY(addFramework);
+    // Inform the scheduler about the new master.
+    schedulerDetector.appoint(master2.get());
 
-  EXPECT_CALL(allocator2, addSlave(_, _, _, _));
+    AWAIT_READY(addFramework);
 
-  Future<vector<Offer> > resourceOffers2;
-  EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(FutureArg<1>(&resourceOffers2));
+    EXPECT_CALL(allocator2, addSlave(_, _, _, _));
 
-  // Inform the slave about the new master.
-  slaveDetector.appoint(master2.get());
+    Future<vector<Offer>> resourceOffers2;
+    EXPECT_CALL(sched, resourceOffers(&driver, _))
+      .WillOnce(FutureArg<1>(&resourceOffers2));
 
-  AWAIT_READY(resourceOffers2);
+    // Inform the slave about the new master.
+    slaveDetector.appoint(master2.get());
 
-  // Since the task is still running on the slave, the framework
-  // should only be offered the resources not being used by the task.
-  EXPECT_THAT(resourceOffers2.get(), OfferEq(1, 524));
+    AWAIT_READY(resourceOffers2);
 
-  EXPECT_CALL(exec, shutdown(_))
-    .Times(AtMost(1));
+    // Since the task is still running on the slave, the framework
+    // should only be offered the resources not being used by the
+    // task.
+    EXPECT_THAT(resourceOffers2.get(), OfferEq(1, 524));
 
-  // Shut everything down.
-  driver.stop();
-  driver.join();
+    EXPECT_CALL(exec, shutdown(_))
+      .Times(AtMost(1));
 
-  this->Shutdown();
+    // Shut everything down.
+    driver.stop();
+    driver.join();
+
+    this->Shutdown();
+  }
 }
 
 
@@ -1336,109 +1367,121 @@ TYPED_TEST(MasterAllocatorTest, FrameworkReregistersFirst)
 // accounted for correctly.
 TYPED_TEST(MasterAllocatorTest, SlaveReregistersFirst)
 {
-  EXPECT_CALL(this->allocator, initialize(_, _, _));
-
-  Try<PID<Master> > master = this->StartMaster(&this->allocator);
-  ASSERT_SOME(master);
-
+  // Objects that persist after the election of a new master.
+  StandaloneMasterDetector slaveDetector;
+  StandaloneMasterDetector schedulerDetector;
+  MockScheduler sched;
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
-  StandaloneMasterDetector slaveDetector(master.get());
+  TestingMesosSchedulerDriver driver(&sched, &schedulerDetector);
 
-  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
+  // Explicit scope is to ensure all object associated with the
+  // leading master (e.g. allocator) are destroyed once the master
+  // is shut down. Otherwise subsequent (not expected) calls into
+  // Allocator::recoverResources() are possibly rendering tests flaky.
+  {
+    TestAllocator<TypeParam> allocator;
 
-  slave::Flags flags = this->CreateSlaveFlags();
-  flags.resources = Some("cpus:2;mem:1024");
+    EXPECT_CALL(allocator, initialize(_, _, _));
 
-  Try<PID<Slave> > slave = this->StartSlave(&exec, &slaveDetector, flags);
-  ASSERT_SOME(slave);
+    Try<PID<Master>> master = this->StartMaster(&allocator);
+    ASSERT_SOME(master);
+    slaveDetector.appoint(master.get());
+    schedulerDetector.appoint(master.get());
 
-  MockScheduler sched;
-  StandaloneMasterDetector schedulerDetector(master.get());
-  TestingMesosSchedulerDriver driver(&sched, &schedulerDetector);
+    EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
-  EXPECT_CALL(this->allocator, addFramework(_, _, _));
+    slave::Flags flags = this->CreateSlaveFlags();
+    flags.resources = Some("cpus:2;mem:1024");
 
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _));
+    Try<PID<Slave>> slave = this->StartSlave(&exec, &slaveDetector, flags);
+    ASSERT_SOME(slave);
 
-  EXPECT_CALL(sched, registered(&driver, _, _));
+    EXPECT_CALL(allocator, addFramework(_, _, _));
+    EXPECT_CALL(allocator, recoverResources(_, _, _, _));
 
-  // The framework should be offered all of the resources on the slave
-  // since it is the only framework running.
-  EXPECT_CALL(sched, resourceOffers(&driver, OfferEq(2, 1024)))
-    .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 1, 500, "*"))
-    .WillRepeatedly(DeclineOffers());
+    EXPECT_CALL(sched, registered(&driver, _, _));
 
-  EXPECT_CALL(exec, registered(_, _, _, _));
+    // The framework should be offered all of the resources on the
+    // slave since it is the only framework running.
+    EXPECT_CALL(sched, resourceOffers(&driver, OfferEq(2, 1024)))
+      .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 1, 500, "*"))
+      .WillRepeatedly(DeclineOffers());
 
-  EXPECT_CALL(exec, launchTask(_, _))
-    .WillOnce(SendStatusUpdateFromTask(TASK_RUNNING));
+    EXPECT_CALL(exec, registered(_, _, _, _));
 
-  Future<TaskStatus> status;
-  EXPECT_CALL(sched, statusUpdate(&driver, _))
-    .WillOnce(FutureArg<1>(&status));
+    EXPECT_CALL(exec, launchTask(_, _))
+      .WillOnce(SendStatusUpdateFromTask(TASK_RUNNING));
 
-  Future<Nothing> _statusUpdateAcknowledgement =
-    FUTURE_DISPATCH(_, &Slave::_statusUpdateAcknowledgement);
+    Future<TaskStatus> status;
+    EXPECT_CALL(sched, statusUpdate(&driver, _))
+      .WillOnce(FutureArg<1>(&status));
 
-  driver.start();
+    Future<Nothing> _statusUpdateAcknowledgement =
+      FUTURE_DISPATCH(_, &Slave::_statusUpdateAcknowledgement);
 
-  AWAIT_READY(status);
+    driver.start();
 
-  EXPECT_EQ(TASK_RUNNING, status.get().state());
+    AWAIT_READY(status);
 
-  // Make sure the slave handles status update acknowledgement so that
-  // it doesn't try to retry the update after master failover.
-  AWAIT_READY(_statusUpdateAcknowledgement);
+    EXPECT_EQ(TASK_RUNNING, status.get().state());
 
-  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
-    .WillRepeatedly(DoDefault());
+    // Make sure the slave handles status update acknowledgement so
+    // that it doesn't try to retry the update after master failover.
+    AWAIT_READY(_statusUpdateAcknowledgement);
 
-  this->ShutdownMasters();
+    EXPECT_CALL(allocator, recoverResources(_, _, _, _))
+      .WillRepeatedly(DoDefault());
 
-  TestAllocator<TypeParam> allocator2;
+    this->ShutdownMasters();
+  }
 
-  EXPECT_CALL(allocator2, initialize(_, _, _));
+  {
+    TestAllocator<TypeParam> allocator2;
 
-  Future<Nothing> addSlave;
-  EXPECT_CALL(allocator2, addSlave(_, _, _, _))
-    .WillOnce(DoAll(InvokeAddSlave(&allocator2),
-                    FutureSatisfy(&addSlave)));
+    EXPECT_CALL(allocator2, initialize(_, _, _));
 
-  Try<PID<Master> > master2 = this->StartMaster(&allocator2);
-  ASSERT_SOME(master2);
+    Future<Nothing> addSlave;
+    EXPECT_CALL(allocator2, addSlave(_, _, _, _))
+      .WillOnce(DoAll(InvokeAddSlave(&allocator2),
+                      FutureSatisfy(&addSlave)));
 
-  // Inform the slave about the new master.
-  slaveDetector.appoint(master2.get());
+    Try<PID<Master>> master2 = this->StartMaster(&allocator2);
+    ASSERT_SOME(master2);
 
-  AWAIT_READY(addSlave);
+    // Inform the slave about the new master.
+    slaveDetector.appoint(master2.get());
 
-  EXPECT_CALL(sched, disconnected(_));
+    AWAIT_READY(addSlave);
 
-  EXPECT_CALL(sched, registered(&driver, _, _));
+    EXPECT_CALL(sched, disconnected(_));
 
-  EXPECT_CALL(allocator2, addFramework(_, _, _));
+    EXPECT_CALL(sched, registered(&driver, _, _));
 
-  Future<vector<Offer> > resourceOffers2;
-  EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(FutureArg<1>(&resourceOffers2));
+    EXPECT_CALL(allocator2, addFramework(_, _, _));
 
-  // Inform the scheduler about the new master.
-  schedulerDetector.appoint(master2.get());
+    Future<vector<Offer>> resourceOffers2;
+    EXPECT_CALL(sched, resourceOffers(&driver, _))
+      .WillOnce(FutureArg<1>(&resourceOffers2));
 
-  AWAIT_READY(resourceOffers2);
+    // Inform the scheduler about the new master.
+    schedulerDetector.appoint(master2.get());
 
-  // Since the task is still running on the slave, the framework
-  // should only be offered the resources not being used by the task.
-  EXPECT_THAT(resourceOffers2.get(), OfferEq(1, 524));
+    AWAIT_READY(resourceOffers2);
 
-  EXPECT_CALL(exec, shutdown(_))
-    .Times(AtMost(1));
+    // Since the task is still running on the slave, the framework
+    // should only be offered the resources not being used by the
+    // task.
+    EXPECT_THAT(resourceOffers2.get(), OfferEq(1, 524));
 
-  // Shut everything down.
-  driver.stop();
-  driver.join();
+    EXPECT_CALL(exec, shutdown(_))
+      .Times(AtMost(1));
 
-  this->Shutdown();
+    // Shut everything down.
+    driver.stop();
+    driver.join();
+
+    this->Shutdown();
+  }
 }
 
 } // namespace tests {

http://git-wip-us.apache.org/repos/asf/mesos/blob/ab48d54c/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index bab45ce..f23fdcb 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -53,6 +53,8 @@
 #include "master/detector.hpp"
 #include "master/master.hpp"
 
+#include "master/allocator/mesos/hierarchical.hpp"
+
 #include "slave/slave.hpp"
 
 #include "slave/containerizer/containerizer.hpp"
@@ -765,79 +767,79 @@ public:
 
 ACTION_P(InvokeInitialize, allocator)
 {
-  allocator->real.initialize(arg0, arg1, arg2);
+  allocator->real->initialize(arg0, arg1, arg2);
 }
 
 
 ACTION_P(InvokeAddFramework, allocator)
 {
-  allocator->real.addFramework(arg0, arg1, arg2);
+  allocator->real->addFramework(arg0, arg1, arg2);
 }
 
 
 ACTION_P(InvokeRemoveFramework, allocator)
 {
-  allocator->real.removeFramework(arg0);
+  allocator->real->removeFramework(arg0);
 }
 
 
 ACTION_P(InvokeActivateFramework, allocator)
 {
-  allocator->real.activateFramework(arg0);
+  allocator->real->activateFramework(arg0);
 }
 
 
 ACTION_P(InvokeDeactivateFramework, allocator)
 {
-  allocator->real.deactivateFramework(arg0);
+  allocator->real->deactivateFramework(arg0);
 }
 
 
 ACTION_P(InvokeAddSlave, allocator)
 {
-  allocator->real.addSlave(arg0, arg1, arg2, arg3);
+  allocator->real->addSlave(arg0, arg1, arg2, arg3);
 }
 
 
 ACTION_P(InvokeRemoveSlave, allocator)
 {
-  allocator->real.removeSlave(arg0);
+  allocator->real->removeSlave(arg0);
 }
 
 
 ACTION_P(InvokeActivateSlave, allocator)
 {
-  allocator->real.activateSlave(arg0);
+  allocator->real->activateSlave(arg0);
 }
 
 
 ACTION_P(InvokeDeactivateSlave, allocator)
 {
-  allocator->real.deactivateSlave(arg0);
+  allocator->real->deactivateSlave(arg0);
 }
 
 
 ACTION_P(InvokeUpdateWhitelist, allocator)
 {
-  allocator->real.updateWhitelist(arg0);
+  allocator->real->updateWhitelist(arg0);
 }
 
 
 ACTION_P(InvokeRequestResources, allocator)
 {
-  allocator->real.requestResources(arg0, arg1);
+  allocator->real->requestResources(arg0, arg1);
 }
 
 
 ACTION_P(InvokeUpdateAllocation, allocator)
 {
-  allocator->real.updateAllocation(arg0, arg1, arg2);
+  allocator->real->updateAllocation(arg0, arg1, arg2);
 }
 
 
 ACTION_P(InvokeRecoverResources, allocator)
 {
-  allocator->real.recoverResources(arg0, arg1, arg2, arg3);
+  allocator->real->recoverResources(arg0, arg1, arg2, arg3);
 }
 
 
@@ -846,21 +848,24 @@ ACTION_P2(InvokeRecoverResourcesWithFilters, allocator, timeout)
   Filters filters;
   filters.set_refuse_seconds(timeout);
 
-  allocator->real.recoverResources(arg0, arg1, arg2, filters);
+  allocator->real->recoverResources(arg0, arg1, arg2, filters);
 }
 
 
 ACTION_P(InvokeReviveOffers, allocator)
 {
-  allocator->real.reviveOffers(arg0);
+  allocator->real->reviveOffers(arg0);
 }
 
 
-template <typename T = mesos::master::allocator::Allocator>
+template <typename T = master::allocator::HierarchicalDRFAllocator>
 class TestAllocator : public mesos::master::allocator::Allocator
 {
 public:
+  // Actual allocation is done by an instance of real allocator,
+  // which is specified by the template parameter.
   TestAllocator()
+    : real(new master::allocator::HierarchicalDRFAllocator())
   {
     // We use 'ON_CALL' and 'WillByDefault' here to specify the
     // default actions (call in to the real allocator). This allows
@@ -942,7 +947,7 @@ public:
       .WillRepeatedly(DoDefault());
   }
 
-  ~TestAllocator() {}
+  virtual ~TestAllocator() {}
 
   MOCK_METHOD3(initialize, void(
       const Duration&,
@@ -1000,7 +1005,7 @@ public:
 
   MOCK_METHOD1(reviveOffers, void(const FrameworkID&));
 
-  T real;
+  process::Owned<mesos::master::allocator::Allocator> real;
 };
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/ab48d54c/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index 87f4a6a..713be8f 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -44,6 +44,8 @@
 #include "master/detector.hpp"
 #include "master/master.hpp"
 
+#include "master/allocator/mesos/hierarchical.hpp"
+
 #include "slave/gc.hpp"
 #include "slave/paths.hpp"
 #include "slave/slave.hpp"


[3/9] mesos git commit: Moved RoleInfo to mesos.master namespace.

Posted by nn...@apache.org.
Moved RoleInfo to mesos.master namespace.

Since the Allocator is a public interface now, there is no reason to
keep RoleInfo in mesos.internal. It is ok to change the package because
RoleInfo is used solely by Master and Allocator, hence there is no
external communication affected.

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


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

Branch: refs/heads/master
Commit: aabebb14c22492552c77ed471ad56a51c87d4ee1
Parents: f1fc3d7
Author: Alexander Rukletsov <ru...@gmail.com>
Authored: Tue Apr 21 12:10:27 2015 -0700
Committer: Niklas Q. Nielsen <ni...@qni.dk>
Committed: Tue Apr 21 12:10:30 2015 -0700

----------------------------------------------------------------------
 include/mesos/master/allocator.hpp          | 2 +-
 include/mesos/master/allocator.proto        | 2 +-
 src/master/allocator/mesos/allocator.hpp    | 6 +++---
 src/master/allocator/mesos/hierarchical.hpp | 9 +++++----
 src/master/master.cpp                       | 1 +
 src/master/master.hpp                       | 4 ++--
 src/tests/hierarchical_allocator_tests.cpp  | 1 +
 src/tests/mesos.hpp                         | 2 +-
 8 files changed, 15 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/aabebb14/include/mesos/master/allocator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/master/allocator.hpp b/include/mesos/master/allocator.hpp
index bb40b1c..e821bf3 100644
--- a/include/mesos/master/allocator.hpp
+++ b/include/mesos/master/allocator.hpp
@@ -59,7 +59,7 @@ public:
       const lambda::function<
           void(const FrameworkID&,
                const hashmap<SlaveID, Resources>&)>& offerCallback,
-      const hashmap<std::string, internal::RoleInfo>& roles) = 0;
+      const hashmap<std::string, RoleInfo>& roles) = 0;
 
   virtual void addFramework(
       const FrameworkID& frameworkId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/aabebb14/include/mesos/master/allocator.proto
----------------------------------------------------------------------
diff --git a/include/mesos/master/allocator.proto b/include/mesos/master/allocator.proto
index 5566719..c32b88e 100644
--- a/include/mesos/master/allocator.proto
+++ b/include/mesos/master/allocator.proto
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package mesos.internal;
+package mesos.master;
 
 // Describes a role, which are used to group frameworks for allocation
 // decisions, depending on the allocation policy being used.

http://git-wip-us.apache.org/repos/asf/mesos/blob/aabebb14/src/master/allocator/mesos/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/allocator.hpp b/src/master/allocator/mesos/allocator.hpp
index 4089ac0..5360d0a 100644
--- a/src/master/allocator/mesos/allocator.hpp
+++ b/src/master/allocator/mesos/allocator.hpp
@@ -48,7 +48,7 @@ public:
       const lambda::function<
           void(const FrameworkID&,
                const hashmap<SlaveID, Resources>&)>& offerCallback,
-      const hashmap<std::string, RoleInfo>& roles);
+      const hashmap<std::string, mesos::master::RoleInfo>& roles);
 
   void addFramework(
       const FrameworkID& frameworkId,
@@ -125,7 +125,7 @@ public:
       const lambda::function<
           void(const FrameworkID&,
                const hashmap<SlaveID, Resources>&)>& offerCallback,
-      const hashmap<std::string, RoleInfo>& roles) = 0;
+      const hashmap<std::string, mesos::master::RoleInfo>& roles) = 0;
 
   virtual void addFramework(
       const FrameworkID& frameworkId,
@@ -202,7 +202,7 @@ inline void MesosAllocator<AllocatorProcess>::initialize(
     const lambda::function<
         void(const FrameworkID&,
              const hashmap<SlaveID, Resources>&)>& offerCallback,
-    const hashmap<std::string, RoleInfo>& roles)
+    const hashmap<std::string, mesos::master::RoleInfo>& roles)
 {
   process::dispatch(
       process,

http://git-wip-us.apache.org/repos/asf/mesos/blob/aabebb14/src/master/allocator/mesos/hierarchical.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.hpp b/src/master/allocator/mesos/hierarchical.hpp
index 95b7905..09adced 100644
--- a/src/master/allocator/mesos/hierarchical.hpp
+++ b/src/master/allocator/mesos/hierarchical.hpp
@@ -76,7 +76,7 @@ public:
       const lambda::function<
           void(const FrameworkID&,
                const hashmap<SlaveID, Resources>&)>& offerCallback,
-      const hashmap<std::string, RoleInfo>& roles);
+      const hashmap<std::string, mesos::master::RoleInfo>& roles);
 
   void addFramework(
       const FrameworkID& frameworkId,
@@ -191,7 +191,7 @@ protected:
 
   hashmap<SlaveID, Slave> slaves;
 
-  hashmap<std::string, RoleInfo> roles;
+  hashmap<std::string, mesos::master::RoleInfo> roles;
 
   // Slaves to send offers for.
   Option<hashset<std::string> > whitelist;
@@ -267,7 +267,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::initialize(
     const lambda::function<
         void(const FrameworkID&,
              const hashmap<SlaveID, Resources>&)>& _offerCallback,
-    const hashmap<std::string, RoleInfo>& _roles)
+    const hashmap<std::string, mesos::master::RoleInfo>& _roles)
 {
   allocationInterval = _allocationInterval;
   offerCallback = _offerCallback;
@@ -275,7 +275,8 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::initialize(
   initialized = true;
 
   roleSorter = new RoleSorter();
-  foreachpair (const std::string& name, const RoleInfo& roleInfo, roles) {
+  foreachpair (
+      const std::string& name, const mesos::master::RoleInfo& roleInfo, roles) {
     roleSorter->add(name, roleInfo.weight());
     frameworkSorters[name] = new FrameworkSorter();
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/aabebb14/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index b726838..f3462d1 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -112,6 +112,7 @@ namespace mesos {
 namespace internal {
 namespace master {
 
+using mesos::master::RoleInfo;
 using mesos::master::allocator::Allocator;
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/aabebb14/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index d21129b..550d2c5 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -1251,7 +1251,7 @@ inline std::ostream& operator << (
 // Information about an active role.
 struct Role
 {
-  explicit Role(const RoleInfo& _info)
+  explicit Role(const mesos::master::RoleInfo& _info)
     : info(_info) {}
 
   void addFramework(Framework* framework)
@@ -1275,7 +1275,7 @@ struct Role
     return resources;
   }
 
-  RoleInfo info;
+  mesos::master::RoleInfo info;
 
   hashmap<FrameworkID, Framework*> frameworks;
 };

http://git-wip-us.apache.org/repos/asf/mesos/blob/aabebb14/src/tests/hierarchical_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hierarchical_allocator_tests.cpp b/src/tests/hierarchical_allocator_tests.cpp
index ca93bd1..e25b99f 100644
--- a/src/tests/hierarchical_allocator_tests.cpp
+++ b/src/tests/hierarchical_allocator_tests.cpp
@@ -44,6 +44,7 @@ using mesos::internal::master::MIN_CPUS;
 using mesos::internal::master::MIN_MEM;
 
 using mesos::master::allocator::Allocator;
+using mesos::master::RoleInfo;
 using mesos::internal::master::allocator::HierarchicalDRFAllocator;
 
 using process::Clock;

http://git-wip-us.apache.org/repos/asf/mesos/blob/aabebb14/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 61eaa28..8a53430 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -850,7 +850,7 @@ public:
       const lambda::function<
           void(const FrameworkID&,
                const hashmap<SlaveID, Resources>&)>&,
-      const hashmap<std::string, RoleInfo>&));
+      const hashmap<std::string, mesos::master::RoleInfo>&));
 
   MOCK_METHOD3(addFramework, void(
       const FrameworkID&,


[4/9] mesos git commit: Added support for allocator modules.

Posted by nn...@apache.org.
Added support for allocator modules.

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


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

Branch: refs/heads/master
Commit: 079fc0af9600b5407b37cbc6145b99a27a09201c
Parents: aabebb1
Author: Alexander Rukletsov <ru...@gmail.com>
Authored: Tue Apr 21 12:10:51 2015 -0700
Committer: Niklas Q. Nielsen <ni...@qni.dk>
Committed: Tue Apr 21 12:10:52 2015 -0700

----------------------------------------------------------------------
 include/mesos/module/allocator.hpp | 65 +++++++++++++++++++++++++++++++++
 src/Makefile.am                    |  1 +
 src/module/manager.cpp             |  1 +
 3 files changed, 67 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/079fc0af/include/mesos/module/allocator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/module/allocator.hpp b/include/mesos/module/allocator.hpp
new file mode 100644
index 0000000..376eb48
--- /dev/null
+++ b/include/mesos/module/allocator.hpp
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __MESOS_MODULE_ALLOCATOR_HPP__
+#define __MESOS_MODULE_ALLOCATOR_HPP__
+
+#include <mesos/mesos.hpp>
+#include <mesos/module.hpp>
+
+#include <mesos/master/allocator.hpp>
+
+namespace mesos {
+namespace modules {
+
+template <>
+inline const char* kind<mesos::master::allocator::Allocator>()
+{
+  return "Allocator";
+}
+
+
+template <>
+struct Module<mesos::master::allocator::Allocator> : ModuleBase
+{
+  Module(
+      const char* _moduleApiVersion,
+      const char* _mesosVersion,
+      const char* _authorName,
+      const char* _authorEmail,
+      const char* _description,
+      bool (*_compatible)(),
+      mesos::master::allocator::Allocator*
+        (*_create)(const Parameters& parameters))
+    : ModuleBase(
+        _moduleApiVersion,
+        _mesosVersion,
+        mesos::modules::kind<mesos::master::allocator::Allocator>(),
+        _authorName,
+        _authorEmail,
+        _description,
+        _compatible),
+      create(_create) {}
+
+  mesos::master::allocator::Allocator* (*create)(const Parameters& parameters);
+};
+
+} // namespace modules {
+} // namespace mesos {
+
+#endif // __MESOS_MODULE_ALLOCATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/079fc0af/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 5751e96..afca3c8 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -429,6 +429,7 @@ nodist_master_HEADERS = ../include/mesos/master/allocator.pb.h
 moduledir = $(pkgincludedir)/module
 
 module_HEADERS =							\
+  $(top_srcdir)/include/mesos/module/allocator.hpp			\
   $(top_srcdir)/include/mesos/module/anonymous.hpp			\
   $(top_srcdir)/include/mesos/module/authenticatee.hpp			\
   $(top_srcdir)/include/mesos/module/authenticator.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/079fc0af/src/module/manager.cpp
----------------------------------------------------------------------
diff --git a/src/module/manager.cpp b/src/module/manager.cpp
index 82a38f0..5fed1aa 100644
--- a/src/module/manager.cpp
+++ b/src/module/manager.cpp
@@ -62,6 +62,7 @@ void ModuleManager::initialize()
   // current change.  Typically that should be the version currently
   // under development.
 
+  kindToVersion["Allocator"] = MESOS_VERSION;
   kindToVersion["Anonymous"] = MESOS_VERSION;
   kindToVersion["Authenticatee"] = MESOS_VERSION;
   kindToVersion["Authenticator"] = MESOS_VERSION;


[9/9] mesos git commit: Wired up test allocator to allocator tests.

Posted by nn...@apache.org.
Wired up test allocator to allocator tests.

Same typed tests are used for built-in and modularized allocators.

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


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

Branch: refs/heads/master
Commit: da607079b473d34d06031943e69cd8e0c24073c5
Parents: 54b5bdc
Author: Alexander Rukletsov <ru...@gmail.com>
Authored: Tue Apr 21 12:12:12 2015 -0700
Committer: Niklas Q. Nielsen <ni...@qni.dk>
Committed: Tue Apr 21 12:12:12 2015 -0700

----------------------------------------------------------------------
 src/tests/master_allocator_tests.cpp | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/da607079/src/tests/master_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_allocator_tests.cpp b/src/tests/master_allocator_tests.cpp
index 8d1a866..534b248 100644
--- a/src/tests/master_allocator_tests.cpp
+++ b/src/tests/master_allocator_tests.cpp
@@ -27,6 +27,8 @@
 
 #include <mesos/master/allocator.hpp>
 
+#include <mesos/module/allocator.hpp>
+
 #include <process/clock.hpp>
 #include <process/future.hpp>
 #include <process/gmock.hpp>
@@ -43,6 +45,7 @@
 
 #include "tests/containerizer.hpp"
 #include "tests/mesos.hpp"
+#include "tests/module.hpp"
 
 using mesos::master::allocator::Allocator;
 using mesos::internal::master::allocator::HierarchicalDRFAllocator;
@@ -74,7 +77,9 @@ namespace tests {
 template <typename T>
 class MasterAllocatorTest : public MesosTest {};
 
-typedef ::testing::Types<HierarchicalDRFAllocator> AllocatorTypes;
+typedef ::testing::Types<HierarchicalDRFAllocator,
+                         tests::Module<Allocator, TestDRFAllocator>>
+  AllocatorTypes;
 
 
 // Causes all TYPED_TEST(MasterAllocatorTest, ...) to be run for