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 2018/07/02 06:24:25 UTC

[5/7] mesos git commit: Introduced a random sorter as an alternative to the DRF sorter.

Introduced a random sorter as an alternative to the DRF sorter.

This sorter returns a weighted random shuffling of its clients
upon each `sort()` request.

This implementation is a copy of the `DRFSorter` with share
calculation logic (including the `dirty` bit) removed and an
adjusted `sort()` implementation. Work needs to be done to
reduce the amount of duplicated logic needed across sorter
implementations, but it is left unaddressed here.

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


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

Branch: refs/heads/1.6.x
Commit: 99ff52dc0e7dbf1f3279fd43c087154b17b4df1f
Parents: 16b9eb2
Author: Benjamin Mahler <bm...@apache.org>
Authored: Sat Jun 2 21:27:10 2018 -0400
Committer: Benjamin Mahler <bm...@apache.org>
Committed: Fri Jun 29 16:32:24 2018 -0700

----------------------------------------------------------------------
 src/CMakeLists.txt                            |   1 +
 src/Makefile.am                               |   2 +
 src/master/allocator/sorter/random/sorter.cpp | 570 +++++++++++++++++++++
 src/master/allocator/sorter/random/sorter.hpp | 424 +++++++++++++++
 4 files changed, 997 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/99ff52dc/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index d488131..70d5126 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -353,6 +353,7 @@ set(MASTER_SRC
   master/allocator/mesos/metrics.cpp
   master/allocator/sorter/drf/metrics.cpp
   master/allocator/sorter/drf/sorter.cpp
+  master/allocator/sorter/random/sorter.cpp
   master/contender/contender.cpp
   master/contender/standalone.cpp
   master/contender/zookeeper.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/99ff52dc/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 121e373..1e8ff24 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1014,6 +1014,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   master/allocator/mesos/metrics.cpp					\
   master/allocator/sorter/drf/metrics.cpp				\
   master/allocator/sorter/drf/sorter.cpp				\
+  master/allocator/sorter/random/sorter.cpp				\
   master/contender/contender.cpp					\
   master/contender/standalone.cpp					\
   master/contender/zookeeper.cpp					\
@@ -1165,6 +1166,7 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   master/allocator/sorter/sorter.hpp					\
   master/allocator/sorter/drf/metrics.hpp				\
   master/allocator/sorter/drf/sorter.hpp				\
+  master/allocator/sorter/random/sorter.hpp				\
   master/allocator/sorter/random/utils.hpp				\
   master/contender/standalone.hpp					\
   master/contender/zookeeper.hpp					\

http://git-wip-us.apache.org/repos/asf/mesos/blob/99ff52dc/src/master/allocator/sorter/random/sorter.cpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/sorter/random/sorter.cpp b/src/master/allocator/sorter/random/sorter.cpp
new file mode 100644
index 0000000..d17f8af
--- /dev/null
+++ b/src/master/allocator/sorter/random/sorter.cpp
@@ -0,0 +1,570 @@
+// 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 "master/allocator/sorter/random/sorter.hpp"
+#include "master/allocator/sorter/random/utils.hpp"
+
+#include <set>
+#include <string>
+#include <vector>
+
+#include <mesos/mesos.hpp>
+#include <mesos/resources.hpp>
+#include <mesos/values.hpp>
+
+#include <process/pid.hpp>
+
+#include <stout/check.hpp>
+#include <stout/foreach.hpp>
+#include <stout/hashmap.hpp>
+#include <stout/option.hpp>
+#include <stout/strings.hpp>
+
+using std::set;
+using std::string;
+using std::vector;
+
+using process::UPID;
+
+namespace mesos {
+namespace internal {
+namespace master {
+namespace allocator {
+
+
+RandomSorter::RandomSorter()
+  : root(new Node("", Node::INTERNAL, nullptr)) {}
+
+
+RandomSorter::RandomSorter(
+    const UPID& allocator,
+    const string& metricsPrefix)
+  : root(new Node("", Node::INTERNAL, nullptr)) {}
+
+
+RandomSorter::~RandomSorter()
+{
+  delete root;
+}
+
+
+void RandomSorter::initialize(
+    const Option<set<string>>& _fairnessExcludeResourceNames) {}
+
+
+void RandomSorter::add(const string& clientPath)
+{
+  vector<string> pathElements = strings::tokenize(clientPath, "/");
+  CHECK(!pathElements.empty());
+
+  Node* current = root;
+  Node* lastCreatedNode = nullptr;
+
+  // Traverse the tree to add new nodes for each element of the path,
+  // if that node doesn't already exist (similar to `mkdir -p`).
+  foreach (const string& element, pathElements) {
+    Node* node = nullptr;
+
+    foreach (Node* child, current->children) {
+      if (child->name == element) {
+        node = child;
+        break;
+      }
+    }
+
+    if (node != nullptr) {
+      current = node;
+      continue;
+    }
+
+    // We didn't find `element`, so add a new child to `current`.
+    //
+    // If adding this child would result in turning `current` from a
+    // leaf node into an internal node, we need to create an
+    // additional child node: `current` must have been associated with
+    // a client and clients must always be associated with leaf nodes.
+    if (current->isLeaf()) {
+      Node* parent = CHECK_NOTNULL(current->parent);
+
+      parent->removeChild(current);
+
+      // Create a node under `parent`. This internal node will take
+      // the place of `current` in the tree.
+      Node* internal = new Node(current->name, Node::INTERNAL, parent);
+      parent->addChild(internal);
+      internal->allocation = current->allocation;
+
+      CHECK_EQ(current->path, internal->path);
+
+      // Update `current` to become a virtual leaf node and a child of
+      // `internal`.
+      current->name = ".";
+      current->parent = internal;
+      current->path = strings::join("/", parent->path, current->name);
+
+      internal->addChild(current);
+
+      CHECK_EQ(internal->path, current->clientPath());
+
+      current = internal;
+    }
+
+    // Now actually add a new child to `current`.
+    Node* newChild = new Node(element, Node::INTERNAL, current);
+    current->addChild(newChild);
+
+    current = newChild;
+    lastCreatedNode = newChild;
+  }
+
+  CHECK(current->kind == Node::INTERNAL);
+
+  // `current` is the node associated with the last element of the
+  // path. If we didn't add `current` to the tree above, create a leaf
+  // node now. For example, if the tree contains "a/b" and we add a
+  // new client "a", we want to create a new leaf node "a/." here.
+  if (current != lastCreatedNode) {
+    Node* newChild = new Node(".", Node::INACTIVE_LEAF, current);
+    current->addChild(newChild);
+    current = newChild;
+  } else {
+    // If we created `current` in the loop above, it was marked an
+    // `INTERNAL` node. It should actually be an inactive leaf node.
+    current->kind = Node::INACTIVE_LEAF;
+
+    // `current` has changed from an internal node to an inactive
+    // leaf, so remove and re-add it to its parent. This moves it to
+    // the end of the parent's list of children.
+    CHECK_NOTNULL(current->parent);
+
+    current->parent->removeChild(current);
+    current->parent->addChild(current);
+  }
+
+  // `current` is the newly created node associated with the last
+  // element of the path. `current` should be an inactive leaf node.
+  CHECK(current->children.empty());
+  CHECK(current->kind == Node::INACTIVE_LEAF);
+
+  // Add a new entry to the lookup table. The full path of the newly
+  // added client should not already exist in `clients`.
+  CHECK_EQ(clientPath, current->clientPath());
+  CHECK(!clients.contains(clientPath));
+
+  clients[clientPath] = current;
+}
+
+
+void RandomSorter::remove(const string& clientPath)
+{
+  Node* current = CHECK_NOTNULL(find(clientPath));
+
+  // Save a copy of the leaf node's allocated resources, because we
+  // destroy the leaf node below.
+  const hashmap<SlaveID, Resources> leafAllocation =
+    current->allocation.resources;
+
+  // Remove the lookup table entry for the client.
+  CHECK(clients.contains(clientPath));
+  clients.erase(clientPath);
+
+  // To remove a client from the tree, we have to do two things:
+  //
+  //   (1) Update the tree structure to reflect the removal of the
+  //       client. This means removing the client's leaf node, then
+  //       walking back up the tree to remove any internal nodes that
+  //       are now unnecessary.
+  //
+  //   (2) Update allocations of ancestor nodes to reflect the removal
+  //       of the client.
+  //
+  // We do both things at once: find the leaf node, remove it, and
+  // walk up the tree, updating ancestor allocations and removing
+  // ancestors when possible.
+  while (current != root) {
+    Node* parent = CHECK_NOTNULL(current->parent);
+
+    // Update `parent` to reflect the fact that the resources in the
+    // leaf node are no longer allocated to the subtree rooted at
+    // `parent`. We skip `root`, because we never update the
+    // allocation made to the root node.
+    if (parent != root) {
+      foreachpair (const SlaveID& slaveId,
+                   const Resources& resources,
+                   leafAllocation) {
+        parent->allocation.subtract(slaveId, resources);
+      }
+    }
+
+    if (current->children.empty()) {
+      parent->removeChild(current);
+      delete current;
+    } else if (current->children.size() == 1) {
+      // If `current` has only one child that was created to
+      // accommodate inserting `clientPath` (see `RandomSorter::add()`),
+      // we can remove the child node and turn `current` back into a
+      // leaf node.
+      Node* child = *(current->children.begin());
+
+      if (child->name == ".") {
+        CHECK(child->isLeaf());
+        CHECK(clients.contains(current->path));
+        CHECK_EQ(child, clients.at(current->path));
+
+        current->kind = child->kind;
+        current->removeChild(child);
+
+        // `current` has changed kind (from `INTERNAL` to a leaf,
+        // which might be active or inactive). Hence we might need to
+        // change its position in the `children` list.
+        if (current->kind == Node::INTERNAL) {
+          CHECK_NOTNULL(current->parent);
+
+          current->parent->removeChild(current);
+          current->parent->addChild(current);
+        }
+
+        clients[current->path] = current;
+
+        delete child;
+      }
+    }
+
+    current = parent;
+  }
+}
+
+
+void RandomSorter::activate(const string& clientPath)
+{
+  Node* client = CHECK_NOTNULL(find(clientPath));
+
+  if (client->kind == Node::INACTIVE_LEAF) {
+    client->kind = Node::ACTIVE_LEAF;
+
+    // `client` has been activated, so move it to the beginning of its
+    // parent's list of children.
+    CHECK_NOTNULL(client->parent);
+
+    client->parent->removeChild(client);
+    client->parent->addChild(client);
+  }
+}
+
+
+void RandomSorter::deactivate(const string& clientPath)
+{
+  Node* client = CHECK_NOTNULL(find(clientPath));
+
+  if (client->kind == Node::ACTIVE_LEAF) {
+    client->kind = Node::INACTIVE_LEAF;
+
+    // `client` has been deactivated, so move it to the end of its
+    // parent's list of children.
+    CHECK_NOTNULL(client->parent);
+
+    client->parent->removeChild(client);
+    client->parent->addChild(client);
+  }
+}
+
+
+void RandomSorter::updateWeight(const string& path, double weight)
+{
+  weights[path] = weight;
+}
+
+
+void RandomSorter::allocated(
+    const string& clientPath,
+    const SlaveID& slaveId,
+    const Resources& resources)
+{
+  Node* current = CHECK_NOTNULL(find(clientPath));
+
+  // NOTE: We don't currently update the `allocation` for the root
+  // node. This is debatable, but the current implementation doesn't
+  // require looking at the allocation of the root node.
+  while (current != root) {
+    current->allocation.add(slaveId, resources);
+    current = CHECK_NOTNULL(current->parent);
+  }
+}
+
+
+void RandomSorter::update(
+    const string& clientPath,
+    const SlaveID& slaveId,
+    const Resources& oldAllocation,
+    const Resources& newAllocation)
+{
+  // TODO(bmahler): Check invariants between old and new allocations.
+  // Namely, the roles and quantities of resources should be the same!
+
+  Node* current = CHECK_NOTNULL(find(clientPath));
+
+  // NOTE: We don't currently update the `allocation` for the root
+  // node. This is debatable, but the current implementation doesn't
+  // require looking at the allocation of the root node.
+  while (current != root) {
+    current->allocation.update(slaveId, oldAllocation, newAllocation);
+    current = CHECK_NOTNULL(current->parent);
+  }
+}
+
+
+void RandomSorter::unallocated(
+    const string& clientPath,
+    const SlaveID& slaveId,
+    const Resources& resources)
+{
+  Node* current = CHECK_NOTNULL(find(clientPath));
+
+  // NOTE: We don't currently update the `allocation` for the root
+  // node. This is debatable, but the current implementation doesn't
+  // require looking at the allocation of the root node.
+  while (current != root) {
+    current->allocation.subtract(slaveId, resources);
+    current = CHECK_NOTNULL(current->parent);
+  }
+}
+
+
+const hashmap<SlaveID, Resources>& RandomSorter::allocation(
+    const string& clientPath) const
+{
+  const Node* client = CHECK_NOTNULL(find(clientPath));
+  return client->allocation.resources;
+}
+
+
+const Resources& RandomSorter::allocationScalarQuantities(
+    const string& clientPath) const
+{
+  const Node* client = CHECK_NOTNULL(find(clientPath));
+  return client->allocation.scalarQuantities;
+}
+
+
+hashmap<string, Resources> RandomSorter::allocation(
+    const SlaveID& slaveId) const
+{
+  hashmap<string, Resources> result;
+
+  // We want to find the allocation that has been made to each client
+  // on a particular `slaveId`. Rather than traversing the tree
+  // looking for leaf nodes (clients), we can instead just iterate
+  // over the `clients` hashmap.
+  //
+  // TODO(jmlvanre): We can index the allocation by slaveId to make
+  // this faster.  It is a tradeoff between speed vs. memory. For now
+  // we use existing data structures.
+  foreachvalue (const Node* client, clients) {
+    if (client->allocation.resources.contains(slaveId)) {
+      // It is safe to use `at()` here because we've just checked the
+      // existence of the key. This avoids unnecessary copies.
+      string path = client->clientPath();
+      CHECK(!result.contains(path));
+      result.emplace(path, client->allocation.resources.at(slaveId));
+    }
+  }
+
+  return result;
+}
+
+
+Resources RandomSorter::allocation(
+    const string& clientPath,
+    const SlaveID& slaveId) const
+{
+  const Node* client = CHECK_NOTNULL(find(clientPath));
+
+  if (client->allocation.resources.contains(slaveId)) {
+    return client->allocation.resources.at(slaveId);
+  }
+
+  return Resources();
+}
+
+
+const Resources& RandomSorter::totalScalarQuantities() const
+{
+  return total_.scalarQuantities;
+}
+
+
+void RandomSorter::add(const SlaveID& slaveId, const Resources& resources)
+{
+  if (!resources.empty()) {
+    // Add shared resources to the total quantities when the same
+    // resources don't already exist in the total.
+    const Resources newShared = resources.shared()
+      .filter([this, slaveId](const Resource& resource) {
+        return !total_.resources[slaveId].contains(resource);
+      });
+
+    total_.resources[slaveId] += resources;
+
+    const Resources scalarQuantities =
+      (resources.nonShared() + newShared).createStrippedScalarQuantity();
+
+    total_.scalarQuantities += scalarQuantities;
+
+    foreach (const Resource& resource, scalarQuantities) {
+      total_.totals[resource.name()] += resource.scalar();
+    }
+  }
+}
+
+
+void RandomSorter::remove(const SlaveID& slaveId, const Resources& resources)
+{
+  if (!resources.empty()) {
+    CHECK(total_.resources.contains(slaveId));
+    CHECK(total_.resources[slaveId].contains(resources))
+      << total_.resources[slaveId] << " does not contain " << resources;
+
+    total_.resources[slaveId] -= resources;
+
+    // Remove shared resources from the total quantities when there
+    // are no instances of same resources left in the total.
+    const Resources absentShared = resources.shared()
+      .filter([this, slaveId](const Resource& resource) {
+        return !total_.resources[slaveId].contains(resource);
+      });
+
+    const Resources scalarQuantities =
+      (resources.nonShared() + absentShared).createStrippedScalarQuantity();
+
+    foreach (const Resource& resource, scalarQuantities) {
+      total_.totals[resource.name()] -= resource.scalar();
+    }
+
+    CHECK(total_.scalarQuantities.contains(scalarQuantities));
+    total_.scalarQuantities -= scalarQuantities;
+
+    if (total_.resources[slaveId].empty()) {
+      total_.resources.erase(slaveId);
+    }
+  }
+}
+
+
+vector<string> RandomSorter::sort()
+{
+  std::function<void (Node*)> shuffleTree = [this, &shuffleTree](Node* node) {
+    // Inactive leaves are always stored at the end of the
+    // `children` vector; this means that we should only shuffle
+    // the prefix of the vector before the first inactive leaf.
+    auto inactiveBegin = std::find_if(
+        node->children.begin(),
+        node->children.end(),
+        [](Node* n) { return n->kind == Node::INACTIVE_LEAF; });
+
+    vector<double> weights(inactiveBegin - node->children.begin());
+
+    for (int i = 0; i < inactiveBegin - node->children.begin(); ++i) {
+      weights[i] = findWeight(node->children[i]);
+    }
+
+    weightedShuffle(node->children.begin(), inactiveBegin, weights, generator);
+
+    foreach (Node* child, node->children) {
+      if (child->kind == Node::INTERNAL) {
+        shuffleTree(child);
+      } else if (child->kind == Node::INACTIVE_LEAF) {
+        break;
+      }
+    }
+  };
+
+  shuffleTree(root);
+
+  // Return all active leaves in the tree via pre-order traversal.
+  // The children of each node are already shuffled, with
+  // inactive leaves stored after active leaves and internal nodes.
+  vector<string> result;
+  result.reserve(clients.size());
+
+  std::function<void (const Node*)> listClients =
+      [&listClients, &result](const Node* node) {
+    foreach (const Node* child, node->children) {
+      switch (child->kind) {
+        case Node::ACTIVE_LEAF:
+          result.push_back(child->clientPath());
+          break;
+
+        case Node::INACTIVE_LEAF:
+          // As soon as we see the first inactive leaf, we can stop
+          // iterating over the current node's list of children.
+          return;
+
+        case Node::INTERNAL:
+          listClients(child);
+          break;
+      }
+    }
+  };
+
+  listClients(root);
+
+  return result;
+}
+
+
+bool RandomSorter::contains(const string& clientPath) const
+{
+  return find(clientPath) != nullptr;
+}
+
+
+size_t RandomSorter::count() const
+{
+  return clients.size();
+}
+
+
+double RandomSorter::findWeight(const Node* node) const
+{
+  Option<double> weight = weights.get(node->path);
+
+  if (weight.isNone()) {
+    return 1.0;
+  }
+
+  return weight.get();
+}
+
+
+RandomSorter::Node* RandomSorter::find(const string& clientPath) const
+{
+  Option<Node*> client_ = clients.get(clientPath);
+
+  if (client_.isNone()) {
+    return nullptr;
+  }
+
+  Node* client = client_.get();
+
+  CHECK(client->isLeaf());
+
+  return client;
+}
+
+} // namespace allocator {
+} // namespace master {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/99ff52dc/src/master/allocator/sorter/random/sorter.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/sorter/random/sorter.hpp b/src/master/allocator/sorter/random/sorter.hpp
new file mode 100644
index 0000000..6e22cf6
--- /dev/null
+++ b/src/master/allocator/sorter/random/sorter.hpp
@@ -0,0 +1,424 @@
+// 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.
+
+#ifndef __MASTER_ALLOCATOR_SORTER_RANDOM_SORTER_HPP__
+#define __MASTER_ALLOCATOR_SORTER_RANDOM_SORTER_HPP__
+
+#include <algorithm>
+#include <random>
+#include <set>
+#include <string>
+#include <vector>
+
+#include <mesos/mesos.hpp>
+#include <mesos/resources.hpp>
+#include <mesos/values.hpp>
+
+#include <stout/check.hpp>
+#include <stout/hashmap.hpp>
+#include <stout/option.hpp>
+
+#include "master/allocator/sorter/sorter.hpp"
+
+
+namespace mesos {
+namespace internal {
+namespace master {
+namespace allocator {
+
+class RandomSorter : public Sorter
+{
+public:
+  RandomSorter();
+
+  explicit RandomSorter(
+      const process::UPID& allocator,
+      const std::string& metricsPrefix);
+
+  virtual ~RandomSorter();
+
+  virtual void initialize(
+      const Option<std::set<std::string>>& fairnessExcludeResourceNames);
+
+  virtual void add(const std::string& clientPath);
+
+  virtual void remove(const std::string& clientPath);
+
+  virtual void activate(const std::string& clientPath);
+
+  virtual void deactivate(const std::string& clientPath);
+
+  virtual void updateWeight(const std::string& path, double weight);
+
+  virtual void allocated(
+      const std::string& clientPath,
+      const SlaveID& slaveId,
+      const Resources& resources);
+
+  virtual void update(
+      const std::string& clientPath,
+      const SlaveID& slaveId,
+      const Resources& oldAllocation,
+      const Resources& newAllocation);
+
+  virtual void unallocated(
+      const std::string& clientPath,
+      const SlaveID& slaveId,
+      const Resources& resources);
+
+  virtual const hashmap<SlaveID, Resources>& allocation(
+      const std::string& clientPath) const;
+
+  virtual const Resources& allocationScalarQuantities(
+      const std::string& clientPath) const;
+
+  virtual hashmap<std::string, Resources> allocation(
+      const SlaveID& slaveId) const;
+
+  virtual Resources allocation(
+      const std::string& clientPath,
+      const SlaveID& slaveId) const;
+
+  virtual const Resources& totalScalarQuantities() const;
+
+  virtual void add(const SlaveID& slaveId, const Resources& resources);
+
+  virtual void remove(const SlaveID& slaveId, const Resources& resources);
+
+  // This will perform a weighted random shuffle on each call.
+  //
+  // TODO(bmahler): Unlike the DRF sorter, the allocator ideally would
+  // not call `sort()` for every agent, but rather loop through a single
+  // weighted shuffle before re-shuffling..
+  virtual std::vector<std::string> sort();
+
+  virtual bool contains(const std::string& clientPath) const;
+
+  virtual size_t count() const;
+
+private:
+  // A node in the sorter's tree.
+  struct Node;
+
+  // Returns the weight associated with the node. If no weight has
+  // been configured for the node's path, the default weight (1.0) is
+  // returned.
+  double findWeight(const Node* node) const;
+
+  // Returns the client associated with the given path. Returns
+  // nullptr if the path is not found or if the path identifies an
+  // internal node in the tree (not a client).
+  Node* find(const std::string& clientPath) const;
+
+  // Used for random number generation.
+  std::mt19937 generator;
+
+  // The root node in the sorter tree.
+  Node* root;
+
+  // To speed lookups, we keep a map from client paths to the leaf
+  // node associated with that client. There is an entry in this map
+  // for every leaf node in the client tree (except for the root when
+  // the tree is empty). Paths in this map do NOT contain the trailing
+  // "." label we use for leaf nodes.
+  hashmap<std::string, Node*> clients;
+
+  // Weights associated with role paths. Setting the weight for a path
+  // influences the sampling probability of all nodes in the subtree
+  // rooted at that path. This hashmap might include weights for paths
+  // that are not currently in the sorter tree.
+  hashmap<std::string, double> weights;
+
+  // Total resources.
+  struct Total
+  {
+    // We need to keep track of the resources (and not just scalar
+    // quantities) to account for multiple copies of the same shared
+    // resources. We need to ensure that we do not update the scalar
+    // quantities for shared resources when the change is only in the
+    // number of copies in the sorter.
+    hashmap<SlaveID, Resources> resources;
+
+    // NOTE: Scalars can be safely aggregated across slaves. We keep
+    // that to speed up the calculation of shares. See MESOS-2891 for
+    // the reasons why we want to do that.
+    //
+    // NOTE: We omit information about dynamic reservations and
+    // persistent volumes here to enable resources to be aggregated
+    // across slaves more effectively. See MESOS-4833 for more
+    // information.
+    //
+    // Sharedness info is also stripped out when resource identities
+    // are omitted because sharedness inherently refers to the
+    // identities of resources and not quantities.
+    Resources scalarQuantities;
+
+    // We also store a map version of `scalarQuantities`, mapping
+    // the `Resource::name` to aggregated scalar. This improves the
+    // performance of calculating shares. See MESOS-4694.
+    //
+    // TODO(bmahler): Ideally we do not store `scalarQuantities`
+    // redundantly here, investigate performance improvements to
+    // `Resources` to make this unnecessary.
+    hashmap<std::string, Value::Scalar> totals;
+  } total_;
+};
+
+
+// Represents a node in the sorter's tree. The structure of the tree
+// reflects the hierarchical relationships between the clients of the
+// sorter. Some (but not all) nodes correspond to sorter clients; some
+// nodes only exist to represent the structure of the sorter
+// tree. Clients are always associated with leaf nodes.
+//
+// For example, if there are two sorter clients "a/b" and "c/d", the
+// tree will contain five nodes: the root node, internal nodes for "a"
+// and "c", and leaf nodes for the clients "a/b" and "c/d".
+struct RandomSorter::Node
+{
+  // Indicates whether a node is an active leaf node, an inactive leaf
+  // node, or an internal node. Sorter clients always correspond to
+  // leaf nodes, and only leaf nodes can be activated or deactivated.
+  // The root node is always an "internal" node.
+  enum Kind
+  {
+    ACTIVE_LEAF,
+    INACTIVE_LEAF,
+    INTERNAL
+  };
+
+  Node(const std::string& _name, Kind _kind, Node* _parent)
+    : name(_name), kind(_kind), parent(_parent)
+  {
+    // Compute the node's path. Three cases:
+    //
+    //  (1) If the root node, use the empty string
+    //  (2) If a child of the root node, use the child's name
+    //  (3) Otherwise, use the parent's name, "/", and the child's name.
+    if (parent == nullptr) {
+      path = "";
+    } else if (parent->parent == nullptr) {
+      path = name;
+    } else {
+      path = strings::join("/", parent->path, name);
+    }
+  }
+
+  ~Node()
+  {
+    foreach (Node* child, children) {
+      delete child;
+    }
+  }
+
+  // The label of the edge from this node's parent to the
+  // node. "Implicit" leaf nodes are always named ".".
+  //
+  // TODO(neilc): Consider naming implicit leaf nodes in a clearer
+  // way, e.g., by making `name` an Option?
+  std::string name;
+
+  // Complete path from root to node. This includes the trailing "."
+  // label for virtual leaf nodes.
+  std::string path;
+
+  Kind kind;
+
+  Node* parent;
+
+  // Pointers to the child nodes. `children` is only non-empty if
+  // `kind` is INTERNAL_NODE.
+  //
+  // All inactive leaves are stored at the end of the vector; that
+  // is, each `children` vector consists of zero or more active leaves
+  // and internal nodes, followed by zero or more inactive leaves. This
+  // means that code that only wants to iterate over active children
+  // can stop when the first inactive leaf is observed.
+  std::vector<Node*> children;
+
+  // If this node represents a sorter client, this returns the path of
+  // that client. Unlike the `path` field, this does NOT include the
+  // trailing "." label for virtual leaf nodes.
+  //
+  // For example, if the sorter contains two clients "a" and "a/b",
+  // the tree will contain four nodes: the root node, "a", "a/."
+  // (virtual leaf), and "a/b". The `clientPath()` of "a/." is "a",
+  // because that is the name of the client associated with that
+  // virtual leaf node.
+  std::string clientPath() const
+  {
+    if (name == ".") {
+      CHECK(kind == ACTIVE_LEAF || kind == INACTIVE_LEAF);
+      return CHECK_NOTNULL(parent)->path;
+    }
+
+    return path;
+  }
+
+  bool isLeaf() const
+  {
+    if (kind == ACTIVE_LEAF || kind == INACTIVE_LEAF) {
+      CHECK(children.empty());
+      return true;
+    }
+
+    return false;
+  }
+
+  void removeChild(const Node* child)
+  {
+    // Sanity check: ensure we are removing an extant node.
+    auto it = std::find(children.begin(), children.end(), child);
+    CHECK(it != children.end());
+
+    children.erase(it);
+  }
+
+  void addChild(Node* child)
+  {
+    // Sanity check: don't allow duplicates to be inserted.
+    auto it = std::find(children.begin(), children.end(), child);
+    CHECK(it == children.end());
+
+    // If we're inserting an inactive leaf, place it at the end of the
+    // `children` vector; otherwise, place it at the beginning. This
+    // maintains ordering invariant above.
+    if (child->kind == INACTIVE_LEAF) {
+      children.push_back(child);
+    } else {
+      children.insert(children.begin(), child);
+    }
+  }
+
+  // Allocation for a node.
+  struct Allocation
+  {
+    Allocation() {}
+
+    void add(const SlaveID& slaveId, const Resources& toAdd)
+    {
+      // Add shared resources to the allocated quantities when the same
+      // resources don't already exist in the allocation.
+      const Resources sharedToAdd = toAdd.shared()
+        .filter([this, slaveId](const Resource& resource) {
+            return !resources[slaveId].contains(resource);
+        });
+
+      const Resources quantitiesToAdd =
+        (toAdd.nonShared() + sharedToAdd).createStrippedScalarQuantity();
+
+      resources[slaveId] += toAdd;
+      scalarQuantities += quantitiesToAdd;
+
+      foreach (const Resource& resource, quantitiesToAdd) {
+        totals[resource.name()] += resource.scalar();
+      }
+    }
+
+    void subtract(const SlaveID& slaveId, const Resources& toRemove)
+    {
+      CHECK(resources.contains(slaveId));
+      CHECK(resources.at(slaveId).contains(toRemove))
+        << "Resources " << resources.at(slaveId) << " at agent " << slaveId
+        << " does not contain " << toRemove;
+
+      resources[slaveId] -= toRemove;
+
+      // Remove shared resources from the allocated quantities when there
+      // are no instances of same resources left in the allocation.
+      const Resources sharedToRemove = toRemove.shared()
+        .filter([this, slaveId](const Resource& resource) {
+            return !resources[slaveId].contains(resource);
+        });
+
+      const Resources quantitiesToRemove =
+        (toRemove.nonShared() + sharedToRemove).createStrippedScalarQuantity();
+
+      foreach (const Resource& resource, quantitiesToRemove) {
+        totals[resource.name()] -= resource.scalar();
+      }
+
+      CHECK(scalarQuantities.contains(quantitiesToRemove))
+        << scalarQuantities << " does not contain " << quantitiesToRemove;
+
+      scalarQuantities -= quantitiesToRemove;
+
+      if (resources[slaveId].empty()) {
+        resources.erase(slaveId);
+      }
+    }
+
+    void update(
+        const SlaveID& slaveId,
+        const Resources& oldAllocation,
+        const Resources& newAllocation)
+    {
+      const Resources oldAllocationQuantity =
+        oldAllocation.createStrippedScalarQuantity();
+      const Resources newAllocationQuantity =
+        newAllocation.createStrippedScalarQuantity();
+
+      CHECK(resources.contains(slaveId));
+      CHECK(resources[slaveId].contains(oldAllocation))
+        << "Resources " << resources[slaveId] << " at agent " << slaveId
+        << " does not contain " << oldAllocation;
+
+      CHECK(scalarQuantities.contains(oldAllocationQuantity))
+        << scalarQuantities << " does not contain " << oldAllocationQuantity;
+
+      resources[slaveId] -= oldAllocation;
+      resources[slaveId] += newAllocation;
+
+      scalarQuantities -= oldAllocationQuantity;
+      scalarQuantities += newAllocationQuantity;
+
+      foreach (const Resource& resource, oldAllocationQuantity) {
+        totals[resource.name()] -= resource.scalar();
+      }
+
+      foreach (const Resource& resource, newAllocationQuantity) {
+        totals[resource.name()] += resource.scalar();
+      }
+    }
+
+    // We maintain multiple copies of each shared resource allocated
+    // to a client, where the number of copies represents the number
+    // of times this shared resource has been allocated to (and has
+    // not been recovered from) a specific client.
+    hashmap<SlaveID, Resources> resources;
+
+    // Similarly, we aggregate scalars across slaves and omit information
+    // about dynamic reservations, persistent volumes and sharedness of
+    // the corresponding resource. See notes above.
+    Resources scalarQuantities;
+
+    // We also store a map version of `scalarQuantities`, mapping
+    // the `Resource::name` to aggregated scalar. This improves the
+    // performance of calculating shares. See MESOS-4694.
+    //
+    // TODO(bmahler): Ideally we do not store `scalarQuantities`
+    // redundantly here, investigate performance improvements to
+    // `Resources` to make this unnecessary.
+    hashmap<std::string, Value::Scalar> totals;
+  } allocation;
+};
+
+} // namespace allocator {
+} // namespace master {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __MASTER_ALLOCATOR_SORTER_RANDOM_SORTER_HPP__