You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by bm...@apache.org on 2014/12/04 00:14:07 UTC

[1/8] mesos git commit: Removed unnecessary 'whitelisted' variable in the allocator.

Repository: mesos
Updated Branches:
  refs/heads/master f5cb86db5 -> 8a75dd226


Removed unnecessary 'whitelisted' variable in the allocator.

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


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

Branch: refs/heads/master
Commit: f575ae43ec7c6b28f5e5ba240e219814274cb8c4
Parents: f5cb86d
Author: Benjamin Mahler <be...@gmail.com>
Authored: Tue Dec 2 17:46:37 2014 -0800
Committer: Benjamin Mahler <be...@gmail.com>
Committed: Wed Dec 3 14:59:03 2014 -0800

----------------------------------------------------------------------
 src/master/hierarchical_allocator_process.hpp | 16 +++-------------
 1 file changed, 3 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f575ae43/src/master/hierarchical_allocator_process.hpp
----------------------------------------------------------------------
diff --git a/src/master/hierarchical_allocator_process.hpp b/src/master/hierarchical_allocator_process.hpp
index fbaa23f..4f284ce 100644
--- a/src/master/hierarchical_allocator_process.hpp
+++ b/src/master/hierarchical_allocator_process.hpp
@@ -64,7 +64,6 @@ struct Slave
   explicit Slave(const SlaveInfo& _info)
     : available(_info.resources()),
       activated(true),
-      whitelisted(false),
       checkpoint(_info.checkpoint()),
       info(_info) {}
 
@@ -79,10 +78,6 @@ struct Slave
   // deactivated slaves until they are reactivated.
   bool activated;
 
-  // Indicates if the resources on this slave should be offered to
-  // frameworks.
-  bool whitelisted;
-
   bool checkpoint;
 private:
   SlaveInfo info;
@@ -191,7 +186,7 @@ protected:
   void expire(const FrameworkID& frameworkId, Filter* filter);
 
   // Checks whether the slave is whitelisted.
-  bool isWhitelisted(const SlaveID& slave);
+  bool isWhitelisted(const SlaveID& slaveId);
 
   // Returns true if there is a filter for this framework
   // on this slave.
