You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by bm...@apache.org on 2017/04/28 23:17:09 UTC

[1/2] mesos git commit: Updated Suppress / Revive calls to take multiple roles.

Repository: mesos
Updated Branches:
  refs/heads/master 6f6a5868b -> 65cc98879


Updated Suppress / Revive calls to take multiple roles.

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


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

Branch: refs/heads/master
Commit: 65cc98879f68208a51587c35aba410418474cd11
Parents: 786f9f6
Author: Benjamin Mahler <bm...@apache.org>
Authored: Thu Apr 27 17:21:13 2017 -0700
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Fri Apr 28 15:59:34 2017 -0700

----------------------------------------------------------------------
 include/mesos/allocator/allocator.hpp       | 20 ++++----
 include/mesos/scheduler/scheduler.proto     | 16 +++----
 include/mesos/v1/scheduler/scheduler.proto  | 16 +++----
 src/master/allocator/mesos/allocator.hpp    | 16 +++----
 src/master/allocator/mesos/hierarchical.cpp | 10 ++--
 src/master/allocator/mesos/hierarchical.hpp |  4 +-
 src/master/master.cpp                       | 60 ++++++++++++------------
 src/master/master.hpp                       |  2 +-
 src/messages/messages.proto                 |  6 +--
 src/tests/allocator.hpp                     |  4 +-
 src/tests/hierarchical_allocator_tests.cpp  | 18 +++----
 11 files changed, 83 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/65cc9887/include/mesos/allocator/allocator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/allocator/allocator.hpp b/include/mesos/allocator/allocator.hpp
index 6eda1b8..dc34a1b 100644
--- a/include/mesos/allocator/allocator.hpp
+++ b/include/mesos/allocator/allocator.hpp
@@ -350,27 +350,23 @@ public:
    * Suppresses offers.
    *
    * Informs the allocator to stop sending offers to this framework for the
