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 2019/07/15 22:48:18 UTC

[mesos] branch master updated (c076c8c -> dcbee1d)

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

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


    from c076c8c  Added test for agent to leave draining state on its own.
     new 382e9de  Fixed /roles and GET_ROLES to expose all known roles.
     new d4cb3a3  Added a test to ensure that roles with only reservations are exposed.
     new f99e181  Added a test to ensure that ancestor roles are exposed in /roles.
     new dcbee1d  Minor cleanups in role_tests.cpp.

The 4 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/master/http.cpp             |   8 +-
 src/master/master.cpp           | 156 +++++++++++++++++++++++++++------
 src/master/master.hpp           | 144 +++++--------------------------
 src/master/readonly_handler.cpp |  41 ++++-----
 src/tests/role_tests.cpp        | 185 ++++++++++++++++++++++++++++++++++++++--
 5 files changed, 358 insertions(+), 176 deletions(-)


[mesos] 02/04: Added a test to ensure that roles with only reservations are exposed.

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

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

commit d4cb3a34ce00aa6177f6d206c552e309ad250598
Author: Benjamin Mahler <bm...@apache.org>
AuthorDate: Mon Jul 15 16:47:02 2019 -0400

    Added a test to ensure that roles with only reservations are exposed.
    
    This adds a test for MESOS-9888, to ensure that if a role has only
    reservations associated with it, it gets exposed from /roles.
    
    Review: https://reviews.apache.org/r/71074
---
 src/tests/role_tests.cpp | 78 +++++++++++++++++++++++++++++++++++++++++++++++-
 1 file changed, 77 insertions(+), 1 deletion(-)

diff --git a/src/tests/role_tests.cpp b/src/tests/role_tests.cpp
index 5066e41..5a6a01a 100644
--- a/src/tests/role_tests.cpp
+++ b/src/tests/role_tests.cpp
@@ -34,6 +34,8 @@
 #include "tests/mesos.hpp"
 #include "tests/resources_utils.hpp"
 
+#include "tests/master/mock_master_api_subscriber.hpp"
+
 using mesos::internal::master::Master;
 using mesos::internal::slave::Slave;
 
@@ -870,7 +872,81 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(RoleTest, EndpointImplicitRolesQuotas)
 
   EXPECT_EQ(*expected, *parse)
     << "expected " << stringify(*expected)
-    << " vs actual " << stringify(*parse);}
+    << " vs actual " << stringify(*parse);
+}
+
+
+// This test ensures that roles with only reservations are
+// included in the /roles endpoint.
+TEST_F_TEMP_DISABLED_ON_WINDOWS(RoleTest, EndpointImplicitRolesReservations)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  v1::MockMasterAPISubscriber subscriber;
+
+  AWAIT_READY(subscriber.subscribe(master.get()->pid));
+
+  Future<Nothing> agentAdded;
+  EXPECT_CALL(subscriber, agentAdded(_))
+    .WillOnce(FutureSatisfy(&agentAdded));
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags agentFlags = CreateSlaveFlags();
+  agentFlags.resources = "cpus(role):1;mem(role):10";
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), agentFlags);
+
+  AWAIT_READY(agentAdded);
+
+  // Check that the /roles endpoint contains the role.
+  {
+    Future<Response> response = process::http::get(
+        master.get()->pid,
+        "roles",
+        None(),
+        createBasicAuthHeaders(DEFAULT_CREDENTIAL));
+
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
+
+    Try<JSON::Value> parse = JSON::parse(response->body);
+    ASSERT_SOME(parse);
+
+    Try<JSON::Value> expected = JSON::parse(
+        "{"
+        "  \"roles\": ["
+        "    {"
+        "      \"frameworks\": [],"
+        "      \"name\": \"role\","
+        "      \"resources\": {},"
+        "      \"allocated\": {},"
+        "      \"offered\": {},"
+        "      \"reserved\": {"
+        "        \"cpus\": 1.0,"
+        "        \"mem\":  10.0"
+        "      },"
+        "      \"quota\": {"
+        "        \"consumed\": {"
+        "          \"cpus\": 1.0,"
+        "          \"mem\": 10.0"
+        "        },"
+        "        \"guarantee\": {},"
+        "        \"limit\": {},"
+        "        \"role\": \"role\""
+        "      },"
+        "      \"weight\": 1.0"
+        "    }"
+        "  ]"
+        "}");
+
+    ASSERT_SOME(expected);
+
+    EXPECT_EQ(*expected, *parse)
+      << "expected " << stringify(*expected)
+      << " vs actual " << stringify(*parse);
+  }
+}
 
 
 // This test ensures that master adds/removes all roles of


