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/07/02 20:00:21 UTC

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

Repository: mesos
Updated Branches:
  refs/heads/1.4.x d29d9f309 -> c35e06b58


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/9cba3aa6
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/9cba3aa6
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/9cba3aa6

Branch: refs/heads/1.4.x
Commit: 9cba3aa6dd571f8b92b46261d2e1256b0c47e338
Parents: c45b4bd
Author: Meng Zhu <mz...@mesosphere.io>
Authored: Wed Jun 20 17:00:03 2018 -0700
Committer: Greg Mann <gr...@gmail.com>
Committed: Mon Jul 2 12:41:43 2018 -0700

----------------------------------------------------------------------
 docs/configuration.md                       | 16 ++++++++++
 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, 156 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9cba3aa6/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index 9925262..173b8a7 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -589,6 +589,22 @@ load an alternate allocator module using <code>--modules</code>.
 </tr>
 <tr>
   <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>
+  <td>
     --[no-]authenticate_agents,
     <p/>
     --[no-]authenticate_slaves

http://git-wip-us.apache.org/repos/asf/mesos/blob/9cba3aa6/include/mesos/allocator/allocator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/allocator/allocator.hpp b/include/mesos/allocator/allocator.hpp
index 8bbd7fb..38aacea 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/9cba3aa6/src/master/allocator/mesos/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/allocator.hpp b/src/master/allocator/mesos/allocator.hpp
index 903edf6..53d891e 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,
@@ -199,7 +200,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,
@@ -348,7 +351,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,
@@ -358,7 +362,8 @@ inline void MesosAllocator<AllocatorProcess>::initialize(
       inverseOfferCallback,
       fairnessExcludeResourceNames,
       filterGpuResources,
-      domain);
+      domain,
+      minAllocatableResources);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/9cba3aa6/src/master/allocator/mesos/hierarchical.cpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.cpp b/src/master/allocator/mesos/hierarchical.cpp
index ee7d758..33b7495 100644
--- a/src/master/allocator/mesos/hierarchical.cpp
+++ b/src/master/allocator/mesos/hierarchical.cpp
@@ -151,7 +151,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;
@@ -159,6 +160,7 @@ void HierarchicalAllocatorProcess::initialize(
   fairnessExcludeResourceNames = _fairnessExcludeResourceNames;
   filterGpuResources = _filterGpuResources;
   domain = _domain;
+  minAllocatableResources = _minAllocatableResources;
   initialized = true;
   paused = false;
 
@@ -2383,14 +2385,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/9cba3aa6/src/master/allocator/mesos/hierarchical.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.hpp b/src/master/allocator/mesos/hierarchical.hpp
index 6db912d..4f22c8b 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,
@@ -281,7 +283,7 @@ protected:
       const FrameworkID& frameworkID,
       const SlaveID& slaveID) const;
 
-  static bool allocatable(const Resources& resources);
+  bool allocatable(const Resources& resources);
 
   bool initialized;
   bool paused;
@@ -469,6 +471,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. During the first stage resources
   // are allocated only to frameworks in roles with quota set. During the
   // second stage remaining resources that would not be required to satisfy

http://git-wip-us.apache.org/repos/asf/mesos/blob/9cba3aa6/src/master/flags.cpp
----------------------------------------------------------------------
diff --git a/src/master/flags.cpp b/src/master/flags.cpp
index ffd9d7e..af0014c 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/9cba3aa6/src/master/flags.hpp
----------------------------------------------------------------------
diff --git a/src/master/flags.hpp b/src/master/flags.hpp
index 8b1944c..b262fd2 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/9cba3aa6/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 83ea4a6..0b5df92 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -762,6 +762,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,
@@ -769,7 +805,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/9cba3aa6/src/tests/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/tests/allocator.hpp b/src/tests/allocator.hpp
index f1c0d14..d7b4f2f 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);
 }
 
 
@@ -229,9 +229,9 @@ public:
     // to get the best of both worlds: the ability to use 'DoDefault'
     // and no warnings when expectations are not explicit.
 
-    ON_CALL(*this, initialize(_, _, _, _, _, _))
+    ON_CALL(*this, initialize(_, _, _, _, _, _, _))
       .WillByDefault(InvokeInitialize(this));
-    EXPECT_CALL(*this, initialize(_, _, _, _, _, _))
+    EXPECT_CALL(*this, initialize(_, _, _, _, _, _, _))
       .WillRepeatedly(DoDefault());
 
     ON_CALL(*this, recover(_, _))
@@ -357,7 +357,7 @@ public:
 
   virtual ~TestAllocator() {}
 