-   * specified role. If the role is not specified, we will stop sending offers
-   * to this framework for all of its roles.
-   *
-   * @param role The optional role parameter allows frameworks with multiple
-   *     roles to do fine-grained suppression.
+   * specified roles. If `roles` is an empty set, we will stop sending offers
+   * to this framework for all of the framework's subscribed roles.
    */
   virtual void suppressOffers(
       const FrameworkID& frameworkId,
-      const Option<std::string>& role) = 0;
+      const std::set<std::string>& roles) = 0;
 
   /**
-   * Revives offers to this framework for the specified role. This is
+   * Revives offers to this framework for the specified roles. This is
    * invoked by a framework when it wishes to receive filtered resources
-   * immediately or get itself out of a suppressed state.
-   *
-   * @param role The optional role parameter allows frameworks with multiple
-   *     roles to do fine-grained revival.
+   * immediately or get itself out of a suppressed state. If `roles` is
+   * an empty set, it is treated as being set to all of the framework's
+   * subscribed roles.
    */
   virtual void reviveOffers(
       const FrameworkID& frameworkId,
-      const Option<std::string>& role) = 0;
+      const std::set<std::string>& roles) = 0;
 
   /**
    * Informs the allocator to set quota for the given role.

http://git-wip-us.apache.org/repos/asf/mesos/blob/65cc9887/include/mesos/scheduler/scheduler.proto
----------------------------------------------------------------------
diff --git a/include/mesos/scheduler/scheduler.proto b/include/mesos/scheduler/scheduler.proto
index cee0a17..f83b2ce 100644
--- a/include/mesos/scheduler/scheduler.proto
+++ b/include/mesos/scheduler/scheduler.proto
@@ -300,11 +300,11 @@ message Call {
     optional Filters filters = 2;
   }
 
-  // Revive offers for a specified role. If role is unset, the
-  // `REVIVE` call will revive offers for all of the roles the
-  // framework is subscribed to.
+  // Revive offers for the specified roles. If `roles` is empty,
+  // the `REVIVE` call will revive offers for all of the roles
+  // the framework is currently subscribed to.
   message Revive {
-    optional string role = 1;
+    repeated string roles = 1;
   }
 
   // Kills a specific task. If the scheduler has a custom executor,
@@ -389,11 +389,11 @@ message Call {
     repeated mesos.Request requests = 1;
   }
 
-  // Suppress offers for a specified role. If role is unset, the
-  // `SUPPRESS` call will suppress offers for all of the roles the
-  // framework is subscribed to.
+  // Suppress offers for the specified roles. If `roles` is empty,
+  // the `SUPPRESS` call will suppress offers for all of the roles
+  // the framework is currently subscribed to.
   message Suppress {
-    optional string role = 1;
+    repeated string roles = 1;
   }
 
   // Identifies who generated this call. Master assigns a framework id

http://git-wip-us.apache.org/repos/asf/mesos/blob/65cc9887/include/mesos/v1/scheduler/scheduler.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/scheduler/scheduler.proto b/include/mesos/v1/scheduler/scheduler.proto
index 00a54ec..d923cb9 100644
--- a/include/mesos/v1/scheduler/scheduler.proto
+++ b/include/mesos/v1/scheduler/scheduler.proto
@@ -290,11 +290,11 @@ message Call {
     optional Filters filters = 2;
   }
 
-  // Revive offers for a specified role. If role is unset, the
-  // `REVIVE` call will revive offers for all of the roles the
-  // framework is subscribed to.
+  // Revive offers for the specified roles. If `roles` is empty,
+  // the `REVIVE` call will revive offers for all of the roles
+  // the framework is currently subscribed to.
   message Revive {
-    optional string role = 1;
+    repeated string roles = 1;
   }
 
   // Kills a specific task. If the scheduler has a custom executor,
@@ -379,11 +379,11 @@ message Call {
     repeated mesos.v1.Request requests = 1;
   }
 
-  // Suppress offers for a specified role. If role is unset, the
-  // `SUPPRESS` call will suppress offers for all of the roles the
-  // framework is subscribed to.
+  // Suppress offers for the specified roles. If `roles` is empty,
+  // the `SUPPRESS` call will suppress offers for all of the roles
+  // the framework is currently subscribed to.
   message Suppress {
-    optional string role = 1;
+    repeated string roles = 1;
   }
 
   // Identifies who generated this call. Master assigns a framework id

http://git-wip-us.apache.org/repos/asf/mesos/blob/65cc9887/src/master/allocator/mesos/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/allocator.hpp b/src/master/allocator/mesos/allocator.hpp
index 57b54b8..119b461 100644
--- a/src/master/allocator/mesos/allocator.hpp
+++ b/src/master/allocator/mesos/allocator.hpp
@@ -145,11 +145,11 @@ public:
 
   void suppressOffers(
       const FrameworkID& frameworkId,
-      const Option<std::string>& role);
+      const std::set<std::string>& roles);
 
   void reviveOffers(
       const FrameworkID& frameworkId,
-      const Option<std::string>& role);
+      const std::set<std::string>& roles);
 
   void setQuota(
       const std::string& role,
@@ -282,11 +282,11 @@ public:
 
   virtual void suppressOffers(
       const FrameworkID& frameworkId,
-      const Option<std::string>& role) = 0;
+      const std::set<std::string>& roles) = 0;
 
   virtual void reviveOffers(
       const FrameworkID& frameworkId,
-      const Option<std::string>& role) = 0;
+      const std::set<std::string>& roles) = 0;
 
   virtual void setQuota(
       const std::string& role,
@@ -613,26 +613,26 @@ inline void MesosAllocator<AllocatorProcess>::recoverResources(
 template <typename AllocatorProcess>
 inline void MesosAllocator<AllocatorProcess>::suppressOffers(
     const FrameworkID& frameworkId,
-    const Option<std::string>& role)
+    const std::set<std::string>& roles)
 {
   process::dispatch(
       process,
       &MesosAllocatorProcess::suppressOffers,
       frameworkId,
-      role);
+      roles);
 }
 
 
 template <typename AllocatorProcess>
 inline void MesosAllocator<AllocatorProcess>::reviveOffers(
     const FrameworkID& frameworkId,
-    const Option<std::string>& role)
+    const std::set<std::string>& roles)
 {
   process::dispatch(
       process,
       &MesosAllocatorProcess::reviveOffers,
       frameworkId,
-      role);
+      roles);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/65cc9887/src/master/allocator/mesos/hierarchical.cpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.cpp b/src/master/allocator/mesos/hierarchical.cpp
index ce01128..b75ed9a 100644
--- a/src/master/allocator/mesos/hierarchical.cpp
+++ b/src/master/allocator/mesos/hierarchical.cpp
@@ -1195,7 +1195,7 @@ void HierarchicalAllocatorProcess::recoverResources(
 
 void HierarchicalAllocatorProcess::suppressOffers(
     const FrameworkID& frameworkId,
-    const Option<string>& role)
+    const set<string>& roles_)
 {
   CHECK(initialized);
   CHECK(frameworks.contains(frameworkId));
@@ -1205,8 +1205,7 @@ void HierarchicalAllocatorProcess::suppressOffers(
   // Deactivating the framework in the sorter is fine as long as
   // SUPPRESS is not parameterized. When parameterization is added,
   // we have to differentiate between the cases here.
-  const set<string>& roles =
-    role.isSome() ? set<string>{role.get()} : framework.roles;
+  const set<string>& roles = roles_.empty() ? framework.roles : roles_;
 
   foreach (const string& role, roles) {
     CHECK(frameworkSorters.contains(role));
@@ -1220,7 +1219,7 @@ void HierarchicalAllocatorProcess::suppressOffers(
 
 void HierarchicalAllocatorProcess::reviveOffers(
     const FrameworkID& frameworkId,
-    const Option<string>& role)
+    const set<string>& roles_)
 {
   CHECK(initialized);
   CHECK(frameworks.contains(frameworkId));
@@ -1229,8 +1228,7 @@ void HierarchicalAllocatorProcess::reviveOffers(
   framework.offerFilters.clear();
   framework.inverseOfferFilters.clear();
 
-  const set<string>& roles =
-    role.isSome() ? set<string>{role.get()} : framework.roles;
+  const set<string>& roles = roles_.empty() ? framework.roles : roles_;
 
   // Activating the framework in the sorter on REVIVE is fine as long as
   // SUPPRESS is not parameterized. When parameterization is added,

http://git-wip-us.apache.org/repos/asf/mesos/blob/65cc9887/src/master/allocator/mesos/hierarchical.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.hpp b/src/master/allocator/mesos/hierarchical.hpp
index 79420fa..123f97c 100644
--- a/src/master/allocator/mesos/hierarchical.hpp
+++ b/src/master/allocator/mesos/hierarchical.hpp
@@ -190,11 +190,11 @@ public:
 
   void suppressOffers(
       const FrameworkID& frameworkId,
-      const Option<std::string>& role);
+      const std::set<std::string>& roles);
 
   void reviveOffers(
       const FrameworkID& frameworkId,
-      const Option<std::string>& role);
+      const std::set<std::string>& roles);
 
   void setQuota(
       const std::string& role,

http://git-wip-us.apache.org/repos/asf/mesos/blob/65cc9887/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 69f52a4..31a7a2f 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -821,7 +821,7 @@ void Master::initialize()
   install<ReviveOffersMessage>(
       &Master::reviveOffers,
       &ReviveOffersMessage::framework_id,
-      &ReviveOffersMessage::role);
+      &ReviveOffersMessage::roles);
 
   install<KillTaskMessage>(
       &Master::killTask,
@@ -3301,34 +3301,33 @@ void Master::suppress(
 
   ++metrics->messages_suppress_offers;
 
-  const Option<string> role =
-    suppress.has_role() ? Option<string>(suppress.role()) : None();
+  set<string> roles;
 
-  // Validate role if it is set. We need to make sure the role is valid
-  // and also one of the framework roles.
-  if (role.isSome()) {
-    // There maybe cases that the framework developer set an invalid role
-    // when constructing `scheduler::Call::Suppress`.
-    Option<Error> roleError = roles::validate(role.get());
+  // Validate the roles, if provided. We need to make sure the
+  // roles is valid and also contained within the framework roles.
+  // Note that if a single role is invalid, we drop the entire
+  // call and do not suppress the valid roles.
+  foreach (const string& role, suppress.roles()) {
+    Option<Error> roleError = roles::validate(role);
     if (roleError.isSome()) {
       drop(framework,
            suppress,
-           "suppression role is invalid: " + roleError->message);
-
+           "suppression role '" + role + "' is invalid: " + roleError->message);
       return;
     }
 
-    if (framework->roles.count(role.get()) == 0) {
+    if (framework->roles.count(role) == 0) {
       drop(framework,
            suppress,
-           "suppression role " + role.get() + " is not one"
+           "suppression role '" + role + "' is not one"
            " of the frameworks's subscribed roles");
-
       return;
     }
+
+    roles.insert(role);
   }
 
-  allocator->suppressOffers(framework->id(), role);
+  allocator->suppressOffers(framework->id(), roles);
 }
 
 
@@ -4906,7 +4905,7 @@ void Master::declineInverseOffers(
 void Master::reviveOffers(
     const UPID& from,
     const FrameworkID& frameworkId,
-    const string& role)
+    const vector<string>& roles)
 {
   Framework* framework = getFramework(frameworkId);
 
@@ -4925,8 +4924,8 @@ void Master::reviveOffers(
   }
 
   scheduler::Call::Revive call;
-  if (!role.empty()) {
-    call.set_role(role);
+  foreach (const string& role, roles) {
+    call.add_roles(role);
   }
 
   revive(framework, call);
@@ -4943,32 +4942,33 @@ void Master::revive(
 
   ++metrics->messages_revive_offers;
 
-  const Option<string> role =
-    revive.has_role() ? Option<string>(revive.role()) : None();
+  set<string> roles;
 
-  // Validate role if it is set. We need to make sure the role is valid
-  // and also one of the framework roles.
-  if (role.isSome()) {
-    Option<Error> roleError = roles::validate(role.get());
+  // Validate the roles, if provided. We need to make sure the
+  // roles is valid and also contained within the framework roles.
+  // Note that if a single role is invalid, we drop the entire
+  // call and do not suppress the valid roles.
+  foreach (const string& role, revive.roles()) {
+    Option<Error> roleError = roles::validate(role);
     if (roleError.isSome()) {
       drop(framework,
            revive,
-           "revive role is invalid: " + roleError->message);
-
+           "revive role '" + role + "' is invalid: " + roleError->message);
       return;
     }
 
-    if (framework->roles.count(role.get()) == 0) {
+    if (framework->roles.count(role) == 0) {
       drop(framework,
            revive,
-           "revive role " + role.get() + " is not one"
+           "revive role '" + role + "' is not one"
            " of the frameworks's subscribed roles");
-
       return;
     }
+
+    roles.insert(role);
   }
 
-  allocator->reviveOffers(framework->id(), role);
+  allocator->reviveOffers(framework->id(), roles);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/65cc9887/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index eca353b..89d0790 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -357,7 +357,7 @@ public:
   void reviveOffers(
       const process::UPID& from,
       const FrameworkID& frameworkId,
-      const std::string& role);
+      const std::vector<std::string>& role);
 
   void killTask(
       const process::UPID& from,

http://git-wip-us.apache.org/repos/asf/mesos/blob/65cc9887/src/messages/messages.proto
----------------------------------------------------------------------
diff --git a/src/messages/messages.proto b/src/messages/messages.proto
index 6cc06eb..eae9ce5 100644
--- a/src/messages/messages.proto
+++ b/src/messages/messages.proto
@@ -279,12 +279,12 @@ message RescindInverseOfferMessage {
  * Used by the pre-Event/Call Mesos scheduler driver.
  * See scheduler::Call::Revive.
  *
- * NOTE: There is no optional `role` field in V0 API as we will eventually
- * move to V1 API.
+ * NOTE: There is no `roles` field in V0 API as we will
+ * eventually move to V1 API.
  */
 message ReviveOffersMessage {
   required FrameworkID framework_id = 1;
-  optional string role = 2;
+  repeated string roles = 2;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/65cc9887/src/tests/allocator.hpp
----------------------------------------------------------------------
diff --git a/src/tests/allocator.hpp b/src/tests/allocator.hpp
index 6b71c57..4ea7224 100644
--- a/src/tests/allocator.hpp
+++ b/src/tests/allocator.hpp
@@ -453,11 +453,11 @@ public:
 
   MOCK_METHOD2(suppressOffers, void(
       const FrameworkID&,
-      const Option<std::string>&));
+      const std::set<std::string>&));
 
   MOCK_METHOD2(reviveOffers, void(
       const FrameworkID&,
-      const Option<std::string>&));
+      const std::set<std::string>&));
 
   MOCK_METHOD2(setQuota, void(
       const std::string&,

http://git-wip-us.apache.org/repos/asf/mesos/blob/65cc9887/src/tests/hierarchical_allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/hierarchical_allocator_tests.cpp b/src/tests/hierarchical_allocator_tests.cpp
index 84bb6f3..ebc4868 100644
--- a/src/tests/hierarchical_allocator_tests.cpp
+++ b/src/tests/hierarchical_allocator_tests.cpp
@@ -3333,7 +3333,7 @@ TEST_F(HierarchicalAllocatorTest, DeactivateAndReactivateFramework)
       None());
 
   // Suppress offers and disconnect framework.
-  allocator->suppressOffers(framework.id(), None());
+  allocator->suppressOffers(framework.id(), {});
   allocator->deactivateFramework(framework.id());
 
   // Advance the clock and trigger a background allocation cycle.
@@ -3392,7 +3392,7 @@ TEST_F(HierarchicalAllocatorTest, SuppressAndReviveOffers)
 
   // Here the revival is totally unnecessary but we should tolerate the
   // framework's redundant REVIVE calls.
-  allocator->reviveOffers(framework.id(), None());
+  allocator->reviveOffers(framework.id(), {});
 
   // Settle to ensure that the dispatched allocation is executed.
   Clock::settle();
@@ -3407,7 +3407,7 @@ TEST_F(HierarchicalAllocatorTest, SuppressAndReviveOffers)
       allocatedResources(agent.resources(), "role1"),
       None());
 