[mesos] 03/04: Added a test to ensure that ancestor roles are exposed in /roles.

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

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

commit f99e181d34a78ef304f9072a6685f33be99ea07f
Author: Benjamin Mahler <bm...@apache.org>
AuthorDate: Mon Jul 15 17:07:58 2019 -0400

    Added a test to ensure that ancestor roles are exposed in /roles.
    
    This adds a test for MESOS-9890, to ensure that ancestor roles with
    no objects directly associated with them get exposed in /roles.
    
    Review: https://reviews.apache.org/r/71077
---
 src/tests/role_tests.cpp | 99 ++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 99 insertions(+)

diff --git a/src/tests/role_tests.cpp b/src/tests/role_tests.cpp
index 5a6a01a..7a6c8a6 100644
--- a/src/tests/role_tests.cpp
+++ b/src/tests/role_tests.cpp
@@ -949,6 +949,105 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(RoleTest, EndpointImplicitRolesReservations)
 }
 
 
+// This test ensures that ancestor roles are exposed when
+// there are no direct objects associated with them.
+//
+// TODO(bmahler): This currently only tests the reservation
+// case, but we should also test the allocation, framework
+// subsription, and quota/weight configuration cases.
+TEST_F_TEMP_DISABLED_ON_WINDOWS(RoleTest, EndpointImplicitRolesAncestors)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  v1::MockMasterAPISubscriber subscriber;
+
+  AWAIT_READY(subscriber.subscribe(master.get()->pid));
+
+  Future<Nothing> agentAdded;
+  EXPECT_CALL(subscriber, agentAdded(_))
+    .WillOnce(FutureSatisfy(&agentAdded));
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags agentFlags = CreateSlaveFlags();
+  agentFlags.resources = "cpus(ancestor/child):1;mem(ancestor/child):10;";
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), agentFlags);
+
+  AWAIT_READY(agentAdded);
+
+  // Check that the /roles endpoint contains the role and
+  // its ancestor.
+  {
+    Future<Response> response = process::http::get(
+        master.get()->pid,
+        "roles",
+        None(),
+        createBasicAuthHeaders(DEFAULT_CREDENTIAL));
+
+    AWAIT_EXPECT_RESPONSE_STATUS_EQ(OK().status, response);
+
+    Try<JSON::Value> parse = JSON::parse(response->body);
+    ASSERT_SOME(parse);
+
+    Try<JSON::Value> expected = JSON::parse(
+        "{"
+        "  \"roles\": ["
+        "    {"
+        "      \"frameworks\": [],"
+        "      \"name\": \"ancestor\","
+        "      \"resources\": {},"
+        "      \"allocated\": {},"
+        "      \"offered\": {},"
+        "      \"reserved\": {"
+        "        \"cpus\": 1.0,"
+        "        \"mem\":  10.0"
+        "      },"
+        "      \"quota\": {"
+        "        \"consumed\": {"
+        "          \"cpus\": 1.0,"
+        "          \"mem\": 10.0"
+        "        },"
+        "        \"guarantee\": {},"
+        "        \"limit\": {},"
+        "        \"role\": \"ancestor\""
+        "      },"
+        "      \"weight\": 1.0"
+        "    },"
+        "    {"
+        "      \"frameworks\": [],"
+        "      \"name\": \"ancestor/child\","
+        "      \"resources\": {},"
+        "      \"allocated\": {},"
+        "      \"offered\": {},"
+        "      \"reserved\": {"
+        "        \"cpus\": 1.0,"
+        "        \"mem\":  10.0"
+        "      },"
+        "      \"quota\": {"
+        "        \"consumed\": {"
+        "          \"cpus\": 1.0,"
+        "          \"mem\": 10.0"
+        "        },"
+        "        \"guarantee\": {},"
+        "        \"limit\": {},"
+        "        \"role\": \"ancestor/child\""
+        "      },"
+        "      \"weight\": 1.0"
+        "    }"
+        "  ]"
+        "}");
+
+    ASSERT_SOME(expected);
+
+    EXPECT_EQ(*expected, *parse)
+      << "expected " << stringify(*expected)
+      << " vs actual " << stringify(*parse);
+  }
+}
+
+
 // This test ensures that master adds/removes all roles of
 // a multi-role framework when it registers/terminates.
 TEST_F_TEMP_DISABLED_ON_WINDOWS(


[mesos] 01/04: Fixed /roles and GET_ROLES to expose all known roles.

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

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

commit 382e9de30ab28edb59e7f443e908468c065968ab
Author: Benjamin Mahler <bm...@apache.org>
AuthorDate: Mon Jul 15 14:24:08 2019 -0400

    Fixed /roles and GET_ROLES to expose all known roles.
    
    Previously, per MESOS-9888 and MESOS-9890, the /roles and GET_ROLES
    APIs only exposed roles that had frameworks associated with them
    (either because the framework is subscribed to the role, or there
    is a framework with allocations to the role) or configured weight
    and/or quota.
    
    This approach omits some important cases:
    
      (1) Roles that have only reservations associated with them.
      (2) Roles that have only a parent relationship to other roles.
    
    This patch exposes a function that returns all "known" roles based
    on the criteria we care about:
    
      (1) Roles with configured weight or quota.
      (2) Roles with reservations.
      (3) Roles with frameworks subscribed or allocated resources.
      (4) Ancestor roles of (1), (2), or (3).
    
    Also, the resource breakdowns are pulled out from the Role struct
    and placed in a function that returns the breakdowns for all known
    roles. This was done because there is currently not a Role struct
    entry for all known roles.
    
    Review: https://reviews.apache.org/r/71073
---
 src/master/http.cpp             |   8 ++-
 src/master/master.cpp           | 156 ++++++++++++++++++++++++++++++++--------
 src/master/master.hpp           | 144 ++++++-------------------------------
 src/master/readonly_handler.cpp |  41 +++++------
 4 files changed, 178 insertions(+), 171 deletions(-)

diff --git a/src/master/http.cpp b/src/master/http.cpp
index cd0f40c..b077dd7 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -2684,7 +2684,7 @@ Future<Response> Master::Http::getRoles(
     .then(defer(master->self(),
         [this, contentType](const Owned<ObjectApprovers>& approvers)
           -> Response {
-      const vector<string> filteredRoles = master->filterRoles(approvers);
+      const vector<string> knownRoles = master->knownRoles();
 
       mesos::master::Response response;
       response.set_type(mesos::master::Response::GET_ROLES);
@@ -2692,7 +2692,11 @@ Future<Response> Master::Http::getRoles(
       mesos::master::Response::GetRoles* getRoles =
         response.mutable_get_roles();
 
-      foreach (const string& name, filteredRoles) {
+      foreach (const string& name, knownRoles) {
+        if (!approvers->approved<VIEW_ROLE>(name)) {
+          continue;
+        }
+
         mesos::Role role;
 
         if (master->weights.contains(name)) {
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 5247377..2a59f89 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -3515,46 +3515,59 @@ void Master::suppress(
 }
 
 
-vector<string> Master::filterRoles(
-    const Owned<ObjectApprovers>& approvers) const
-{
-  JSON::Object object;
-
-  // Compute the role names to return results for. When an explicit
-  // role whitelist has been configured, we use that list of names.
-  // When using implicit roles, the right behavior is a bit more
-  // subtle. There are no constraints on possible role names, so we
-  // instead list all the "interesting" roles: all roles with one or
-  // more registered frameworks, and all roles with a non-default
-  // weight or quota.
-  //
+vector<string> Master::knownRoles() const
+{
   // NOTE: we use a `std::set` to store the role names to ensure a
   // deterministic output order.
   set<string> roleList;
+
+  auto insertAncestors = [&roleList](const string& role) {
+    foreach (const string& ancestor, roles::ancestors(role)) {
+      bool inserted = roleList.insert(ancestor).second;
+
+      // We can break here as an optimization since the ancestor
+      // will have had its ancestors inserted already.
+      if (!inserted) break;
+    }
+  };
+
   if (roleWhitelist.isSome()) {
-    const hashset<string>& whitelist = roleWhitelist.get();
-    roleList.insert(whitelist.begin(), whitelist.end());
+    foreach (const string& role, *this->roleWhitelist) {
+      roleList.insert(role);
+      insertAncestors(role);
+    }
   } else {
-    hashset<string> roles = this->roles.keys();
-    roleList.insert(roles.begin(), roles.end());
+    // In terms of building a complete set of known roles, we have to visit:
+    //   (1) all entries of `roles` (which means there are frameworks
+    //       subscribed to a role or have allocations to a role)
+    //   (2) all reservation roles
+    //   (3) all roles with configured weights or quotas
+    //   (4) all ancestor roles of (1), (2), and (3).
 
-    hashset<string> weights = this->weights.keys();
-    roleList.insert(weights.begin(), weights.end());
+    foreachkey (const string& role, this->roles) {
+      roleList.insert(role);
+      insertAncestors(role);
+    }
 
-    hashset<string> quotas = this->quotas.keys();
-    roleList.insert(quotas.begin(), quotas.end());
-  }
+    foreachvalue (Slave* slave, this->slaves.registered) {
+      foreachkey (const string& role, slave->totalResources.reservations()) {
+        roleList.insert(role);
+        insertAncestors(role);
+      }
+    }
 
-  vector<string> filteredRoleList;
-  filteredRoleList.reserve(roleList.size());
+    foreachkey (const string& role, this->weights) {
+      roleList.insert(role);
+      insertAncestors(role);
+    }
 
-  foreach (const string& role, roleList) {
-    if (approvers->approved<VIEW_ROLE>(role)) {
-      filteredRoleList.push_back(role);
+    foreachkey (const string& role, this->quotas) {
+      roleList.insert(role);
+      insertAncestors(role);
     }
   }
 
-  return filteredRoleList;
+  return vector<string>(roleList.begin(), roleList.end());
 }
 
 
@@ -3568,6 +3581,93 @@ bool Master::isWhitelistedRole(const string& name) const
 }
 
 
+hashmap<string, Master::ResourceBreakdown>
+  Master::getRoleTreeResourceQuantities() const
+{
+  auto allocatedToRoleSubtree = [](const string& role) {
+    return [&](const Resource& r) {
+      CHECK(r.has_allocation_info());
+      return r.allocation_info().role() == role ||
+        roles::isStrictSubroleOf(r.allocation_info().role(), role);
+    };
+  };
+
+  auto reservedToRoleSubtree = [](const string& role) {
+    return [&](const Resource& r) {
+      return Resources::isReserved(r) &&
+        (Resources::reservationRole(r) == role ||
+         roles::isStrictSubroleOf(Resources::reservationRole(r), role));
+    };
+  };
+
+  auto offered = [&](const string& role) {
+    ResourceQuantities total;
+
+    foreachvalue (Framework* framework, frameworks.registered) {
+      total += ResourceQuantities::fromResources(
+          framework->totalOfferedResources
+            .filter(allocatedToRoleSubtree(role)));
+    }
+
+    return total;
+  };
+
+  auto allocated = [&](const string& role) {
+    ResourceQuantities total;
+
+    foreachvalue (Framework* framework, frameworks.registered) {
+      total += ResourceQuantities::fromResources(
+          framework->totalUsedResources.filter(allocatedToRoleSubtree(role)));
+    }
+
+    return total;
+  };
+
+  auto reserved = [&](const string& role) {
+    ResourceQuantities total;
+
+    foreachvalue (Slave* slave, slaves.registered) {
+      total += ResourceQuantities::fromResources(
+          slave->totalResources.filter(reservedToRoleSubtree(role)));
+    }
+
+    return total;
+  };
+
+  // Consumed quota = allocation + unallocated reservation.
+  auto consumedQuota = [&](const string& role) {
+    ResourceQuantities unallocatedReservation;
+
+    foreachvalue (Slave* slave, slaves.registered) {
+      ResourceQuantities totalReservation =
+        ResourceQuantities::fromResources(
+           slave->totalResources.filter(reservedToRoleSubtree(role)));
+
+       ResourceQuantities usedReservation;
+       foreachvalue (const Resources& r, slave->usedResources) {
+         usedReservation += ResourceQuantities::fromResources(
+             r.filter(reservedToRoleSubtree(role)));
+       }
+
+       unallocatedReservation += totalReservation - usedReservation;
+     }
+
+    return allocated(role) + unallocatedReservation;
+  };
+
+  hashmap<string, ResourceBreakdown> result;
+
+  foreach (const string& role, knownRoles()) {
+    result[role].offered = offered(role);
+    result[role].allocated = allocated(role);
+    result[role].reserved = reserved(role);
+    result[role].consumedQuota = consumedQuota(role);
+  }
+
+  return result;
+}
+
+
 void Master::launchTasks(
     const UPID& from,
     LaunchTasksMessage&& launchTasksMessage)
diff --git a/src/master/master.hpp b/src/master/master.hpp
index e8def83..dda508f 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -1155,8 +1155,15 @@ private:
       const hashset<SlaveID>& toRemoveGone,
       const process::Future<bool>& registrarResult);
 
-  std::vector<std::string> filterRoles(
-      const process::Owned<ObjectApprovers>& approvers) const;
+  // Returns all roles known to the master, if roles are whitelisted
+  // this simply returns the whitelist and any ancestors of roles in
+  // the whitelist. Otherwise, this returns:
+  //
+  //   (1) Roles with configured weight or quota.
+  //   (2) Roles with reservations.
+  //   (3) Roles with frameworks subscribed or allocated resources.
+  //   (4) Ancestor roles of (1), (2), or (3).
+  std::vector<std::string> knownRoles() const;
 
   /**
    * Returns whether the given role is on the whitelist.
@@ -1167,6 +1174,20 @@ private:
    */
   bool isWhitelistedRole(const std::string& name) const;
 
+  struct ResourceBreakdown
+  {
+    ResourceQuantities offered;
+    ResourceQuantities allocated;
+    ResourceQuantities reserved;
+    ResourceQuantities consumedQuota;
+  };
+
+  // TODO(bmahler): Store a role tree rather than the existing
+  // `roles` map which does not track the tree correctly (it does
+  // not insert ancestor entries, nor does it track roles if there
+  // are reservations but no frameworks related to them).
+  hashmap<std::string, ResourceBreakdown> getRoleTreeResourceQuantities() const;
+
   // Performs validations of the FrameworkInfo and suppressed roles set
   // which do not depend on the current state of this framework.
   Option<Error> validateFramework(
@@ -2707,59 +2728,6 @@ struct Role
     frameworks.erase(framework->id());
   }
 
-  // TODO(bmahler): This function is somewhat expensive,
-  // it should ideally migrate into a field updated in an
-  // event-driven manner within a role tree structure. Or,
-  // at least compute the overall tree when looping over
-  // all agents rather than looping over all agents for
-  // each role.
-  ResourceQuantities consumedQuota() const
-  {
-    const std::string& role = this->role; // For cleaner captures.
-
-    // Consumed quota = allocation + unallocated reservation.
-
-    ResourceQuantities allocation;
-
-    auto allocatedToRoleSubtree = [&role](const Resource& r) {
-      CHECK(r.has_allocation_info());
-      return r.allocation_info().role() == role ||
-        roles::isStrictSubroleOf(r.allocation_info().role(), role);
-    };
-
-    // Loop over all frameworks since `frameworks` only tracks
-    // those that are directly subscribed to this role, and we
-    // need to sum all descendant role allocations.
-    foreachvalue (Framework* framework, master->frameworks.registered) {
-      allocation += ResourceQuantities::fromResources(
-        framework->totalUsedResources.filter(allocatedToRoleSubtree));
-    }
-
-    ResourceQuantities unallocatedReservation;
-
-    auto reservedToRoleSubtree = [&role](const Resource& r) {
-      return Resources::isReserved(r) &&
-        (Resources::reservationRole(r) == role ||
-         roles::isStrictSubroleOf(Resources::reservationRole(r), role));
-    };
-
-    foreachvalue (Slave* slave, master->slaves.registered) {
-      ResourceQuantities totalReservation =
-        ResourceQuantities::fromResources(
-           slave->totalResources.filter(reservedToRoleSubtree));
-
-       ResourceQuantities usedReservation;
-       foreachvalue (const Resources& r, slave->usedResources) {
-         usedReservation += ResourceQuantities::fromResources(
-             r.filter(reservedToRoleSubtree));
-       }
-
-       unallocatedReservation += totalReservation - usedReservation;
-     }
-
-    return allocation + unallocatedReservation;
-  }
-
   Resources allocatedAndOfferedResources() const
   {
     Resources resources;
@@ -2779,72 +2747,6 @@ struct Role
     return resources;
   }
 
-  ResourceQuantities reserved() const
-  {
-    const std::string& role = this->role; // For cleaner captures.
-
-    ResourceQuantities total;
-
-    auto reservedToRoleSubtree = [&role](const Resource& r) {
-      return Resources::isReserved(r) &&
-        (Resources::reservationRole(r) == role ||
-         roles::isStrictSubroleOf(Resources::reservationRole(r), role));
-    };
-
-    foreachvalue (Slave* slave, master->slaves.registered) {
-      total += ResourceQuantities::fromResources(
-          slave->totalResources.filter(reservedToRoleSubtree));
-    }
-
-    return total;
-  }
-
-  ResourceQuantities allocated() const
-  {
-    const std::string& role = this->role; // For cleaner captures.
-
-    ResourceQuantities total;
-
-    auto allocatedToRoleSubtree = [&role](const Resource& r) {
-      CHECK(r.has_allocation_info());
-      return r.allocation_info().role() == role ||
-        roles::isStrictSubroleOf(r.allocation_info().role(), role);
-    };
-
-    // Loop over all frameworks since `frameworks` only tracks
-    // those that are directly subscribed to this role, and we
-    // need to sum all descendant role allocations.
-    foreachvalue (Framework* framework, master->frameworks.registered) {
-      total += ResourceQuantities::fromResources(
-          framework->totalUsedResources.filter(allocatedToRoleSubtree));
-    }
-
-    return total;
-  }
-
-  ResourceQuantities offered() const
-  {
-    const std::string& role = this->role; // For cleaner captures.
-
-    ResourceQuantities total;
-
-    auto allocatedToRoleSubtree = [&role](const Resource& r) {
-      CHECK(r.has_allocation_info());
-      return r.allocation_info().role() == role ||
-        roles::isStrictSubroleOf(r.allocation_info().role(), role);
-    };
-
-    // Loop over all frameworks since `frameworks` only tracks
-    // those that are directly subscribed to this role, and we
-    // need to sum all descendant role allocations.
-    foreachvalue (Framework* framework, master->frameworks.registered) {
-      total += ResourceQuantities::fromResources(
-          framework->totalOfferedResources.filter(allocatedToRoleSubtree));
-    }
-
-    return total;
-  }
-
   const Master* master;
   const std::string role;
 
diff --git a/src/master/readonly_handler.cpp b/src/master/readonly_handler.cpp
index f4432a5..ab8a4ce 100644
--- a/src/master/readonly_handler.cpp
+++ b/src/master/readonly_handler.cpp
@@ -699,13 +699,20 @@ process::http::Response Master::ReadOnlyHandler::roles(
     const process::Owned<ObjectApprovers>& approvers) const
 {
   const Master* master = this->master;
-  const vector<string> filteredRoles = master->filterRoles(approvers);
+
+  const vector<string> knownRoles = master->knownRoles();
+  const hashmap<string, ResourceBreakdown> resourceBreakdowns =
+    master->getRoleTreeResourceQuantities();
 
   auto roles = [&](JSON::ObjectWriter* writer) {
     writer->field(
         "roles",
         [&](JSON::ArrayWriter* writer) {
-          foreach (const string& name, filteredRoles) {
+          foreach (const string& name, knownRoles) {
+            if (!approvers->approved<VIEW_ROLE>(name)) {
+              continue;
+            }
+
             writer->element([&](JSON::ObjectWriter* writer) {
               writer->field("name", name);
 
@@ -714,6 +721,11 @@ process::http::Response Master::ReadOnlyHandler::roles(
 
               Option<Role*> role = master->roles.get(name);
 
+              CHECK_CONTAINS(resourceBreakdowns, name);
+
+              const ResourceBreakdown& resourceBreakdown =
+                resourceBreakdowns.at(name);
+
               // Prior to Mesos 1.9, this field is filled based on
               // `QuotaInfo` which is now deprecated. For backward
               // compatibility reasons, we do not use any formatter
@@ -732,28 +744,17 @@ process::http::Response Master::ReadOnlyHandler::roles(
 
                 writer->field("guarantee", quota.guarantees);
                 writer->field("limit", quota.limits);
-
-                ResourceQuantities consumed = role.isSome() ?
-                  (*role)->consumedQuota() : ResourceQuantities();
-
-                writer->field("consumed", consumed);
+                writer->field("consumed", resourceBreakdown.consumedQuota);
               });
 
-              const ResourceQuantities allocated = role.isSome() ?
-                (*role)->allocated() : ResourceQuantities();
-
-              const ResourceQuantities offered = role.isSome() ?
-                (*role)->offered() : ResourceQuantities();
-
-              const ResourceQuantities reserved = role.isSome() ?
-                (*role)->reserved() : ResourceQuantities();
-
               // Deprecated by allocated, offered, reserved.
-              writer->field("resources", allocated + offered);
+              writer->field(
+                  "resources",
+                  resourceBreakdown.allocated + resourceBreakdown.offered);
 
-              writer->field("allocated", allocated);
-              writer->field("offered", offered);
-              writer->field("reserved", reserved);
+              writer->field("allocated", resourceBreakdown.allocated);
+              writer->field("offered", resourceBreakdown.offered);
+              writer->field("reserved", resourceBreakdown.reserved);
 
               if (role.isNone()) {
                 writer->field("frameworks", [](JSON::ArrayWriter*) {});


[mesos] 04/04: Minor cleanups in role_tests.cpp.

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

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

commit dcbee1dcdb6dd5915252530fa6b4ffad900fc80f
Author: Benjamin Mahler <bm...@apache.org>
AuthorDate: Mon Jul 15 17:10:25 2019 -0400

    Minor cleanups in role_tests.cpp.
    
    Review: https://reviews.apache.org/r/71078
---
 src/tests/role_tests.cpp | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/src/tests/role_tests.cpp b/src/tests/role_tests.cpp
index 7a6c8a6..d6cc31b 100644
--- a/src/tests/role_tests.cpp
+++ b/src/tests/role_tests.cpp
@@ -287,7 +287,7 @@ TEST_F(RoleTest, ImplicitRoleStaticReservation)
 
 
 // This test checks that the "/roles" endpoint returns the expected
-// information when there are no active roles.
+// information when there are no known roles.
 TEST_F_TEMP_DISABLED_ON_WINDOWS(RoleTest, EndpointEmpty)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
@@ -320,9 +320,9 @@ TEST_F_TEMP_DISABLED_ON_WINDOWS(RoleTest, EndpointEmpty)
 
 
 // This test checks that the "/roles" endpoint returns the expected
-// information when there are configured weights and explicit roles,
-// but no registered frameworks.
-TEST_F(RoleTest, EndpointNoFrameworks)
+// information when the role whitelist is used but no frameworks
+// are present.
+TEST_F(RoleTest, EndpointWithWhitelistNoFrameworks)
 {
   master::Flags masterFlags = CreateMasterFlags();
   masterFlags.roles = "role1,role2";