-  MOCK_METHOD6(initialize, void(
+  MOCK_METHOD7(initialize, void(
       const Duration&,
       const lambda::function<
           void(const FrameworkID&,
@@ -367,7 +367,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/9cba3aa6/src/tests/api_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/api_tests.cpp b/src/tests/api_tests.cpp
index 903e7ea..fa34e41 100644
--- a/src/tests/api_tests.cpp
+++ b/src/tests/api_tests.cpp
@@ -969,7 +969,7 @@ TEST_P(MasterAPITest, ReserveResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   // Set a low allocation interval to speed up this test.
   master::Flags flags = MesosTest::CreateMasterFlags();
@@ -1061,7 +1061,7 @@ TEST_P(MasterAPITest, UnreserveResources)
 {
   TestAllocator<> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   // Set a low allocation interval to speed up this test.
   master::Flags flags = MesosTest::CreateMasterFlags();

http://git-wip-us.apache.org/repos/asf/mesos/blob/9cba3aa6/src/tests/hierarchical_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hierarchical_allocator_tests.cpp b/src/tests/hierarchical_allocator_tests.cpp
index 6ea7d50..5a8cfb1 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(
+        Resources::parse("cpus:" + stringify(MIN_CPUS)).get());
+    minAllocatableResources.push_back(Resources::parse(
+        "mem:" + stringify((double)MIN_MEM.bytes() / Bytes::MEGABYTES)).get());
+
     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/9cba3aa6/src/tests/master_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_allocator_tests.cpp b/src/tests/master_allocator_tests.cpp
index 455b7da..c4fb54d 100644
--- a/src/tests/master_allocator_tests.cpp
+++ b/src/tests/master_allocator_tests.cpp
@@ -164,7 +164,7 @@ TYPED_TEST(MasterAllocatorTest, SingleFramework)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -212,7 +212,7 @@ TYPED_TEST(MasterAllocatorTest, ResourcesUnused)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -321,7 +321,7 @@ TYPED_TEST(MasterAllocatorTest, OutOfOrderDispatch)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -450,7 +450,7 @@ TYPED_TEST(MasterAllocatorTest, SchedulerFailover)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -576,7 +576,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkExited)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -736,7 +736,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveLost)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
   ASSERT_SOME(master);
@@ -860,7 +860,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveAdded)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -956,7 +956,7 @@ TYPED_TEST(MasterAllocatorTest, TaskFinished)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -1060,7 +1060,7 @@ TYPED_TEST(MasterAllocatorTest, CpusOnlyOfferedAndTaskLaunched)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -1141,7 +1141,7 @@ TYPED_TEST(MasterAllocatorTest, MemoryOnlyOfferedAndTaskLaunched)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.allocation_interval = Milliseconds(50);
@@ -1235,7 +1235,7 @@ TYPED_TEST(MasterAllocatorTest, Whitelist)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   Future<Nothing> updateWhitelist1;
   EXPECT_CALL(allocator, updateWhitelist(Option<hashset<string>>(hosts)))
@@ -1274,7 +1274,7 @@ TYPED_TEST(MasterAllocatorTest, RoleTest)
 {
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   master::Flags masterFlags = this->CreateMasterFlags();
   masterFlags.roles = Some("role2");
@@ -1367,7 +1367,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkReregistersFirst)
   {
     TestAllocator<TypeParam> allocator;
 
-    EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+    EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
     Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
     ASSERT_SOME(master);
@@ -1424,7 +1424,7 @@ TYPED_TEST(MasterAllocatorTest, FrameworkReregistersFirst)
   {
     TestAllocator<TypeParam> allocator2;
 
-    EXPECT_CALL(allocator2, initialize(_, _, _, _, _, _));
+    EXPECT_CALL(allocator2, initialize(_, _, _, _, _, _, _));
 
     Future<Nothing> addFramework;
     EXPECT_CALL(allocator2, addFramework(_, _, _, _, _))
@@ -1491,7 +1491,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveReregistersFirst)
   {
     TestAllocator<TypeParam> allocator;
 
-    EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+    EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
     Try<Owned<cluster::Master>> master = this->StartMaster(&allocator);
     ASSERT_SOME(master);
@@ -1547,7 +1547,7 @@ TYPED_TEST(MasterAllocatorTest, SlaveReregistersFirst)
   {
     TestAllocator<TypeParam> allocator2;
 
-    EXPECT_CALL(allocator2, initialize(_, _, _, _, _, _));
+    EXPECT_CALL(allocator2, initialize(_, _, _, _, _, _, _));
 
     Future<Nothing> addSlave;
     EXPECT_CALL(allocator2, addSlave(_, _, _, _, _, _))
@@ -1614,7 +1614,7 @@ TYPED_TEST(MasterAllocatorTest, RebalancedForUpdatedWeights)
 
   TestAllocator<TypeParam> allocator;
 
-  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _));
+  EXPECT_CALL(allocator, initialize(_, _, _, _, _, _, _));
 
   // Start Mesos master.
   master::Flags masterFlags = this->CreateMasterFlags();
@@ -1808,7 +1808,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/9cba3aa6/src/tests/master_quota_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_quota_tests.cpp b/src/tests/master_quota_tests.cpp
index fe4479e..6b1c6c5 100644
--- a/src/tests/master_quota_tests.cpp
+++ b/src/tests/master_quota_tests.cpp
@@ -408,7 +408,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);
@@ -580,7 +580,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);
@@ -642,7 +642,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);
@@ -719,7 +719,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);
@@ -769,7 +769,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);
@@ -838,7 +838,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);
@@ -1074,7 +1074,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();
@@ -1108,7 +1108,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
@@ -1218,7 +1218,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.
@@ -1782,7 +1782,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/9cba3aa6/src/tests/reservation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/reservation_tests.cpp b/src/tests/reservation_tests.cpp
index 70bf579..e430f11 100644
--- a/src/tests/reservation_tests.cpp
+++ b/src/tests/reservation_tests.cpp
@@ -531,7 +531,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);
@@ -2053,7 +2053,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);
@@ -2150,7 +2150,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/9cba3aa6/src/tests/resource_offers_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resource_offers_tests.cpp b/src/tests/resource_offers_tests.cpp
index e1fcab4..6ac357c 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/9cba3aa6/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index ca16401..3188b03 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -3502,7 +3502,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);