-  allocator->suppressOffers(framework.id(), None());
+  allocator->suppressOffers(framework.id(), {});
 
   // Advance the clock and trigger a background allocation cycle.
   Clock::advance(flags.allocation_interval);
@@ -3417,7 +3417,7 @@ TEST_F(HierarchicalAllocatorTest, SuppressAndReviveOffers)
   EXPECT_TRUE(allocation.isPending());
 
   // Revive again and this time it should work.
-  allocator->reviveOffers(framework.id(), None());
+  allocator->reviveOffers(framework.id(), {});
 
   // Framework will be offered all of agent's resources again after
   // reviving offers.
@@ -4228,7 +4228,7 @@ TEST_F(HierarchicalAllocatorTest, ReviveOffers)
   Future<Allocation> allocation = allocations.get();
   EXPECT_TRUE(allocation.isPending());
 
-  allocator->reviveOffers(framework.id(), None());
+  allocator->reviveOffers(framework.id(), {});
 
   // Framework will be offered all of agent's resources again
   // after reviving offers.
@@ -4254,7 +4254,7 @@ TEST_F(HierarchicalAllocatorTest, SuppressAndReviveOffersWithMultiRole)
   FrameworkInfo framework = createFrameworkInfo({"role1", "role2"});
 
   allocator->addFramework(framework.id(), framework, {}, true);
-  allocator->suppressOffers(framework.id(), "role1");
+  allocator->suppressOffers(framework.id(), {"role1"});
 
   // Total cluster resources will become cpus=2, mem=1024.
   SlaveInfo agent = createSlaveInfo("cpus:2;mem:1024;disk:0");
@@ -4289,7 +4289,7 @@ TEST_F(HierarchicalAllocatorTest, SuppressAndReviveOffersWithMultiRole)
 
   // Revive offers for role1, after which the agent's resources
   // should be offered to it.
-  allocator->reviveOffers(framework.id(), "role1");
+  allocator->reviveOffers(framework.id(), {"role1"});
 
   expected = Allocation(
       framework.id(),
@@ -5389,7 +5389,7 @@ TEST_P(HierarchicalAllocator_BENCHMARK_Test, SuppressOffers)
     // 'frameworkCount % allocationsCount' of frameworks not suppressed. For
     // the purposes of the benchmark this is not an issue.
     for (size_t j = 0; j < frameworkCount / allocationsCount; ++j) {
-      allocator->suppressOffers(frameworks[suppressCount].id(), None());
+      allocator->suppressOffers(frameworks[suppressCount].id(), {});
       ++suppressCount;
     }
 
@@ -5584,7 +5584,7 @@ TEST_P(HierarchicalAllocator_BENCHMARK_Test, AllocatorBacklog)
   // events. The allocator doesn't have more resources to allocate
   // but still incurs the overhead of additional allocation runs.
   for (size_t i = 0; i < frameworkCount; i++) {
-    allocator->reviveOffers(frameworks.at(i).id(), None());
+    allocator->reviveOffers(frameworks.at(i).id(), {});
   }
 
   // Wait for all the `reviveOffers` operations to be processed.


[2/2] mesos git commit: Fixed the implementation of per-role Suppress / Revive calls.

Posted by bm...@apache.org.
Fixed the implementation of per-role Suppress / Revive calls.

The existing implementation uses a single suppression boolean which
results in incorrect handling of per-role suppression / revival.

See: https://issues.apache.org/jira/browse/MESOS-7430

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


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

Branch: refs/heads/master
Commit: 786f9f6771988719cd6e0dde2ba333d1717fccfb
Parents: 6f6a586
Author: Benjamin Mahler <bm...@apache.org>
Authored: Thu Apr 27 17:34:03 2017 -0700
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Fri Apr 28 15:59:34 2017 -0700

----------------------------------------------------------------------
 src/master/allocator/mesos/hierarchical.cpp | 22 ++++++----------------
 src/master/allocator/mesos/hierarchical.hpp |  3 ---
 2 files changed, 6 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/786f9f67/src/master/allocator/mesos/hierarchical.cpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.cpp b/src/master/allocator/mesos/hierarchical.cpp
index 84dc31d..ce01128 100644
--- a/src/master/allocator/mesos/hierarchical.cpp
+++ b/src/master/allocator/mesos/hierarchical.cpp
@@ -127,7 +127,6 @@ private:
 HierarchicalAllocatorProcess::Framework::Framework(
     const FrameworkInfo& frameworkInfo)
   : roles(protobuf::framework::getRoles(frameworkInfo)),
-    suppressed(false),
     capabilities(frameworkInfo.capabilities()) {}
 
 
@@ -372,10 +371,6 @@ void HierarchicalAllocatorProcess::deactivateFramework(
   framework.offerFilters.clear();
   framework.inverseOfferFilters.clear();
 
-  // Clear the suppressed flag to make sure the framework can be offered
-  // resources immediately after getting activated.
-  framework.suppressed = false;
-
   LOG(INFO) << "Deactivated framework " << frameworkId;
 }
 
@@ -1206,7 +1201,6 @@ void HierarchicalAllocatorProcess::suppressOffers(
   CHECK(frameworks.contains(frameworkId));
 
   Framework& framework = frameworks.at(frameworkId);
-  framework.suppressed = true;
 
   // Deactivating the framework in the sorter is fine as long as
   // SUPPRESS is not parameterized. When parameterization is added,
@@ -1238,16 +1232,12 @@ void HierarchicalAllocatorProcess::reviveOffers(
   const set<string>& roles =
     role.isSome() ? set<string>{role.get()} : framework.roles;
 
-  if (framework.suppressed) {
-    framework.suppressed = false;
-
-    // Activating the framework in the sorter on REVIVE is fine as long as
-    // SUPPRESS is not parameterized. When parameterization is added,
-    // we may need to differentiate between the cases here.
-    foreach (const string& role, roles) {
-      CHECK(frameworkSorters.contains(role));
-      frameworkSorters.at(role)->activate(frameworkId.value());
-    }
+  // Activating the framework in the sorter on REVIVE is fine as long as
+  // SUPPRESS is not parameterized. When parameterization is added,
+  // we may need to differentiate between the cases here.
+  foreach (const string& role, roles) {
+    CHECK(frameworkSorters.contains(role));
+    frameworkSorters.at(role)->activate(frameworkId.value());
   }
 
   // We delete each actual `OfferFilter` when

http://git-wip-us.apache.org/repos/asf/mesos/blob/786f9f67/src/master/allocator/mesos/hierarchical.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.hpp b/src/master/allocator/mesos/hierarchical.hpp
index 219f508..79420fa 100644
--- a/src/master/allocator/mesos/hierarchical.hpp
+++ b/src/master/allocator/mesos/hierarchical.hpp
@@ -302,9 +302,6 @@ protected:
 
     std::set<std::string> roles;
 
-    // Whether the framework suppresses offers.
-    bool suppressed;
-
     protobuf::framework::Capabilities capabilities;
 
     // Active offer and inverse offer filters for the framework.