You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by gr...@apache.org on 2018/06/21 00:58:18 UTC

[1/4] mesos git commit: Fixed a bug in `createStrippedScalarQuantity()`.

Repository: mesos
Updated Branches:
  refs/heads/master 4ea9c7f96 -> 79547292b


Fixed a bug in `createStrippedScalarQuantity()`.

This patch fixes `createStrippedScalarQuantity()` by
stripping the revocable field in resources.

Also added a test.

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


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

Branch: refs/heads/master
Commit: b0888195edf50fa8cf986525300dfd4f9420d812
Parents: 4ea9c7f
Author: Meng Zhu <mz...@mesosphere.io>
Authored: Wed Jun 20 16:59:54 2018 -0700
Committer: Greg Mann <gr...@gmail.com>
Committed: Wed Jun 20 17:01:52 2018 -0700

----------------------------------------------------------------------
 src/common/resources.cpp      |  1 +
 src/tests/resources_tests.cpp | 13 +++++++++++++
 src/v1/resources.cpp          |  1 +
 3 files changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b0888195/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index b139201..8518abf 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -1646,6 +1646,7 @@ Resources Resources::createStrippedScalarQuantity() const
 
       scalar.clear_disk();
       scalar.clear_shared();
+      scalar.clear_revocable();
       stripped.add(scalar);
     }
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0888195/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index 2da5836..b0e28e2 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -2683,6 +2683,19 @@ TEST(ResourcesOperationTest, StrippedResourcesNonScalar)
 }
 
 
+TEST(ResourceOperationTest, StrippedResourcesRevocable)
+{
+  Resource plain = Resources::parse("cpus", "1", "*").get();
+
+  Resource revocable = plain;
+  revocable.mutable_revocable();
+
+  Resources stripped = Resources(revocable).createStrippedScalarQuantity();
+
+  EXPECT_EQ(Resources(plain), stripped);
+}
+
+
 TEST(ResourcesOperationTest, CreatePersistentVolumeFromMount)
 {
   Resource::DiskInfo::Source source = createDiskSourceMount("mnt");

http://git-wip-us.apache.org/repos/asf/mesos/blob/b0888195/src/v1/resources.cpp
----------------------------------------------------------------------
diff --git a/src/v1/resources.cpp b/src/v1/resources.cpp
index 5079327..04a45a5 100644
--- a/src/v1/resources.cpp
+++ b/src/v1/resources.cpp
@@ -1664,6 +1664,7 @@ Resources Resources::createStrippedScalarQuantity() const
 
       scalar.clear_disk();
       scalar.clear_shared();
+      scalar.clear_revocable();
       stripped.add(scalar);
     }
   }


[4/4] mesos git commit: Added a test to verify the `min_allocatable_resources` flag.

Posted by gr...@apache.org.
Added a test to verify the `min_allocatable_resources` flag.

This test verifies that the allocator honors the
`min_allocatable_resources` flag and only offers resources
that contain at least one of the specified resource quantities.

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


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

Branch: refs/heads/master
Commit: 79547292b58eafe1fbfacce4844533a677084bbe
Parents: a7e3136
Author: Meng Zhu <mz...@mesosphere.io>
Authored: Wed Jun 20 17:00:18 2018 -0700
Committer: Greg Mann <gr...@gmail.com>
Committed: Wed Jun 20 17:48:14 2018 -0700

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


http://git-wip-us.apache.org/repos/asf/mesos/blob/79547292/src/tests/master_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_allocator_tests.cpp b/src/tests/master_allocator_tests.cpp
index 58821a4..824a755 100644
--- a/src/tests/master_allocator_tests.cpp
+++ b/src/tests/master_allocator_tests.cpp
@@ -1235,6 +1235,111 @@ TYPED_TEST(MasterAllocatorTest, MemoryOnlyOfferedAndTaskLaunched)
 }
 
 
+// This test verifies that the allocator honors the `min_allocatable_resources`
+// flag and only offers resources that are more than at least one of the
+// specified resources quantities.
+TYPED_TEST(MasterAllocatorTest, MinAllocatableResources)
+{
+  Clock::pause();
+
+  master::Flags masterFlags = this->CreateMasterFlags();
+  masterFlags.min_allocatable_resources = "cpus:1;mem:100|disk:100";
+
+  TestAllocator<TypeParam> allocator;
+
+  Try<Owned<cluster::Master>> master =
+    this->StartMaster(&allocator, masterFlags);
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  // Prevent agents from destruction until end of test.
+  vector<Owned<cluster::Slave>> slaves;
+
+  // Add agents with non-allocatable resources.
+  vector<string> nonAllocatableAgentResources = {
+    "cpus:1;mem:0;disk:0",
+    "cpus:0;mem:100;disk:0",
+    "cpus:0.5;mem:200;disk:0",
+    "cpus:0;mem:0;disk:50"};
+
+  foreach (const string& resourcesString, nonAllocatableAgentResources) {
+    Future<SlaveRegisteredMessage> slaveRegisteredMessage =
+      FUTURE_PROTOBUF(SlaveRegisteredMessage(), _, _);
+
+    slave::Flags flags = this->CreateSlaveFlags();
+    flags.resources = resourcesString;
+
+    Try<Owned<cluster::Slave>> slave = this->StartSlave(detector.get(), flags);
+    ASSERT_SOME(slave);
+
+    // Advance the clock to trigger agent registration.
+    Clock::advance(flags.registration_backoff_factor);
+    Clock::settle();
+
+    AWAIT_READY(slaveRegisteredMessage);
+
+    slaves.push_back(slave.get());
+  }
+
+  // Add agents with allocatable resources.
+  vector<string> allocatableAgentResources = {
+    "cpus:1;mem:100;disk:0",
+    "cpus:0;mem:0;disk:100",
+    "cpus:0.5;mem:0;disk:100",
+    "cpus:1;mem:100;disk:50",
+  };
+
+  hashset<SlaveID> allocatableAgents;
+  foreach (const string& resourcesString, allocatableAgentResources) {
+    Future<SlaveRegisteredMessage> slaveRegisteredMessage =
+      FUTURE_PROTOBUF(SlaveRegisteredMessage(), _, _);
+
+    slave::Flags flags = this->CreateSlaveFlags();
+    flags.resources = resourcesString;
+
+    Try<Owned<cluster::Slave>> slave = this->StartSlave(detector.get(), flags);
+    ASSERT_SOME(slave);
+
+    // Advance the clock to trigger agent registration.
+    Clock::advance(flags.registration_backoff_factor);
+    Clock::settle();
+
+    AWAIT_READY(slaveRegisteredMessage);
+
+    slaves.push_back(slave.get());
+    allocatableAgents.insert(slaveRegisteredMessage->slave_id());
+  }
+
+  // Add a framework. This will trigger an event-driven allocation.
+  // Since it is the only framework, it will get all offers.
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(_, _, _));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  driver.start();
+
+  AWAIT_READY(offers);
+
+  // All allocatable agents are offered. None of the non-allocatable
+  // agents should be offered.
+  EXPECT_EQ(offers->size(), allocatableAgents.size());
+
+  foreach (const Offer& offer, offers.get()) {
+    EXPECT_TRUE(allocatableAgents.count(offer.slave_id()) != 0);
+  }
+
+  driver.stop();
+  driver.join();
+}
+
+
 // Checks that changes to the whitelist are sent to the allocator.
 // The allocator whitelisting is tested in the allocator unit tests.
 // TODO(bmahler): Move this to a whitelist unit test.


[2/4] mesos git commit: Added a resource utility `isScalarQuantity`.

Posted by gr...@apache.org.
Added a resource utility `isScalarQuantity`.

`isScalarQuantity()` checks if a `Resources` object
is a "pure" scalar quantity; i.e., its resources only have
name, type (set to scalar) and scalar fields set.

Also added tests.

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


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

Branch: refs/heads/master
Commit: c3709616e75ab540b0bba45a5096e2d90fe6c133
Parents: b088819
Author: Meng Zhu <mz...@mesosphere.io>
Authored: Wed Jun 20 16:59:58 2018 -0700
Committer: Greg Mann <gr...@gmail.com>
Committed: Wed Jun 20 17:02:10 2018 -0700

----------------------------------------------------------------------
 include/mesos/resources.hpp    |  5 +++++
 include/mesos/v1/resources.hpp |  5 +++++
 src/common/resources.cpp       | 11 +++++++++++
 src/tests/resources_tests.cpp  | 31 +++++++++++++++++++++++++++++++
 src/v1/resources.cpp           | 11 +++++++++++
 5 files changed, 63 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c3709616/include/mesos/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/resources.hpp b/include/mesos/resources.hpp
index 7afe0d8..bd6d6d6 100644
--- a/include/mesos/resources.hpp
+++ b/include/mesos/resources.hpp
@@ -324,6 +324,11 @@ public:
   // Tests if the given Resource object is shared.
   static bool isShared(const Resource& resource);
 