[3/5] mesos git commit: Introduced a CHECK_NOTERROR macro.

Posted by gr...@apache.org.
Introduced a CHECK_NOTERROR macro.

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


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

Branch: refs/heads/1.4.x
Commit: 8911db363509c50fbeaf2813cd06f16cf829541d
Parents: d29d9f3
Author: Benjamin Mahler <bm...@apache.org>
Authored: Mon Feb 5 13:32:37 2018 -0800
Committer: Greg Mann <gr...@gmail.com>
Committed: Mon Jul 2 12:41:43 2018 -0700

----------------------------------------------------------------------
 3rdparty/stout/include/stout/check.hpp | 79 ++++++++++++++++++++++++++---
 1 file changed, 73 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8911db36/3rdparty/stout/include/stout/check.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/check.hpp b/3rdparty/stout/include/stout/check.hpp
index 6a33579..7651150 100644
--- a/3rdparty/stout/include/stout/check.hpp
+++ b/3rdparty/stout/include/stout/check.hpp
@@ -23,9 +23,15 @@
 #include <stout/error.hpp>
 #include <stout/none.hpp>
 #include <stout/option.hpp>
-#include <stout/result.hpp>
 #include <stout/some.hpp>
-#include <stout/try.hpp>
+
+
+template <typename T>
+class Result;
+
+template <typename T, typename E>
+class Try;
+
 
 // A generic macro to facilitate definitions of CHECK_*, akin to CHECK.
 // This appends the error if possible to the end of the log message,
@@ -102,6 +108,67 @@ const T& _check_not_none(
       (expression))
 
 
+// A private helper for CHECK_NOTERROR which is similar to the
+// CHECK_NOTNULL provided by glog.
+template <typename T, typename E>
+T&& _check_not_error(
+    const char* file,
+    int line,
+    const char* message,
+    Try<T, E>&& t) {
+  if (t.isError()) {
+    google::LogMessageFatal(
+        file,
+        line,
+        new std::string(
+            std::string(message) + ": " + Error(t.error()).message));
+  }
+  return std::move(t).get();
+}
+
+
+template <typename T, typename E>
+T& _check_not_error(
+    const char* file,
+    int line,
+    const char* message,
+    Try<T, E>& t) {
+  if (t.isError()) {
+    google::LogMessageFatal(
+        file,
+        line,
+        new std::string(
+            std::string(message) + ": " + Error(t.error()).message));
+  }
+  return t.get();
+}
+
+
+template <typename T, typename E>
+const T& _check_not_error(
+    const char* file,
+    int line,
+    const char* message,
+    const Try<T, E>& t) {
+  if (t.isError()) {
+    google::LogMessageFatal(
+        file,
+        line,
+        new std::string(
+            std::string(message) + ": " + Error(t.error()).message));
+  }
+  return t.get();
+}
+
+
+#define CHECK_NOTERROR(expression) \
+  _check_not_error( \
+      __FILE__, \
+      __LINE__, \
+      "'" #expression "' Must be SOME", \
+      (expression))
+
+
 // Private structs/functions used for CHECK_*.
 
 template <typename T>