@@ -418,7 +413,6 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveAdded(
   CHECK(!slaves.contains(slaveId));
 
   slaves[slaveId] = Slave(slaveInfo);
-  slaves[slaveId].whitelisted = isWhitelisted(slaveId);
 
   roleSorter->add(slaveInfo.resources());
 
@@ -511,10 +505,6 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::updateWhitelist(
     if (whitelist.get().empty()) {
       LOG(WARNING) << "Whitelist is empty, no offers will be made!";
     }
-
-    foreachkey (const SlaveID& slaveId, slaves) {
-      slaves[slaveId].whitelisted = isWhitelisted(slaveId);
-    }
   } else {
     LOG(INFO) << "Advertising offers for all slaves";
   }
@@ -711,8 +701,8 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::allocate(
 
   hashmap<FrameworkID, hashmap<SlaveID, Resources> > offerable;
   foreach (const SlaveID& slaveId, slaveIds) {
-    // If the slave is not activated or whitelisted, ignore it.
-    if (!slaves[slaveId].activated || !slaves[slaveId].whitelisted) {
+    // Don't send offers for non-whitelisted and deactivated slaves.
+    if (!isWhitelisted(slaveId) || !slaves[slaveId].activated) {
       continue;
     }
 


[4/8] mesos git commit: Updated allocator.hpp to idiomatically use the "Process wrapper" pattern.

Posted by bm...@apache.org.
Updated allocator.hpp to idiomatically use the "Process wrapper" pattern.

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


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

Branch: refs/heads/master
Commit: 219fa4bc2ae96d6e90cffdbacde782db6987271d
Parents: 31317b3
Author: Benjamin Mahler <be...@gmail.com>
Authored: Tue Dec 2 18:17:28 2014 -0800
Committer: Benjamin Mahler <be...@gmail.com>
Committed: Wed Dec 3 15:02:04 2014 -0800

----------------------------------------------------------------------
 src/master/allocator.hpp | 146 +++++++++++++++++++++---------------------
 1 file changed, 73 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/219fa4bc/src/master/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator.hpp b/src/master/allocator.hpp
index 04eb2a3..b8c5286 100644
--- a/src/master/allocator.hpp
+++ b/src/master/allocator.hpp
@@ -44,154 +44,154 @@ class Master; // Forward declaration.
 
 namespace allocator {
 
+class AllocatorProcess; // Forward declaration.
+
 // 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. 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.
-// NOTE: New Allocators should implement this interface.
-class AllocatorProcess : public process::Process<AllocatorProcess>
+//
+// NOTE: DO NOT subclass this class when implementing a new allocator.
+// Implement AllocatorProcess (above) instead!
+class Allocator
 {
 public:
-  AllocatorProcess() {}
+  // The AllocatorProcess object passed to the constructor is spawned
+  // and terminated by the allocator. But it is the responsibility
+  // of the caller to de-allocate the object, if necessary.
+  explicit Allocator(AllocatorProcess* _process);
 
-  virtual ~AllocatorProcess() {}
+  virtual ~Allocator();
 
-  virtual void initialize(
+  void initialize(
       const Flags& flags,
       const process::PID<Master>& master,
-      const hashmap<std::string, RoleInfo>& roles) = 0;
+      const hashmap<std::string, RoleInfo>& roles);
 
-  virtual void frameworkAdded(
+  void frameworkAdded(
       const FrameworkID& frameworkId,
       const FrameworkInfo& frameworkInfo,
-      const Resources& used) = 0;
+      const Resources& used);
 
-  virtual void frameworkRemoved(
-      const FrameworkID& frameworkId) = 0;
+  void frameworkRemoved(
+      const FrameworkID& frameworkId);
 
-  virtual void frameworkActivated(
+  void frameworkActivated(
       const FrameworkID& frameworkId,
-      const FrameworkInfo& frameworkInfo) = 0;
+      const FrameworkInfo& frameworkInfo);
 
-  virtual void frameworkDeactivated(
-      const FrameworkID& frameworkId) = 0;
+  void frameworkDeactivated(
+      const FrameworkID& frameworkId);
 
   // Note that the 'total' resources are passed explicitly because it
   // includes resources that are dynamically "persisted" 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 slaveAdded(
+  void slaveAdded(
       const SlaveID& slaveId,
       const SlaveInfo& slaveInfo,
       const Resources& total,
-      const hashmap<FrameworkID, Resources>& used) = 0;
+      const hashmap<FrameworkID, Resources>& used);
 
-  virtual void slaveRemoved(
-      const SlaveID& slaveId) = 0;
+  void slaveRemoved(
+      const SlaveID& slaveId);
 
   // No longer offers resources for the deactivated slave.
-  virtual void slaveDeactivated(
-      const SlaveID& slaveId) = 0;
+  void slaveDeactivated(
+      const SlaveID& slaveId);
 
   // Offers resources for the activated slave.
-  virtual void slaveActivated(
-      const SlaveID& slaveId) = 0;
+  void slaveActivated(
+      const SlaveID& slaveId);
 
-  virtual void updateWhitelist(
-      const Option<hashset<std::string> >& whitelist) = 0;
+  void updateWhitelist(
+      const Option<hashset<std::string> >& whitelist);
 
-  virtual void resourcesRequested(
+  void resourcesRequested(
       const FrameworkID& frameworkId,
-      const std::vector<Request>& requests) = 0;
+      const std::vector<Request>& requests);
 
   // Whenever resources are "recovered" in the cluster (e.g., a task
   // finishes, an offer is removed because a framework has failed or
-  // is failing over), or a framework refuses them, the master
-  // invokes this callback.
-  virtual void resourcesRecovered(
+  // is failing over), or a framework refuses them.
+  void resourcesRecovered(
       const FrameworkID& frameworkId,
       const SlaveID& slaveId,
       const Resources& resources,
-      const Option<Filters>& filters) = 0;
+      const Option<Filters>& filters);
 
   // Whenever a framework that has filtered resources wants to revive
   // offers for those resources the master invokes this callback.
-  virtual void offersRevived(
-      const FrameworkID& frameworkId) = 0;
+  void offersRevived(
+      const FrameworkID& frameworkId);
+
+private:
+  Allocator(const Allocator&); // Not copyable.
+  Allocator& operator=(const Allocator&); // Not assignable.
+
+  AllocatorProcess* process;
 };
 
 
-// This is a wrapper around the AllocatorProcess interface.
-// NOTE: DO NOT subclass this class when implementing a new allocator.
-// Implement AllocatorProcess (above) instead!
-class Allocator
+class AllocatorProcess : public process::Process<AllocatorProcess>
 {
 public:
-  // The AllocatorProcess object passed to the constructor is
-  // spawned and terminated by the allocator. But it is the responsibility
-  // of the caller to de-allocate the object, if necessary.
-  explicit Allocator(AllocatorProcess* _process);
+  AllocatorProcess() {}
 
-  virtual ~Allocator();
+  virtual ~AllocatorProcess() {}
 
-  void initialize(
+  virtual void initialize(
       const Flags& flags,
       const process::PID<Master>& master,
-      const hashmap<std::string, RoleInfo>& roles);
+      const hashmap<std::string, RoleInfo>& roles) = 0;
 
-  void frameworkAdded(
+  virtual void frameworkAdded(
       const FrameworkID& frameworkId,
       const FrameworkInfo& frameworkInfo,
-      const Resources& used);
+      const Resources& used) = 0;
 
-  void frameworkRemoved(
-      const FrameworkID& frameworkId);
+  virtual void frameworkRemoved(
+      const FrameworkID& frameworkId) = 0;
 
-  void frameworkActivated(
+  virtual void frameworkActivated(
       const FrameworkID& frameworkId,
-      const FrameworkInfo& frameworkInfo);
+      const FrameworkInfo& frameworkInfo) = 0;
 
-  void frameworkDeactivated(
-      const FrameworkID& frameworkId);
+  virtual void frameworkDeactivated(
+      const FrameworkID& frameworkId) = 0;
 
-  void slaveAdded(
+  virtual void slaveAdded(
       const SlaveID& slaveId,
       const SlaveInfo& slaveInfo,
       const Resources& total,
-      const hashmap<FrameworkID, Resources>& used);
+      const hashmap<FrameworkID, Resources>& used) = 0;
 
-  void slaveRemoved(
-      const SlaveID& slaveId);
+  virtual void slaveRemoved(
+      const SlaveID& slaveId) = 0;
 
-  void slaveDeactivated(
-      const SlaveID& slaveId);
+  virtual void slaveDeactivated(
+      const SlaveID& slaveId) = 0;
 
-  void slaveActivated(
-      const SlaveID& slaveId);
+  virtual void slaveActivated(
+      const SlaveID& slaveId) = 0;
 
-  void updateWhitelist(
-      const Option<hashset<std::string> >& whitelist);
+  virtual void updateWhitelist(
+      const Option<hashset<std::string> >& whitelist) = 0;
 
-  void resourcesRequested(
+  virtual void resourcesRequested(
       const FrameworkID& frameworkId,
-      const std::vector<Request>& requests);
+      const std::vector<Request>& requests) = 0;
 
-  void resourcesRecovered(
+  virtual void resourcesRecovered(
       const FrameworkID& frameworkId,
       const SlaveID& slaveId,
       const Resources& resources,
-      const Option<Filters>& filters);
-
-  void offersRevived(
-      const FrameworkID& frameworkId);
-
-private:
-  Allocator(const Allocator&); // Not copyable.
-  Allocator& operator=(const Allocator&); // Not assignable.
+      const Option<Filters>& filters) = 0;
 
-  AllocatorProcess* process;
+  virtual void offersRevived(
+      const FrameworkID& frameworkId) = 0;
 };
 
 


[3/8] mesos git commit: Updated Allocator::slaveAdded to take the total resources explicitly.

Posted by bm...@apache.org.
Updated Allocator::slaveAdded to take the total resources explicitly.

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


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

Branch: refs/heads/master
Commit: 31317b35ba68b423565e4eeb1c3e92f69251402e
Parents: 0cb4c9c
Author: Benjamin Mahler <be...@gmail.com>
Authored: Tue Dec 2 18:04:32 2014 -0800
Committer: Benjamin Mahler <be...@gmail.com>
Committed: Wed Dec 3 14:59:28 2014 -0800

----------------------------------------------------------------------
 src/master/allocator.hpp                      | 12 +++-
 src/master/hierarchical_allocator_process.hpp |  4 +-
 src/master/master.cpp                         |  9 ++-
 src/tests/allocator_tests.cpp                 | 56 ++++++++--------
 src/tests/mesos.hpp                           | 76 +++++++++++++++-------
 src/tests/slave_recovery_tests.cpp            |  2 +-
 6 files changed, 99 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/31317b35/src/master/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator.hpp b/src/master/allocator.hpp
index 02d20d0..04eb2a3 100644
--- a/src/master/allocator.hpp
+++ b/src/master/allocator.hpp
@@ -57,9 +57,6 @@ public:
 
   virtual ~AllocatorProcess() {}
 
-  // Explicitely use 'initialize' since we're overloading below.
-  using process::ProcessBase::initialize;
-
   virtual void initialize(
       const Flags& flags,
       const process::PID<Master>& master,
@@ -80,9 +77,15 @@ public:
   virtual void frameworkDeactivated(
       const FrameworkID& frameworkId) = 0;
 
+  // Note that the 'total' resources are passed explicitly because it
+  // includes resources that are dynamically "persisted" 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 slaveAdded(
       const SlaveID& slaveId,
       const SlaveInfo& slaveInfo,
+      const Resources& total,
       const hashmap<FrameworkID, Resources>& used) = 0;
 
   virtual void slaveRemoved(
@@ -156,6 +159,7 @@ public:
   void slaveAdded(
       const SlaveID& slaveId,
       const SlaveInfo& slaveInfo,
+      const Resources& total,
       const hashmap<FrameworkID, Resources>& used);
 
   void slaveRemoved(
@@ -268,6 +272,7 @@ inline void Allocator::frameworkDeactivated(
 inline void Allocator::slaveAdded(
     const SlaveID& slaveId,
     const SlaveInfo& slaveInfo,
+    const Resources& total,
     const hashmap<FrameworkID, Resources>& used)
 {
   process::dispatch(
@@ -275,6 +280,7 @@ inline void Allocator::slaveAdded(
       &AllocatorProcess::slaveAdded,
       slaveId,
       slaveInfo,
+      total,
       used);
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/31317b35/src/master/hierarchical_allocator_process.hpp
----------------------------------------------------------------------
diff --git a/src/master/hierarchical_allocator_process.hpp b/src/master/hierarchical_allocator_process.hpp
index c71739b..f4577bd 100644
--- a/src/master/hierarchical_allocator_process.hpp
+++ b/src/master/hierarchical_allocator_process.hpp
@@ -92,6 +92,7 @@ public:
   void slaveAdded(
       const SlaveID& slaveId,
       const SlaveInfo& slaveInfo,
+      const Resources& total,
       const hashmap<FrameworkID, Resources>& used);
 
   void slaveRemoved(
@@ -394,13 +395,12 @@ void
 HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveAdded(
     const SlaveID& slaveId,
     const SlaveInfo& slaveInfo,
+    const Resources& total,
     const hashmap<FrameworkID, Resources>& used)
 {
   CHECK(initialized);
   CHECK(!slaves.contains(slaveId));
 
-  const Resources& total = slaveInfo.resources();
-
   roleSorter->add(total);
 
   foreachpair (const FrameworkID& frameworkId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/31317b35/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 99b5a20..9d92f1a 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -4408,7 +4408,14 @@ void Master::addSlave(
     }
   }
 
-  allocator->slaveAdded(slave->id, slave->info, slave->usedResources);
+  // TODO(bmahler): This will need to include resources that
+  // are "persisted" on the slave (e.g. persistent volumes,
+  // dynamic reservations, etc).
+  allocator->slaveAdded(
+      slave->id,
+      slave->info,
+      slave->info.resources(),
+      slave->usedResources);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/31317b35/src/tests/allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/allocator_tests.cpp b/src/tests/allocator_tests.cpp
index 1fcbb4a..a7ffa39 100644
--- a/src/tests/allocator_tests.cpp
+++ b/src/tests/allocator_tests.cpp
@@ -92,7 +92,7 @@ TEST_F(DRFAllocatorTest, DRFAllocatorProcess)
   slave::Flags flags1 = CreateSlaveFlags();
   flags1.resources = Some("cpus:2;mem:1024;disk:0");
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
 
   Try<PID<Slave> > slave1 = StartSlave(flags1);
   ASSERT_SOME(slave1);
@@ -150,7 +150,7 @@ TEST_F(DRFAllocatorTest, DRFAllocatorProcess)
   slave::Flags flags2 = CreateSlaveFlags();
   flags2.resources = Some("cpus:1;mem:512;disk:0");
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
 
   Future<vector<Offer> > offers2;
   EXPECT_CALL(sched2, resourceOffers(_, _))
@@ -177,7 +177,7 @@ TEST_F(DRFAllocatorTest, DRFAllocatorProcess)
   slave::Flags flags3 = CreateSlaveFlags();
   flags3.resources = Some("cpus:3;mem:2048;disk:0");
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
 
   Future<vector<Offer> > offers3;
   EXPECT_CALL(sched2, resourceOffers(_, _))
@@ -225,7 +225,7 @@ TEST_F(DRFAllocatorTest, DRFAllocatorProcess)
   slave::Flags flags4 = CreateSlaveFlags();
   flags4.resources = Some("cpus:4;mem:4096;disk:0");
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
 
   Future<vector<Offer> > offers4;
   EXPECT_CALL(sched3, resourceOffers(_, _))
@@ -275,7 +275,7 @@ TEST_F(DRFAllocatorTest, DRFAllocatorProcess)
   slave::Flags flags5 = CreateSlaveFlags();
   flags5.resources = Some("cpus:1;mem:512;disk:0");
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
 
   Future<vector<Offer> > offers5;
   EXPECT_CALL(sched2, resourceOffers(_, _))
@@ -350,7 +350,7 @@ TEST_F(DRFAllocatorTest, PerSlaveAllocation)
   flags1.resources = Some("cpus:2;mem:1024;disk:0");
 
   Future<Nothing> slaveAdded1;
-  EXPECT_CALL(allocator, slaveAdded(_, _, _))
+  EXPECT_CALL(allocator, slaveAdded(_, _, _, _))
     .WillOnce(DoAll(InvokeSlaveAdded(&allocator),
                     FutureSatisfy(&slaveAdded1)));
 
@@ -364,7 +364,7 @@ TEST_F(DRFAllocatorTest, PerSlaveAllocation)
   flags2.resources = Some("cpus:2;mem:1024;disk:0");
 
   Future<Nothing> slaveAdded2;
-  EXPECT_CALL(allocator, slaveAdded(_, _, _))
+  EXPECT_CALL(allocator, slaveAdded(_, _, _, _))
     .WillOnce(DoAll(InvokeSlaveAdded(&allocator),
                     FutureSatisfy(&slaveAdded2)));
 
@@ -549,7 +549,7 @@ TEST_F(DRFAllocatorTest, SameShareAllocations)
     .WillRepeatedly(DoDefault());
 
   // Start the slave.
-  EXPECT_CALL(allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
 
   Try<PID<Slave> > slave = StartSlave();
   ASSERT_SOME(slave);
@@ -594,7 +594,7 @@ TEST_F(ReservationAllocatorTest, ReservedResources)
   ASSERT_SOME(master);
 
   Future<Nothing> slaveAdded;
-  EXPECT_CALL(allocator, slaveAdded(_, _, _))
+  EXPECT_CALL(allocator, slaveAdded(_, _, _, _))
     .WillOnce(DoDefault())
     .WillOnce(DoDefault())
     .WillOnce(DoDefault())
@@ -700,7 +700,7 @@ TEST_F(ReservationAllocatorTest, ReservedResources)
   flags5.default_role = "role1";
   flags5.resources = Some("cpus:1;mem:512;disk:0");
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
 
   Future<Nothing> resourceOffers4;
   EXPECT_CALL(sched1, resourceOffers(_, OfferEq(1, 512)))
@@ -751,7 +751,7 @@ TEST_F(ReservationAllocatorTest, ResourcesReturned)
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _))
+  EXPECT_CALL(allocator, slaveAdded(_, _, _, _))
     .Times(2);
 
   Future<Nothing> slaveAdded1 = FUTURE_DISPATCH(
@@ -926,7 +926,7 @@ TYPED_TEST(AllocatorTest, MockAllocator)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:2;mem:1024;disk:0");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(flags);
   ASSERT_SOME(slave);
@@ -984,7 +984,7 @@ TYPED_TEST(AllocatorTest, ResourcesUnused)
   slave::Flags flags1 = this->CreateSlaveFlags();
   flags1.resources = Some("cpus:2;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   Try<PID<Slave> > slave1 = this->StartSlave(&exec, flags1);
   ASSERT_SOME(slave1);
@@ -1095,7 +1095,7 @@ TYPED_TEST(AllocatorTest, OutOfOrderDispatch)
   slave::Flags flags1 = this->CreateSlaveFlags();
   flags1.resources = Some("cpus:2;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   Try<PID<Slave> > slave1 = this->StartSlave(flags1);
   ASSERT_SOME(slave1);
@@ -1228,7 +1228,7 @@ TYPED_TEST(AllocatorTest, SchedulerFailover)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:3;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&exec, flags);
   ASSERT_SOME(slave);
@@ -1372,7 +1372,7 @@ TYPED_TEST(AllocatorTest, FrameworkExited)
 
   flags.resources = Some("cpus:3;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&containerizer, flags);
   ASSERT_SOME(slave);
@@ -1513,7 +1513,7 @@ TYPED_TEST(AllocatorTest, SlaveLost)
   slave::Flags flags1 = this->CreateSlaveFlags();
   flags1.resources = Some("cpus:2;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   Try<PID<Slave> > slave1 = this->StartSlave(&exec, flags1);
   ASSERT_SOME(slave1);
@@ -1578,7 +1578,7 @@ TYPED_TEST(AllocatorTest, SlaveLost)
   slave::Flags flags2 = this->CreateSlaveFlags();
   flags2.resources = string("cpus:3;mem:256;disk:1024;ports:[31000-32000]");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   // Eventually after slave2 is launched, we should get
   // an offer that contains all of slave2's resources
@@ -1632,7 +1632,7 @@ TYPED_TEST(AllocatorTest, SlaveAdded)
   slave::Flags flags1 = this->CreateSlaveFlags();
   flags1.resources = Some("cpus:3;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   Try<PID<Slave> > slave1 = this->StartSlave(&exec, flags1);
   ASSERT_SOME(slave1);
@@ -1681,7 +1681,7 @@ TYPED_TEST(AllocatorTest, SlaveAdded)
   slave::Flags flags2 = this->CreateSlaveFlags();
   flags2.resources = Some("cpus:4;mem:2048");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   // After slave2 launches, all of its resources are combined with the
   // resources on slave1 that the task isn't using.
@@ -1733,7 +1733,7 @@ TYPED_TEST(AllocatorTest, TaskFinished)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:3;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&exec, flags);
   ASSERT_SOME(slave);
@@ -1842,7 +1842,7 @@ TYPED_TEST(AllocatorTest, CpusOnlyOfferedAndTaskLaunched)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:2;mem:0");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&exec, flags);
   ASSERT_SOME(slave);
@@ -1930,7 +1930,7 @@ TYPED_TEST(AllocatorTest, MemoryOnlyOfferedAndTaskLaunched)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:0;mem:200");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&exec, flags);
   ASSERT_SOME(slave);
@@ -2024,7 +2024,7 @@ TYPED_TEST(AllocatorTest, WhitelistSlave)
   Try<PID<Master> > master = this->StartMaster(&this->allocator, masterFlags);
   ASSERT_SOME(master);
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:2;mem:1024");
@@ -2194,7 +2194,7 @@ TYPED_TEST(AllocatorTest, FrameworkReregistersFirst)
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   StandaloneMasterDetector slaveDetector(master.get());
 
@@ -2269,7 +2269,7 @@ TYPED_TEST(AllocatorTest, FrameworkReregistersFirst)
 
   AWAIT_READY(frameworkAdded);
 
-  EXPECT_CALL(allocator2, slaveAdded(_, _, _));
+  EXPECT_CALL(allocator2, slaveAdded(_, _, _, _));
 
   Future<vector<Offer> > resourceOffers2;
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -2321,7 +2321,7 @@ TYPED_TEST(AllocatorTest, SlaveReregistersFirst)
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
   StandaloneMasterDetector slaveDetector(master.get());
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
 
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:2;mem:1024");
@@ -2378,7 +2378,7 @@ TYPED_TEST(AllocatorTest, SlaveReregistersFirst)
   EXPECT_CALL(allocator2, initialize(_, _, _));
 
   Future<Nothing> slaveAdded;
-  EXPECT_CALL(allocator2, slaveAdded(_, _, _))
+  EXPECT_CALL(allocator2, slaveAdded(_, _, _, _))
     .WillOnce(DoAll(InvokeSlaveAdded(&allocator2),
                     FutureSatisfy(&slaveAdded)));
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/31317b35/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index f132c6c..94fd753 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -670,7 +670,7 @@ public:
     ON_CALL(*this, frameworkDeactivated(_))
       .WillByDefault(InvokeFrameworkDeactivated(this));
 
-    ON_CALL(*this, slaveAdded(_, _, _))
+    ON_CALL(*this, slaveAdded(_, _, _, _))
       .WillByDefault(InvokeSlaveAdded(this));
 
     ON_CALL(*this, slaveRemoved(_))
@@ -701,29 +701,54 @@ public:
     process::wait(real);
   }
 
-  MOCK_METHOD3(initialize, void(const master::Flags&,
-                                const process::PID<master::Master>&,
-                                const hashmap<std::string, RoleInfo>&));
-  MOCK_METHOD3(frameworkAdded, void(const FrameworkID&,
-                                    const FrameworkInfo&,
-                                    const Resources&));
-  MOCK_METHOD1(frameworkRemoved, void(const FrameworkID&));
-  MOCK_METHOD2(frameworkActivated, void(const FrameworkID&,
-                                        const FrameworkInfo&));
-  MOCK_METHOD1(frameworkDeactivated, void(const FrameworkID&));
-  MOCK_METHOD3(slaveAdded, void(const SlaveID&,
-                                const SlaveInfo&,
-                                const hashmap<FrameworkID, Resources>&));
-  MOCK_METHOD1(slaveRemoved, void(const SlaveID&));
-  MOCK_METHOD1(slaveDeactivated, void(const SlaveID&));
-  MOCK_METHOD1(slaveActivated, void(const SlaveID&));
-  MOCK_METHOD1(updateWhitelist, void(const Option<hashset<std::string> >&));
-  MOCK_METHOD2(resourcesRequested, void(const FrameworkID&,
-                                        const std::vector<Request>&));
-  MOCK_METHOD4(resourcesRecovered, void(const FrameworkID&,
-                                        const SlaveID&,
-                                        const Resources&,
-                                        const Option<Filters>& filters));
+  MOCK_METHOD3(initialize, void(
+      const master::Flags&,
+      const process::PID<master::Master>&,
+      const hashmap<std::string, RoleInfo>&));
+
+  MOCK_METHOD3(frameworkAdded, void(
+      const FrameworkID&,
+      const FrameworkInfo&,
+      const Resources&));
+
+  MOCK_METHOD1(frameworkRemoved, void(
+      const FrameworkID&));
+
+  MOCK_METHOD2(frameworkActivated, void(
+      const FrameworkID&,
+      const FrameworkInfo&));
+
+  MOCK_METHOD1(frameworkDeactivated, void(
+      const FrameworkID&));
+
+  MOCK_METHOD4(slaveAdded, void(
+      const SlaveID&,
+      const SlaveInfo&,
+      const Resources&,
+      const hashmap<FrameworkID, Resources>&));
+
+  MOCK_METHOD1(slaveRemoved, void(
+      const SlaveID&));
+
+  MOCK_METHOD1(slaveDeactivated, void(
+      const SlaveID&));
+
+  MOCK_METHOD1(slaveActivated, void(
+      const SlaveID&));
+
+  MOCK_METHOD1(updateWhitelist, void(
+      const Option<hashset<std::string> >&));
+
+  MOCK_METHOD2(resourcesRequested, void(
+      const FrameworkID&,
+      const std::vector<Request>&));
+
+  MOCK_METHOD4(resourcesRecovered, void(
+      const FrameworkID&,
+      const SlaveID&,
+      const Resources&,
+      const Option<Filters>& filters));
+
   MOCK_METHOD1(offersRevived, void(const FrameworkID&));
 
   T real;
@@ -796,7 +821,8 @@ ACTION_P(InvokeSlaveAdded, allocator)
       &master::allocator::AllocatorProcess::slaveAdded,
       arg0,
       arg1,
-      arg2);
+      arg2,
+      arg3);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/31317b35/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index 2b6c76a..6cf5fb8 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -2207,7 +2207,7 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileTasksMissingFromSlave)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _));
+  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
 
   Try<TypeParam*> containerizer1 = TypeParam::create(flags, true);
   ASSERT_SOME(containerizer1);


[6/8] mesos git commit: Cleaned up the allocator method naming.

Posted by bm...@apache.org.
Cleaned up the allocator method naming.

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


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

Branch: refs/heads/master
Commit: 9b84b52becf898ddab740a9a933aea52562e7740
Parents: 219fa4b
Author: Benjamin Mahler <be...@gmail.com>
Authored: Tue Dec 2 18:44:19 2014 -0800
Committer: Benjamin Mahler <be...@gmail.com>
Committed: Wed Dec 3 15:02:05 2014 -0800

----------------------------------------------------------------------
 src/master/allocator.hpp                      |  97 +++--
 src/master/hierarchical_allocator_process.hpp |  64 +--
 src/master/master.cpp                         |  62 +--
 src/tests/allocator_tests.cpp                 | 468 ++++++++++-----------
 src/tests/fault_tolerance_tests.cpp           |  12 +-
 src/tests/master_authorization_tests.cpp      |  48 +--
 src/tests/master_tests.cpp                    |  30 +-
 src/tests/mesos.hpp                           |  80 ++--
 src/tests/partition_tests.cpp                 |   6 +-
 src/tests/rate_limiting_tests.cpp             |  18 +-
 src/tests/resource_offers_tests.cpp           |   8 +-
 src/tests/slave_recovery_tests.cpp            |  18 +-
 12 files changed, 455 insertions(+), 456 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9b84b52b/src/master/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator.hpp b/src/master/allocator.hpp
index b8c5286..8fb660d 100644
--- a/src/master/allocator.hpp
+++ b/src/master/allocator.hpp
@@ -69,19 +69,20 @@ public:
       const process::PID<Master>& master,
       const hashmap<std::string, RoleInfo>& roles);
 
-  void frameworkAdded(
+  void addFramework(
       const FrameworkID& frameworkId,
       const FrameworkInfo& frameworkInfo,
       const Resources& used);
 
-  void frameworkRemoved(
+  void removeFramework(
       const FrameworkID& frameworkId);
 
-  void frameworkActivated(
+  // Offers are sent only to activated frameworks.
+  void activateFramework(
       const FrameworkID& frameworkId,
       const FrameworkInfo& frameworkInfo);
 
-  void frameworkDeactivated(
+  void deactivateFramework(
       const FrameworkID& frameworkId);
 
   // Note that the 'total' resources are passed explicitly because it
@@ -89,34 +90,32 @@ public:
   // (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.
-  void slaveAdded(
+  void addSlave(
       const SlaveID& slaveId,
       const SlaveInfo& slaveInfo,
       const Resources& total,
       const hashmap<FrameworkID, Resources>& used);
 
-  void slaveRemoved(
+  void removeSlave(
       const SlaveID& slaveId);
 
-  // No longer offers resources for the deactivated slave.
-  void slaveDeactivated(
+  // Offers are sent only for activated slaves.
+  void activateSlave(
       const SlaveID& slaveId);
 
-  // Offers resources for the activated slave.
-  void slaveActivated(
+  void deactivateSlave(
       const SlaveID& slaveId);
 
   void updateWhitelist(
       const Option<hashset<std::string> >& whitelist);
 
-  void resourcesRequested(
+  void requestResources(
       const FrameworkID& frameworkId,
       const std::vector<Request>& requests);
 
-  // Whenever resources are "recovered" in the cluster (e.g., a task
-  // finishes, an offer is removed because a framework has failed or
-  // is failing over), or a framework refuses them.
-  void resourcesRecovered(
+  // Informs the allocator to recover resources that are considered
+  // used by the framework.
+  void recoverResources(
       const FrameworkID& frameworkId,
       const SlaveID& slaveId,
       const Resources& resources,
@@ -124,7 +123,7 @@ public:
 
   // Whenever a framework that has filtered resources wants to revive
   // offers for those resources the master invokes this callback.
-  void offersRevived(
+  void reviveOffers(
       const FrameworkID& frameworkId);
 
 private:
@@ -147,50 +146,50 @@ public:
       const process::PID<Master>& master,
       const hashmap<std::string, RoleInfo>& roles) = 0;
 
-  virtual void frameworkAdded(
+  virtual void addFramework(
       const FrameworkID& frameworkId,
       const FrameworkInfo& frameworkInfo,
       const Resources& used) = 0;
 
-  virtual void frameworkRemoved(
+  virtual void removeFramework(
       const FrameworkID& frameworkId) = 0;
 
-  virtual void frameworkActivated(
+  virtual void activateFramework(
       const FrameworkID& frameworkId,
       const FrameworkInfo& frameworkInfo) = 0;
 
-  virtual void frameworkDeactivated(
+  virtual void deactivateFramework(
       const FrameworkID& frameworkId) = 0;
 
-  virtual void slaveAdded(
+  virtual void addSlave(
       const SlaveID& slaveId,
       const SlaveInfo& slaveInfo,
       const Resources& total,
       const hashmap<FrameworkID, Resources>& used) = 0;
 
-  virtual void slaveRemoved(
+  virtual void removeSlave(
       const SlaveID& slaveId) = 0;
 
-  virtual void slaveDeactivated(
+  virtual void activateSlave(
       const SlaveID& slaveId) = 0;
 
-  virtual void slaveActivated(
+  virtual void deactivateSlave(
       const SlaveID& slaveId) = 0;
 
   virtual void updateWhitelist(
       const Option<hashset<std::string> >& whitelist) = 0;
 
-  virtual void resourcesRequested(
+  virtual void requestResources(
       const FrameworkID& frameworkId,
       const std::vector<Request>& requests) = 0;
 
-  virtual void resourcesRecovered(
+  virtual void recoverResources(
       const FrameworkID& frameworkId,
       const SlaveID& slaveId,
       const Resources& resources,
       const Option<Filters>& filters) = 0;
 
-  virtual void offersRevived(
+  virtual void reviveOffers(
       const FrameworkID& frameworkId) = 0;
 };
 
@@ -223,53 +222,53 @@ inline void Allocator::initialize(
 }
 
 
-inline void Allocator::frameworkAdded(
+inline void Allocator::addFramework(
     const FrameworkID& frameworkId,
     const FrameworkInfo& frameworkInfo,
     const Resources& used)
 {
   process::dispatch(
       process,
-      &AllocatorProcess::frameworkAdded,
+      &AllocatorProcess::addFramework,
       frameworkId,
       frameworkInfo,
       used);
 }
 
 
-inline void Allocator::frameworkRemoved(
+inline void Allocator::removeFramework(
     const FrameworkID& frameworkId)
 {
   process::dispatch(
       process,
-      &AllocatorProcess::frameworkRemoved,
+      &AllocatorProcess::removeFramework,
       frameworkId);
 }
 
 
-inline void Allocator::frameworkActivated(
+inline void Allocator::activateFramework(
     const FrameworkID& frameworkId,
     const FrameworkInfo& frameworkInfo)
 {
   process::dispatch(
       process,
-      &AllocatorProcess::frameworkActivated,
+      &AllocatorProcess::activateFramework,
       frameworkId,
       frameworkInfo);
 }
 
 
-inline void Allocator::frameworkDeactivated(
+inline void Allocator::deactivateFramework(
     const FrameworkID& frameworkId)
 {
   process::dispatch(
       process,
-      &AllocatorProcess::frameworkDeactivated,
+      &AllocatorProcess::deactivateFramework,
       frameworkId);
 }
 
 
-inline void Allocator::slaveAdded(
+inline void Allocator::addSlave(
     const SlaveID& slaveId,
     const SlaveInfo& slaveInfo,
     const Resources& total,
@@ -277,7 +276,7 @@ inline void Allocator::slaveAdded(
 {
   process::dispatch(
       process,
-      &AllocatorProcess::slaveAdded,
+      &AllocatorProcess::addSlave,
       slaveId,
       slaveInfo,
       total,
@@ -285,29 +284,29 @@ inline void Allocator::slaveAdded(
 }
 
 
-inline void Allocator::slaveRemoved(const SlaveID& slaveId)
+inline void Allocator::removeSlave(const SlaveID& slaveId)
 {
   process::dispatch(
       process,
-      &AllocatorProcess::slaveRemoved,
+      &AllocatorProcess::removeSlave,
       slaveId);
 }
 
 
-inline void Allocator::slaveDeactivated(const SlaveID& slaveId)
+inline void Allocator::activateSlave(const SlaveID& slaveId)
 {
   process::dispatch(
       process,
-      &AllocatorProcess::slaveDeactivated,
+      &AllocatorProcess::activateSlave,
       slaveId);
 }
 
 
-inline void Allocator::slaveActivated(const SlaveID& slaveId)
+inline void Allocator::deactivateSlave(const SlaveID& slaveId)
 {
   process::dispatch(
       process,
-      &AllocatorProcess::slaveActivated,
+      &AllocatorProcess::deactivateSlave,
       slaveId);
 }
 
@@ -322,19 +321,19 @@ inline void Allocator::updateWhitelist(
 }
 
 
-inline void Allocator::resourcesRequested(
+inline void Allocator::requestResources(
     const FrameworkID& frameworkId,
     const std::vector<Request>& requests)
 {
   process::dispatch(
       process,
-      &AllocatorProcess::resourcesRequested,
+      &AllocatorProcess::requestResources,
       frameworkId,
       requests);
 }
 
 
-inline void Allocator::resourcesRecovered(
+inline void Allocator::recoverResources(
     const FrameworkID& frameworkId,
     const SlaveID& slaveId,
     const Resources& resources,
@@ -342,7 +341,7 @@ inline void Allocator::resourcesRecovered(
 {
   process::dispatch(
       process,
-      &AllocatorProcess::resourcesRecovered,
+      &AllocatorProcess::recoverResources,
       frameworkId,
       slaveId,
       resources,
@@ -350,12 +349,12 @@ inline void Allocator::resourcesRecovered(
 }
 
 
-inline void Allocator::offersRevived(
+inline void Allocator::reviveOffers(
     const FrameworkID& frameworkId)
 {
   process::dispatch(
       process,
-      &AllocatorProcess::offersRevived,
+      &AllocatorProcess::reviveOffers,
       frameworkId);
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/9b84b52b/src/master/hierarchical_allocator_process.hpp
----------------------------------------------------------------------
diff --git a/src/master/hierarchical_allocator_process.hpp b/src/master/hierarchical_allocator_process.hpp
index f4577bd..f0baa05 100644
--- a/src/master/hierarchical_allocator_process.hpp
+++ b/src/master/hierarchical_allocator_process.hpp
@@ -74,50 +74,50 @@ public:
       const process::PID<Master>& _master,
       const hashmap<std::string, RoleInfo>& _roles);
 
-  void frameworkAdded(
+  void addFramework(
       const FrameworkID& frameworkId,
       const FrameworkInfo& frameworkInfo,
       const Resources& used);
 
-  void frameworkRemoved(
+  void removeFramework(
       const FrameworkID& frameworkId);
 
-  void frameworkActivated(
+  void activateFramework(
       const FrameworkID& frameworkId,
       const FrameworkInfo& frameworkInfo);
 
-  void frameworkDeactivated(
+  void deactivateFramework(
       const FrameworkID& frameworkId);
 
-  void slaveAdded(
+  void addSlave(
       const SlaveID& slaveId,
       const SlaveInfo& slaveInfo,
       const Resources& total,
       const hashmap<FrameworkID, Resources>& used);
 
-  void slaveRemoved(
+  void removeSlave(
       const SlaveID& slaveId);
 
-  void slaveDeactivated(
+  void deactivateSlave(
       const SlaveID& slaveId);
 
-  void slaveActivated(
+  void activateSlave(
       const SlaveID& slaveId);
 
   void updateWhitelist(
       const Option<hashset<std::string> >& whitelist);
 
-  void resourcesRequested(
+  void requestResources(
       const FrameworkID& frameworkId,
       const std::vector<Request>& requests);
 
-  void resourcesRecovered(
+  void recoverResources(
       const FrameworkID& frameworkId,
       const SlaveID& slaveId,
       const Resources& resources,
       const Option<Filters>& filters);
 
-  void offersRevived(
+  void reviveOffers(
       const FrameworkID& frameworkId);
 
 protected:
@@ -272,7 +272,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::initialize(
 
 template <class RoleSorter, class FrameworkSorter>
 void
-HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkAdded(
+HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::addFramework(
     const FrameworkID& frameworkId,
     const FrameworkInfo& frameworkInfo,
     const Resources& used)
@@ -303,7 +303,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkAdded(
 
 template <class RoleSorter, class FrameworkSorter>
 void
-HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkRemoved(
+HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::removeFramework(
     const FrameworkID& frameworkId)
 {
   CHECK(initialized);
@@ -322,7 +322,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkRemoved(
 
   // Do not delete the filters contained in this
   // framework's 'filters' hashset yet, see comments in
-  // HierarchicalAllocatorProcess::offersRevived and
+  // HierarchicalAllocatorProcess::reviveOffers and
   // HierarchicalAllocatorProcess::expire.
   frameworks.erase(frameworkId);
 
@@ -332,7 +332,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkRemoved(
 
 template <class RoleSorter, class FrameworkSorter>
 void
-HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkActivated(
+HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::activateFramework(
     const FrameworkID& frameworkId,
     const FrameworkInfo& frameworkInfo)
 {
@@ -349,7 +349,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkActivated(
 
 template <class RoleSorter, class FrameworkSorter>
 void
-HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkDeactivated(
+HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::deactivateFramework(
     const FrameworkID& frameworkId)
 {
   CHECK(initialized);
@@ -367,7 +367,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkDeactivated(
 
   // Do not delete the filters contained in this
   // framework's 'filters' hashset yet, see comments in
-  // HierarchicalAllocatorProcess::offersRevived and
+  // HierarchicalAllocatorProcess::reviveOffers and
   // HierarchicalAllocatorProcess::expire.
   frameworks[frameworkId].filters.clear();
 
@@ -392,7 +392,7 @@ Resources sum(const Iterable& resources)
 
 template <class RoleSorter, class FrameworkSorter>
 void
-HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveAdded(
+HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::addSlave(
     const SlaveID& slaveId,
     const SlaveInfo& slaveInfo,
     const Resources& total,
@@ -432,7 +432,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveAdded(
 
 template <class RoleSorter, class FrameworkSorter>
 void
-HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveRemoved(
+HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::removeSlave(
     const SlaveID& slaveId)
 {
   CHECK(initialized);
@@ -453,29 +453,29 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveRemoved(
 
 template <class RoleSorter, class FrameworkSorter>
 void
-HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveDeactivated(
+HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::activateSlave(
     const SlaveID& slaveId)
 {
   CHECK(initialized);
   CHECK(slaves.contains(slaveId));
 
-  slaves[slaveId].activated = false;
+  slaves[slaveId].activated = true;
 
-  LOG(INFO) << "Slave " << slaveId << " deactivated";
+  LOG(INFO)<< "Slave " << slaveId << " reactivated";
 }
 
 
 template <class RoleSorter, class FrameworkSorter>
 void
-HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveActivated(
+HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::deactivateSlave(
     const SlaveID& slaveId)
 {
   CHECK(initialized);
   CHECK(slaves.contains(slaveId));
 
-  slaves[slaveId].activated = true;
+  slaves[slaveId].activated = false;
 
-  LOG(INFO)<< "Slave " << slaveId << " reactivated";
+  LOG(INFO) << "Slave " << slaveId << " deactivated";
 }
 
 
@@ -502,7 +502,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::updateWhitelist(
 
 template <class RoleSorter, class FrameworkSorter>
 void
-HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::resourcesRequested(
+HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::requestResources(
     const FrameworkID& frameworkId,
     const std::vector<Request>& requests)
 {
@@ -514,7 +514,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::resourcesRequested(
 
 template <class RoleSorter, class FrameworkSorter>
 void
-HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::resourcesRecovered(
+HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::recoverResources(
     const FrameworkID& frameworkId,
     const SlaveID& slaveId,
     const Resources& resources,
@@ -528,8 +528,8 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::resourcesRecovered(
 
   // Updated resources allocated to framework (if framework still
   // exists, which it might not in the event that we dispatched
-  // Master::offer before we received AllocatorProcess::frameworkRemoved
-  // or AllocatorProcess::frameworkDeactivated, in which case we will
+  // Master::offer before we received AllocatorProcess::removeFramework
+  // or AllocatorProcess::deactivateFramework, in which case we will
   // have already recovered all of its resources).
   if (frameworks.contains(frameworkId)) {
     const std::string& role = frameworks[frameworkId].role;
@@ -545,7 +545,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::resourcesRecovered(
 
   // Update resources allocatable on slave (if slave still exists,
   // which it might not in the event that we dispatched Master::offer
-  // before we received Allocator::slaveRemoved).
+  // before we received Allocator::removeSlave).
   if (slaves.contains(slaveId)) {
     slaves[slaveId].available += resources;
 
@@ -604,7 +604,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::resourcesRecovered(
 
 template <class RoleSorter, class FrameworkSorter>
 void
-HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::offersRevived(
+HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::reviveOffers(
     const FrameworkID& frameworkId)
 {
   CHECK(initialized);
@@ -754,7 +754,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::expire(
 {
   // The filter might have already been removed (e.g., if the
   // framework no longer exists or in
-  // HierarchicalAllocatorProcess::offersRevived) but not yet deleted (to
+  // HierarchicalAllocatorProcess::reviveOffers) but not yet deleted (to
   // keep the address from getting reused possibly causing premature
   // expiration).
   if (frameworks.contains(frameworkId) &&

http://git-wip-us.apache.org/repos/asf/mesos/blob/9b84b52b/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 9d92f1a..3d85e7b 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -1565,7 +1565,7 @@ void Master::_reregisterFramework(
       // replied to the offers but the driver might have dropped
       // those messages since it wasn't connected to the master.
       foreach (Offer* offer, utils::copy(framework->offers)) {
-        allocator->resourcesRecovered(
+        allocator->recoverResources(
             offer->framework_id(),
             offer->slave_id(),
             offer->resources(),
@@ -1580,7 +1580,7 @@ void Master::_reregisterFramework(
       // the allocator has the correct view of the framework's share.
       if (!framework->active) {
         framework->active = true;
-        allocator->frameworkActivated(framework->id, framework->info);
+        allocator->activateFramework(framework->id, framework->info);
       }
 
       FrameworkReregisteredMessage message;
@@ -1718,11 +1718,11 @@ void Master::deactivate(Framework* framework)
   framework->active = false;
 
   // Tell the allocator to stop allocating resources to this framework.
-  allocator->frameworkDeactivated(framework->id);
+  allocator->deactivateFramework(framework->id);
 
   // Remove the framework's offers.
   foreach (Offer* offer, utils::copy(framework->offers)) {
-    allocator->resourcesRecovered(
+    allocator->recoverResources(
         offer->framework_id(), offer->slave_id(), offer->resources(), None());
     removeOffer(offer, true); // Rescind.
   }
@@ -1756,11 +1756,11 @@ void Master::deactivate(Slave* slave)
 
   slave->active = false;
 
-  allocator->slaveDeactivated(slave->id);
+  allocator->deactivateSlave(slave->id);
 
   // Remove and rescind offers.
   foreach (Offer* offer, utils::copy(slave->offers)) {
-    allocator->resourcesRecovered(
+    allocator->recoverResources(
         offer->framework_id(), slave->id, offer->resources(), None());
 
     removeOffer(offer, true); // Rescind!
@@ -1792,7 +1792,7 @@ void Master::resourceRequest(
   }
 
   LOG(INFO) << "Requesting resources for framework " << *framework;
-  allocator->resourcesRequested(frameworkId, requests);
+  allocator->requestResources(frameworkId, requests);
 }
 
 
@@ -2259,7 +2259,7 @@ void Master::launchTasks(
         used += offer->resources();
 
         if (error.isSome()) {
-          allocator->resourcesRecovered(
+          allocator->recoverResources(
               offer->framework_id(),
               offer->slave_id(),
               offer->resources(),
@@ -2486,7 +2486,7 @@ void Master::_launchTasks(
       << " because the framework cannot be found";
 
     // Tell the allocator about the recovered resources.
-    allocator->resourcesRecovered(frameworkId, slaveId, totalResources, None());
+    allocator->recoverResources(frameworkId, slaveId, totalResources, None());
 
     return;
   }
@@ -2512,7 +2512,7 @@ void Master::_launchTasks(
     }
 
     // Tell the allocator about the recovered resources.
-    allocator->resourcesRecovered(frameworkId, slaveId, totalResources, None());
+    allocator->recoverResources(frameworkId, slaveId, totalResources, None());
 
     return;
   }
@@ -2614,7 +2614,7 @@ void Master::_launchTasks(
 
   if (!unusedResources.empty()) {
     // Tell the allocator about the unused (e.g., refused) resources.
-    allocator->resourcesRecovered(
+    allocator->recoverResources(
         frameworkId, slaveId, unusedResources, filters);
   }
 }
@@ -2641,7 +2641,7 @@ void Master::reviveOffers(const UPID& from, const FrameworkID& frameworkId)
   }
 
   LOG(INFO) << "Reviving offers for framework " << *framework;
-  allocator->offersRevived(framework->id);
+  allocator->reviveOffers(framework->id);
 }
 
 
@@ -3117,7 +3117,7 @@ void Master::reregisterSlave(
       slave->connected = true;
       dispatch(slave->observer, &SlaveObserver::reconnect);
       slave->active = true;
-      allocator->slaveActivated(slave->id);
+      allocator->activateSlave(slave->id);
     }
 
     CHECK(slave->active)
@@ -3638,7 +3638,7 @@ void Master::offer(const FrameworkID& frameworkId,
                  << " has terminated or is inactive";
 
     foreachpair (const SlaveID& slaveId, const Resources& offered, resources) {
-      allocator->resourcesRecovered(frameworkId, slaveId, offered, None());
+      allocator->recoverResources(frameworkId, slaveId, offered, None());
     }
     return;
   }
@@ -3653,7 +3653,7 @@ void Master::offer(const FrameworkID& frameworkId,
         << "Master returning resources offered to framework " << *framework
         << " because slave " << slaveId << " is not valid";
 
-      allocator->resourcesRecovered(frameworkId, slaveId, offered, None());
+      allocator->recoverResources(frameworkId, slaveId, offered, None());
       continue;
     }
 
@@ -3670,7 +3670,7 @@ void Master::offer(const FrameworkID& frameworkId,
         << "Master returning resources offered because slave " << *slave
         << " is " << (slave->connected ? "deactivated" : "disconnected");
 
-      allocator->resourcesRecovered(frameworkId, slaveId, offered, None());
+      allocator->recoverResources(frameworkId, slaveId, offered, None());
       continue;
     }
 
@@ -3692,7 +3692,7 @@ void Master::offer(const FrameworkID& frameworkId,
                      << "executors";
         // Pass a default filter to avoid getting this same offer immediately
         // from the allocator.
-        allocator->resourcesRecovered(frameworkId, slaveId, offered, Filters());
+        allocator->recoverResources(frameworkId, slaveId, offered, Filters());
         continue;
       }
     }
@@ -4080,7 +4080,7 @@ void Master::addFramework(Framework* framework)
   // There should be no offered resources yet!
   CHECK_EQ(Resources(), framework->offeredResources);
 
-  allocator->frameworkAdded(
+  allocator->addFramework(
       framework->id,
       framework->info,
       framework->usedResources);
@@ -4150,7 +4150,7 @@ void Master::failoverFramework(Framework* framework, const UPID& newPid)
   // registered message so that the allocator can immediately re-offer
   // these resources to this framework if it wants.
   foreach (Offer* offer, utils::copy(framework->offers)) {
-    allocator->resourcesRecovered(
+    allocator->recoverResources(
         offer->framework_id(), offer->slave_id(), offer->resources(), None());
     removeOffer(offer);
   }
@@ -4162,7 +4162,7 @@ void Master::failoverFramework(Framework* framework, const UPID& newPid)
   // the allocator has the correct view of the framework's share.
   if (!framework->active) {
     framework->active = true;
-    allocator->frameworkActivated(framework->id, framework->info);
+    allocator->activateFramework(framework->id, framework->info);
   }
 
   // 'Failover' the framework's metrics. i.e., change the lookup key
@@ -4184,7 +4184,7 @@ void Master::removeFramework(Framework* framework)
     // Tell the allocator to stop allocating resources to this framework.
     // TODO(vinod): Consider setting  framework->active to false here
     // or just calling 'deactivate(Framework*)'.
-    allocator->frameworkDeactivated(framework->id);
+    allocator->deactivateFramework(framework->id);
   }
 
   // Tell slaves to shutdown the framework.
@@ -4233,7 +4233,7 @@ void Master::removeFramework(Framework* framework)
 
   // Remove the framework's offers (if they weren't removed before).
   foreach (Offer* offer, utils::copy(framework->offers)) {
-    allocator->resourcesRecovered(
+    allocator->recoverResources(
         offer->framework_id(),
         offer->slave_id(),
         offer->resources(),
@@ -4288,7 +4288,7 @@ void Master::removeFramework(Framework* framework)
 
   // Remove the framework.
   frameworks.registered.erase(framework->id);
-  allocator->frameworkRemoved(framework->id);
+  allocator->removeFramework(framework->id);
 }
 
 
@@ -4411,7 +4411,7 @@ void Master::addSlave(
   // TODO(bmahler): This will need to include resources that
   // are "persisted" on the slave (e.g. persistent volumes,
   // dynamic reservations, etc).
-  allocator->slaveAdded(
+  allocator->addSlave(
       slave->id,
       slave->info,
       slave->info.resources(),
@@ -4427,7 +4427,7 @@ void Master::removeSlave(Slave* slave)
 
   // We do this first, to make sure any of the resources recovered
   // below (e.g., removeTask()) are ignored by the allocator.
-  allocator->slaveRemoved(slave->id);
+  allocator->removeSlave(slave->id);
 
   // Transition the tasks to lost and remove them, BUT do not send
   // updates. Rather, build up the updates so that we can send them
@@ -4462,9 +4462,9 @@ void Master::removeSlave(Slave* slave)
   }
 
   foreach (Offer* offer, utils::copy(slave->offers)) {
-    // TODO(vinod): We don't need to call 'Allocator::resourcesRecovered'
+    // TODO(vinod): We don't need to call 'Allocator::recoverResources'
     // once MESOS-621 is fixed.
-    allocator->resourcesRecovered(
+    allocator->recoverResources(
         offer->framework_id(), slave->id, offer->resources(), None());
 
     // Remove and rescind offers.
@@ -4618,7 +4618,7 @@ void Master::updateTask(Task* task, const StatusUpdate& update)
 
   // Once the task becomes terminal, we recover the resources.
   if (terminated) {
-    allocator->resourcesRecovered(
+    allocator->recoverResources(
         task->framework_id(),
         task->slave_id(),
         task->resources(),
@@ -4660,7 +4660,7 @@ void Master::removeTask(Task* task)
 
     // If the task is not terminal, then the resources have
     // not yet been recovered.
-    allocator->resourcesRecovered(
+    allocator->recoverResources(
         task->framework_id(),
         task->slave_id(),
         task->resources(),
@@ -4699,7 +4699,7 @@ void Master::removeExecutor(
             << "' with resources " << executor.resources()
             << " of framework " << frameworkId << " on slave " << *slave;
 
-  allocator->resourcesRecovered(
+  allocator->recoverResources(
     frameworkId, slave->id, executor.resources(), None());
 
   Framework* framework = getFramework(frameworkId);
@@ -4715,7 +4715,7 @@ void Master::offerTimeout(const OfferID& offerId)
 {
   Offer* offer = getOffer(offerId);
   if (offer != NULL) {
-    allocator->resourcesRecovered(
+    allocator->recoverResources(
         offer->framework_id(), offer->slave_id(), offer->resources(), None());
     removeOffer(offer, true);
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/9b84b52b/src/tests/allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/allocator_tests.cpp b/src/tests/allocator_tests.cpp
index a7ffa39..b2924e3 100644
--- a/src/tests/allocator_tests.cpp
+++ b/src/tests/allocator_tests.cpp
@@ -92,7 +92,7 @@ TEST_F(DRFAllocatorTest, DRFAllocatorProcess)
   slave::Flags flags1 = CreateSlaveFlags();
   flags1.resources = Some("cpus:2;mem:1024;disk:0");
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave1 = StartSlave(flags1);
   ASSERT_SOME(slave1);
@@ -108,7 +108,7 @@ TEST_F(DRFAllocatorTest, DRFAllocatorProcess)
   MesosSchedulerDriver driver1(
       &sched1, frameworkInfo1, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched1, registered(_, _, _));
 
@@ -136,21 +136,21 @@ TEST_F(DRFAllocatorTest, DRFAllocatorProcess)
   MesosSchedulerDriver driver2(
       &sched2, frameworkInfo2, master.get(), DEFAULT_CREDENTIAL);
 
-  Future<Nothing> frameworkAdded2;
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _))
+  Future<Nothing> addFramework2;
+  EXPECT_CALL(allocator, addFramework(_, _, _))
     .WillOnce(DoAll(InvokeFrameworkAdded(&allocator),
-      FutureSatisfy(&frameworkAdded2)));
+      FutureSatisfy(&addFramework2)));
 
   EXPECT_CALL(sched2, registered(_, _, _));
 
   driver2.start();
 
-  AWAIT_READY(frameworkAdded2);
+  AWAIT_READY(addFramework2);
 
   slave::Flags flags2 = CreateSlaveFlags();
   flags2.resources = Some("cpus:1;mem:512;disk:0");
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Future<vector<Offer> > offers2;
   EXPECT_CALL(sched2, resourceOffers(_, _))
@@ -177,7 +177,7 @@ TEST_F(DRFAllocatorTest, DRFAllocatorProcess)
   slave::Flags flags3 = CreateSlaveFlags();
   flags3.resources = Some("cpus:3;mem:2048;disk:0");
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Future<vector<Offer> > offers3;
   EXPECT_CALL(sched2, resourceOffers(_, _))
@@ -211,21 +211,21 @@ TEST_F(DRFAllocatorTest, DRFAllocatorProcess)
   MesosSchedulerDriver driver3(
       &sched3, frameworkInfo3, master.get(), DEFAULT_CREDENTIAL);
 
-  Future<Nothing> frameworkAdded3;
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _))
+  Future<Nothing> addFramework3;
+  EXPECT_CALL(allocator, addFramework(_, _, _))
     .WillOnce(DoAll(InvokeFrameworkAdded(&allocator),
-      FutureSatisfy(&frameworkAdded3)));
+      FutureSatisfy(&addFramework3)));
 
   EXPECT_CALL(sched3, registered(_, _, _));
 
   driver3.start();
 
-  AWAIT_READY(frameworkAdded3);
+  AWAIT_READY(addFramework3);
 
   slave::Flags flags4 = CreateSlaveFlags();
   flags4.resources = Some("cpus:4;mem:4096;disk:0");
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Future<vector<Offer> > offers4;
   EXPECT_CALL(sched3, resourceOffers(_, _))
@@ -261,21 +261,21 @@ TEST_F(DRFAllocatorTest, DRFAllocatorProcess)
   MesosSchedulerDriver driver4(
       &sched4, frameworkInfo4, master.get(), DEFAULT_CREDENTIAL);
 
-  Future<Nothing> frameworkAdded4;
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _))
+  Future<Nothing> addFramework4;
+  EXPECT_CALL(allocator, addFramework(_, _, _))
     .WillOnce(DoAll(InvokeFrameworkAdded(&allocator),
-                    FutureSatisfy(&frameworkAdded4)));
+                    FutureSatisfy(&addFramework4)));
 
   EXPECT_CALL(sched4, registered(_, _, _));
 
   driver4.start();
 
-  AWAIT_READY(frameworkAdded4);
+  AWAIT_READY(addFramework4);
 
   slave::Flags flags5 = CreateSlaveFlags();
   flags5.resources = Some("cpus:1;mem:512;disk:0");
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Future<vector<Offer> > offers5;
   EXPECT_CALL(sched2, resourceOffers(_, _))
@@ -297,13 +297,13 @@ TEST_F(DRFAllocatorTest, DRFAllocatorProcess)
   EXPECT_THAT(offers5.get(), OfferEq(1, 512));
 
   // Shut everything down.
-  EXPECT_CALL(allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(allocator, frameworkDeactivated(_))
+  EXPECT_CALL(allocator, deactivateFramework(_))
     .Times(AtMost(4));
 
-  EXPECT_CALL(allocator, frameworkRemoved(_))
+  EXPECT_CALL(allocator, removeFramework(_))
     .Times(AtMost(4));
 
   driver1.stop();
@@ -318,7 +318,7 @@ TEST_F(DRFAllocatorTest, DRFAllocatorProcess)
   driver4.stop();
   driver4.join();
 
-  EXPECT_CALL(allocator, slaveRemoved(_))
+  EXPECT_CALL(allocator, removeSlave(_))
     .Times(AtMost(5));
 
   Shutdown();
@@ -349,29 +349,29 @@ TEST_F(DRFAllocatorTest, PerSlaveAllocation)
   slave::Flags flags1 = CreateSlaveFlags();
   flags1.resources = Some("cpus:2;mem:1024;disk:0");
 
-  Future<Nothing> slaveAdded1;
-  EXPECT_CALL(allocator, slaveAdded(_, _, _, _))
+  Future<Nothing> addSlave1;
+  EXPECT_CALL(allocator, addSlave(_, _, _, _))
     .WillOnce(DoAll(InvokeSlaveAdded(&allocator),
-                    FutureSatisfy(&slaveAdded1)));
+                    FutureSatisfy(&addSlave1)));
 
   Try<PID<Slave> > slave1 = StartSlave(flags1);
   ASSERT_SOME(slave1);
 
-  AWAIT_READY(slaveAdded1);
+  AWAIT_READY(addSlave1);
 
   // Start slave 2.
   slave::Flags flags2 = CreateSlaveFlags();
   flags2.resources = Some("cpus:2;mem:1024;disk:0");
 
-  Future<Nothing> slaveAdded2;
-  EXPECT_CALL(allocator, slaveAdded(_, _, _, _))
+  Future<Nothing> addSlave2;
+  EXPECT_CALL(allocator, addSlave(_, _, _, _))
     .WillOnce(DoAll(InvokeSlaveAdded(&allocator),
-                    FutureSatisfy(&slaveAdded2)));
+                    FutureSatisfy(&addSlave2)));
 
   Try<PID<Slave> > slave2 = StartSlave(flags2);
   ASSERT_SOME(slave2);
 
-  AWAIT_READY(slaveAdded2);
+  AWAIT_READY(addSlave2);
 
   // Start framework 1.
   FrameworkInfo frameworkInfo1; // Bug in gcc 4.1.*, must assign on next line.
@@ -384,17 +384,17 @@ TEST_F(DRFAllocatorTest, PerSlaveAllocation)
   MesosSchedulerDriver driver1(
       &sched1, frameworkInfo1, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched1, registered(_, _, _));
 
-  Future<Nothing> resourcesRecovered1;
-  Future<Nothing> resourcesRecovered2;
-  EXPECT_CALL(allocator, resourcesRecovered(_, _, _, _))
+  Future<Nothing> recoverResources1;
+  Future<Nothing> recoverResources2;
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillOnce(DoAll(InvokeResourcesRecovered(&allocator),
-                    FutureSatisfy(&resourcesRecovered1)))
+                    FutureSatisfy(&recoverResources1)))
     .WillOnce(DoAll(InvokeResourcesRecovered(&allocator),
-                    FutureSatisfy(&resourcesRecovered2)));
+                    FutureSatisfy(&recoverResources2)));
 
   // Decline the offers immediately so that resources for both slaves
   // are eligible for allocation to this and other frameworks.
@@ -408,10 +408,10 @@ TEST_F(DRFAllocatorTest, PerSlaveAllocation)
   // Wait until the resources are returned to the allocator.
   // NOTE: No allocations will be made after this point until a new
   // framework registers because
-  // 1) 'resourcesRecovered' does not trigger an allocation and
+  // 1) 'recoverResources' does not trigger an allocation and
   // 2) 'flags.allocation_interval' is set to a very high value.
-  AWAIT_READY(resourcesRecovered1);
-  AWAIT_READY(resourcesRecovered2);
+  AWAIT_READY(recoverResources1);
+  AWAIT_READY(recoverResources2);
 
   // Start framework 2.
   FrameworkInfo frameworkInfo2; // Bug in gcc 4.1.*, must assign on next line.
@@ -424,7 +424,7 @@ TEST_F(DRFAllocatorTest, PerSlaveAllocation)
   MesosSchedulerDriver driver2(
       &sched2, frameworkInfo2, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched2, registered(_, _, _));
 
@@ -448,13 +448,13 @@ TEST_F(DRFAllocatorTest, PerSlaveAllocation)
   EXPECT_THAT(offers2.get(), OfferEq(2, 1024));
 
   // Shut everything down.
-  EXPECT_CALL(allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(allocator, frameworkDeactivated(_))
+  EXPECT_CALL(allocator, deactivateFramework(_))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(allocator, frameworkRemoved(_))
+  EXPECT_CALL(allocator, removeFramework(_))
     .WillRepeatedly(DoDefault());
 
   driver1.stop();
@@ -463,7 +463,7 @@ TEST_F(DRFAllocatorTest, PerSlaveAllocation)
   driver2.stop();
   driver2.join();
 
-  EXPECT_CALL(allocator, slaveRemoved(_))
+  EXPECT_CALL(allocator, removeSlave(_))
     .WillRepeatedly(DoDefault());
 
   Shutdown();
@@ -497,7 +497,7 @@ TEST_F(DRFAllocatorTest, SameShareAllocations)
   MesosSchedulerDriver driver1(
       &sched1, frameworkInfo1, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   Future<Nothing> registered1;
   EXPECT_CALL(sched1, registered(_, _, _))
@@ -519,7 +519,7 @@ TEST_F(DRFAllocatorTest, SameShareAllocations)
   // We need to retire this expectation on the first match because
   // framework1 can match this expectation first in which case
   // framework2 should be able to match the expectation above.
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _))
+  EXPECT_CALL(allocator, addFramework(_, _, _))
     .RetiresOnSaturation();
 
   Future<Nothing> registered2;
@@ -545,11 +545,11 @@ TEST_F(DRFAllocatorTest, SameShareAllocations)
     .WillRepeatedly(DoAll(Increment(&allocations2),
                           DeclineOffers(filters)));
 
-  EXPECT_CALL(allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
   // Start the slave.
-  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave = StartSlave();
   ASSERT_SOME(slave);
@@ -593,13 +593,13 @@ TEST_F(ReservationAllocatorTest, ReservedResources)
 
   ASSERT_SOME(master);
 
-  Future<Nothing> slaveAdded;
-  EXPECT_CALL(allocator, slaveAdded(_, _, _, _))
+  Future<Nothing> addSlave;
+  EXPECT_CALL(allocator, addSlave(_, _, _, _))
     .WillOnce(DoDefault())
     .WillOnce(DoDefault())
     .WillOnce(DoDefault())
     .WillOnce(DoAll(InvokeSlaveAdded(&allocator),
-                    FutureSatisfy(&slaveAdded)));
+                    FutureSatisfy(&addSlave)));
 
   slave::Flags flags1 = CreateSlaveFlags();
   flags1.default_role = "role1";
@@ -627,7 +627,7 @@ TEST_F(ReservationAllocatorTest, ReservedResources)
   Try<PID<Slave> > slave4 = StartSlave(flags4);
   ASSERT_SOME(slave4);
 
-  AWAIT_READY(slaveAdded);
+  AWAIT_READY(addSlave);
 
   FrameworkInfo frameworkInfo1; // Bug in gcc 4.1.*, must assign on next line.
   frameworkInfo1 = DEFAULT_FRAMEWORK_INFO;
@@ -638,7 +638,7 @@ TEST_F(ReservationAllocatorTest, ReservedResources)
   MesosSchedulerDriver driver1(
       &sched1, frameworkInfo1, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched1, registered(_, _, _));
 
@@ -661,7 +661,7 @@ TEST_F(ReservationAllocatorTest, ReservedResources)
   MesosSchedulerDriver driver2(
       &sched2, frameworkInfo2, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched2, registered(_, _, _));
 
@@ -683,7 +683,7 @@ TEST_F(ReservationAllocatorTest, ReservedResources)
   MesosSchedulerDriver driver3(
       &sched3, frameworkInfo3, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched3, registered(_, _, _));
 
@@ -700,7 +700,7 @@ TEST_F(ReservationAllocatorTest, ReservedResources)
   flags5.default_role = "role1";
   flags5.resources = Some("cpus:1;mem:512;disk:0");
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Future<Nothing> resourceOffers4;
   EXPECT_CALL(sched1, resourceOffers(_, OfferEq(1, 512)))
@@ -713,20 +713,20 @@ TEST_F(ReservationAllocatorTest, ReservedResources)
   AWAIT_READY(resourceOffers4);
 
   // Shut everything down.
-  EXPECT_CALL(allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(allocator, frameworkDeactivated(_))
+  EXPECT_CALL(allocator, deactivateFramework(_))
     .Times(AtMost(3));
 
-  EXPECT_CALL(allocator, frameworkRemoved(_))
+  EXPECT_CALL(allocator, removeFramework(_))
     .Times(AtMost(3));
 
   driver3.stop();
   driver2.stop();
   driver1.stop();
 
-  EXPECT_CALL(allocator, slaveRemoved(_))
+  EXPECT_CALL(allocator, removeSlave(_))
     .Times(AtMost(5));
 
   this->Shutdown();
@@ -751,11 +751,11 @@ TEST_F(ReservationAllocatorTest, ResourcesReturned)
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _, _))
+  EXPECT_CALL(allocator, addSlave(_, _, _, _))
     .Times(2);
 
-  Future<Nothing> slaveAdded1 = FUTURE_DISPATCH(
-      allocator.real, &AllocatorProcess::slaveAdded);
+  Future<Nothing> addSlave1 = FUTURE_DISPATCH(
+      allocator.real, &AllocatorProcess::addSlave);
 
   slave::Flags flags1 = CreateSlaveFlags();
   flags1.resources = Some("cpus(role1):1;mem(role1):200;cpus(role2):2;"
@@ -764,10 +764,10 @@ TEST_F(ReservationAllocatorTest, ResourcesReturned)
   ASSERT_SOME(slave1);
 
   // Wait until allocator has added slave1.
-  AWAIT_READY(slaveAdded1);
+  AWAIT_READY(addSlave1);
 
-  Future<Nothing> slaveAdded2 = FUTURE_DISPATCH(
-      allocator.real, &AllocatorProcess::slaveAdded);
+  Future<Nothing> addSlave2 = FUTURE_DISPATCH(
+      allocator.real, &AllocatorProcess::addSlave);
 
   // This slave's resources will never be offered to anyone,
   // because there is no framework with role3.
@@ -777,7 +777,7 @@ TEST_F(ReservationAllocatorTest, ResourcesReturned)
   ASSERT_SOME(slave2);
 
   // Wait until allocator has added slave2.
-  AWAIT_READY(slaveAdded2);
+  AWAIT_READY(addSlave2);
 
   FrameworkInfo frameworkInfo1; // Bug in gcc 4.1.*, must assign on next line.
   frameworkInfo1 = DEFAULT_FRAMEWORK_INFO;
@@ -790,7 +790,7 @@ TEST_F(ReservationAllocatorTest, ResourcesReturned)
   MesosSchedulerDriver driver1(
       &sched1, frameworkInfo1, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched1, registered(_, _, _));
 
@@ -799,7 +799,7 @@ TEST_F(ReservationAllocatorTest, ResourcesReturned)
   EXPECT_CALL(sched1, resourceOffers(_, OfferEq(2, 400)))
     .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 1, 100, "role1"));
 
-  EXPECT_CALL(allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillOnce(InvokeResourcesRecoveredWithFilters(&allocator, 0));
 
   EXPECT_CALL(exec, registered(_, _, _, _));
@@ -839,7 +839,7 @@ TEST_F(ReservationAllocatorTest, ResourcesReturned)
   MesosSchedulerDriver driver2(
       &sched2, frameworkInfo2, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched2, registered(_, _, _));
 
@@ -857,7 +857,7 @@ TEST_F(ReservationAllocatorTest, ResourcesReturned)
   status.mutable_task_id()->MergeFrom(taskInfo.task_id());
   status.set_state(TASK_FINISHED);
 
-  EXPECT_CALL(allocator, resourcesRecovered(_, _, _, _));
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _));
 
   // After the task finishes, its resources should be reoffered to
   // framework1.
@@ -870,13 +870,13 @@ TEST_F(ReservationAllocatorTest, ResourcesReturned)
   AWAIT_READY(resourceOffers3);
 
   // Shut everything down.
-  EXPECT_CALL(allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(allocator, frameworkDeactivated(_))
+  EXPECT_CALL(allocator, deactivateFramework(_))
     .Times(AtMost(2));
 
-  EXPECT_CALL(allocator, frameworkRemoved(_))
+  EXPECT_CALL(allocator, removeFramework(_))
     .Times(AtMost(2));
 
   Future<Nothing> shutdown;
@@ -888,7 +888,7 @@ TEST_F(ReservationAllocatorTest, ResourcesReturned)
 
   AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
 
-  EXPECT_CALL(allocator, slaveRemoved(_))
+  EXPECT_CALL(allocator, removeSlave(_))
     .Times(AtMost(2));
 
   this->Shutdown();
@@ -926,7 +926,7 @@ TYPED_TEST(AllocatorTest, MockAllocator)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:2;mem:1024;disk:0");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(flags);
   ASSERT_SOME(slave);
@@ -935,7 +935,7 @@ TYPED_TEST(AllocatorTest, MockAllocator)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
@@ -950,19 +950,19 @@ TYPED_TEST(AllocatorTest, MockAllocator)
   AWAIT_READY(resourceOffers);
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_))
+  EXPECT_CALL(this->allocator, deactivateFramework(_))
     .Times(AtMost(1));
 
-  EXPECT_CALL(this->allocator, frameworkRemoved(_))
+  EXPECT_CALL(this->allocator, removeFramework(_))
     .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  EXPECT_CALL(this->allocator, slaveRemoved(_))
+  EXPECT_CALL(this->allocator, removeSlave(_))
     .Times(AtMost(1));
 
   this->Shutdown();
@@ -984,7 +984,7 @@ TYPED_TEST(AllocatorTest, ResourcesUnused)
   slave::Flags flags1 = this->CreateSlaveFlags();
   flags1.resources = Some("cpus:2;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave1 = this->StartSlave(&exec, flags1);
   ASSERT_SOME(slave1);
@@ -993,7 +993,7 @@ TYPED_TEST(AllocatorTest, ResourcesUnused)
   MesosSchedulerDriver driver1(
       &sched1, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched1, registered(_, _, _));
 
@@ -1010,10 +1010,10 @@ TYPED_TEST(AllocatorTest, ResourcesUnused)
   EXPECT_CALL(sched1, resourceOffers(_, OfferEq(2, 1024)))
     .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 1, 512, "*"));
 
-  Future<Nothing> resourcesRecovered;
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  Future<Nothing> recoverResources;
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillOnce(DoAll(InvokeResourcesRecovered(&this->allocator),
-                    FutureSatisfy(&resourcesRecovered)));
+                    FutureSatisfy(&recoverResources)));
 
   EXPECT_CALL(exec, registered(_, _, _, _));
 
@@ -1028,7 +1028,7 @@ TYPED_TEST(AllocatorTest, ResourcesUnused)
   // We need to wait until the allocator knows about the unused
   // resources to start the second framework so that we get the
   // expected offer.
-  AWAIT_READY(resourcesRecovered);
+  AWAIT_READY(recoverResources);
 
   FrameworkInfo frameworkInfo2; // Bug in gcc 4.1.*, must assign on next line.
   frameworkInfo2 = DEFAULT_FRAMEWORK_INFO;
@@ -1039,7 +1039,7 @@ TYPED_TEST(AllocatorTest, ResourcesUnused)
   MesosSchedulerDriver driver2(
       &sched2, frameworkInfo2, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched2, registered(_, _, _));
 
@@ -1054,13 +1054,13 @@ TYPED_TEST(AllocatorTest, ResourcesUnused)
   AWAIT_READY(resourceOffers);
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_))
+  EXPECT_CALL(this->allocator, deactivateFramework(_))
     .Times(AtMost(2));
 
-  EXPECT_CALL(this->allocator, frameworkRemoved(_))
+  EXPECT_CALL(this->allocator, removeFramework(_))
     .Times(AtMost(2));
 
   Future<Nothing> shutdown;
@@ -1075,16 +1075,16 @@ TYPED_TEST(AllocatorTest, ResourcesUnused)
 
   AWAIT_READY(shutdown); // Ensures MockExecutor can be deallocated.
 
-  EXPECT_CALL(this->allocator, slaveRemoved(_))
+  EXPECT_CALL(this->allocator, removeSlave(_))
     .Times(AtMost(1));
 
   this->Shutdown();
 }
 
 
-// Tests the situation where a frameworkRemoved call is dispatched
+// Tests the situation where a removeFramework call is dispatched
 // while we're doing an allocation to that framework, so that
-// resourcesRecovered is called for an already removed framework.
+// recoverResources is called for an already removed framework.
 TYPED_TEST(AllocatorTest, OutOfOrderDispatch)
 {
   EXPECT_CALL(this->allocator, initialize(_, _, _));
@@ -1095,7 +1095,7 @@ TYPED_TEST(AllocatorTest, OutOfOrderDispatch)
   slave::Flags flags1 = this->CreateSlaveFlags();
   flags1.resources = Some("cpus:2;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave1 = this->StartSlave(flags1);
   ASSERT_SOME(slave1);
@@ -1109,7 +1109,7 @@ TYPED_TEST(AllocatorTest, OutOfOrderDispatch)
   MesosSchedulerDriver driver1(
       &sched1, frameworkInfo1, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, Eq(frameworkInfo1), _))
+  EXPECT_CALL(this->allocator, addFramework(_, Eq(frameworkInfo1), _))
     .WillOnce(InvokeFrameworkAdded(&this->allocator));
 
   FrameworkID frameworkId1;
@@ -1126,47 +1126,47 @@ TYPED_TEST(AllocatorTest, OutOfOrderDispatch)
   AWAIT_READY(resourceOffers);
 
   // TODO(benh): I don't see why we want to "catch" (i.e., block) this
-  // resourcesRecovered call. It seems like we want this one to
+  // recoverResources call. It seems like we want this one to
   // properly be executed and later we want to _inject_ a
-  // resourcesRecovered to simulate the code in Master::offer after a
+  // recoverResources to simulate the code in Master::offer after a
   // framework has terminated or is inactive.
   FrameworkID frameworkId;
   SlaveID slaveId;
   Resources savedResources;
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
-    // "Catches" the resourcesRecovered call from the master, so
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
+    // "Catches" the recoverResources call from the master, so
     // that it doesn't get processed until we redispatch it after
-    // the frameworkRemoved trigger.
+    // the removeFramework trigger.
     .WillOnce(DoAll(SaveArg<0>(&frameworkId),
                     SaveArg<1>(&slaveId),
                     SaveArg<2>(&savedResources)));
 
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_));
+  EXPECT_CALL(this->allocator, deactivateFramework(_));
 
-  Future<Nothing> frameworkRemoved;
-  EXPECT_CALL(this->allocator, frameworkRemoved(Eq(frameworkId1)))
+  Future<Nothing> removeFramework;
+  EXPECT_CALL(this->allocator, removeFramework(Eq(frameworkId1)))
     .WillOnce(DoAll(InvokeFrameworkRemoved(&this->allocator),
-                    FutureSatisfy(&frameworkRemoved)));
+                    FutureSatisfy(&removeFramework)));
 
   driver1.stop();
   driver1.join();
 
-  AWAIT_READY(frameworkRemoved);
+  AWAIT_READY(removeFramework);
 
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillOnce(DoDefault());
 
-  // Re-dispatch the resourcesRecovered call which we "caught"
+  // 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.resourcesRecovered(
+  this->allocator.recoverResources(
       frameworkId,
       slaveId,
       savedResources,
       None());
 
   // TODO(benh): Seems like we should wait for the above
-  // resourcesRecovered to be executed.
+  // recoverResources to be executed.
 
   FrameworkInfo frameworkInfo2; // Bug in gcc 4.1.*, must assign on next line.
   frameworkInfo2 = DEFAULT_FRAMEWORK_INFO;
@@ -1177,7 +1177,7 @@ TYPED_TEST(AllocatorTest, OutOfOrderDispatch)
   MesosSchedulerDriver driver2(
       &sched2, frameworkInfo2, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, Eq(frameworkInfo2), _))
+  EXPECT_CALL(this->allocator, addFramework(_, Eq(frameworkInfo2), _))
     .WillOnce(InvokeFrameworkAdded(&this->allocator));
 
   FrameworkID frameworkId2;
@@ -1194,19 +1194,19 @@ TYPED_TEST(AllocatorTest, OutOfOrderDispatch)
   AWAIT_READY(resourceOffers);
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_))
+  EXPECT_CALL(this->allocator, deactivateFramework(_))
     .Times(AtMost(1));
 
-  EXPECT_CALL(this->allocator, frameworkRemoved(Eq(frameworkId2)))
+  EXPECT_CALL(this->allocator, removeFramework(Eq(frameworkId2)))
     .Times(AtMost(1));
 
   driver2.stop();
   driver2.join();
 
-  EXPECT_CALL(this->allocator, slaveRemoved(_))
+  EXPECT_CALL(this->allocator, removeSlave(_))
     .Times(AtMost(1));
 
   this->Shutdown();
@@ -1228,7 +1228,7 @@ TYPED_TEST(AllocatorTest, SchedulerFailover)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:3;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&exec, flags);
   ASSERT_SOME(slave);
@@ -1244,7 +1244,7 @@ TYPED_TEST(AllocatorTest, SchedulerFailover)
   MesosSchedulerDriver driver1(
       &sched1, frameworkInfo1, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
   FrameworkID frameworkId;
   EXPECT_CALL(sched1, registered(&driver1, _, _))
@@ -1263,7 +1263,7 @@ TYPED_TEST(AllocatorTest, 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, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillOnce(InvokeResourcesRecoveredWithFilters(&this->allocator, 0))
     // For subsequent offers.
     .WillRepeatedly(InvokeResourcesRecoveredWithFilters(&this->allocator, 0));
@@ -1285,14 +1285,14 @@ TYPED_TEST(AllocatorTest, SchedulerFailover)
   // if it fails over.
   DROP_PROTOBUFS(UnregisterFrameworkMessage(), _, _);
 
-  Future<Nothing> frameworkDeactivated;
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_))
+  Future<Nothing> deactivateFramework;
+  EXPECT_CALL(this->allocator, deactivateFramework(_))
     .WillOnce(DoAll(InvokeFrameworkDeactivated(&this->allocator),
-                    FutureSatisfy(&frameworkDeactivated)));
+                    FutureSatisfy(&deactivateFramework)));
 
   driver1.stop();
 
-  AWAIT_READY(frameworkDeactivated);
+  AWAIT_READY(deactivateFramework);
 
   FrameworkInfo frameworkInfo2; // Bug in gcc 4.1.*, must assign on next line.
   frameworkInfo2 = DEFAULT_FRAMEWORK_INFO;
@@ -1304,7 +1304,7 @@ TYPED_TEST(AllocatorTest, SchedulerFailover)
   MesosSchedulerDriver driver2(
       &sched2, frameworkInfo2, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkActivated(_, _));
+  EXPECT_CALL(this->allocator, activateFramework(_, _));
 
   EXPECT_CALL(sched2, registered(_, frameworkId, _));
 
@@ -1320,13 +1320,13 @@ TYPED_TEST(AllocatorTest, SchedulerFailover)
   AWAIT_READY(resourceOffers);
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_))
+  EXPECT_CALL(this->allocator, deactivateFramework(_))
     .Times(AtMost(1));
 
-  EXPECT_CALL(this->allocator, frameworkRemoved(_))
+  EXPECT_CALL(this->allocator, removeFramework(_))
     .Times(AtMost(1));
 
   EXPECT_CALL(exec, shutdown(_))
@@ -1335,7 +1335,7 @@ TYPED_TEST(AllocatorTest, SchedulerFailover)
   driver2.stop();
   driver2.join();
 
-  EXPECT_CALL(this->allocator, slaveRemoved(_))
+  EXPECT_CALL(this->allocator, removeSlave(_))
     .Times(AtMost(1));
 
   this->Shutdown();
@@ -1372,7 +1372,7 @@ TYPED_TEST(AllocatorTest, FrameworkExited)
 
   flags.resources = Some("cpus:3;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&containerizer, flags);
   ASSERT_SOME(slave);
@@ -1381,7 +1381,7 @@ TYPED_TEST(AllocatorTest, FrameworkExited)
   MesosSchedulerDriver driver1(
       &sched1, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched1, registered(_, _, _));
 
@@ -1398,10 +1398,10 @@ TYPED_TEST(AllocatorTest, FrameworkExited)
     .WillOnce(LaunchTasks(executor1, 1, 2, 512, "*"));
 
   // The framework does not use all the resources.
-  Future<Nothing> resourcesRecovered;
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  Future<Nothing> recoverResources;
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillOnce(DoAll(InvokeResourcesRecovered(&this->allocator),
-                    FutureSatisfy(&resourcesRecovered)));
+                    FutureSatisfy(&recoverResources)));
 
   EXPECT_CALL(exec1, registered(_, _, _, _));
 
@@ -1419,13 +1419,13 @@ TYPED_TEST(AllocatorTest, FrameworkExited)
   // We need to wait until the allocator knows about the unused
   // resources to start the second framework so that we get the
   // expected offer.
-  AWAIT_READY(resourcesRecovered);
+  AWAIT_READY(recoverResources);
 
   MockScheduler sched2;
   MesosSchedulerDriver driver2(
       &sched2, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched2, registered(_, _, _));
 
@@ -1442,10 +1442,10 @@ TYPED_TEST(AllocatorTest, FrameworkExited)
     .WillOnce(LaunchTasks(executor2, 1, 1, 256, "*"));
 
   // The framework 2 does not use all the resources.
-  Future<Nothing> resourcesRecovered2;
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  Future<Nothing> recoverResources2;
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillOnce(DoAll(InvokeResourcesRecovered(&this->allocator),
-                    FutureSatisfy(&resourcesRecovered2)));
+                    FutureSatisfy(&recoverResources2)));
 
   EXPECT_CALL(exec2, registered(_, _, _, _));
 
@@ -1456,17 +1456,17 @@ TYPED_TEST(AllocatorTest, FrameworkExited)
 
   AWAIT_READY(launchTask);
 
-  AWAIT_READY(resourcesRecovered2);
+  AWAIT_READY(recoverResources2);
 
   // Shut everything down but check that framework 2 gets the
   // resources from framework 1 after it is shutdown.
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_))
+  EXPECT_CALL(this->allocator, deactivateFramework(_))
     .Times(AtMost(2)); // Once for each framework.
 
-  EXPECT_CALL(this->allocator, frameworkRemoved(_))
+  EXPECT_CALL(this->allocator, removeFramework(_))
     .Times(AtMost(2)); // Once for each framework.
 
   // After we stop framework 1, all of it's resources should
@@ -1490,7 +1490,7 @@ TYPED_TEST(AllocatorTest, FrameworkExited)
   driver2.stop();
   driver2.join();
 
-  EXPECT_CALL(this->allocator, slaveRemoved(_))
+  EXPECT_CALL(this->allocator, removeSlave(_))
     .Times(AtMost(1));
 
   this->Shutdown();
@@ -1513,7 +1513,7 @@ TYPED_TEST(AllocatorTest, SlaveLost)
   slave::Flags flags1 = this->CreateSlaveFlags();
   flags1.resources = Some("cpus:2;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave1 = this->StartSlave(&exec, flags1);
   ASSERT_SOME(slave1);
@@ -1522,7 +1522,7 @@ TYPED_TEST(AllocatorTest, SlaveLost)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
@@ -1530,10 +1530,10 @@ TYPED_TEST(AllocatorTest, SlaveLost)
   EXPECT_CALL(sched, resourceOffers(_, OfferEq(2, 1024)))
     .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 2, 512, "*"));
 
-  Future<Nothing> resourcesRecovered;
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  Future<Nothing> recoverResources;
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillOnce(DoAll(InvokeResourcesRecovered(&this->allocator),
-                    FutureSatisfy(&resourcesRecovered)));
+                    FutureSatisfy(&recoverResources)));
 
   EXPECT_CALL(exec, registered(_, _, _, _));
 
@@ -1554,17 +1554,17 @@ TYPED_TEST(AllocatorTest, SlaveLost)
   AWAIT_READY(launchTask);
 
   // Framework does not use all the resources.
-  AWAIT_READY(resourcesRecovered);
+  AWAIT_READY(recoverResources);
 
-  // 'resourcesRecovered' should be called twice, once for the task
+  // 'recoverResources' should be called twice, once for the task
   // and once for the executor.
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .Times(2);
 
-  Future<Nothing> slaveRemoved;
-  EXPECT_CALL(this->allocator, slaveRemoved(_))
+  Future<Nothing> removeSlave;
+  EXPECT_CALL(this->allocator, removeSlave(_))
     .WillOnce(DoAll(InvokeSlaveRemoved(&this->allocator),
-                    FutureSatisfy(&slaveRemoved)));
+                    FutureSatisfy(&removeSlave)));
 
   EXPECT_CALL(exec, shutdown(_))
     .Times(AtMost(1));
@@ -1573,12 +1573,12 @@ TYPED_TEST(AllocatorTest, SlaveLost)
 
   this->ShutdownSlaves();
 
-  AWAIT_READY(slaveRemoved);
+  AWAIT_READY(removeSlave);
 
   slave::Flags flags2 = this->CreateSlaveFlags();
   flags2.resources = string("cpus:3;mem:256;disk:1024;ports:[31000-32000]");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   // Eventually after slave2 is launched, we should get
   // an offer that contains all of slave2's resources
@@ -1596,19 +1596,19 @@ TYPED_TEST(AllocatorTest, SlaveLost)
             Resources::parse(flags2.resources.get()).get());
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_))
+  EXPECT_CALL(this->allocator, deactivateFramework(_))
     .Times(AtMost(1));
 
-  EXPECT_CALL(this->allocator, frameworkRemoved(_))
+  EXPECT_CALL(this->allocator, removeFramework(_))
     .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  EXPECT_CALL(this->allocator, slaveRemoved(_))
+  EXPECT_CALL(this->allocator, removeSlave(_))
     .Times(AtMost(1));
 
   this->Shutdown();
@@ -1632,7 +1632,7 @@ TYPED_TEST(AllocatorTest, SlaveAdded)
   slave::Flags flags1 = this->CreateSlaveFlags();
   flags1.resources = Some("cpus:3;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave1 = this->StartSlave(&exec, flags1);
   ASSERT_SOME(slave1);
@@ -1641,7 +1641,7 @@ TYPED_TEST(AllocatorTest, SlaveAdded)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
@@ -1660,7 +1660,7 @@ TYPED_TEST(AllocatorTest, 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, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillOnce(InvokeResourcesRecoveredWithFilters(&this->allocator, 0.1))
     .WillRepeatedly(InvokeResourcesRecoveredWithFilters(&this->allocator, 0));
 
@@ -1681,7 +1681,7 @@ TYPED_TEST(AllocatorTest, SlaveAdded)
   slave::Flags flags2 = this->CreateSlaveFlags();
   flags2.resources = Some("cpus:4;mem:2048");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   // After slave2 launches, all of its resources are combined with the
   // resources on slave1 that the task isn't using.
@@ -1695,13 +1695,13 @@ TYPED_TEST(AllocatorTest, SlaveAdded)
   AWAIT_READY(resourceOffers);
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_))
+  EXPECT_CALL(this->allocator, deactivateFramework(_))
     .Times(AtMost(1));
 
-  EXPECT_CALL(this->allocator, frameworkRemoved(_))
+  EXPECT_CALL(this->allocator, removeFramework(_))
     .Times(AtMost(1));
 
   EXPECT_CALL(exec, shutdown(_))
@@ -1710,7 +1710,7 @@ TYPED_TEST(AllocatorTest, SlaveAdded)
   driver.stop();
   driver.join();
 
-  EXPECT_CALL(this->allocator, slaveRemoved(_))
+  EXPECT_CALL(this->allocator, removeSlave(_))
     .Times(AtMost(2));
 
   this->Shutdown();
@@ -1733,7 +1733,7 @@ TYPED_TEST(AllocatorTest, TaskFinished)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:3;mem:1024");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&exec, flags);
   ASSERT_SOME(slave);
@@ -1742,7 +1742,7 @@ TYPED_TEST(AllocatorTest, TaskFinished)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
@@ -1761,10 +1761,10 @@ TYPED_TEST(AllocatorTest, TaskFinished)
   // don't send the TASK_FINISHED status update below until after the
   // allocator knows about the unused resources so that it can
   // aggregate them with the resources from the finished task.
-  Future<Nothing> resourcesRecovered;
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  Future<Nothing> recoverResources;
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoAll(InvokeResourcesRecovered(&this->allocator),
-                          FutureSatisfy(&resourcesRecovered)));
+                          FutureSatisfy(&recoverResources)));
 
   EXPECT_CALL(exec, registered(_, _, _, _));
 
@@ -1785,13 +1785,13 @@ TYPED_TEST(AllocatorTest, TaskFinished)
 
   AWAIT_READY(launchTask);
 
-  AWAIT_READY(resourcesRecovered);
+  AWAIT_READY(recoverResources);
 
   TaskStatus status;
   status.mutable_task_id()->MergeFrom(taskInfo.task_id());
   status.set_state(TASK_FINISHED);
 
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _));
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _));
 
   // After the first task gets killed.
   Future<Nothing> resourceOffers;
@@ -1803,13 +1803,13 @@ TYPED_TEST(AllocatorTest, TaskFinished)
   AWAIT_READY(resourceOffers);
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_))
+  EXPECT_CALL(this->allocator, deactivateFramework(_))
     .Times(AtMost(1));
 
-  EXPECT_CALL(this->allocator, frameworkRemoved(_))
+  EXPECT_CALL(this->allocator, removeFramework(_))
     .Times(AtMost(1));
 
   EXPECT_CALL(exec, shutdown(_))
@@ -1818,7 +1818,7 @@ TYPED_TEST(AllocatorTest, TaskFinished)
   driver.stop();
   driver.join();
 
-  EXPECT_CALL(this->allocator, slaveRemoved(_))
+  EXPECT_CALL(this->allocator, removeSlave(_))
     .Times(AtMost(1));
 
   this->Shutdown();
@@ -1842,7 +1842,7 @@ TYPED_TEST(AllocatorTest, CpusOnlyOfferedAndTaskLaunched)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:2;mem:0");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&exec, flags);
   ASSERT_SOME(slave);
@@ -1851,11 +1851,11 @@ TYPED_TEST(AllocatorTest, CpusOnlyOfferedAndTaskLaunched)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
   // Launch a cpus only task.
@@ -1894,10 +1894,10 @@ TYPED_TEST(AllocatorTest, CpusOnlyOfferedAndTaskLaunched)
   AWAIT_READY(resourceOffers);
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_))
+  EXPECT_CALL(this->allocator, deactivateFramework(_))
     .Times(AtMost(1));
 
-  EXPECT_CALL(this->allocator, frameworkRemoved(_))
+  EXPECT_CALL(this->allocator, removeFramework(_))
     .Times(AtMost(1));
 
   EXPECT_CALL(exec, shutdown(_))
@@ -1906,7 +1906,7 @@ TYPED_TEST(AllocatorTest, CpusOnlyOfferedAndTaskLaunched)
   driver.stop();
   driver.join();
 
-  EXPECT_CALL(this->allocator, slaveRemoved(_))
+  EXPECT_CALL(this->allocator, removeSlave(_))
     .Times(AtMost(1));
 
   this->Shutdown();
@@ -1930,7 +1930,7 @@ TYPED_TEST(AllocatorTest, MemoryOnlyOfferedAndTaskLaunched)
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:0;mem:200");
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   Try<PID<Slave> > slave = this->StartSlave(&exec, flags);
   ASSERT_SOME(slave);
@@ -1939,11 +1939,11 @@ TYPED_TEST(AllocatorTest, MemoryOnlyOfferedAndTaskLaunched)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
   // Launch a memory only task.
@@ -1982,10 +1982,10 @@ TYPED_TEST(AllocatorTest, MemoryOnlyOfferedAndTaskLaunched)
   AWAIT_READY(resourceOffers);
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_))
+  EXPECT_CALL(this->allocator, deactivateFramework(_))
     .Times(AtMost(1));
 
-  EXPECT_CALL(this->allocator, frameworkRemoved(_))
+  EXPECT_CALL(this->allocator, removeFramework(_))
     .Times(AtMost(1));
 
   EXPECT_CALL(exec, shutdown(_))
@@ -1994,7 +1994,7 @@ TYPED_TEST(AllocatorTest, MemoryOnlyOfferedAndTaskLaunched)
   driver.stop();
   driver.join();
 
-  EXPECT_CALL(this->allocator, slaveRemoved(_))
+  EXPECT_CALL(this->allocator, removeSlave(_))
     .Times(AtMost(1));
 
   this->Shutdown();
@@ -2024,7 +2024,7 @@ TYPED_TEST(AllocatorTest, WhitelistSlave)
   Try<PID<Master> > master = this->StartMaster(&this->allocator, masterFlags);
   ASSERT_SOME(master);
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:2;mem:1024");
@@ -2040,7 +2040,7 @@ TYPED_TEST(AllocatorTest, WhitelistSlave)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
   EXPECT_CALL(sched, registered(_, _, _));
 
@@ -2081,19 +2081,19 @@ TYPED_TEST(AllocatorTest, WhitelistSlave)
   Clock::resume();
 
   // Shut everything down.
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_))
+  EXPECT_CALL(this->allocator, deactivateFramework(_))
     .Times(AtMost(1));
 
-  EXPECT_CALL(this->allocator, frameworkRemoved(_))
+  EXPECT_CALL(this->allocator, removeFramework(_))
     .Times(AtMost(1));
 
   driver.stop();
   driver.join();
 
-  EXPECT_CALL(this->allocator, slaveRemoved(_))
+  EXPECT_CALL(this->allocator, removeSlave(_))
     .Times(AtMost(1));
 
   this->Shutdown();
@@ -2150,29 +2150,29 @@ TYPED_TEST(AllocatorTest, RoleTest)
   EXPECT_CALL(sched2, registered(_, _, _))
     .WillOnce(FutureSatisfy(&registered2));
 
-  Future<Nothing> frameworkAdded;
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _))
-    .WillOnce(FutureSatisfy(&frameworkAdded));
+  Future<Nothing> addFramework;
+  EXPECT_CALL(this->allocator, addFramework(_, _, _))
+    .WillOnce(FutureSatisfy(&addFramework));
 
   driver2.start();
 
   AWAIT_READY(registered2);
-  AWAIT_READY(frameworkAdded);
+  AWAIT_READY(addFramework);
 
   // Shut everything down.
-  Future<Nothing> frameworkDeactivated;
-  EXPECT_CALL(this->allocator, frameworkDeactivated(_))
-    .WillOnce(FutureSatisfy(&frameworkDeactivated));
+  Future<Nothing> deactivateFramework;
+  EXPECT_CALL(this->allocator, deactivateFramework(_))
+    .WillOnce(FutureSatisfy(&deactivateFramework));
 
-  Future<Nothing> frameworkRemoved;
-  EXPECT_CALL(this->allocator, frameworkRemoved(_))
-    .WillOnce(FutureSatisfy(&frameworkRemoved));
+  Future<Nothing> removeFramework;
+  EXPECT_CALL(this->allocator, removeFramework(_))
+    .WillOnce(FutureSatisfy(&removeFramework));
 
   driver2.stop();
   driver2.join();
 
-  AWAIT_READY(frameworkDeactivated);
-  AWAIT_READY(frameworkRemoved);
+  AWAIT_READY(deactivateFramework);
+  AWAIT_READY(removeFramework);
 
   driver1.stop();
   driver1.join();
@@ -2194,7 +2194,7 @@ TYPED_TEST(AllocatorTest, FrameworkReregistersFirst)
 
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   StandaloneMasterDetector slaveDetector(master.get());
 
@@ -2204,7 +2204,7 @@ TYPED_TEST(AllocatorTest, FrameworkReregistersFirst)
   Try<PID<Slave> > slave = this->StartSlave(&exec, &slaveDetector, flags);
   ASSERT_SOME(slave);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
   MockScheduler sched;
   StandaloneMasterDetector schedulerDetector(master.get());
@@ -2218,7 +2218,7 @@ TYPED_TEST(AllocatorTest, FrameworkReregistersFirst)
     .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, 1, 500, "*"))
     .WillRepeatedly(DeclineOffers());
 
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _));
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _));
 
   EXPECT_CALL(exec, registered(_, _, _, _));
 
@@ -2242,7 +2242,7 @@ TYPED_TEST(AllocatorTest, FrameworkReregistersFirst)
   // it doesn't try to retry the update after master failover.
   AWAIT_READY(_statusUpdateAcknowledgement);
 
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
   this->ShutdownMasters();
@@ -2252,10 +2252,10 @@ TYPED_TEST(AllocatorTest, FrameworkReregistersFirst)
 
   EXPECT_CALL(allocator2, initialize(_, _, _));
 
-  Future<Nothing> frameworkAdded;
-  EXPECT_CALL(allocator2, frameworkAdded(_, _, _))
+  Future<Nothing> addFramework;
+  EXPECT_CALL(allocator2, addFramework(_, _, _))
     .WillOnce(DoAll(InvokeFrameworkAdded(&allocator2),
-                    FutureSatisfy(&frameworkAdded)));
+                    FutureSatisfy(&addFramework)));
 
   EXPECT_CALL(sched, registered(&driver, _, _));
 
@@ -2267,9 +2267,9 @@ TYPED_TEST(AllocatorTest, FrameworkReregistersFirst)
   // Inform the scheduler about the new master.
   schedulerDetector.appoint(master2.get());
 
-  AWAIT_READY(frameworkAdded);
+  AWAIT_READY(addFramework);
 
-  EXPECT_CALL(allocator2, slaveAdded(_, _, _, _));
+  EXPECT_CALL(allocator2, addSlave(_, _, _, _));
 
   Future<vector<Offer> > resourceOffers2;
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -2285,13 +2285,13 @@ TYPED_TEST(AllocatorTest, FrameworkReregistersFirst)
   EXPECT_THAT(resourceOffers2.get(), OfferEq(1, 524));
 
   // Shut everything down.
-  EXPECT_CALL(allocator2, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(allocator2, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(allocator2, frameworkDeactivated(_))
+  EXPECT_CALL(allocator2, deactivateFramework(_))
     .Times(AtMost(1));
 
-  EXPECT_CALL(allocator2, frameworkRemoved(_))
+  EXPECT_CALL(allocator2, removeFramework(_))
     .Times(AtMost(1));
 
   EXPECT_CALL(exec, shutdown(_))
@@ -2300,7 +2300,7 @@ TYPED_TEST(AllocatorTest, FrameworkReregistersFirst)
   driver.stop();
   driver.join();
 
-  EXPECT_CALL(allocator2, slaveRemoved(_))
+  EXPECT_CALL(allocator2, removeSlave(_))
     .Times(AtMost(1));
 
   this->Shutdown();
@@ -2321,7 +2321,7 @@ TYPED_TEST(AllocatorTest, SlaveReregistersFirst)
   MockExecutor exec(DEFAULT_EXECUTOR_ID);
   StandaloneMasterDetector slaveDetector(master.get());
 
-  EXPECT_CALL(this->allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(this->allocator, addSlave(_, _, _, _));
 
   slave::Flags flags = this->CreateSlaveFlags();
   flags.resources = Some("cpus:2;mem:1024");
@@ -2333,9 +2333,9 @@ TYPED_TEST(AllocatorTest, SlaveReregistersFirst)
   StandaloneMasterDetector schedulerDetector(master.get());
   TestingMesosSchedulerDriver driver(&sched, &schedulerDetector);
 
-  EXPECT_CALL(this->allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(this->allocator, addFramework(_, _, _));
 
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _));
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _));
 
   EXPECT_CALL(sched, registered(&driver, _, _));
 
@@ -2367,7 +2367,7 @@ TYPED_TEST(AllocatorTest, SlaveReregistersFirst)
   // it doesn't try to retry the update after master failover.
   AWAIT_READY(_statusUpdateAcknowledgement);
 
-  EXPECT_CALL(this->allocator, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(this->allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
   this->ShutdownMasters();
@@ -2377,10 +2377,10 @@ TYPED_TEST(AllocatorTest, SlaveReregistersFirst)
 
   EXPECT_CALL(allocator2, initialize(_, _, _));
 
-  Future<Nothing> slaveAdded;
-  EXPECT_CALL(allocator2, slaveAdded(_, _, _, _))
+  Future<Nothing> addSlave;
+  EXPECT_CALL(allocator2, addSlave(_, _, _, _))
     .WillOnce(DoAll(InvokeSlaveAdded(&allocator2),
-                    FutureSatisfy(&slaveAdded)));
+                    FutureSatisfy(&addSlave)));
 
   Try<PID<Master> > master2 = this->StartMaster(&allocator2);
   ASSERT_SOME(master2);
@@ -2388,13 +2388,13 @@ TYPED_TEST(AllocatorTest, SlaveReregistersFirst)
   // Inform the slave about the new master.
   slaveDetector.appoint(master2.get());
 
-  AWAIT_READY(slaveAdded);
+  AWAIT_READY(addSlave);
 
   EXPECT_CALL(sched, disconnected(_));
 
   EXPECT_CALL(sched, registered(&driver, _, _));
 
-  EXPECT_CALL(allocator2, frameworkAdded(_, _, _));
+  EXPECT_CALL(allocator2, addFramework(_, _, _));
 
   Future<vector<Offer> > resourceOffers2;
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -2410,13 +2410,13 @@ TYPED_TEST(AllocatorTest, SlaveReregistersFirst)
   EXPECT_THAT(resourceOffers2.get(), OfferEq(1, 524));
 
   // Shut everything down.
-  EXPECT_CALL(allocator2, resourcesRecovered(_, _, _, _))
+  EXPECT_CALL(allocator2, recoverResources(_, _, _, _))
     .WillRepeatedly(DoDefault());
 
-  EXPECT_CALL(allocator2, frameworkDeactivated(_))
+  EXPECT_CALL(allocator2, deactivateFramework(_))
     .Times(AtMost(1));
 
-  EXPECT_CALL(allocator2, frameworkRemoved(_))
+  EXPECT_CALL(allocator2, removeFramework(_))
     .Times(AtMost(1));
 
   EXPECT_CALL(exec, shutdown(_))
@@ -2425,7 +2425,7 @@ TYPED_TEST(AllocatorTest, SlaveReregistersFirst)
   driver.stop();
   driver.join();
 
-  EXPECT_CALL(allocator2, slaveRemoved(_))
+  EXPECT_CALL(allocator2, removeSlave(_))
     .Times(AtMost(1));
 
   this->Shutdown();

http://git-wip-us.apache.org/repos/asf/mesos/blob/9b84b52b/src/tests/fault_tolerance_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fault_tolerance_tests.cpp b/src/tests/fault_tolerance_tests.cpp
index 5baeda6..5763486 100644
--- a/src/tests/fault_tolerance_tests.cpp
+++ b/src/tests/fault_tolerance_tests.cpp
@@ -612,8 +612,8 @@ TEST_F(FaultToleranceTest, SchedulerReregisterAfterFailoverTimeout)
   AWAIT_READY(frameworkRegisteredMessage);
   AWAIT_READY(frameworkId);
 
-  Future<Nothing> frameworkDeactivated = FUTURE_DISPATCH(
-      _, &master::allocator::AllocatorProcess::frameworkDeactivated);
+  Future<Nothing> deactivateFramework = FUTURE_DISPATCH(
+      _, &master::allocator::AllocatorProcess::deactivateFramework);
 
   Future<Nothing> frameworkFailoverTimeout =
     FUTURE_DISPATCH(_, &Master::frameworkFailoverTimeout);
@@ -623,7 +623,7 @@ TEST_F(FaultToleranceTest, SchedulerReregisterAfterFailoverTimeout)
       frameworkRegisteredMessage.get().to, master.get()));
 
   // Wait until master schedules the framework for removal.
-  AWAIT_READY(frameworkDeactivated);
+  AWAIT_READY(deactivateFramework);
 
   // Simulate framework failover timeout.
   Clock::pause();
@@ -698,15 +698,15 @@ TEST_F(FaultToleranceTest, SchedulerReregisterAfterUnregistration)
   AWAIT_READY(frameworkRegisteredMessage);
   AWAIT_READY(frameworkId);
 
-  Future<Nothing> frameworkRemoved = FUTURE_DISPATCH(
-      _, &master::allocator::AllocatorProcess::frameworkRemoved);
+  Future<Nothing> removeFramework = FUTURE_DISPATCH(
+      _, &master::allocator::AllocatorProcess::removeFramework);
 
   // Unregister the framework.
   driver1.stop();
   driver1.join();
 
   // Wait until master actually marks the framework as completed.
-  AWAIT_READY(frameworkRemoved);
+  AWAIT_READY(removeFramework);
 
   // Now launch the second (i.e., failover) scheduler using the
   // framework id recorded from the first scheduler.


[7/8] mesos git commit: Fixed a bug in the allocator's framework activation.

Posted by bm...@apache.org.
Fixed a bug in the allocator's framework activation.

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


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

Branch: refs/heads/master
Commit: f68d6cd7746cfb47e136a5c95b9bc7bcdcdf269d
Parents: 9b84b52
Author: Benjamin Mahler <be...@gmail.com>
Authored: Tue Dec 2 19:33:15 2014 -0800
Committer: Benjamin Mahler <be...@gmail.com>
Committed: Wed Dec 3 15:02:05 2014 -0800

----------------------------------------------------------------------
 src/master/allocator.hpp                      | 12 ++++--------
 src/master/hierarchical_allocator_process.hpp | 10 +++++-----
 src/master/master.cpp                         |  4 ++--
 src/tests/allocator_tests.cpp                 |  2 +-
 src/tests/mesos.hpp                           | 10 ++++------
 5 files changed, 16 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f68d6cd7/src/master/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator.hpp b/src/master/allocator.hpp
index 8fb660d..af4782b 100644
--- a/src/master/allocator.hpp
+++ b/src/master/allocator.hpp
@@ -79,8 +79,7 @@ public:
 
   // Offers are sent only to activated frameworks.
   void activateFramework(
-      const FrameworkID& frameworkId,
-      const FrameworkInfo& frameworkInfo);
+      const FrameworkID& frameworkId);
 
   void deactivateFramework(
       const FrameworkID& frameworkId);
@@ -155,8 +154,7 @@ public:
       const FrameworkID& frameworkId) = 0;
 
   virtual void activateFramework(
-      const FrameworkID& frameworkId,
-      const FrameworkInfo& frameworkInfo) = 0;
+      const FrameworkID& frameworkId) = 0;
 
   virtual void deactivateFramework(
       const FrameworkID& frameworkId) = 0;
@@ -247,14 +245,12 @@ inline void Allocator::removeFramework(
 
 
 inline void Allocator::activateFramework(
-    const FrameworkID& frameworkId,
-    const FrameworkInfo& frameworkInfo)
+    const FrameworkID& frameworkId)
 {
   process::dispatch(
       process,
       &AllocatorProcess::activateFramework,
-      frameworkId,
-      frameworkInfo);
+      frameworkId);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f68d6cd7/src/master/hierarchical_allocator_process.hpp
----------------------------------------------------------------------
diff --git a/src/master/hierarchical_allocator_process.hpp b/src/master/hierarchical_allocator_process.hpp
index f0baa05..ddd42e7 100644
--- a/src/master/hierarchical_allocator_process.hpp
+++ b/src/master/hierarchical_allocator_process.hpp
@@ -83,8 +83,7 @@ public:
       const FrameworkID& frameworkId);
 
   void activateFramework(
-      const FrameworkID& frameworkId,
-      const FrameworkInfo& frameworkInfo);
+      const FrameworkID& frameworkId);
 
   void deactivateFramework(
       const FrameworkID& frameworkId);
@@ -333,12 +332,13 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::removeFramework(
 template <class RoleSorter, class FrameworkSorter>
 void
 HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::activateFramework(
-    const FrameworkID& frameworkId,
-    const FrameworkInfo& frameworkInfo)
+    const FrameworkID& frameworkId)
 {
   CHECK(initialized);
 
-  const std::string& role = frameworkInfo.role();
+  CHECK(frameworks.contains(frameworkId));
+  const std::string& role = frameworks[frameworkId].role;
+
   sorters[role]->activate(frameworkId.value());
 
   LOG(INFO) << "Activated framework " << frameworkId;

http://git-wip-us.apache.org/repos/asf/mesos/blob/f68d6cd7/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 3d85e7b..b910665 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -1580,7 +1580,7 @@ void Master::_reregisterFramework(
       // the allocator has the correct view of the framework's share.
       if (!framework->active) {
         framework->active = true;
-        allocator->activateFramework(framework->id, framework->info);
+        allocator->activateFramework(framework->id);
       }
 
       FrameworkReregisteredMessage message;
@@ -4162,7 +4162,7 @@ void Master::failoverFramework(Framework* framework, const UPID& newPid)
   // the allocator has the correct view of the framework's share.
   if (!framework->active) {
     framework->active = true;
-    allocator->activateFramework(framework->id, framework->info);
+    allocator->activateFramework(framework->id);
   }
 
   // 'Failover' the framework's metrics. i.e., change the lookup key

http://git-wip-us.apache.org/repos/asf/mesos/blob/f68d6cd7/src/tests/allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/allocator_tests.cpp b/src/tests/allocator_tests.cpp
index b2924e3..65f05fd 100644
--- a/src/tests/allocator_tests.cpp
+++ b/src/tests/allocator_tests.cpp
@@ -1304,7 +1304,7 @@ TYPED_TEST(AllocatorTest, SchedulerFailover)
   MesosSchedulerDriver driver2(
       &sched2, frameworkInfo2, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(this->allocator, activateFramework(_, _));
+  EXPECT_CALL(this->allocator, activateFramework(_));
 
   EXPECT_CALL(sched2, registered(_, frameworkId, _));
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f68d6cd7/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 02a7129..aa10343 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -664,7 +664,7 @@ public:
     ON_CALL(*this, removeFramework(_))
       .WillByDefault(InvokeFrameworkRemoved(this));
 
-    ON_CALL(*this, activateFramework(_, _))
+    ON_CALL(*this, activateFramework(_))
       .WillByDefault(InvokeFrameworkActivated(this));
 
     ON_CALL(*this, deactivateFramework(_))
@@ -714,9 +714,8 @@ public:
   MOCK_METHOD1(removeFramework, void(
       const FrameworkID&));
 
-  MOCK_METHOD2(activateFramework, void(
-      const FrameworkID&,
-      const FrameworkInfo&));
+  MOCK_METHOD1(activateFramework, void(
+      const FrameworkID&));
 
   MOCK_METHOD1(deactivateFramework, void(
       const FrameworkID&));
@@ -800,8 +799,7 @@ ACTION_P(InvokeFrameworkActivated, allocator)
   process::dispatch(
       allocator->real,
       &master::allocator::AllocatorProcess::activateFramework,
-      arg0,
-      arg1);
+      arg0);
 }
 
 


[8/8] mesos git commit: Updated allocator to only check initialization in exposed methods.

Posted by bm...@apache.org.
Updated allocator to only check initialization in exposed methods.

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


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

Branch: refs/heads/master
Commit: 8a75dd2264f847fcea1a5bd1025d1eacc0d78b63
Parents: f68d6cd
Author: Benjamin Mahler <be...@gmail.com>
Authored: Wed Dec 3 11:33:26 2014 -0800
Committer: Benjamin Mahler <be...@gmail.com>
Committed: Wed Dec 3 15:02:05 2014 -0800

----------------------------------------------------------------------
 src/master/hierarchical_allocator_process.hpp | 8 --------
 1 file changed, 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8a75dd22/src/master/hierarchical_allocator_process.hpp
----------------------------------------------------------------------
diff --git a/src/master/hierarchical_allocator_process.hpp b/src/master/hierarchical_allocator_process.hpp
index ddd42e7..f18346f 100644
--- a/src/master/hierarchical_allocator_process.hpp
+++ b/src/master/hierarchical_allocator_process.hpp
@@ -628,7 +628,6 @@ template <class RoleSorter, class FrameworkSorter>
 void
 HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::batch()
 {
-  CHECK(initialized);
   allocate();
   delay(flags.allocation_interval, self(), &Self::batch);
 }
@@ -638,8 +637,6 @@ template <class RoleSorter, class FrameworkSorter>
 void
 HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::allocate()
 {
-  CHECK(initialized);
-
   Stopwatch stopwatch;
   stopwatch.start();
 
@@ -655,8 +652,6 @@ void
 HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::allocate(
     const SlaveID& slaveId)
 {
-  CHECK(initialized);
-
   hashset<SlaveID> slaveIds;
   slaveIds.insert(slaveId);
 
@@ -675,8 +670,6 @@ void
 HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::allocate(
     const hashset<SlaveID>& slaveIds_)
 {
-  CHECK(initialized);
-
   if (roleSorter->count() == 0) {
     VLOG(1) << "No roles to allocate resources!";
     return;
@@ -771,7 +764,6 @@ bool
 HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::isWhitelisted(
     const SlaveID& slaveId)
 {
-  CHECK(initialized);
   CHECK(slaves.contains(slaveId));
 
   return whitelist.isNone() ||


Re: [2/8] mesos git commit: Cleaned up the utility structs in the allocator.

Posted by Bernd Mathiske <be...@mesosphere.io>.
Hi Ben!

FYI, one of your recent commits seems to have caused the build on Macs to fail:

libtool: compile:  g++ -DPACKAGE_NAME=\"mesos\" -DPACKAGE_TARNAME=\"mesos\" -DPACKAGE_VERSION=\"0.22.0\" "-DPACKAGE_STRING=\"mesos 0.22.0\"" -DPACKAGE_BUGREPORT=\"\" -DPACKAGE_URL=\"\" -DPACKAGE=\"mesos\" -DVERSION=\"0.22.0\" -DSTDC_HEADERS=1 -DHAVE_SYS_TYPES_H=1 -DHAVE_SYS_STAT_H=1 -DHAVE_STDLIB_H=1 -DHAVE_STRING_H=1 -DHAVE_MEMORY_H=1 -DHAVE_STRINGS_H=1 -DHAVE_INTTYPES_H=1 -DHAVE_STDINT_H=1 -DHAVE_UNISTD_H=1 -DHAVE_DLFCN_H=1 -DLT_OBJDIR=\".libs/\" -DHAVE_PTHREAD=1 -DHAVE_LIBZ=1 -DHAVE_LIBCURL=1 -DHAVE_APR_POOLS_H=1 -DHAVE_LIBAPR_1=1 -DHAVE_SVN_VERSION_H=1 -DHAVE_LIBSVN_SUBR_1=1 -DHAVE_SVN_DELTA_H=1 -DHAVE_LIBSVN_DELTA_1=1 -DHAVE_LIBSASL2=1 -DMESOS_HAS_JAVA=1 -DHAVE_PYTHON=\"2.7\" -DMESOS_HAS_PYTHON=1 -I. -I../../src -Wall -Werror -DLIBDIR=\"/usr/local/lib\" -DPKGLIBEXECDIR=\"/usr/local/libexec/mesos\" -DPKGDATADIR=\"/usr/local/share/mesos\" -I../../include -I../../3rdparty/libprocess/include -I../../3rdparty/libprocess/3rdparty/stout/include -I../include -I../include/mesos -I../3rdparty/libprocess/3rdparty/boost-1.53.0 -I../3rdparty/libprocess/3rdparty/picojson-4f93734 -I../3rdparty/libprocess/3rdparty/protobuf-2.5.0/src -I../3rdparty/libprocess/3rdparty/glog-0.3.3/src -I../3rdparty/libprocess/3rdparty/glog-0.3.3/src -I../3rdparty/leveldb/include -I../3rdparty/zookeeper-3.4.5/src/c/include -I../3rdparty/zookeeper-3.4.5/src/c/generated -I../3rdparty/libprocess/3rdparty/protobuf-2.5.0/src -I/usr/include/subversion-1 -I/usr/include/apr-1 -I/usr/include/apr-1.0 -D_THREAD_SAFE -g1 -O0 -std=c++11 -stdlib=libc++ -DGTEST_USE_OWN_TR1_TUPLE=1 -MT authorizer/libmesos_no_3rdparty_la-authorizer.lo -MD -MP -MF authorizer/.deps/libmesos_no_3rdparty_la-authorizer.Tpo -c ../../src/authorizer/authorizer.cpp -o authorizer/libmesos_no_3rdparty_la-authorizer.o >/dev/null 2>&1
libtool: compile:  g++ -DPACKAGE_NAME=\"mesos\" -DPACKAGE_TARNAME=\"mesos\" -DPACKAGE_VERSION=\"0.22.0\" "-DPACKAGE_STRING=\"mesos 0.22.0\"" -DPACKAGE_BUGREPORT=\"\" -DPACKAGE_URL=\"\" -DPACKAGE=\"mesos\" -DVERSION=\"0.22.0\" -DSTDC_HEADERS=1 -DHAVE_SYS_TYPES_H=1 -DHAVE_SYS_STAT_H=1 -DHAVE_STDLIB_H=1 -DHAVE_STRING_H=1 -DHAVE_MEMORY_H=1 -DHAVE_STRINGS_H=1 -DHAVE_INTTYPES_H=1 -DHAVE_STDINT_H=1 -DHAVE_UNISTD_H=1 -DHAVE_DLFCN_H=1 -DLT_OBJDIR=\".libs/\" -DHAVE_PTHREAD=1 -DHAVE_LIBZ=1 -DHAVE_LIBCURL=1 -DHAVE_APR_POOLS_H=1 -DHAVE_LIBAPR_1=1 -DHAVE_SVN_VERSION_H=1 -DHAVE_LIBSVN_SUBR_1=1 -DHAVE_SVN_DELTA_H=1 -DHAVE_LIBSVN_DELTA_1=1 -DHAVE_LIBSASL2=1 -DMESOS_HAS_JAVA=1 -DHAVE_PYTHON=\"2.7\" -DMESOS_HAS_PYTHON=1 -I. -I../../src -Wall -Werror -DLIBDIR=\"/usr/local/lib\" -DPKGLIBEXECDIR=\"/usr/local/libexec/mesos\" -DPKGDATADIR=\"/usr/local/share/mesos\" -I../../include -I../../3rdparty/libprocess/include -I../../3rdparty/libprocess/3rdparty/stout/include -I../include -I../include/mesos -I../3rdparty/libprocess/3rdparty/boost-1.53.0 -I../3rdparty/libprocess/3rdparty/picojson-4f93734 -I../3rdparty/libprocess/3rdparty/protobuf-2.5.0/src -I../3rdparty/libprocess/3rdparty/glog-0.3.3/src -I../3rdparty/libprocess/3rdparty/glog-0.3.3/src -I../3rdparty/leveldb/include -I../3rdparty/zookeeper-3.4.5/src/c/include -I../3rdparty/zookeeper-3.4.5/src/c/generated -I../3rdparty/libprocess/3rdparty/protobuf-2.5.0/src -I/usr/include/subversion-1 -I/usr/include/apr-1 -I/usr/include/apr-1.0 -D_THREAD_SAFE -g1 -O0 -std=c++11 -stdlib=libc++ -DGTEST_USE_OWN_TR1_TUPLE=1 -MT logging/libmesos_no_3rdparty_la-logging.lo -MD -MP -MF logging/.deps/libmesos_no_3rdparty_la-logging.Tpo -c ../../src/logging/logging.cpp  -fno-common -DPIC -o logging/.libs/libmesos_no_3rdparty_la-logging.o
In file included from ../../src/local/local.cpp:43:
../../src/master/allocator.hpp:143:16: error: 
      'mesos::internal::master::allocator::AllocatorProcess::initialize' hides overloaded virtual function
      [-Werror,-Woverloaded-virtual]
  virtual void initialize(
               ^
../../3rdparty/libprocess/include/process/process.hpp:49:16: note: hidden overloaded virtual function
      'process::ProcessBase::initialize' declared here: different number of parameters (0 vs 3)
  virtual void initialize() {}

Bernd

> On Dec 3, 2014, at 3:14 PM, bmahler@apache.org wrote:
> 
> Cleaned up the utility structs in the allocator.
> 
> Review: https://reviews.apache.org/r/28663
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
> Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/0cb4c9c0
> Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/0cb4c9c0
> Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/0cb4c9c0
> 
> Branch: refs/heads/master
> Commit: 0cb4c9c05960a31cd507f36e9a1b7da6aa3f689f
> Parents: f575ae4
> Author: Benjamin Mahler <be...@gmail.com>
> Authored: Tue Dec 2 17:51:55 2014 -0800
> Committer: Benjamin Mahler <be...@gmail.com>
> Committed: Wed Dec 3 14:59:19 2014 -0800
> 
> ----------------------------------------------------------------------
> src/master/hierarchical_allocator_process.hpp | 136 ++++++++++-----------
> 1 file changed, 64 insertions(+), 72 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/mesos/blob/0cb4c9c0/src/master/hierarchical_allocator_process.hpp
> ----------------------------------------------------------------------
> diff --git a/src/master/hierarchical_allocator_process.hpp b/src/master/hierarchical_allocator_process.hpp
> index 4f284ce..c71739b 100644
> --- a/src/master/hierarchical_allocator_process.hpp
> +++ b/src/master/hierarchical_allocator_process.hpp
> @@ -57,52 +57,6 @@ typedef HierarchicalAllocatorProcess<DRFSorter, DRFSorter>
> HierarchicalDRFAllocatorProcess;
> 
> 
> -struct Slave
> -{
> -  Slave() {}
> -
> -  explicit Slave(const SlaveInfo& _info)
> -    : available(_info.resources()),
> -      activated(true),
> -      checkpoint(_info.checkpoint()),
> -      info(_info) {}
> -
> -  Resources resources() const { return info.resources(); }
> -
> -  std::string hostname() const { return info.hostname(); }
> -
> -  // Contains all of the resources currently free on this slave.
> -  Resources available;
> -
> -  // Whether the slave is activated. Resources are not offered for
> -  // deactivated slaves until they are reactivated.
> -  bool activated;
> -
> -  bool checkpoint;
> -private:
> -  SlaveInfo info;
> -};
> -
> -
> -struct Framework
> -{
> -  Framework() {}
> -
> -  explicit Framework(const FrameworkInfo& _info)
> -    : checkpoint(_info.checkpoint()),
> -      info(_info) {}
> -
> -  std::string role() const { return info.role(); }
> -
> -  // Filters that have been added by this framework.
> -  hashset<Filter*> filters;
> -
> -  bool checkpoint;
> -private:
> -  FrameworkInfo info;
> -};
> -
> -
> // Implements the basic allocator algorithm - first pick a role by
> // some criteria, then pick one of their frameworks to allocate to.
> template <typename RoleSorter, typename FrameworkSorter>
> @@ -202,14 +156,31 @@ protected:
>   Flags flags;
>   process::PID<Master> master;
> 
> -  // Contains all frameworks.
> +  struct Framework
> +  {
> +    std::string role;
> +    bool checkpoint;  // Whether the framework desires checkpointing.
> +
> +    hashset<Filter*> filters; // Active filters for the framework.
> +  };
> +
>   hashmap<FrameworkID, Framework> frameworks;
> 
>   // Maps role names to the Sorter object which contains
>   // all of that role's frameworks.
>   hashmap<std::string, FrameworkSorter*> sorters;
> 
> -  // Contains all active slaves.
> +  struct Slave
> +  {
> +    Resources total;
> +    Resources available;
> +
> +    bool activated;  // Whether to offer resources.
> +    bool checkpoint; // Whether slave supports checkpointing.
> +
> +    std::string hostname;
> +  };
> +
>   hashmap<SlaveID, Slave> slaves;
> 
>   hashmap<std::string, RoleInfo> roles;
> @@ -319,7 +290,9 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkAdded(
>   sorters[role]->add(used);
>   sorters[role]->allocated(frameworkId.value(), used);
> 
> -  frameworks[frameworkId] = Framework(frameworkInfo);
> +  frameworks[frameworkId] = Framework();
> +  frameworks[frameworkId].role = frameworkInfo.role();
> +  frameworks[frameworkId].checkpoint = frameworkInfo.checkpoint();
> 
>   LOG(INFO) << "Added framework " << frameworkId;
> 
> @@ -335,7 +308,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkRemoved(
>   CHECK(initialized);
> 
>   CHECK(frameworks.contains(frameworkId));
> -  const std::string& role = frameworks[frameworkId].role();
> +  const std::string& role = frameworks[frameworkId].role;
> 
>   // Might not be in 'sorters[role]' because it was previously
>   // deactivated and never re-added.
> @@ -381,7 +354,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkDeactivated(
>   CHECK(initialized);
> 
>   CHECK(frameworks.contains(frameworkId));
> -  const std::string& role = frameworks[frameworkId].role();
> +  const std::string& role = frameworks[frameworkId].role;
> 
>   sorters[role]->deactivate(frameworkId.value());
> 
> @@ -401,6 +374,21 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkDeactivated(
> }
> 
> 
> +namespace internal {
> +
> +// TODO(bmahler): Generalize this.
> +template <typename Iterable>
> +Resources sum(const Iterable& resources)
> +{
> +  Resources total;
> +  foreach (const Resources& r, resources) {
> +    total += r;
> +  }
> +  return total;
> +}
> +
> +} // namespace internal {
> +
> template <class RoleSorter, class FrameworkSorter>
> void
> HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveAdded(
> @@ -409,33 +397,34 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveAdded(
>     const hashmap<FrameworkID, Resources>& used)
> {
>   CHECK(initialized);
> -
>   CHECK(!slaves.contains(slaveId));
> 
> -  slaves[slaveId] = Slave(slaveInfo);
> -
> -  roleSorter->add(slaveInfo.resources());
> +  const Resources& total = slaveInfo.resources();
> 
> -  Resources unused = slaveInfo.resources();
> +  roleSorter->add(total);
> 
>   foreachpair (const FrameworkID& frameworkId,
>                const Resources& resources,
>                used) {
>     if (frameworks.contains(frameworkId)) {
> -      const std::string& role = frameworks[frameworkId].role();
> +      const std::string& role = frameworks[frameworkId].role;
> +
>       sorters[role]->add(resources);
>       sorters[role]->allocated(frameworkId.value(), resources);
>       roleSorter->allocated(role, resources);
>     }
> -
> -    unused -= resources; // Only want to allocate resources that are not used!
>   }
> 
> -  slaves[slaveId].available = unused;
> +  slaves[slaveId] = Slave();
> +  slaves[slaveId].total = total;
> +  slaves[slaveId].available = total - internal::sum(used.values());
> +  slaves[slaveId].activated = true;
> +  slaves[slaveId].checkpoint = slaveInfo.checkpoint();
> +  slaves[slaveId].hostname = slaveInfo.hostname();
> 
> -  LOG(INFO) << "Added slave " << slaveId << " (" << slaveInfo.hostname()
> -            << ") with " << slaveInfo.resources() << " (and " << unused
> -            << " available)";
> +  LOG(INFO) << "Added slave " << slaveId << " (" << slaves[slaveId].hostname
> +            << ") with " << slaves[slaveId].total
> +            << " (and " << slaves[slaveId].available << " available)";
> 
>   allocate(slaveId);
> }
> @@ -449,7 +438,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveRemoved(
>   CHECK(initialized);
>   CHECK(slaves.contains(slaveId));
> 
> -  roleSorter->remove(slaves[slaveId].resources());
> +  roleSorter->remove(slaves[slaveId].total);
> 
>   slaves.erase(slaveId);
> 
> @@ -542,12 +531,16 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::resourcesRecovered(
>   // Master::offer before we received AllocatorProcess::frameworkRemoved
>   // or AllocatorProcess::frameworkDeactivated, in which case we will
>   // have already recovered all of its resources).
> -  if (frameworks.contains(frameworkId) &&
> -      sorters[frameworks[frameworkId].role()]->contains(frameworkId.value())) {
> -    const std::string& role = frameworks[frameworkId].role();
> -    sorters[role]->unallocated(frameworkId.value(), resources);
> -    sorters[role]->remove(resources);
> -    roleSorter->unallocated(role, resources);
> +  if (frameworks.contains(frameworkId)) {
> +    const std::string& role = frameworks[frameworkId].role;
> +
> +    CHECK(sorters.contains(role));
> +
> +    if (sorters[role]->contains(frameworkId.value())) {
> +      sorters[role]->unallocated(frameworkId.value(), resources);
> +      sorters[role]->remove(resources);
> +      roleSorter->unallocated(role, resources);
> +    }
>   }
> 
>   // Update resources allocatable on slave (if slave still exists,
> @@ -779,11 +772,10 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::isWhitelisted(
>     const SlaveID& slaveId)
> {
>   CHECK(initialized);
> -
>   CHECK(slaves.contains(slaveId));
> 
>   return whitelist.isNone() ||
> -         whitelist.get().contains(slaves[slaveId].hostname());
> +         whitelist.get().contains(slaves[slaveId].hostname);
> }
> 
> 
> 


[2/8] mesos git commit: Cleaned up the utility structs in the allocator.

Posted by bm...@apache.org.
Cleaned up the utility structs in the allocator.

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


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

Branch: refs/heads/master
Commit: 0cb4c9c05960a31cd507f36e9a1b7da6aa3f689f
Parents: f575ae4
Author: Benjamin Mahler <be...@gmail.com>
Authored: Tue Dec 2 17:51:55 2014 -0800
Committer: Benjamin Mahler <be...@gmail.com>
Committed: Wed Dec 3 14:59:19 2014 -0800

----------------------------------------------------------------------
 src/master/hierarchical_allocator_process.hpp | 136 ++++++++++-----------
 1 file changed, 64 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0cb4c9c0/src/master/hierarchical_allocator_process.hpp
----------------------------------------------------------------------
diff --git a/src/master/hierarchical_allocator_process.hpp b/src/master/hierarchical_allocator_process.hpp
index 4f284ce..c71739b 100644
--- a/src/master/hierarchical_allocator_process.hpp
+++ b/src/master/hierarchical_allocator_process.hpp
@@ -57,52 +57,6 @@ typedef HierarchicalAllocatorProcess<DRFSorter, DRFSorter>
 HierarchicalDRFAllocatorProcess;
 
 
-struct Slave
-{
-  Slave() {}
-
-  explicit Slave(const SlaveInfo& _info)
-    : available(_info.resources()),
-      activated(true),
-      checkpoint(_info.checkpoint()),
-      info(_info) {}
-
-  Resources resources() const { return info.resources(); }
-
-  std::string hostname() const { return info.hostname(); }
-
-  // Contains all of the resources currently free on this slave.
-  Resources available;
-
-  // Whether the slave is activated. Resources are not offered for
-  // deactivated slaves until they are reactivated.
-  bool activated;
-
-  bool checkpoint;
-private:
-  SlaveInfo info;
-};
-
-
-struct Framework
-{
-  Framework() {}
-
-  explicit Framework(const FrameworkInfo& _info)
-    : checkpoint(_info.checkpoint()),
-      info(_info) {}
-
-  std::string role() const { return info.role(); }
-
-  // Filters that have been added by this framework.
-  hashset<Filter*> filters;
-
-  bool checkpoint;
-private:
-  FrameworkInfo info;
-};
-
-
 // Implements the basic allocator algorithm - first pick a role by
 // some criteria, then pick one of their frameworks to allocate to.
 template <typename RoleSorter, typename FrameworkSorter>
@@ -202,14 +156,31 @@ protected:
   Flags flags;
   process::PID<Master> master;
 
-  // Contains all frameworks.
+  struct Framework
+  {
+    std::string role;
+    bool checkpoint;  // Whether the framework desires checkpointing.
+
+    hashset<Filter*> filters; // Active filters for the framework.
+  };
+
   hashmap<FrameworkID, Framework> frameworks;
 
   // Maps role names to the Sorter object which contains
   // all of that role's frameworks.
   hashmap<std::string, FrameworkSorter*> sorters;
 
-  // Contains all active slaves.
+  struct Slave
+  {
+    Resources total;
+    Resources available;
+
+    bool activated;  // Whether to offer resources.
+    bool checkpoint; // Whether slave supports checkpointing.
+
+    std::string hostname;
+  };
+
   hashmap<SlaveID, Slave> slaves;
 
   hashmap<std::string, RoleInfo> roles;
@@ -319,7 +290,9 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkAdded(
   sorters[role]->add(used);
   sorters[role]->allocated(frameworkId.value(), used);
 
-  frameworks[frameworkId] = Framework(frameworkInfo);
+  frameworks[frameworkId] = Framework();
+  frameworks[frameworkId].role = frameworkInfo.role();
+  frameworks[frameworkId].checkpoint = frameworkInfo.checkpoint();
 
   LOG(INFO) << "Added framework " << frameworkId;
 
@@ -335,7 +308,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkRemoved(
   CHECK(initialized);
 
   CHECK(frameworks.contains(frameworkId));
-  const std::string& role = frameworks[frameworkId].role();
+  const std::string& role = frameworks[frameworkId].role;
 
   // Might not be in 'sorters[role]' because it was previously
   // deactivated and never re-added.
@@ -381,7 +354,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkDeactivated(
   CHECK(initialized);
 
   CHECK(frameworks.contains(frameworkId));
-  const std::string& role = frameworks[frameworkId].role();
+  const std::string& role = frameworks[frameworkId].role;
 
   sorters[role]->deactivate(frameworkId.value());
 
@@ -401,6 +374,21 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::frameworkDeactivated(
 }
 
 
+namespace internal {
+
+// TODO(bmahler): Generalize this.
+template <typename Iterable>
+Resources sum(const Iterable& resources)
+{
+  Resources total;
+  foreach (const Resources& r, resources) {
+    total += r;
+  }
+  return total;
+}
+
+} // namespace internal {
+
 template <class RoleSorter, class FrameworkSorter>
 void
 HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveAdded(
@@ -409,33 +397,34 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveAdded(
     const hashmap<FrameworkID, Resources>& used)
 {
   CHECK(initialized);
-
   CHECK(!slaves.contains(slaveId));
 
-  slaves[slaveId] = Slave(slaveInfo);
-
-  roleSorter->add(slaveInfo.resources());
+  const Resources& total = slaveInfo.resources();
 
-  Resources unused = slaveInfo.resources();
+  roleSorter->add(total);
 
   foreachpair (const FrameworkID& frameworkId,
                const Resources& resources,
                used) {
     if (frameworks.contains(frameworkId)) {
-      const std::string& role = frameworks[frameworkId].role();
+      const std::string& role = frameworks[frameworkId].role;
+
       sorters[role]->add(resources);
       sorters[role]->allocated(frameworkId.value(), resources);
       roleSorter->allocated(role, resources);
     }
-
-    unused -= resources; // Only want to allocate resources that are not used!
   }
 
-  slaves[slaveId].available = unused;
+  slaves[slaveId] = Slave();
+  slaves[slaveId].total = total;
+  slaves[slaveId].available = total - internal::sum(used.values());
+  slaves[slaveId].activated = true;
+  slaves[slaveId].checkpoint = slaveInfo.checkpoint();
+  slaves[slaveId].hostname = slaveInfo.hostname();
 
-  LOG(INFO) << "Added slave " << slaveId << " (" << slaveInfo.hostname()
-            << ") with " << slaveInfo.resources() << " (and " << unused
-            << " available)";
+  LOG(INFO) << "Added slave " << slaveId << " (" << slaves[slaveId].hostname
+            << ") with " << slaves[slaveId].total
+            << " (and " << slaves[slaveId].available << " available)";
 
   allocate(slaveId);
 }
@@ -449,7 +438,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::slaveRemoved(
   CHECK(initialized);
   CHECK(slaves.contains(slaveId));
 
-  roleSorter->remove(slaves[slaveId].resources());
+  roleSorter->remove(slaves[slaveId].total);
 
   slaves.erase(slaveId);
 
@@ -542,12 +531,16 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::resourcesRecovered(
   // Master::offer before we received AllocatorProcess::frameworkRemoved
   // or AllocatorProcess::frameworkDeactivated, in which case we will
   // have already recovered all of its resources).
-  if (frameworks.contains(frameworkId) &&
-      sorters[frameworks[frameworkId].role()]->contains(frameworkId.value())) {
-    const std::string& role = frameworks[frameworkId].role();
-    sorters[role]->unallocated(frameworkId.value(), resources);
-    sorters[role]->remove(resources);
-    roleSorter->unallocated(role, resources);
+  if (frameworks.contains(frameworkId)) {
+    const std::string& role = frameworks[frameworkId].role;
+
+    CHECK(sorters.contains(role));
+
+    if (sorters[role]->contains(frameworkId.value())) {
+      sorters[role]->unallocated(frameworkId.value(), resources);
+      sorters[role]->remove(resources);
+      roleSorter->unallocated(role, resources);
+    }
   }
 
   // Update resources allocatable on slave (if slave still exists,
@@ -779,11 +772,10 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::isWhitelisted(
     const SlaveID& slaveId)
 {
   CHECK(initialized);
-
   CHECK(slaves.contains(slaveId));
 
   return whitelist.isNone() ||
-         whitelist.get().contains(slaves[slaveId].hostname());
+         whitelist.get().contains(slaves[slaveId].hostname);
 }
 
 


[5/8] mesos git commit: Cleaned up the allocator method naming.

Posted by bm...@apache.org.
http://git-wip-us.apache.org/repos/asf/mesos/blob/9b84b52b/src/tests/master_authorization_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_authorization_tests.cpp b/src/tests/master_authorization_tests.cpp
index 796fe21..42ffe24 100644
--- a/src/tests/master_authorization_tests.cpp
+++ b/src/tests/master_authorization_tests.cpp
@@ -272,15 +272,15 @@ TEST_F(MasterAuthorizationTest, KillTask)
   AWAIT_READY(status);
   EXPECT_EQ(TASK_KILLED, status.get().state());
 
-  Future<Nothing> resourcesRecovered =
-    FUTURE_DISPATCH(_, &AllocatorProcess::resourcesRecovered);
+  Future<Nothing> recoverResources =
+    FUTURE_DISPATCH(_, &AllocatorProcess::recoverResources);
 
   // Now complete authorization.
   promise.set(true);
 
   // No task launch should happen resulting in all resources being
   // returned to the allocator.
-  AWAIT_READY(resourcesRecovered);
+  AWAIT_READY(recoverResources);
 
   driver.stop();
   driver.join();
@@ -348,8 +348,8 @@ TEST_F(MasterAuthorizationTest, SlaveRemoved)
   EXPECT_CALL(sched, statusUpdate(&driver, _))
     .WillOnce(FutureArg<1>(&status));
 
-  Future<Nothing> resourcesRecovered =
-    FUTURE_DISPATCH(_, &AllocatorProcess::resourcesRecovered);
+  Future<Nothing> recoverResources =
+    FUTURE_DISPATCH(_, &AllocatorProcess::recoverResources);
 
   // Now complete authorization.
   promise.set(true);
@@ -360,7 +360,7 @@ TEST_F(MasterAuthorizationTest, SlaveRemoved)
 
   // No task launch should happen resulting in all resources being
   // returned to the allocator.
-  AWAIT_READY(resourcesRecovered);
+  AWAIT_READY(recoverResources);
 
   driver.stop();
   driver.join();
@@ -422,20 +422,20 @@ TEST_F(MasterAuthorizationTest, SlaveDisconnected)
   EXPECT_CALL(sched, slaveLost(&driver, _))
     .Times(AtMost(1));
 
-  Future<Nothing> slaveDeactivated =
-    FUTURE_DISPATCH(_, &AllocatorProcess::slaveDeactivated);
+  Future<Nothing> deactivateSlave =
+    FUTURE_DISPATCH(_, &AllocatorProcess::deactivateSlave);
 
   // Now stop the slave.
   Stop(slave.get());
 
-  AWAIT_READY(slaveDeactivated);
+  AWAIT_READY(deactivateSlave);
 
   Future<TaskStatus> status;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
     .WillOnce(FutureArg<1>(&status));
 
-  Future<Nothing> resourcesRecovered =
-    FUTURE_DISPATCH(_, &AllocatorProcess::resourcesRecovered);
+  Future<Nothing> recoverResources =
+    FUTURE_DISPATCH(_, &AllocatorProcess::recoverResources);
 
   // Now complete authorization.
   promise.set(true);
@@ -446,7 +446,7 @@ TEST_F(MasterAuthorizationTest, SlaveDisconnected)
 
   // No task launch should happen resulting in all resources being
   // returned to the allocator.
-  AWAIT_READY(resourcesRecovered);
+  AWAIT_READY(recoverResources);
 
   driver.stop();
   driver.join();
@@ -501,24 +501,24 @@ TEST_F(MasterAuthorizationTest, FrameworkRemoved)
   // Wait until authorization is in progress.
   AWAIT_READY(authorize);
 
-  Future<Nothing> frameworkRemoved =
-    FUTURE_DISPATCH(_, &AllocatorProcess::frameworkRemoved);
+  Future<Nothing> removeFramework =
+    FUTURE_DISPATCH(_, &AllocatorProcess::removeFramework);
 
   // Now stop the framework.
   driver.stop();
   driver.join();
 
-  AWAIT_READY(frameworkRemoved);
+  AWAIT_READY(removeFramework);
 
-  Future<Nothing> resourcesRecovered =
-    FUTURE_DISPATCH(_, &AllocatorProcess::resourcesRecovered);
+  Future<Nothing> recoverResources =
+    FUTURE_DISPATCH(_, &AllocatorProcess::recoverResources);
 
   // Now complete authorization.
   promise.set(true);
 
   // No task launch should happen resulting in all resources being
   // returned to the allocator.
-  AWAIT_READY(resourcesRecovered);
+  AWAIT_READY(recoverResources);
 
   Shutdown(); // Must shutdown before 'containerizer' gets deallocated.
 }
@@ -926,15 +926,15 @@ TEST_F(MasterAuthorizationTest, FrameworkRemovedBeforeRegistration)
   Clock::settle();
   Clock::resume();
 
-  Future<Nothing> frameworkRemoved =
-    FUTURE_DISPATCH(_, &AllocatorProcess::frameworkRemoved);
+  Future<Nothing> removeFramework =
+    FUTURE_DISPATCH(_, &AllocatorProcess::removeFramework);
 
   // Now complete authorization.
   promise.set(true);
 
   // When the master tries to link to a non-existent framework PID
   // it should realize the framework is gone and remove it.
-  AWAIT_READY(frameworkRemoved);
+  AWAIT_READY(removeFramework);
 
   Shutdown();
 }
@@ -986,15 +986,15 @@ TEST_F(MasterAuthorizationTest, FrameworkRemovedBeforeReregistration)
   // Wait until the second authorization attempt is in progress.
   AWAIT_READY(authorize2);
 
-  Future<Nothing> frameworkRemoved =
-    FUTURE_DISPATCH(_, &AllocatorProcess::frameworkRemoved);
+  Future<Nothing> removeFramework =
+    FUTURE_DISPATCH(_, &AllocatorProcess::removeFramework);
 
   // Stop the framework.
   driver.stop();
   driver.join();
 
   // Wait until the framework is removed.
-  AWAIT_READY(frameworkRemoved);
+  AWAIT_READY(removeFramework);
 
   // Now complete the second authorization attempt.
   promise2.set(true);

http://git-wip-us.apache.org/repos/asf/mesos/blob/9b84b52b/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index 065fc3e..56cba65 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -1349,8 +1349,8 @@ TEST_F(MasterTest, LaunchAcrossSlavesTest)
   combinedOffers.push_back(offers1.get()[0].id());
   combinedOffers.push_back(offers2.get()[0].id());
 
-  Future<Nothing> resourcesRecovered =
-    FUTURE_DISPATCH(_, &AllocatorProcess::resourcesRecovered);
+  Future<Nothing> recoverResources =
+    FUTURE_DISPATCH(_, &AllocatorProcess::recoverResources);
 
   driver.launchTasks(combinedOffers, tasks);
 
@@ -1359,7 +1359,7 @@ TEST_F(MasterTest, LaunchAcrossSlavesTest)
   EXPECT_EQ(TaskStatus::REASON_INVALID_OFFERS, status.get().reason());
 
   // The resources of the invalid offers should be recovered.
-  AWAIT_READY(resourcesRecovered);
+  AWAIT_READY(recoverResources);
 
   EXPECT_CALL(exec, shutdown(_))
     .Times(AtMost(1));
@@ -1429,8 +1429,8 @@ TEST_F(MasterTest, LaunchDuplicateOfferTest)
   EXPECT_CALL(sched, statusUpdate(&driver, _))
     .WillOnce(FutureArg<1>(&status));
 
-  Future<Nothing> resourcesRecovered =
-    FUTURE_DISPATCH(_, &AllocatorProcess::resourcesRecovered);
+  Future<Nothing> recoverResources =
+    FUTURE_DISPATCH(_, &AllocatorProcess::recoverResources);
 
   driver.launchTasks(combinedOffers, tasks);
 
@@ -1439,7 +1439,7 @@ TEST_F(MasterTest, LaunchDuplicateOfferTest)
   EXPECT_EQ(TaskStatus::REASON_INVALID_OFFERS, status.get().reason());
 
   // The resources of the invalid offers should be recovered.
-  AWAIT_READY(resourcesRecovered);
+  AWAIT_READY(recoverResources);
 
   EXPECT_CALL(exec, shutdown(_))
     .Times(AtMost(1));
@@ -2180,8 +2180,8 @@ TEST_F(MasterTest, OfferTimeout)
   EXPECT_CALL(sched, offerRescinded(&driver, _))
     .WillOnce(FutureSatisfy(&offerRescinded));
 
-  Future<Nothing> resourcesRecovered =
-    FUTURE_DISPATCH(_, &AllocatorProcess::resourcesRecovered);
+  Future<Nothing> recoverResources =
+    FUTURE_DISPATCH(_, &AllocatorProcess::recoverResources);
 
   driver.start();
 
@@ -2197,7 +2197,7 @@ TEST_F(MasterTest, OfferTimeout)
 
   AWAIT_READY(offerRescinded);
 
-  AWAIT_READY(resourcesRecovered);
+  AWAIT_READY(recoverResources);
 
   // Expect that the resources are re-offered to the framework after
   // the rescind.
@@ -2466,8 +2466,8 @@ TEST_F(MasterTest, ReleaseResourcesForTerminalTaskWithPendingUpdates)
   // Ensure status update manager handles TASK_FINISHED update.
   AWAIT_READY(__statusUpdate2);
 
-  Future<Nothing> resourcesRecovered = FUTURE_DISPATCH(
-      _, &AllocatorProcess::resourcesRecovered);
+  Future<Nothing> recoverResources = FUTURE_DISPATCH(
+      _, &AllocatorProcess::recoverResources);
 
   // Advance the clock so that the status update manager resends
   // TASK_RUNNING update with 'latest_state' as TASK_FINISHED.
@@ -2476,7 +2476,7 @@ TEST_F(MasterTest, ReleaseResourcesForTerminalTaskWithPendingUpdates)
   Clock::resume();
 
   // Ensure the resources are recovered.
-  AWAIT_READY(resourcesRecovered);
+  AWAIT_READY(recoverResources);
 
   EXPECT_CALL(exec, shutdown(_))
     .Times(AtMost(1));
@@ -2712,15 +2712,15 @@ TEST_F(MasterTest, SlaveActiveEndpoint)
 
   ASSERT_SOME_EQ(JSON::Boolean(true), status);
 
-  Future<Nothing> slaveDeactivated =
-    FUTURE_DISPATCH(_, &AllocatorProcess::slaveDeactivated);
+  Future<Nothing> deactivateSlave =
+    FUTURE_DISPATCH(_, &AllocatorProcess::deactivateSlave);
 
   // Inject a slave exited event at the master causing the master
   // to mark the slave as disconnected.
   process::inject::exited(slaveRegisteredMessage.get().to, master.get());
 
   // Wait until master deactivates the slave.
-  AWAIT_READY(slaveDeactivated);
+  AWAIT_READY(deactivateSlave);
 
   // Verify slave is inactive.
   response = process::http::get(master.get(), "state.json");

http://git-wip-us.apache.org/repos/asf/mesos/blob/9b84b52b/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 94fd753..02a7129 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -658,40 +658,40 @@ public:
     ON_CALL(*this, initialize(_, _, _))
       .WillByDefault(InvokeInitialize(this));
 
-    ON_CALL(*this, frameworkAdded(_, _, _))
+    ON_CALL(*this, addFramework(_, _, _))
       .WillByDefault(InvokeFrameworkAdded(this));
 
-    ON_CALL(*this, frameworkRemoved(_))
+    ON_CALL(*this, removeFramework(_))
       .WillByDefault(InvokeFrameworkRemoved(this));
 
-    ON_CALL(*this, frameworkActivated(_, _))
+    ON_CALL(*this, activateFramework(_, _))
       .WillByDefault(InvokeFrameworkActivated(this));
 
-    ON_CALL(*this, frameworkDeactivated(_))
+    ON_CALL(*this, deactivateFramework(_))
       .WillByDefault(InvokeFrameworkDeactivated(this));
 
-    ON_CALL(*this, slaveAdded(_, _, _, _))
+    ON_CALL(*this, addSlave(_, _, _, _))
       .WillByDefault(InvokeSlaveAdded(this));
 
-    ON_CALL(*this, slaveRemoved(_))
+    ON_CALL(*this, removeSlave(_))
       .WillByDefault(InvokeSlaveRemoved(this));
 
-    ON_CALL(*this, slaveDeactivated(_))
-      .WillByDefault(InvokeSlaveDeactivated(this));
-
-    ON_CALL(*this, slaveActivated(_))
+    ON_CALL(*this, activateSlave(_))
       .WillByDefault(InvokeSlaveReactivated(this));
 
+    ON_CALL(*this, deactivateSlave(_))
+      .WillByDefault(InvokeSlaveDeactivated(this));
+
     ON_CALL(*this, updateWhitelist(_))
       .WillByDefault(InvokeUpdateWhitelist(this));
 
-    ON_CALL(*this, resourcesRequested(_, _))
+    ON_CALL(*this, requestResources(_, _))
       .WillByDefault(InvokeResourcesRequested(this));
 
-    ON_CALL(*this, resourcesRecovered(_, _, _, _))
+    ON_CALL(*this, recoverResources(_, _, _, _))
       .WillByDefault(InvokeResourcesRecovered(this));
 
-    ON_CALL(*this, offersRevived(_))
+    ON_CALL(*this, reviveOffers(_))
       .WillByDefault(InvokeOffersRevived(this));
   }
 
@@ -706,50 +706,50 @@ public:
       const process::PID<master::Master>&,
       const hashmap<std::string, RoleInfo>&));
 
-  MOCK_METHOD3(frameworkAdded, void(
+  MOCK_METHOD3(addFramework, void(
       const FrameworkID&,
       const FrameworkInfo&,
       const Resources&));
 
-  MOCK_METHOD1(frameworkRemoved, void(
+  MOCK_METHOD1(removeFramework, void(
       const FrameworkID&));
 
-  MOCK_METHOD2(frameworkActivated, void(
+  MOCK_METHOD2(activateFramework, void(
       const FrameworkID&,
       const FrameworkInfo&));
 
-  MOCK_METHOD1(frameworkDeactivated, void(
+  MOCK_METHOD1(deactivateFramework, void(
       const FrameworkID&));
 
-  MOCK_METHOD4(slaveAdded, void(
+  MOCK_METHOD4(addSlave, void(
       const SlaveID&,
       const SlaveInfo&,
       const Resources&,
       const hashmap<FrameworkID, Resources>&));
 
-  MOCK_METHOD1(slaveRemoved, void(
+  MOCK_METHOD1(removeSlave, void(
       const SlaveID&));
 
-  MOCK_METHOD1(slaveDeactivated, void(
+  MOCK_METHOD1(activateSlave, void(
       const SlaveID&));
 
-  MOCK_METHOD1(slaveActivated, void(
+  MOCK_METHOD1(deactivateSlave, void(
       const SlaveID&));
 
   MOCK_METHOD1(updateWhitelist, void(
       const Option<hashset<std::string> >&));
 
-  MOCK_METHOD2(resourcesRequested, void(
+  MOCK_METHOD2(requestResources, void(
       const FrameworkID&,
       const std::vector<Request>&));
 
-  MOCK_METHOD4(resourcesRecovered, void(
+  MOCK_METHOD4(recoverResources, void(
       const FrameworkID&,
       const SlaveID&,
       const Resources&,
       const Option<Filters>& filters));
 
-  MOCK_METHOD1(offersRevived, void(const FrameworkID&));
+  MOCK_METHOD1(reviveOffers, void(const FrameworkID&));
 
   T real;
 };
@@ -761,9 +761,9 @@ AllocatorTypes;
 
 // The following actions make up for the fact that DoDefault
 // cannot be used inside a DoAll, for example:
-// EXPECT_CALL(allocator, frameworkAdded(_, _, _))
+// EXPECT_CALL(allocator, addFramework(_, _, _))
 //   .WillOnce(DoAll(InvokeFrameworkAdded(&allocator),
-//                   FutureSatisfy(&frameworkAdded)));
+//                   FutureSatisfy(&addFramework)));
 
 ACTION_P(InvokeInitialize, allocator)
 {
@@ -780,7 +780,7 @@ ACTION_P(InvokeFrameworkAdded, allocator)
 {
   process::dispatch(
       allocator->real,
-      &master::allocator::AllocatorProcess::frameworkAdded,
+      &master::allocator::AllocatorProcess::addFramework,
       arg0,
       arg1,
       arg2);
@@ -791,7 +791,7 @@ ACTION_P(InvokeFrameworkRemoved, allocator)
 {
   process::dispatch(
       allocator->real,
-      &master::allocator::AllocatorProcess::frameworkRemoved, arg0);
+      &master::allocator::AllocatorProcess::removeFramework, arg0);
 }
 
 
@@ -799,7 +799,7 @@ ACTION_P(InvokeFrameworkActivated, allocator)
 {
   process::dispatch(
       allocator->real,
-      &master::allocator::AllocatorProcess::frameworkActivated,
+      &master::allocator::AllocatorProcess::activateFramework,
       arg0,
       arg1);
 }
@@ -809,7 +809,7 @@ ACTION_P(InvokeFrameworkDeactivated, allocator)
 {
   process::dispatch(
       allocator->real,
-      &master::allocator::AllocatorProcess::frameworkDeactivated,
+      &master::allocator::AllocatorProcess::deactivateFramework,
       arg0);
 }
 
@@ -818,7 +818,7 @@ ACTION_P(InvokeSlaveAdded, allocator)
 {
   process::dispatch(
       allocator->real,
-      &master::allocator::AllocatorProcess::slaveAdded,
+      &master::allocator::AllocatorProcess::addSlave,
       arg0,
       arg1,
       arg2,
@@ -830,25 +830,25 @@ ACTION_P(InvokeSlaveRemoved, allocator)
 {
   process::dispatch(
       allocator->real,
-      &master::allocator::AllocatorProcess::slaveRemoved,
+      &master::allocator::AllocatorProcess::removeSlave,
       arg0);
 }
 
 
-ACTION_P(InvokeSlaveDeactivated, allocator)
+ACTION_P(InvokeSlaveReactivated, allocator)
 {
   process::dispatch(
       allocator->real,
-      &master::allocator::AllocatorProcess::slaveDeactivated,
+      &master::allocator::AllocatorProcess::activateSlave,
       arg0);
 }
 
 
-ACTION_P(InvokeSlaveReactivated, allocator)
+ACTION_P(InvokeSlaveDeactivated, allocator)
 {
   process::dispatch(
       allocator->real,
-      &master::allocator::AllocatorProcess::slaveActivated,
+      &master::allocator::AllocatorProcess::deactivateSlave,
       arg0);
 }
 
@@ -866,7 +866,7 @@ ACTION_P(InvokeResourcesRequested, allocator)
 {
   process::dispatch(
       allocator->real,
-      &master::allocator::AllocatorProcess::resourcesRequested,
+      &master::allocator::AllocatorProcess::requestResources,
       arg0,
       arg1);
 }
@@ -876,7 +876,7 @@ ACTION_P(InvokeResourcesRecovered, allocator)
 {
   process::dispatch(
       allocator->real,
-      &master::allocator::AllocatorProcess::resourcesRecovered,
+      &master::allocator::AllocatorProcess::recoverResources,
       arg0,
       arg1,
       arg2,
@@ -891,7 +891,7 @@ ACTION_P2(InvokeResourcesRecoveredWithFilters, allocator, timeout)
 
   process::dispatch(
       allocator->real,
-      &master::allocator::AllocatorProcess::resourcesRecovered,
+      &master::allocator::AllocatorProcess::recoverResources,
       arg0,
       arg1,
       arg2,
@@ -903,7 +903,7 @@ ACTION_P(InvokeOffersRevived, allocator)
 {
   process::dispatch(
       allocator->real,
-      &master::allocator::AllocatorProcess::offersRevived,
+      &master::allocator::AllocatorProcess::reviveOffers,
       arg0);
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/9b84b52b/src/tests/partition_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/partition_tests.cpp b/src/tests/partition_tests.cpp
index 0dc1a92..fea7801 100644
--- a/src/tests/partition_tests.cpp
+++ b/src/tests/partition_tests.cpp
@@ -444,8 +444,8 @@ TEST_F(PartitionTest, OneWayPartitionMasterToSlave)
 
   AWAIT_READY(ping);
 
-  Future<Nothing> slaveDeactivated =
-    FUTURE_DISPATCH(_, &AllocatorProcess::slaveDeactivated);
+  Future<Nothing> deactivateSlave =
+    FUTURE_DISPATCH(_, &AllocatorProcess::deactivateSlave);
 
   // Inject a slave exited event at the master causing the master
   // to mark the slave as disconnected. The slave should not notice
@@ -453,7 +453,7 @@ TEST_F(PartitionTest, OneWayPartitionMasterToSlave)
   process::inject::exited(slaveRegisteredMessage.get().to, master.get());
 
   // Wait until master deactivates the slave.
-  AWAIT_READY(slaveDeactivated);
+  AWAIT_READY(deactivateSlave);
 
   Future<SlaveReregisteredMessage> slaveReregisteredMessage =
     FUTURE_PROTOBUF(SlaveReregisteredMessage(), _, _);

http://git-wip-us.apache.org/repos/asf/mesos/blob/9b84b52b/src/tests/rate_limiting_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/rate_limiting_tests.cpp b/src/tests/rate_limiting_tests.cpp
index e1fd3c2..7f5ca25 100644
--- a/src/tests/rate_limiting_tests.cpp
+++ b/src/tests/rate_limiting_tests.cpp
@@ -189,8 +189,8 @@ TEST_F(RateLimitingTest, NoRateLimiting)
     EXPECT_EQ(1, metrics.values[messages_processed].as<JSON::Number>().value);
   }
 
-  Future<Nothing> frameworkRemoved =
-    FUTURE_DISPATCH(_, &AllocatorProcess::frameworkRemoved);
+  Future<Nothing> removeFramework =
+    FUTURE_DISPATCH(_, &AllocatorProcess::removeFramework);
 
   driver->stop();
   driver->join();
@@ -200,7 +200,7 @@ TEST_F(RateLimitingTest, NoRateLimiting)
   // 'sched' that reaches Master after its registration) gets
   // processed without Clock advances proves that the framework is
   // given unlimited rate.
-  AWAIT_READY(frameworkRemoved);
+  AWAIT_READY(removeFramework);
 
   // For metrics endpoint.
   Clock::advance(Milliseconds(501));
@@ -568,8 +568,8 @@ TEST_F(RateLimitingTest, DifferentPrincipalFrameworks)
 
   // 3. Remove a framework and its message counters are deleted while
   // the other framework's counters stay.
-  Future<Nothing> frameworkRemoved =
-    FUTURE_DISPATCH(_, &AllocatorProcess::frameworkRemoved);
+  Future<Nothing> removeFramework =
+    FUTURE_DISPATCH(_, &AllocatorProcess::removeFramework);
 
   driver1->stop();
   driver1->join();
@@ -578,7 +578,7 @@ TEST_F(RateLimitingTest, DifferentPrincipalFrameworks)
   // No need to advance again because we already advanced 1sec for
   // sched2 so the RateLimiter for sched1 doesn't impose a delay this
   // time.
-  AWAIT_READY(frameworkRemoved);
+  AWAIT_READY(removeFramework);
 
   // Settle to avoid the race between the removal of the counters and
   // the metrics endpoint query.
@@ -735,8 +735,8 @@ TEST_F(RateLimitingTest, SamePrincipalFrameworks)
 
   AWAIT_READY(duplicateFrameworkRegisteredMessage2);
 
-  Future<Nothing> frameworkRemoved =
-    FUTURE_DISPATCH(_, &AllocatorProcess::frameworkRemoved);
+  Future<Nothing> removeFramework =
+    FUTURE_DISPATCH(_, &AllocatorProcess::removeFramework);
 
   driver1->stop();
   driver1->join();
@@ -746,7 +746,7 @@ TEST_F(RateLimitingTest, SamePrincipalFrameworks)
   Clock::settle();
   Clock::advance(Seconds(1));
 
-  AWAIT_READY(frameworkRemoved);
+  AWAIT_READY(removeFramework);
 
   // Message counters are not removed after the first framework is
   // unregistered.

http://git-wip-us.apache.org/repos/asf/mesos/blob/9b84b52b/src/tests/resource_offers_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resource_offers_tests.cpp b/src/tests/resource_offers_tests.cpp
index c8c25d7..c4afc38 100644
--- a/src/tests/resource_offers_tests.cpp
+++ b/src/tests/resource_offers_tests.cpp
@@ -801,7 +801,7 @@ TEST_F(ResourceOffersTest, Request)
   MesosSchedulerDriver driver(
       &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _))
+  EXPECT_CALL(allocator, addFramework(_, _, _))
     .Times(1);
 
   Future<Nothing> registered;
@@ -818,7 +818,7 @@ TEST_F(ResourceOffersTest, Request)
   sent.push_back(request);
 
   Future<vector<Request>> received;
-  EXPECT_CALL(allocator, resourcesRequested(_, _))
+  EXPECT_CALL(allocator, requestResources(_, _))
     .WillOnce(FutureArg<1>(&received));
 
   driver.requestResources(sent);
@@ -828,10 +828,10 @@ TEST_F(ResourceOffersTest, Request)
   EXPECT_NE(0u, received.get().size());
   EXPECT_EQ(request.slave_id(), received.get()[0].slave_id());
 
-  EXPECT_CALL(allocator, frameworkDeactivated(_))
+  EXPECT_CALL(allocator, deactivateFramework(_))
     .Times(AtMost(1)); // Races with shutting down the cluster.
 
-  EXPECT_CALL(allocator, frameworkRemoved(_))
+  EXPECT_CALL(allocator, removeFramework(_))
     .Times(AtMost(1)); // Races with shutting down the cluster.
 
   driver.stop();

http://git-wip-us.apache.org/repos/asf/mesos/blob/9b84b52b/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index 6cf5fb8..8bd0f14 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -2207,7 +2207,7 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileTasksMissingFromSlave)
 
   slave::Flags flags = this->CreateSlaveFlags();
 
-  EXPECT_CALL(allocator, slaveAdded(_, _, _, _));
+  EXPECT_CALL(allocator, addSlave(_, _, _, _));
 
   Try<TypeParam*> containerizer1 = TypeParam::create(flags, true);
   ASSERT_SOME(containerizer1);
@@ -2225,7 +2225,7 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileTasksMissingFromSlave)
   MesosSchedulerDriver driver(
       &sched, frameworkInfo, master.get(), DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(allocator, frameworkAdded(_, _, _));
+  EXPECT_CALL(allocator, addFramework(_, _, _));
 
   Future<FrameworkID> frameworkId;
   EXPECT_CALL(sched, registered(_, _, _))
@@ -2258,7 +2258,7 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileTasksMissingFromSlave)
   // Wait for the ACK to be checkpointed.
   AWAIT_READY(_statusUpdateAcknowledgement);
 
-  EXPECT_CALL(allocator, slaveDeactivated(_));
+  EXPECT_CALL(allocator, deactivateSlave(_));
 
   this->Stop(slave.get());
   delete containerizer1.get();
@@ -2299,8 +2299,8 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileTasksMissingFromSlave)
   Future<SlaveReregisteredMessage> slaveReregisteredMessage =
     FUTURE_PROTOBUF(SlaveReregisteredMessage(), _, _);
 
-  EXPECT_CALL(allocator, slaveActivated(_));
-  EXPECT_CALL(allocator, resourcesRecovered(_, _, _, _));
+  EXPECT_CALL(allocator, activateSlave(_));
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _));
 
   Future<TaskStatus> status;
   EXPECT_CALL(sched, statusUpdate(_, _))
@@ -2344,14 +2344,14 @@ TYPED_TEST(SlaveRecoveryTest, ReconcileTasksMissingFromSlave)
 
   Clock::resume();
 
-  EXPECT_CALL(allocator, frameworkDeactivated(_))
+  EXPECT_CALL(allocator, deactivateFramework(_))
     .WillRepeatedly(Return());
-  EXPECT_CALL(allocator, frameworkRemoved(_))
+  EXPECT_CALL(allocator, removeFramework(_))
     .WillRepeatedly(Return());
 
   // If there was an outstanding offer, we can get a call to
-  // resourcesRecovered when we stop the scheduler.
-  EXPECT_CALL(allocator, resourcesRecovered(_, _, _, _))
+  // recoverResources when we stop the scheduler.
+  EXPECT_CALL(allocator, recoverResources(_, _, _, _))
     .WillRepeatedly(Return());
 
   driver.stop();