+  // Tests if the given Resources object is a "pure" scalar quantity which
+  // consists of resource objects with ONLY name, type (set to "Scalar")
+  // and scalar fields set.
+  static bool isScalarQuantity(const Resources& resources);
+
   // Tests if the given Resource object has refined reservations.
   static bool hasRefinedReservations(const Resource& resource);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c3709616/include/mesos/v1/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/v1/resources.hpp b/include/mesos/v1/resources.hpp
index c712057..c065dd1 100644
--- a/include/mesos/v1/resources.hpp
+++ b/include/mesos/v1/resources.hpp
@@ -324,6 +324,11 @@ public:
   // Tests if the given Resource object is shared.
   static bool isShared(const Resource& resource);
 
+  // Tests if the given Resources object is a "pure" scalar quantity which
+  // only consists of resource object with ONLY name, type (set to "Scalar")
+  // and scalar fields set.
+  static bool isScalarQuantity(const Resources& resources);
+
   // Tests if the given Resource object has refined reservations.
   static bool hasRefinedReservations(const Resource& resource);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c3709616/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index 8518abf..b9f1c2d 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -1248,6 +1248,17 @@ bool Resources::isShared(const Resource& resource)
 }
 
 
+bool Resources::isScalarQuantity(const Resources& resources)
+{
+  // Instead of checking the absence of non-scalar-quantity fields,
+  // we do an equality check between the original resources object and
+  // its stripped counterpart.
+  //
+  // We remove the static reservation metadata here via `toUnreserved()`.
+  return resources == resources.createStrippedScalarQuantity().toUnreserved();
+}
+
+
 bool Resources::hasRefinedReservations(const Resource& resource)
 {
   CHECK(!resource.has_role()) << resource;

http://git-wip-us.apache.org/repos/asf/mesos/blob/c3709616/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index b0e28e2..1410423 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -1943,6 +1943,37 @@ TEST(ResourcesTest, AbsentResources)
 }
 
 
+TEST(ResourcesTest, isScalarQuantity)
+{
+  Resources scalarQuantity1 = Resources::parse("cpus:1").get();
+  EXPECT_TRUE(Resources::isScalarQuantity(scalarQuantity1));
+
+  Resources scalarQuantity2 = Resources::parse("cpus:1;mem:1").get();
+  EXPECT_TRUE(Resources::isScalarQuantity(scalarQuantity2));
+
+  Resources range = Resources::parse("ports", "[1-16000]", "*").get();
+  EXPECT_FALSE(Resources::isScalarQuantity(range));
+
+  Resources set = Resources::parse("names:{foo,bar}").get();
+  EXPECT_FALSE(Resources::isScalarQuantity(set));
+
+  Resources reserved = createReservedResource(
+      "cpus", "1", createDynamicReservationInfo("role", "principal"));
+  EXPECT_FALSE(Resources::isScalarQuantity(reserved));
+
+  Resources disk = createDiskResource("10", "role1", "1", "path");
+  EXPECT_FALSE(Resources::isScalarQuantity(disk));
+
+  Resources allocated = Resources::parse("cpus:1;mem:512").get();
+  allocated.allocate("role");
+  EXPECT_FALSE(Resources::isScalarQuantity(allocated));
+
+  Resource revocable = Resources::parse("cpus", "1", "*").get();
+  revocable.mutable_revocable();
+  EXPECT_FALSE(Resources::isScalarQuantity(revocable));
+}
+
+
 TEST(ReservedResourcesTest, Validation)
 {
   // Unreserved.

http://git-wip-us.apache.org/repos/asf/mesos/blob/c3709616/src/v1/resources.cpp
----------------------------------------------------------------------
diff --git a/src/v1/resources.cpp b/src/v1/resources.cpp
index 04a45a5..3d06fc6 100644
--- a/src/v1/resources.cpp
+++ b/src/v1/resources.cpp
@@ -1268,6 +1268,17 @@ bool Resources::isShared(const Resource& resource)
 }
 
 
