You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by mz...@apache.org on 2018/12/06 22:45:28 UTC

[mesos] branch master updated (5f4c7d6 -> 740fa11)

This is an automated email from the ASF dual-hosted git repository.

mzhu pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git.


    from 5f4c7d6  Added MESOS-9293 to the 1.7.1 CHANGELOG.
     new ed7557d  Added default arguments to `FrameworkProfile` in allocator benchmark.
     new b04e8aa  Removed `used` argument in `AgentProfile` in the allocator benchmark.
     new e6fa3e3  Renamed one allocator benchmark to be more descriptive.
     new 80da1fc  Moved a few allocator test helpers to `tests/allocator.hpp`.
     new 4e51261  Added an allocator benchmark for quota performance.
     new 740fa11  Added a benchmark to compare quota and nonquota allocation performance.

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/Makefile.am                                    |   1 +
 src/tests/CMakeLists.txt                           |   1 +
 .../{active_user_test_helper.hpp => allocator.cpp} |  32 +-
 src/tests/allocator.hpp                            |   8 +
 src/tests/hierarchical_allocator_benchmarks.cpp    | 348 ++++++++++++++++++++-
 src/tests/hierarchical_allocator_tests.cpp         |  18 --
 6 files changed, 359 insertions(+), 49 deletions(-)
 copy src/tests/{active_user_test_helper.hpp => allocator.cpp} (67%)


[mesos] 02/06: Removed `used` argument in `AgentProfile` in the allocator benchmark.

Posted by mz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mzhu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit b04e8aa2b38613ae103e9d004854030d56ca388a
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Fri Oct 12 17:23:03 2018 -0700

    Removed `used` argument in `AgentProfile` in the allocator benchmark.
    
    Currently, it is not easy to initialize frameworks with used
    resources in the fixture because when we specify the
    `agentProfile`, no framework has been created yet.
    
    Also, we currently do not have any use case for setting `used`
    resources during the initialization. We can revisit this once
    it becomes necessary.
    
    Review: https://reviews.apache.org/r/69093
---
 src/tests/hierarchical_allocator_benchmarks.cpp | 15 +++++++--------
 1 file changed, 7 insertions(+), 8 deletions(-)

diff --git a/src/tests/hierarchical_allocator_benchmarks.cpp b/src/tests/hierarchical_allocator_benchmarks.cpp
index b2fa614..65e9790 100644
--- a/src/tests/hierarchical_allocator_benchmarks.cpp
+++ b/src/tests/hierarchical_allocator_benchmarks.cpp
@@ -100,19 +100,19 @@ struct FrameworkProfile
 
 struct AgentProfile
 {
+  // TODO(mzhu): Add option to specify `used` resources. `used` resources
+  // requires the knowledge of `frameworkId` which currently is created
+  // during the initialization which is after the agent profile creation.
   AgentProfile(const string& _name,
                size_t _instances,
-               const Resources& _resources,
-               const hashmap<FrameworkID, Resources>& _usedResources)
+               const Resources& _resources)
     : name(_name),
       instances(_instances),
-      resources(_resources),
-      usedResources(_usedResources) {}
+      resources(_resources) {}
 
   string name;
   size_t instances;
   Resources resources;
-  hashmap<FrameworkID, Resources> usedResources;
 };
 
 
@@ -233,7 +233,7 @@ protected:
             AGENT_CAPABILITIES(),
             None(),
             agent.resources(),
-            profile.usedResources);
+            {});
       }
     }
 
@@ -331,8 +331,7 @@ TEST_F(HierarchicalAllocations_BENCHMARK_TestBase, Allocations)
   config.agentProfiles.push_back(AgentProfile(
       "agent",
       80,
-      CHECK_NOTERROR(Resources::parse("cpus:64;mem:488000")),
-      {}));
+      CHECK_NOTERROR(Resources::parse("cpus:64;mem:488000"))));
 
   // Add framework profiles.
 


[mesos] 03/06: Renamed one allocator benchmark to be more descriptive.

Posted by mz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mzhu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit e6fa3e3efc39971fea6157cd8d3e3a3bebbac94d
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Thu Oct 18 21:45:14 2018 -0700

    Renamed one allocator benchmark to be more descriptive.
    
    Renamed `Allocations` to `MultiFrameworkAllocations`.
    Also removed `_TestBase` from the fixture name.
    
    Review: https://reviews.apache.org/r/69094
---
 src/tests/hierarchical_allocator_benchmarks.cpp | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)