@@ -116,8 +183,8 @@ Option<Error> _check_some(const Option<T>& o)
 }
 
 
-template <typename T>
-Option<Error> _check_some(const Try<T>& t)
+template <typename T, typename E>
+Option<Error> _check_some(const Try<T, E>& t)
 {
   if (t.isError()) {
     return Error(t.error());
@@ -168,8 +235,8 @@ Option<Error> _check_none(const Result<T>& r)
 }
 
 
-template <typename T>
-Option<Error> _check_error(const Try<T>& t)
+template <typename T, typename E>
+Option<Error> _check_error(const Try<T, E>& t)
 {
   if (t.isSome()) {
     return Error("is SOME");


[2/5] 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/c45b4bdd
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/c45b4bdd
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/c45b4bdd

Branch: refs/heads/1.4.x
Commit: c45b4bdd50ee2fc5db7e3c2274ef2938a8999c22
Parents: 450651b
Author: Meng Zhu <mz...@mesosphere.io>
Authored: Wed Jun 20 16:59:58 2018 -0700
Committer: Greg Mann <gr...@gmail.com>
Committed: Mon Jul 2 12:41:43 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/c45b4bdd/include/mesos/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/resources.hpp b/include/mesos/resources.hpp
index 688093f..a9144f6 100644
--- a/include/mesos/resources.hpp
+++ b/include/mesos/resources.hpp
@@ -303,6 +303,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/c45b4bdd/include/mesos/v1/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/v1/resources.hpp b/include/mesos/v1/resources.hpp
index 94125f8..6ec8d15 100644
--- a/include/mesos/v1/resources.hpp
+++ b/include/mesos/v1/resources.hpp
@@ -303,6 +303,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/c45b4bdd/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index c597c9e..158ca92 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -1138,6 +1138,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/c45b4bdd/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index 2b1deff..f1aa17c 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -1941,6 +1941,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/c45b4bdd/src/v1/resources.cpp
----------------------------------------------------------------------
diff --git a/src/v1/resources.cpp b/src/v1/resources.cpp
index 28889b4..530c460 100644
--- a/src/v1/resources.cpp
+++ b/src/v1/resources.cpp
@@ -1166,6 +1166,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;


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

Posted by gr...@apache.org.
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/450651bd
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/450651bd
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/450651bd

Branch: refs/heads/1.4.x
Commit: 450651bd115292edf4d42cab3367ba58017b4354
Parents: 8911db3
Author: Meng Zhu <mz...@mesosphere.io>
Authored: Wed Jun 20 16:59:54 2018 -0700
Committer: Greg Mann <gr...@gmail.com>
Committed: Mon Jul 2 12:41:43 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/450651bd/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index 499cc29..c597c9e 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -1525,6 +1525,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/450651bd/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index 148183a..2b1deff 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -2582,6 +2582,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/450651bd/src/v1/resources.cpp
----------------------------------------------------------------------
diff --git a/src/v1/resources.cpp b/src/v1/resources.cpp
index 781fef6..28889b4 100644
--- a/src/v1/resources.cpp
+++ b/src/v1/resources.cpp
@@ -1553,6 +1553,7 @@ Resources Resources::createStrippedScalarQuantity() const
 
       scalar.clear_disk();
       scalar.clear_shared();
+      scalar.clear_revocable();
       stripped.add(scalar);
     }
   }


[5/5] mesos git commit: Added MESOS-8935 to the 1.4.2 CHANGELOG.

Posted by gr...@apache.org.
Added MESOS-8935 to the 1.4.2 CHANGELOG.


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

Branch: refs/heads/1.4.x
Commit: c35e06b58f3afd271290f96099ff405783454492
Parents: 9cba3aa
Author: Greg Mann <gr...@gmail.com>
Authored: Mon Jul 2 12:43:43 2018 -0700
Committer: Greg Mann <gr...@gmail.com>
Committed: Mon Jul 2 12:43:43 2018 -0700

----------------------------------------------------------------------
 CHANGELOG | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c35e06b5/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 1d38f2c..c3a4231 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -34,6 +34,7 @@ Release Notes - Mesos - Version 1.4.2 (WIP)
   * [MESOS-8881] - Enable epoll backend in libevent integration.
   * [MESOS-8885] - Disable libevent debug mode.
   * [MESOS-8904] - Master crash when removing quota.
+  * [MESOS-8935] - Quota limit "chopping" can lead to cpu-only and memory-only offers.
   * [MESOS-8936] - Implement a Random Sorter for offer allocations.
   * [MESOS-8942] - Master streaming API does not send (health) check updates for tasks.
   * [MESOS-8945] - Master check failure due to CHECK_SOME(providerId).