+bool Resources::isScalarQuantity(const Resources& resources)
+{
+  // Instead of checking the absence of non-scalar-quantity fields,
+  // we do an equality check between the original `Resources` object and
+  // its stripped counterpart.
+  //
+  // We remove the static reservation metadata here via `toUnreserved()`.
+  return resources == resources.createStrippedScalarQuantity().toUnreserved();
+}
+
+
 bool Resources::hasRefinedReservations(const Resource& resource)
 {
   CHECK(!resource.has_role()) << resource;


[3/4] mesos git commit: Added a master flag to configure minimum allocatable resources.

Posted by gr...@apache.org.
Added a master flag to configure minimum allocatable resources.

This patch adds a new master flag `min_allocatable_resources`.
It specifies one or more resource quantities that define the
minimum allocatable resources for the allocator. The allocator
will only offer resources that contain at least one of the
specified resource quantities.

For example, the setting `disk:1000|cpus:1;mem:32` means that
the allocator will only allocate resources when they contain
1000MB of disk, or when they contain both 1 cpu and 32MB of
memory.

The default value for this new flag is such that it maintains
previous default behavior.

Also fixed all related tests and updated documentation.

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


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

Branch: refs/heads/master
Commit: a7e31366daa15fafd1e9aa21e81a04fadd961e37
Parents: c370961
Author: Meng Zhu <mz...@mesosphere.io>
Authored: Wed Jun 20 17:00:03 2018 -0700
Committer: Greg Mann <gr...@gmail.com>
Committed: Wed Jun 20 17:44:38 2018 -0700

----------------------------------------------------------------------
 docs/configuration/master.md                | 17 +++++++++++
 include/mesos/allocator/allocator.hpp       |  4 ++-
 src/master/allocator/mesos/allocator.hpp    | 13 +++++---
 src/master/allocator/mesos/hierarchical.cpp | 25 ++++++++++-----
 src/master/allocator/mesos/hierarchical.hpp |  9 ++++--
 src/master/flags.cpp                        | 13 ++++++++
 src/master/flags.hpp                        |  1 +
 src/master/master.cpp                       | 39 +++++++++++++++++++++++-
 src/tests/allocator.hpp                     | 11 ++++---
 src/tests/api_tests.cpp                     |  4 +--
 src/tests/hierarchical_allocator_tests.cpp  | 11 ++++++-
 src/tests/master_allocator_tests.cpp        | 36 +++++++++++-----------
 src/tests/master_quota_tests.cpp            | 20 ++++++------
 src/tests/reservation_tests.cpp             |  6 ++--
 src/tests/resource_offers_tests.cpp         |  2 +-
 src/tests/slave_recovery_tests.cpp          |  2 +-
 16 files changed, 157 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/docs/configuration/master.md
----------------------------------------------------------------------
diff --git a/docs/configuration/master.md b/docs/configuration/master.md
index 2ba612b..2090090 100644
--- a/docs/configuration/master.md
+++ b/docs/configuration/master.md
@@ -202,6 +202,23 @@ load an alternate allocator module using <code>--modules</code>.
   </td>
 </tr>
 
+<tr id="min_allocatable_resources">
+  <td>
+    --min_allocatable_resources=VALUE
+  </td>
+  <td>
+One or more sets of resources that define the minimum allocatable
+resources for the allocator. The allocator will only offer resources
+that contain at least one of the specified sets. The resources in each
+set should be delimited by semicolons, and the sets should be delimited
+by the pipe character.
+(Example: <code>disk:1|cpu:1;mem:32</code>
+configures the allocator to only offer resources if they contain a disk
+resource of at least 1 megabyte, or if they contain both 1 cpu and
+32 megabytes of memory.) (default: cpus:0.01|mem:32).
+  </td>
+</tr>
+
 <tr id="authenticate_agents">
   <td>
     --[no-]authenticate_agents,

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/include/mesos/allocator/allocator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/allocator/allocator.hpp b/include/mesos/allocator/allocator.hpp
index 6478692..c19ab64 100644
--- a/include/mesos/allocator/allocator.hpp
+++ b/include/mesos/allocator/allocator.hpp
@@ -101,7 +101,9 @@ public:
       const Option<std::set<std::string>>&
         fairnessExcludeResourceNames = None(),
       bool filterGpuResources = true,
-      const Option<DomainInfo>& domain = None()) = 0;
+      const Option<DomainInfo>& domain = None(),
+      const Option<std::vector<Resources>>&
+        minAllocatableResources = None()) = 0;
 
   /**
    * Informs the allocator of the recovered state from the master.

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/src/master/allocator/mesos/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/allocator.hpp b/src/master/allocator/mesos/allocator.hpp
index c453c01..900c8ee 100644
--- a/src/master/allocator/mesos/allocator.hpp
+++ b/src/master/allocator/mesos/allocator.hpp
@@ -59,7 +59,8 @@ public:
       const Option<std::set<std::string>>&
         fairnessExcludeResourceNames = None(),
       bool filterGpuResources = true,
-      const Option<DomainInfo>& domain = None());
+      const Option<DomainInfo>& domain = None(),
+      const Option<std::vector<Resources>>& minAllocatableResources = None());
 
   void recover(
       const int expectedAgentCount,
@@ -205,7 +206,9 @@ public:
       const Option<std::set<std::string>>&
         fairnessExcludeResourceNames = None(),
       bool filterGpuResources = true,
-      const Option<DomainInfo>& domain = None()) = 0;
+      const Option<DomainInfo>& domain = None(),
+      const Option<std::vector<Resources>>&
+        minAllocatableResources = None()) = 0;
 
   virtual void recover(
       const int expectedAgentCount,
@@ -360,7 +363,8 @@ inline void MesosAllocator<AllocatorProcess>::initialize(
       inverseOfferCallback,
     const Option<std::set<std::string>>& fairnessExcludeResourceNames,
     bool filterGpuResources,
-    const Option<DomainInfo>& domain)
+    const Option<DomainInfo>& domain,
+    const Option<std::vector<Resources>>& minAllocatableResources)
 {
   process::dispatch(
       process,
@@ -370,7 +374,8 @@ inline void MesosAllocator<AllocatorProcess>::initialize(
       inverseOfferCallback,
       fairnessExcludeResourceNames,
       filterGpuResources,
-      domain);
+      domain,
+      minAllocatableResources);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/src/master/allocator/mesos/hierarchical.cpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.cpp b/src/master/allocator/mesos/hierarchical.cpp
index b558228..f631ce7 100644
--- a/src/master/allocator/mesos/hierarchical.cpp
+++ b/src/master/allocator/mesos/hierarchical.cpp
@@ -155,7 +155,8 @@ void HierarchicalAllocatorProcess::initialize(
       _inverseOfferCallback,
     const Option<set<string>>& _fairnessExcludeResourceNames,
     bool _filterGpuResources,
-    const Option<DomainInfo>& _domain)
+    const Option<DomainInfo>& _domain,
+    const Option<std::vector<Resources>>& _minAllocatableResources)
 {
   allocationInterval = _allocationInterval;
   offerCallback = _offerCallback;
@@ -163,6 +164,7 @@ void HierarchicalAllocatorProcess::initialize(
   fairnessExcludeResourceNames = _fairnessExcludeResourceNames;
   filterGpuResources = _filterGpuResources;
   domain = _domain;
+  minAllocatableResources = _minAllocatableResources;
   initialized = true;
   paused = false;
 
@@ -2470,14 +2472,23 @@ bool HierarchicalAllocatorProcess::isFiltered(
 }
 
 
-bool HierarchicalAllocatorProcess::allocatable(
-    const Resources& resources)
+bool HierarchicalAllocatorProcess::allocatable(const Resources& resources)
 {
-  Option<double> cpus = resources.cpus();
-  Option<Bytes> mem = resources.mem();
+  if (minAllocatableResources.isNone() ||
+      CHECK_NOTNONE(minAllocatableResources).empty()) {
+    return true;
+  }
 
-  return (cpus.isSome() && cpus.get() >= MIN_CPUS) ||
-         (mem.isSome() && mem.get() >= MIN_MEM);
+  // We remove the static reservation metadata here via `toUnreserved()`.
+  Resources quantity = resources.createStrippedScalarQuantity().toUnreserved();
+  foreach (
+      const Resources& minResources, CHECK_NOTNONE(minAllocatableResources)) {
+    if (quantity.contains(minResources)) {
+      return true;
+    }
+  }
+
+  return false;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/src/master/allocator/mesos/hierarchical.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.hpp b/src/master/allocator/mesos/hierarchical.hpp
index e9d1742..19aed2d 100644
--- a/src/master/allocator/mesos/hierarchical.hpp
+++ b/src/master/allocator/mesos/hierarchical.hpp
@@ -111,7 +111,9 @@ public:
       const Option<std::set<std::string>>&
         fairnessExcludeResourceNames = None(),
       bool filterGpuResources = true,
-      const Option<DomainInfo>& domain = None());
+      const Option<DomainInfo>& domain = None(),
+      const Option<std::vector<Resources>>&
+        minAllocatableResources = None());
 
   void recover(
       const int _expectedAgentCount,
@@ -287,7 +289,7 @@ protected:
       const FrameworkID& frameworkID,
       const SlaveID& slaveID) const;
 
-  static bool allocatable(const Resources& resources);
+  bool allocatable(const Resources& resources);
 
   bool initialized;
   bool paused;
@@ -474,6 +476,9 @@ protected:
   // The master's domain, if any.
   Option<DomainInfo> domain;
 
+  // The minimum allocatable resources, if any.
+  Option<std::vector<Resources>> minAllocatableResources;
+
   // There are two stages of allocation:
   //
   //   Stage 1: Allocate to satisfy quota guarantees.

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/src/master/flags.cpp
----------------------------------------------------------------------
diff --git a/src/master/flags.cpp b/src/master/flags.cpp
index cc3317e..8fede0d 100644
--- a/src/master/flags.cpp
+++ b/src/master/flags.cpp
@@ -476,6 +476,19 @@ mesos::internal::master::Flags::Flags()
       "  https://issues.apache.org/jira/browse/MESOS-7576",
       true);
 
+  add(&Flags::min_allocatable_resources,
+      "min_allocatable_resources",
+      "One or more sets of resources that define the minimum allocatable\n"
+      "resources for the allocator. The allocator will only offer resources\n"
+      "that contain at least one of the specified sets. The resources in\n"
+      "each set should be delimited by semicolons, and the sets should be\n"
+      "delimited by the pipe character.\n"
+      "(Example: `disk:1|cpu:1;mem:32` configures the allocator to only offer\n"
+      "resources if they contain a disk resource of at least 1 megabyte, or\n"
+      "if they contain both 1 cpu and 32 megabytes of memory.)\n",
+      "cpus:" + stringify(MIN_CPUS) +
+        "|mem:" + stringify((double)MIN_MEM.bytes() / Bytes::MEGABYTES));
+
   add(&Flags::hooks,
       "hooks",
       "A comma-separated list of hook modules to be\n"

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/src/master/flags.hpp
----------------------------------------------------------------------
diff --git a/src/master/flags.hpp b/src/master/flags.hpp
index 94b8ac2..3929c29 100644
--- a/src/master/flags.hpp
+++ b/src/master/flags.hpp
@@ -82,6 +82,7 @@ public:
   std::string allocator;
   Option<std::set<std::string>> fair_sharing_excluded_resource_names;
   bool filter_gpu_resources;
+  std::string min_allocatable_resources;
   Option<std::string> hooks;
   Duration agent_ping_timeout;
   size_t max_agent_ping_timeouts;

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 5db5a8d..4ade16f 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -768,6 +768,42 @@ void Master::initialize()
       << " for --offer_timeout: Must be greater than zero";
   }
 
+  // Parse min_allocatable_resources.
+  Try<vector<Resources>> minAllocatableResources =
+    [](const string& resourceString) -> Try<vector<Resources>> {
+      vector<Resources> result;
+
+      foreach (const string& token, strings::tokenize(resourceString, "|")) {
+        Try<vector<Resource>> resourceVector =
+          Resources::fromSimpleString(token);
+
+        if (resourceVector.isError()) {
+          return Error(resourceVector.error());
+        }
+
+        result.push_back(Resources(CHECK_NOTERROR(resourceVector)));
+      }
+
+      return result;
+  }(flags.min_allocatable_resources);
+
+  if (minAllocatableResources.isError()) {
+    EXIT(EXIT_FAILURE) << "Error parsing min_allocatable_resources: '"
+                       << flags.min_allocatable_resources
+                       << "': " << minAllocatableResources.error();
+  }
+
+  // Validate that configured minimum resources are "pure" scalar quantities.
+  foreach (
+      const Resources& resources, CHECK_NOTERROR(minAllocatableResources)) {
+    if (!Resources::isScalarQuantity(resources)) {
+      EXIT(EXIT_FAILURE) << "Invalid min_allocatable_resources: '"
+                         << flags.min_allocatable_resources << "': "
+                         << "minimum allocatable resources should only"
+                         << "have name, type (scalar) and value set";
+    }
+  }
+
   // Initialize the allocator.
   allocator->initialize(
       flags.allocation_interval,
@@ -775,7 +811,8 @@ void Master::initialize()
       defer(self(), &Master::inverseOffer, lambda::_1, lambda::_2),
       flags.fair_sharing_excluded_resource_names,
       flags.filter_gpu_resources,
-      flags.domain);
+      flags.domain,
+      CHECK_NOTERROR(minAllocatableResources));
 
   // Parse the whitelist. Passing Allocator::updateWhitelist()
   // callback is safe because we shut down the whitelistWatcher in

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/src/tests/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/tests/allocator.hpp b/src/tests/allocator.hpp
index 341efa6..73fc060 100644
--- a/src/tests/allocator.hpp
+++ b/src/tests/allocator.hpp
@@ -45,7 +45,7 @@ namespace tests {
 
 ACTION_P(InvokeInitialize, allocator)
 {
-  allocator->real->initialize(arg0, arg1, arg2, arg3, arg4);
+  allocator->real->initialize(arg0, arg1, arg2, arg3, arg4, arg5, arg6);
 }
 
 
@@ -235,9 +235,9 @@ public:
     // to get the best of both worlds: the ability to use 'DoDefault'
     // and no warnings when expectations are not explicit.
 
-    ON_CALL(*this, initialize(_, _, _, _, _, _))
+    ON_CALL(*this, initialize(_, _, _, _, _, _, _))
       .WillByDefault(InvokeInitialize(this));
-    EXPECT_CALL(*this, initialize(_, _, _, _, _, _))
+    EXPECT_CALL(*this, initialize(_, _, _, _, _, _, _))
       .WillRepeatedly(DoDefault());
 
     ON_CALL(*this, recover(_, _))
@@ -368,7 +368,7 @@ public:
 
   virtual ~TestAllocator() {}
 
-  MOCK_METHOD6(initialize, void(
+  MOCK_METHOD7(initialize, void(
       const Duration&,
       const lambda::function<
           void(const FrameworkID&,
@@ -378,7 +378,8 @@ public:
                const hashmap<SlaveID, UnavailableResources>&)>&,
       const Option<std::set<std::string>>&,
       bool,
-      const Option<DomainInfo>&));
+      const Option<DomainInfo>&,
+      const Option<std::vector<Resources>>&));
 
   MOCK_METHOD2(recover, void(
       const int expectedAgentCount,

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 85635a8..4d6b5b3 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -1179,7 +1179,7 @@ TEST_P(MasterAPITest, ReserveResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -1270,7 +1270,7 @@ TEST_P(MasterAPITest, UnreserveResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/src/tests/hierarchical_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hierarchical_allocator_tests.cpp b/src/tests/hierarchical_allocator_tests.cpp
index c97b2ba..597469e 100644
--- a/src/tests/hierarchical_allocator_tests.cpp
+++ b/src/tests/hierarchical_allocator_tests.cpp
@@ -176,11 +176,20 @@ protected:
         };
     }
 
+    vector<Resources> minAllocatableResources;
+    minAllocatableResources.push_back(
+        CHECK_NOTERROR(Resources::parse("cpus:" + stringify(MIN_CPUS))));
+    minAllocatableResources.push_back(CHECK_NOTERROR(Resources::parse(
+        "mem:" + stringify((double)MIN_MEM.bytes() / Bytes::MEGABYTES))));
+
     allocator->initialize(
         flags.allocation_interval,
         offerCallback.get(),
         inverseOfferCallback.get(),
-        flags.fair_sharing_excluded_resource_names);
+        flags.fair_sharing_excluded_resource_names,
+        true,
+        None(),
+        minAllocatableResources);
   }
 
   SlaveInfo createSlaveInfo(const Resources& resources)

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/src/tests/master_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_allocator_tests.cpp b/src/tests/master_allocator_tests.cpp
index e1aef8a..58821a4 100644
--- a/src/tests/master_allocator_tests.cpp
+++ b/src/tests/master_allocator_tests.cpp
@@ -164,7 +164,7 @@ TYPED_TEST(MasterAllocatorTest, SingleFramework)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -214,7 +214,7 @@ TYPED_TEST(MasterAllocatorTest, ResourcesUnused)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -324,7 +324,7 @@ TYPED_TEST(MasterAllocatorTest, OutOfOrderDispatch)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -455,7 +455,7 @@ TYPED_TEST(MasterAllocatorTest, SchedulerFailover)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -588,7 +588,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkExited)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -749,7 +749,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveLost)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -873,7 +873,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveAdded)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -969,7 +969,7 @@ TYPED_TEST(MasterAllocatorTest, TaskFinished)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -1075,7 +1075,7 @@ TYPED_TEST(MasterAllocatorTest, CpusOnlyOfferedAndTaskLaunched)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -1158,7 +1158,7 @@ TYPED_TEST(MasterAllocatorTest, MemoryOnlyOfferedAndTaskLaunched)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -1254,7 +1254,7 @@ TYPED_TEST(MasterAllocatorTest, Whitelist)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Future<Nothing> updateWhitelist1;
   EXPECT_CALL(allocator, updateWhitelist(Option<hashset<string>>(hosts)))
@@ -1293,7 +1293,7 @@ TYPED_TEST(MasterAllocatorTest, RoleTest)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.roles = Some("role2");
@@ -1388,7 +1388,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkReregistersFirst)
   {
     TestAllocator<TypeParam> allocator;
 
-    EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+    EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
     Try<Owned<cluster::Master>> master = this->StartMaster(
         &allocator, masterFlags);
@@ -1446,7 +1446,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkReregistersFirst)
   {
     TestAllocator<TypeParam> allocator2;
 
-    EXPECT_CALL(allocator2, initialize(_, _, _, _, _, _));
+    EXPECT_CALL(allocator2, initialize(_, _, _, _, _, _, _));
 
     Future<Nothing> addFramework;
     EXPECT_CALL(allocator2, addFramework(_, _, _, _, _))
@@ -1514,7 +1514,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveReregistersFirst)
   {
     TestAllocator<TypeParam> allocator;
 
-    EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+    EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
     Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
 
@@ -1573,7 +1573,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveReregistersFirst)
   {
     TestAllocator<TypeParam> allocator2;
 
-    EXPECT_CALL(allocator2, initialize(_, _, _, _, _, _));
+    EXPECT_CALL(allocator2, initialize(_, _, _, _, _, _, _));
 
     Future<Nothing> addSlave;
     EXPECT_CALL(allocator2, addSlave(_, _, _, _, _, _))
@@ -1640,7 +1640,7 @@ TYPED_TEST(MasterAllocatorTest, RebalancedForUpdatedWeights)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   // Start Mesos master.
   master::Flags masterFlags = this->CreateMasterFlags();
@@ -1834,7 +1834,7 @@ TYPED_TEST(MasterAllocatorTest, NestedRoles)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   Try<Owned<cluster::Master>> master =

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/src/tests/master_quota_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_quota_tests.cpp b/src/tests/master_quota_tests.cpp
index 94d85c5..d836482 100644
--- a/src/tests/master_quota_tests.cpp
+++ b/src/tests/master_quota_tests.cpp
@@ -424,7 +424,7 @@ TEST_F(MasterQuotaTest, SetExistingQuota)
 TEST_F(MasterQuotaTest, RemoveSingleQuota)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -599,7 +599,7 @@ TEST_F(MasterQuotaTest, Status)
 TEST_F(MasterQuotaTest, InsufficientResourcesSingleAgent)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -659,7 +659,7 @@ TEST_F(MasterQuotaTest, InsufficientResourcesSingleAgent)
 TEST_F(MasterQuotaTest, InsufficientResourcesMultipleAgents)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -734,7 +734,7 @@ TEST_F(MasterQuotaTest, InsufficientResourcesMultipleAgents)
 TEST_F(MasterQuotaTest, AvailableResourcesSingleAgent)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -784,7 +784,7 @@ TEST_F(MasterQuotaTest, AvailableResourcesSingleAgent)
 TEST_F(MasterQuotaTest, AvailableResourcesMultipleAgents)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -853,7 +853,7 @@ TEST_F(MasterQuotaTest, AvailableResourcesMultipleAgents)
 TEST_F(MasterQuotaTest, AvailableResourcesAfterRescinding)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -1088,7 +1088,7 @@ TEST_F(MasterQuotaTest, RecoverQuotaEmptyCluster)
   }
 
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   // Restart the master; configured quota should be recovered from the registry.
   master->reset();
@@ -1121,7 +1121,7 @@ TEST_F(MasterQuotaTest, RecoverQuotaEmptyCluster)
 TEST_F(MasterQuotaTest, NoAuthenticationNoAuthorization)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   // Disable http_readwrite authentication and authorization.
   // TODO(alexr): Setting master `--acls` flag to `ACLs()` or `None()` seems
@@ -1227,7 +1227,7 @@ TEST_F(MasterQuotaTest, UnauthenticatedQuotaRequest)
 TEST_F(MasterQuotaTest, AuthorizeGetUpdateQuotaRequests)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   // Setup ACLs so that only the default principal can modify quotas
   // for `ROLE1` and read status.
@@ -1775,7 +1775,7 @@ TEST_F(MasterQuotaTest, DISABLED_ChildRoleDeleteParentQuota)
 TEST_F(MasterQuotaTest, DISABLED_ClusterCapacityWithNestedRoles)
 {
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator);
   ASSERT_SOME(master);

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7e31366/src/tests/reservation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reservation_tests.cpp b/src/tests/reservation_tests.cpp
index 7d121bf..058a66d 100644
--- a/src/tests/reservation_tests.cpp
+++ b/src/tests/reservation_tests.cpp
@@ -603,7 +603,7 @@ TEST_F(ReservationTest, DropReserveTooLarge)
   masterFlags.allocation_interval = Milliseconds(5);
   masterFlags.roles = frameworkInfo.roles(0);
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -2120,7 +2120,7 @@ TEST_F(ReservationTest, DropReserveWithDifferentRole)
   masterFlags.allocation_interval = Milliseconds(5);
 
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);
@@ -2222,7 +2222,7 @@ TEST_F(ReservationTest, PreventUnreservingAlienResources)
   masterFlags.allocation_interval = Milliseconds(5);
 
   TestAllocator<> allocator;
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = StartMaster(&allocator, masterFlags);
   ASSERT_SOME(master);

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

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