diff --git a/src/tests/hierarchical_allocator_benchmarks.cpp b/src/tests/hierarchical_allocator_benchmarks.cpp
index 65e9790..a61f886 100644
--- a/src/tests/hierarchical_allocator_benchmarks.cpp
+++ b/src/tests/hierarchical_allocator_benchmarks.cpp
@@ -316,11 +316,15 @@ private:
 };
 
 
+class HierarchicalAllocations_BENCHMARK :
+  public HierarchicalAllocations_BENCHMARK_TestBase {};
+
+
 // This benchmark launches frameworks with different profiles (number of tasks,
 // task sizes and etc.) and prints out statistics such as total tasks launched,
 // cluster utilization and allocation latency. The test has a timeout of 30
 // seconds.
-TEST_F(HierarchicalAllocations_BENCHMARK_TestBase, Allocations)
+TEST_F(HierarchicalAllocations_BENCHMARK, MultiFrameworkAllocations)
 {
   // Pause the clock because we want to manually drive the allocations.
   Clock::pause();


[mesos] 01/06: Added default arguments to `FrameworkProfile` in allocator benchmark.

Posted by mz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mzhu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit ed7557d71ff5bfb05b9726b714cca6a3de2f2273
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Fri Oct 12 17:04:25 2018 -0700

    Added default arguments to `FrameworkProfile` in allocator benchmark.
    
    For frameworks that do not want to configure task launches, we
    should provide some default task launch settings to simplify the
    benchmark settings.
    
    Review: https://reviews.apache.org/r/69092
---
 src/tests/hierarchical_allocator_benchmarks.cpp | 16 ++++++++++------
 1 file changed, 10 insertions(+), 6 deletions(-)

diff --git a/src/tests/hierarchical_allocator_benchmarks.cpp b/src/tests/hierarchical_allocator_benchmarks.cpp
index 6c6330e..b2fa614 100644
--- a/src/tests/hierarchical_allocator_benchmarks.cpp
+++ b/src/tests/hierarchical_allocator_benchmarks.cpp
@@ -71,12 +71,16 @@ namespace tests {
 // offer acceptance/rejection.
 struct FrameworkProfile
 {
-  FrameworkProfile(const string& _name,
-                   const set<string>& _roles,
-                   size_t _instances,
-                   size_t _maxTasksPerInstance,
-                   const Resources& _taskResources,
-                   size_t _maxTasksPerOffer)
+  FrameworkProfile(
+      const string& _name,
+      const set<string>& _roles,
+      size_t _instances,
+      // For frameworks that do not care about launching tasks, we provide
+      // some default task launch settings.
+      size_t _maxTasksPerInstance = 100,
+      const Resources& _taskResources =
+        CHECK_NOTERROR(Resources::parse("cpus:1;mem:100")),
+      size_t _maxTasksPerOffer = 10)
     : name(_name),
       roles(_roles),
       instances(_instances),


[mesos] 05/06: Added an allocator benchmark for quota performance.

Posted by mz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mzhu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 4e512618d508f32170193c49d72011189f6e2fa1
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Fri Oct 12 17:28:39 2018 -0700

    Added an allocator benchmark for quota performance.
    
    This benchmark evaluates the allocator performance in
    the presence of roles with both small quota (which can
    be satisfied by half an agent) as well as large quota
    (which need resources from two agents). We setup the cluster,
    trigger one allocation cycle and measure the elapsed time.
    
    Review: https://reviews.apache.org/r/69097
---
 src/tests/hierarchical_allocator_benchmarks.cpp | 160 ++++++++++++++++++++++++
 1 file changed, 160 insertions(+)

diff --git a/src/tests/hierarchical_allocator_benchmarks.cpp b/src/tests/hierarchical_allocator_benchmarks.cpp
index a61f886..9542c9c 100644
--- a/src/tests/hierarchical_allocator_benchmarks.cpp
+++ b/src/tests/hierarchical_allocator_benchmarks.cpp
@@ -486,6 +486,166 @@ TEST_F(HierarchicalAllocations_BENCHMARK, MultiFrameworkAllocations)
   cout << "Target allocation: " << targetAllocation << endl;
 }
 
+
+struct QuotaParam
+{
+  QuotaParam(
+      const size_t _smallQuotaRoleCount,
+      const size_t _largeQuotaRoleCount,
+      const size_t _frameworksPerRole)
+    : smallQuotaRoleCount(_smallQuotaRoleCount),
+      largeQuotaRoleCount(_largeQuotaRoleCount),
+      frameworksPerRole(_frameworksPerRole) {}
+
+  // `smallQuotaRole` will have a resource quota of 1/5 an agent, this will
+  // lead to agent chopping.
+  //
+  // `largeQuotaRole` will have a quota of 5 agents, this will lead to
+  // role unsatisfied quota chopping.
+  //
+  // Together, this determines the number of agents to be:
+  //
+  //  `smallQuotaRoleCount / 5 + largeQuotaRoleCount * 5`
+  //
+  // TODO(mzhu): Consider adding another parameter to control the chopping.
+
+  const size_t smallQuotaRoleCount;
+  const size_t largeQuotaRoleCount;
+
+  // Number of frameworks per role.
+  const size_t frameworksPerRole;
+};
+
+
+class HierarchicalAllocator_BENCHMARK_WithQuotaParam
+  : public HierarchicalAllocations_BENCHMARK_TestBase,
+    public WithParamInterface<QuotaParam> {};
+
+
+// Since the quota performance is mostly decided by number of agents and
+// roles, in the default setting below, we let each role has a single framework.
+INSTANTIATE_TEST_CASE_P(
+    QuotaParam,
+    HierarchicalAllocator_BENCHMARK_WithQuotaParam,
+    ::testing::Values(
+        QuotaParam(25U, 5U, 1U), // 30 agents in total
+        QuotaParam(250U, 50U, 1U), // 300 agents in total
+        QuotaParam(2500U, 500U, 1U))); // 3000 agents in total
+
+
+// This benchmark evaluates the allocator performance in the presence of
+// roles with both small quota (which can be satisfied by half an agent) as
+// well as large quota (which need resources from two agents). We setup the
+// cluster, trigger one allocation cycle and measure the elapsed time. All
+// cluster resources will be offered in this cycle to satisfy all roles' quota.
+TEST_P(HierarchicalAllocator_BENCHMARK_WithQuotaParam, LargeAndSmallQuota)
+{
+  // Pause the clock because we want to manually drive the allocations.
+  Clock::pause();
+
+  // Each agent can satisfy either two `smallQuotaRole` or half of
+  // `largeQuotaRole`.
+  const string agentResourcesString = "cpus:2;mem:2048;disk:2048";
+  const string smallQuotaResourcesString = "cpus:1;mem:1024;disk:1024";
+  const string largeQuotaResourcesString = "cpus:4;mem:4096;disk:4096";
+
+  BenchmarkConfig config;
+
+  // Store roles for setting up quota later (after allocator is initialized).
+  vector<string> smallQuotaRoles;
+
+  // Add framework profiles for `smallQuotaRole`.
+  for (size_t i = 0; i < GetParam().smallQuotaRoleCount; i++) {
+    string role("smallQuotaRole" + stringify(i));
+    smallQuotaRoles.push_back(role);
+    config.frameworkProfiles.push_back(FrameworkProfile(
+        "framework_small_" + stringify(i),
+        {role},
+        GetParam().frameworksPerRole));
+  }
+
+  // Store roles for setting up quota later (after allocator is initialized).
+  vector<string> largeQuotaRoles;
+
+  // Add framework profiles for `largeQuotaRole`.
+  for (size_t i = 0; i < GetParam().largeQuotaRoleCount; i++) {
+    string role("largeQuotaRole" + stringify(i));
+    largeQuotaRoles.push_back(role);
+    config.frameworkProfiles.push_back(FrameworkProfile(
+        "framework_large_" + stringify(i),
+        {role},
+        GetParam().frameworksPerRole));
+  }
+
+  // See comment above in `QuotaParam` regarding how we decide agent counts.
+  size_t agentCount =
+    GetParam().smallQuotaRoleCount / 5 + GetParam().largeQuotaRoleCount * 5;
+
+  // Add agent profiles.
+  config.agentProfiles.push_back(AgentProfile(
+      "agent",
+      agentCount,
+      CHECK_NOTERROR(Resources::parse(agentResourcesString))));
+
+  initializeCluster(config);
+
+  size_t totalRoleCount =
+    GetParam().smallQuotaRoleCount + GetParam().largeQuotaRoleCount;
+
+  cout << "Benchmark setup: " << agentCount << " agents, " << totalRoleCount
+       << " roles, " << totalRoleCount * GetParam().frameworksPerRole
+       << " frameworks" << endl;
+
+  // Pause the allocator here to prevent any event-driven allocations while
+  // setting up the quota (while setting quota currently does not lead to
+  // event-driven allocations, this behavior might change in the future).
+  allocator->pause();
+
+  foreach (const string& role, smallQuotaRoles) {
+    allocator->setQuota(role, createQuota(role, smallQuotaResourcesString));
+  }
+
+  foreach (const string& role, largeQuotaRoles) {
+    allocator->setQuota(role, createQuota(role, largeQuotaResourcesString));
+  }
+
+  allocator->resume();
+
+  cout << "Start allocation" << endl;
+
+  Stopwatch watch;
+  watch.start();
+
+  // Advance the clock and trigger a batch allocation cycle.
+  Clock::advance(config.allocationInterval);
+  Clock::settle();
+
+  watch.stop();
+
+  size_t offerCount = 0;
+
+  while (offers.get().isReady()) {
+    ++offerCount;
+  }
+
+  cout << "Made " << offerCount << " allocations in " << watch.elapsed()
+       << endl;
+
+  watch.start();
+
+  // Advance the clock and trigger a batch allocation cycle.
+  Clock::advance(config.allocationInterval);
+  Clock::settle();
+
+  watch.stop();
+
+  // No allocations should be made because all resources were allocated in the
+  // first round.
+  EXPECT_TRUE(offers.get().isPending());
+
+  cout << "Made 0 allocation in " << watch.elapsed() << endl;
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[mesos] 04/06: Moved a few allocator test helpers to `tests/allocator.hpp`.

Posted by mz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mzhu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 80da1fca1a2b765e2868bd30fc6b306e164cce28
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Thu Oct 18 10:49:38 2018 -0700

    Moved a few allocator test helpers to `tests/allocator.hpp`.
    
    This helps to share the helpers between
    `hierarchical_allocator_tests.cpp` and
    `hierarchical_allocator_benchmarks.cpp`.
    
    Review: https://reviews.apache.org/r/69096
---
 src/Makefile.am                            |  1 +
 src/tests/CMakeLists.txt                   |  1 +
 src/tests/allocator.cpp                    | 48 ++++++++++++++++++++++++++++++
 src/tests/allocator.hpp                    |  8 +++++
 src/tests/hierarchical_allocator_tests.cpp | 18 -----------
 5 files changed, 58 insertions(+), 18 deletions(-)

diff --git a/src/Makefile.am b/src/Makefile.am
index df00617..3550051 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -2450,6 +2450,7 @@ mesos_tests_SOURCES =						\
   tests/active_user_test_helper.cpp				\
   tests/active_user_test_helper.hpp				\
   tests/agent_container_api_tests.cpp				\
+  tests/allocator.cpp				    \
   tests/allocator.hpp						\
   tests/anonymous_tests.cpp					\
   tests/api_tests.cpp						\
diff --git a/src/tests/CMakeLists.txt b/src/tests/CMakeLists.txt
index 553516a..c588183 100644
--- a/src/tests/CMakeLists.txt
+++ b/src/tests/CMakeLists.txt
@@ -41,6 +41,7 @@ endif ()
 #################
 set(MESOS_TESTS_UTILS_SRC
   active_user_test_helper.cpp
+  allocator.cpp
   cluster.cpp
   containerizer.cpp
   environment.cpp
diff --git a/src/tests/allocator.cpp b/src/tests/allocator.cpp
new file mode 100644
index 0000000..5534187
--- /dev/null
+++ b/src/tests/allocator.cpp
@@ -0,0 +1,48 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License
+
+#include <string>
+
+#include "tests/allocator.hpp"
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+Quota createQuota(const string& role, const string& resources)
+{
+  mesos::quota::QuotaInfo quotaInfo;
+  quotaInfo.set_role(role);
+  *quotaInfo.mutable_guarantee() = CHECK_NOTERROR(Resources::parse(resources));
+
+  return Quota{quotaInfo};
+}
+
+
+WeightInfo createWeightInfo(const string& role, double weight)
+{
+  WeightInfo weightInfo;
+  weightInfo.set_role(role);
+  weightInfo.set_weight(weight);
+
+  return weightInfo;
+}
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {
diff --git a/src/tests/allocator.hpp b/src/tests/allocator.hpp
index 9437f58..0718bef 100644
--- a/src/tests/allocator.hpp
+++ b/src/tests/allocator.hpp
@@ -39,6 +39,14 @@ namespace mesos {
 namespace internal {
 namespace tests {
 
+// Allocator test helpers.
+
+Quota createQuota(const std::string& role, const std::string& resources);
+
+
+WeightInfo createWeightInfo(const std::string& role, double weight);
+
+
 // The following actions make up for the fact that DoDefault
 // cannot be used inside a DoAll, for example:
 // EXPECT_CALL(allocator, addFramework(_, _, _, _, _))
diff --git a/src/tests/hierarchical_allocator_tests.cpp b/src/tests/hierarchical_allocator_tests.cpp
index ee49b83..a3a6d7b 100644
--- a/src/tests/hierarchical_allocator_tests.cpp
+++ b/src/tests/hierarchical_allocator_tests.cpp
@@ -248,15 +248,6 @@ protected:
     return frameworkInfo;
   }
 
-  static Quota createQuota(const string& role, const string& resources)
-  {
-    mesos::quota::QuotaInfo quotaInfo;
-    quotaInfo.set_role(role);
-    quotaInfo.mutable_guarantee()->CopyFrom(Resources::parse(resources).get());
-
-    return Quota{quotaInfo};
-  }
-
   Resources createRevocableResources(
       const string& name,
       const string& value,
@@ -267,15 +258,6 @@ protected:
     return resource;
   }
 
-  static WeightInfo createWeightInfo(const string& role, double weight)
-  {
-    WeightInfo weightInfo;
-    weightInfo.set_role(role);
-    weightInfo.set_weight(weight);
-
-    return weightInfo;
-  }
-
 protected:
   master::Flags flags;
 


[mesos] 06/06: Added a benchmark to compare quota and nonquota allocation performance.

Posted by mz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

mzhu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 740fa11a33df8528742b3e784206d00111edc4a3
Author: Meng Zhu <mz...@mesosphere.io>
AuthorDate: Fri Oct 19 22:44:04 2018 -0700

    Added a benchmark to compare quota and nonquota allocation performance.
    
    This benchmark evaluates the performance difference between nonquota
    and quota settings. In both settings, the same allocations are made
    for fair comparison. In particular, since the agent will always be
    allocated as a whole in nonquota settings, we should also avoid
    agent chopping in quota setting as well. Thus in this benchmark,
    quotas are only set to be multiples of whole agent resources.
    This is also why we have this dedicated benchmark for comparison
    rather than extending the existing quota benchmarks (which involves
    agent chopping).
    
    Review: https://reviews.apache.org/r/69098
---
 src/tests/hierarchical_allocator_benchmarks.cpp | 151 ++++++++++++++++++++++++
 1 file changed, 151 insertions(+)

diff --git a/src/tests/hierarchical_allocator_benchmarks.cpp b/src/tests/hierarchical_allocator_benchmarks.cpp
index 9542c9c..822c2ef 100644
--- a/src/tests/hierarchical_allocator_benchmarks.cpp
+++ b/src/tests/hierarchical_allocator_benchmarks.cpp
@@ -646,6 +646,157 @@ TEST_P(HierarchicalAllocator_BENCHMARK_WithQuotaParam, LargeAndSmallQuota)
   cout << "Made 0 allocation in " << watch.elapsed() << endl;
 }
 
+
+struct NonQuotaVsQuotaParam
+{
+  NonQuotaVsQuotaParam(
+      const size_t _roleCount,
+      const size_t _agentsPerRole,
+      const size_t _frameworksPerRole,
+      const bool _setQuota)
+    : roleCount(_roleCount),
+      agentsPerRole(_agentsPerRole),
+      frameworksPerRole(_frameworksPerRole),
+      setQuota(_setQuota) {}
+
+  const size_t roleCount;
+
+  // This determines the number of agents needed to satisfy a role's quota.
+  // And total number of agents in the cluster will be
+  // roleCount * agentsPerRole.
+  const size_t agentsPerRole;
+
+  const size_t frameworksPerRole;
+
+  const bool setQuota;
+};
+
+
+class HierarchicalAllocator_BENCHMARK_WithNonQuotaVsQuotaParam
+  : public HierarchicalAllocations_BENCHMARK_TestBase,
+    public WithParamInterface<NonQuotaVsQuotaParam> {};
+
+
+INSTANTIATE_TEST_CASE_P(
+    NonQuotaVsQuotaParam,
+    HierarchicalAllocator_BENCHMARK_WithNonQuotaVsQuotaParam,
+    ::testing::Values(
+        // 10 roles, 10*2 = 20 agents, 10*2 = 20 frameworks,
+        // without and with quota.
+        NonQuotaVsQuotaParam(10U, 2U, 2U, false),
+        NonQuotaVsQuotaParam(10U, 2U, 2U, true),
+        // 100 roles, 100*2 = 200 agents, 100*2 = 200 frameworks.
+        // without and with quota.
+        NonQuotaVsQuotaParam(100U, 2U, 2U, false),
+        NonQuotaVsQuotaParam(100U, 2U, 2U, true),
+        // 1000 roles, 1000*2 = 2000 agents, 1000*2 = 2000 frameworks.
+        // without and with quota.
+        NonQuotaVsQuotaParam(1000U, 2U, 2U, false),
+        NonQuotaVsQuotaParam(1000U, 2U, 2U, true)));
+
+
+// This benchmark evaluates the performance difference between nonquota
+// and quota settings. In both settings, the same allocations are made
+// for fair comparison. In particular, since the agent will always be
+// allocated as a whole in nonquota settings, we should also avoid
+// agent chopping in quota setting as well. Thus in this benchmark,
+// quotas are only set to be multiples of whole agent resources.
+// This is also why we have this dedicated benchmark for comparison
+// rather than extending the existing quota benchmarks (which involves
+// agent chopping).
+TEST_P(
+    HierarchicalAllocator_BENCHMARK_WithNonQuotaVsQuotaParam, NonQuotaVsQuota)
+{
+  // Pause the clock because we want to manually drive the allocations.
+  Clock::pause();
+
+  const string agentResourcesString = "cpus:2;mem:2048;disk:2048";
+
+  BenchmarkConfig config;
+
+  // Store roles for setting up quota later if needed.
+  vector<string> roles;
+
+  for (size_t i = 0; i < GetParam().roleCount; i++) {
+    string role("role" + stringify(i));
+    roles.push_back(role);
+    config.frameworkProfiles.push_back(FrameworkProfile(
+        "framework_" + stringify(i), {role}, GetParam().frameworksPerRole));
+  }
+
+  size_t agentCount = GetParam().roleCount * GetParam().agentsPerRole;
+
+  // Add agent profiles.
+  config.agentProfiles.push_back(AgentProfile(
+      "agent",
+      agentCount,
+      CHECK_NOTERROR(Resources::parse(agentResourcesString))));
+
+  initializeCluster(config);
+
+  if (GetParam().setQuota) {
+    // We ensure the same allocations are made for both nonquota and quota
+    // settings for fair comparison. Thus quota is set to consume multiple
+    // agents and each agent will be offered as a whole.
+    const string quotaResourcesString =
+      "cpus:" + stringify(2 * GetParam().agentsPerRole) +
+      ";mem:" + stringify(2048 * GetParam().agentsPerRole) +
+      ";disk:" + stringify(2048 * GetParam().agentsPerRole);
+
+    // Pause the allocator here to prevent any event-driven allocations while
+    // setting up the quota (while setting quota currently does not lead to
+    // event-driven allocations, this behavior might change in the future).
+    allocator->pause();
+
+    foreach (const string& role, roles) {
+      allocator->setQuota(role, createQuota(role, quotaResourcesString));
+    }
+
+    allocator->resume();
+
+    cout << "Quota run setup: ";
+  } else {
+    cout << "Nonquota run setup: ";
+  }
+
+  cout << agentCount << " agents, " << GetParam().roleCount << " roles, "
+       << GetParam().roleCount * GetParam().frameworksPerRole << " frameworks"
+       << endl;
+
+  Stopwatch watch;
+  watch.start();
+
+  // Advance the clock and trigger a batch allocation cycle.
+  Clock::advance(config.allocationInterval);
+  Clock::settle();
+
+  watch.stop();
+
+  size_t offerCount = 0;
+
+  while (offers.get().isReady()) {
+    offerCount++;
+  }
+
+  cout << "Made " << offerCount << " allocations in " << watch.elapsed()
+       << endl;
+
+  watch.start();
+
+  // Advance the clock and trigger a batch allocation cycle.
+  Clock::advance(config.allocationInterval);
+  Clock::settle();
+
+  watch.stop();
+
+  // No allocations should be made because all resources were allocated in the
+  // first round.
+  EXPECT_TRUE(offers.get().isPending());
+
+  cout << "Made 0 allocation in " << watch.elapsed() << endl;
+}
+
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {