You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by gr...@apache.org on 2018/06/21 17:14:53 UTC

[1/2] mesos git commit: Refactored `struct Slave` in the allocator for better performance.

Repository: mesos
Updated Branches:
  refs/heads/master 528b763aa -> f0284a51d


Refactored `struct Slave` in the allocator for better performance.

This patch refactors the `struct Slave` in the allocator.
In particular, it addresses the slowness of computing
agents' available resources. Instead of calculating them
every time on the fly, this patch "denormalizes" the agent
available resources by updating and persisting the field
each time an agent's allocated or total resources change.

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


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

Branch: refs/heads/master
Commit: f19b6d85672fcf73b28085fed4eecc2c1b18e109
Parents: 528b763
Author: Meng Zhu <mz...@mesosphere.io>
Authored: Thu Jun 21 09:09:36 2018 -0700
Committer: Greg Mann <gr...@gmail.com>
Committed: Thu Jun 21 09:22:21 2018 -0700

----------------------------------------------------------------------
 src/master/allocator/mesos/hierarchical.cpp |  73 ++++++++--------
 src/master/allocator/mesos/hierarchical.hpp | 103 +++++++++++++++++------
 2 files changed, 113 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f19b6d85/src/master/allocator/mesos/hierarchical.cpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.cpp b/src/master/allocator/mesos/hierarchical.cpp
index f631ce7..60d8b5d 100644
--- a/src/master/allocator/mesos/hierarchical.cpp
+++ b/src/master/allocator/mesos/hierarchical.cpp
@@ -508,16 +508,16 @@ void HierarchicalAllocatorProcess::addSlave(
   CHECK_EQ(slaveId, slaveInfo.id());
   CHECK(!paused || expectedAgentCount.isSome());
 
-  slaves[slaveId] = Slave();
+  slaves.insert({slaveId,
+                 Slave(
+                     slaveInfo,
+                     protobuf::slave::Capabilities(capabilities),
+                     true,
+                     total,
+                     Resources::sum(used))});
 
   Slave& slave = slaves.at(slaveId);
 
-  slave.total = total;
-  slave.allocated = Resources::sum(used);
-  slave.activated = true;
-  slave.info = slaveInfo;
-  slave.capabilities = protobuf::slave::Capabilities(capabilities);
-
   // NOTE: We currently implement maintenance in the allocator to be able to
   // leverage state and features such as the FrameworkSorter and OfferFilter.
   if (unavailability.isSome()) {
@@ -574,8 +574,8 @@ void HierarchicalAllocatorProcess::addSlave(
   }
 
   LOG(INFO) << "Added agent " << slaveId << " (" << slave.info.hostname() << ")"
-            << " with " << slave.total
-            << " (allocated: " << slave.allocated << ")";
+            << " with " << slave.getTotal()
+            << " (allocated: " << slave.getAllocated() << ")";
 
   allocate(slaveId);
 }
@@ -593,12 +593,13 @@ void HierarchicalAllocatorProcess::removeSlave(
   // all the resources. Fixing this would require more information
   // than what we currently track in the allocator.
 
-  roleSorter->remove(slaveId, slaves.at(slaveId).total);
+  roleSorter->remove(slaveId, slaves.at(slaveId).getTotal());
 
   // See comment at `quotaRoleSorter` declaration regarding non-revocable.
-  quotaRoleSorter->remove(slaveId, slaves.at(slaveId).total.nonRevocable());
+  quotaRoleSorter->remove(
+      slaveId, slaves.at(slaveId).getTotal().nonRevocable());
 
-  untrackReservations(slaves.at(slaveId).total.reservations());
+  untrackReservations(slaves.at(slaveId).getTotal().reservations());
 
   slaves.erase(slaveId);
   allocationCandidates.erase(slaveId);
@@ -708,8 +709,8 @@ void HierarchicalAllocatorProcess::addResourceProvider(
   }
 
   Slave& slave = slaves.at(slaveId);
-  updateSlaveTotal(slaveId, slave.total + total);
-  slave.allocated += Resources::sum(used);
+  updateSlaveTotal(slaveId, slave.getTotal() + total);
+  slave.allocate(Resources::sum(used));
 
   VLOG(1)
     << "Grew agent " << slaveId << " by "
@@ -846,8 +847,8 @@ void HierarchicalAllocatorProcess::updateAllocation(
   const Resources& updatedOfferedResources = _updatedOfferedResources.get();
 
   // Update the per-slave allocation.
-  slave.allocated -= offeredResources;
-  slave.allocated += updatedOfferedResources;
+  slave.unallocate(offeredResources);
+  slave.allocate(updatedOfferedResources);
 
   // Update the allocation in the framework sorter.
   frameworkSorter->update(
@@ -906,7 +907,7 @@ void HierarchicalAllocatorProcess::updateAllocation(
     strippedConversions.emplace_back(consumed, converted);
   }
 
-  Try<Resources> updatedTotal = slave.total.apply(strippedConversions);
+  Try<Resources> updatedTotal = slave.getTotal().apply(strippedConversions);
   CHECK_SOME(updatedTotal);
 
   updateSlaveTotal(slaveId, updatedTotal.get());
@@ -957,7 +958,7 @@ Future<Nothing> HierarchicalAllocatorProcess::updateAvailable(
   //                \___/ \___/
   //
   //   where A = allocate, R = reserve, U = updateAvailable
-  Try<Resources> updatedAvailable = slave.available().apply(operations);
+  Try<Resources> updatedAvailable = slave.getAvailable().apply(operations);
   if (updatedAvailable.isError()) {
     VLOG(1) << "Failed to update available resources on agent " << slaveId
             << ": " << updatedAvailable.error();
@@ -965,7 +966,7 @@ Future<Nothing> HierarchicalAllocatorProcess::updateAvailable(
   }
 
   // Update the total resources.
-  Try<Resources> updatedTotal = slave.total.apply(operations);
+  Try<Resources> updatedTotal = slave.getTotal().apply(operations);
   CHECK_SOME(updatedTotal);
 
   // Update the total resources in the allocator and role and quota sorters.
@@ -1188,14 +1189,14 @@ void HierarchicalAllocatorProcess::recoverResources(
   if (slaves.contains(slaveId)) {
     Slave& slave = slaves.at(slaveId);
 
-    CHECK(slave.allocated.contains(resources))
-      << slave.allocated << " does not contain " << resources;
+    CHECK(slave.getAllocated().contains(resources))
+      << slave.getAllocated() << " does not contain " << resources;
 
-    slave.allocated -= resources;
+    slave.unallocate(resources);
 
     VLOG(1) << "Recovered " << resources
-            << " (total: " << slave.total
-            << ", allocated: " << slave.allocated << ")"
+            << " (total: " << slave.getTotal()
+            << ", allocated: " << slave.getAllocated() << ")"
             << " on agent " << slaveId
             << " from framework " << frameworkId;
   }
@@ -1750,7 +1751,7 @@ void HierarchicalAllocatorProcess::__allocate()
     // NOTE: `totalScalarQuantities` omits dynamic reservation,
     // persistent volume info, and allocation info. We additionally
     // remove the static reservations here via `toUnreserved()`.
-    availableHeadroom -= slave.available().revocable()
+    availableHeadroom -= slave.getAvailable().revocable()
       .createStrippedScalarQuantity().toUnreserved();
   }
 
@@ -1812,7 +1813,7 @@ void HierarchicalAllocatorProcess::__allocate()
         // See MESOS-5634.
         if (filterGpuResources &&
             !framework.capabilities.gpuResources &&
-            slave.total.gpus().getOrElse(0) > 0) {
+            slave.getTotal().gpus().getOrElse(0) > 0) {
           continue;
         }
 
@@ -1825,14 +1826,14 @@ void HierarchicalAllocatorProcess::__allocate()
         // Calculate the currently available resources on the slave, which
         // is the difference in non-shared resources between total and
         // allocated, plus all shared resources on the agent (if applicable).
-        Resources available = slave.available().nonShared();
+        Resources available = slave.getAvailable().nonShared();
 
         // Since shared resources are offerable even when they are in use, we
         // make one copy of the shared resources available regardless of the
         // past allocations. Offer a shared resource only if it has not been
         // offered in this offer cycle to a framework.
         if (framework.capabilities.sharedResources) {
-          available += slave.total.shared();
+          available += slave.getTotal().shared();
           if (offeredSharedResources.contains(slaveId)) {
             available -= offeredSharedResources[slaveId];
           }
@@ -2014,7 +2015,7 @@ void HierarchicalAllocatorProcess::__allocate()
         // in the cluster.
         availableHeadroom -= allocatedUnreserved;
 
-        slave.allocated += toAllocate;
+        slave.allocate(toAllocate);
 
         trackAllocatedResources(slaveId, frameworkId, toAllocate);
       }
@@ -2056,7 +2057,7 @@ void HierarchicalAllocatorProcess::__allocate()
         // See MESOS-5634.
         if (filterGpuResources &&
             !framework.capabilities.gpuResources &&
-            slave.total.gpus().getOrElse(0) > 0) {
+            slave.getTotal().gpus().getOrElse(0) > 0) {
           continue;
         }
 
@@ -2069,14 +2070,14 @@ void HierarchicalAllocatorProcess::__allocate()
         // Calculate the currently available resources on the slave, which
         // is the difference in non-shared resources between total and
         // allocated, plus all shared resources on the agent (if applicable).
-        Resources available = slave.available().nonShared();
+        Resources available = slave.getAvailable().nonShared();
 
         // Since shared resources are offerable even when they are in use, we
         // make one copy of the shared resources available regardless of the
         // past allocations. Offer a shared resource only if it has not been
         // offered in this offer cycle to a framework.
         if (framework.capabilities.sharedResources) {
-          available += slave.total.shared();
+          available += slave.getTotal().shared();
           if (offeredSharedResources.contains(slaveId)) {
             available -= offeredSharedResources[slaveId];
           }
@@ -2167,7 +2168,7 @@ void HierarchicalAllocatorProcess::__allocate()
             headroomToAllocate.createStrippedScalarQuantity();
         }
 
-        slave.allocated += toAllocate;
+        slave.allocate(toAllocate);
 
         trackAllocatedResources(slaveId, frameworkId, toAllocate);
       }
@@ -2499,7 +2500,7 @@ double HierarchicalAllocatorProcess::_resources_offered_or_allocated(
 
   foreachvalue (const Slave& slave, slaves) {
     Option<Value::Scalar> value =
-      slave.allocated.get<Value::Scalar>(resource);
+      slave.getAllocated().get<Value::Scalar>(resource);
 
     if (value.isSome()) {
       offered_or_allocated += value->value();
@@ -2675,13 +2676,13 @@ bool HierarchicalAllocatorProcess::updateSlaveTotal(
 
   Slave& slave = slaves.at(slaveId);
 
-  const Resources oldTotal = slave.total;
+  const Resources oldTotal = slave.getTotal();
 
   if (oldTotal == total) {
     return false;
   }
 
-  slave.total = total;
+  slave.updateTotal(total);
 
   hashmap<std::string, Resources> oldReservations = oldTotal.reservations();
   hashmap<std::string, Resources> newReservations = total.reservations();

http://git-wip-us.apache.org/repos/asf/mesos/blob/f19b6d85/src/master/allocator/mesos/hierarchical.hpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.hpp b/src/master/allocator/mesos/hierarchical.hpp
index 19aed2d..0f6c0e9 100644
--- a/src/master/allocator/mesos/hierarchical.hpp
+++ b/src/master/allocator/mesos/hierarchical.hpp
@@ -354,42 +354,54 @@ protected:
 
   hashmap<FrameworkID, Framework> frameworks;
 
-  struct Slave
+  class Slave
   {
-    // Total amount of regular *and* oversubscribed resources.
-    Resources total;
-
-    // Regular *and* oversubscribed resources that are allocated.
-    //
-    // NOTE: We maintain multiple copies of each shared resource allocated
-    // to a slave, 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 framework.
-    //
-    // NOTE: We keep track of slave's allocated resources despite
-    // having that information in sorters. This is because the
-    // information in sorters is not accurate if some framework
-    // hasn't reregistered. See MESOS-2919 for details.
-    Resources allocated;
-
-    // We track the total and allocated resources on the slave, the
-    // available resources are computed as follows:
-    //
-    //   available = total - allocated
-    //
-    // Note that it's possible for the slave to be over-allocated!
-    // In this case, allocated > total.
-    Resources available() const
+  public:
+    Slave(
+        const SlaveInfo& _info,
+        const protobuf::slave::Capabilities& _capabilities,
+        bool _activated,
+        const Resources& _total,
+        const Resources& _allocated)
+      : info(_info),
+        capabilities(_capabilities),
+        activated(_activated),
+        total(_total),
+        allocated(_allocated)
     {
       // In order to subtract from the total,
       // we strip the allocation information.
       Resources allocated_ = allocated;
       allocated_.unallocate();
 
-      return total - allocated_;
+      available = total - allocated_;
+    }
+
+    Resources getTotal() const { return total; }
+
+    Resources getAllocated() const { return allocated; }
+
+    Resources getAvailable() const { return available; }
+
+    void updateTotal(const Resources& newTotal) {
+      total = newTotal;
+
+      updateAvailable();
+    }
+
+    void allocate(const Resources& toAllocate)
+    {
+      allocated += toAllocate;
+
+      updateAvailable();
     }
 
-    bool activated;  // Whether to offer resources.
+    void unallocate(const Resources& toUnallocate)
+    {
+      allocated -= toUnallocate;
+
+      updateAvailable();
+    }
 
     // The `SlaveInfo` that was passed to the allocator when the slave was added
     // or updated. Currently only two fields are used: `hostname` for host
@@ -399,6 +411,8 @@ protected:
 
     protobuf::slave::Capabilities capabilities;
 
+    bool activated; // Whether to offer resources.
+
     // Represents a scheduled unavailability due to maintenance for a specific
     // slave, and the responses from frameworks as to whether they will be able
     // to gracefully handle this unavailability.
@@ -434,6 +448,41 @@ protected:
     // a given point in time, for an optional duration. This information is used
     // to send out `InverseOffers`.
     Option<Maintenance> maintenance;
+
+  private:
+    void updateAvailable() {
+      // In order to subtract from the total,
+      // we strip the allocation information.
+      Resources allocated_ = allocated;
+      allocated_.unallocate();
+
+      available = total - allocated_;
+    }
+
+    // Total amount of regular *and* oversubscribed resources.
+    Resources total;
+
+    // Regular *and* oversubscribed resources that are allocated.
+    //
+    // NOTE: We maintain multiple copies of each shared resource allocated
+    // to a slave, 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 framework.
+    //
+    // NOTE: We keep track of the slave's allocated resources despite
+    // having that information in sorters. This is because the
+    // information in sorters is not accurate if some framework
+    // hasn't reregistered. See MESOS-2919 for details.
+    Resources allocated;
+
+    // We track the total and allocated resources on the slave, the
+    // available resources are computed as follows:
+    //
+    //   available = total - allocated
+    //
+    // Note that it's possible for the slave to be over-allocated!
+    // In this case, allocated > total.
+    Resources available;
   };
 
   hashmap<SlaveID, Slave> slaves;


[2/2] mesos git commit: Modified `createStrippedScalarQuantity()` to clear all metadata fields.

Posted by gr...@apache.org.
Modified `createStrippedScalarQuantity()` to clear all metadata fields.

Currently `createStrippedScalarQuantity()` strips resource meta-data
and transforms dynamic reservations into a static reservation.
However, no current code depends on the reservations in resources
returned by this helper function. This leads to boilerplate code
around call sites and performance overhead.

This patch updates the function to clear all reservation information.

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


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

Branch: refs/heads/master
Commit: f0284a51d188d32522c460c7e276ea80316eabd9
Parents: f19b6d8
Author: Meng Zhu <mz...@mesosphere.io>
Authored: Thu Jun 21 09:09:39 2018 -0700
Committer: Greg Mann <gr...@gmail.com>
Committed: Thu Jun 21 09:31:27 2018 -0700

----------------------------------------------------------------------
 include/mesos/resources.hpp                 |  8 ++---
 include/mesos/v1/resources.hpp              |  8 ++---
 src/common/resources.cpp                    | 22 +++----------
 src/master/allocator/mesos/hierarchical.cpp | 41 ++++++------------------
 src/tests/resources_tests.cpp               | 13 ++------
 src/tests/sorter_tests.cpp                  |  2 +-
 src/v1/resources.cpp                        | 22 +++----------
 7 files changed, 29 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f0284a51/include/mesos/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/resources.hpp b/include/mesos/resources.hpp
index bd6d6d6..175833c 100644
--- a/include/mesos/resources.hpp
+++ b/include/mesos/resources.hpp
@@ -467,11 +467,9 @@ public:
   Resources toUnreserved() const;
 
   // Returns a Resources object that contains all the scalar resources
-  // in this object, but with their AllocationInfo, ReservationInfo,
-  // DiskInfo, and SharedInfo omitted. The `role` and RevocableInfo,
-  // if any, are preserved. Because we clear ReservationInfo but
-  // preserve `role`, this means that stripping a dynamically reserved
-  // resource makes it effectively statically reserved.
+  // but with all the meta-data fields, such as AllocationInfo,
+  // ReservationInfo and etc. cleared. Only scalar resources' name,
+  // type (SCALAR) and value are preserved.
   //
   // This is intended for code that would like to aggregate together
   // Resource values without regard for metadata like whether the

http://git-wip-us.apache.org/repos/asf/mesos/blob/f0284a51/include/mesos/v1/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/v1/resources.hpp b/include/mesos/v1/resources.hpp
index c065dd1..b607b68 100644
--- a/include/mesos/v1/resources.hpp
+++ b/include/mesos/v1/resources.hpp
@@ -467,11 +467,9 @@ public:
   Resources toUnreserved() const;
 
   // Returns a Resources object that contains all the scalar resources
-  // in this object, but with their AllocationInfo, ReservationInfo,
-  // DiskInfo, and SharedInfo omitted. The `role` and RevocableInfo,
-  // if any, are preserved. Because we clear ReservationInfo but
-  // preserve `role`, this means that stripping a dynamically reserved
-  // resource makes it effectively statically reserved.
+  // but with all the meta-data fields, such as AllocationInfo,
+  // ReservationInfo and etc. cleared. Only scalar resources' name,
+  // type (SCALAR) and value are preserved.
   //
   // This is intended for code that would like to aggregate together
   // Resource values without regard for metadata like whether the

http://git-wip-us.apache.org/repos/asf/mesos/blob/f0284a51/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index b9f1c2d..253b8bc 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -1640,24 +1640,12 @@ Resources Resources::createStrippedScalarQuantity() const
 
   foreach (const Resource& resource, resources) {
     if (resource.type() == Value::SCALAR) {
-      Resource scalar = resource;
-      scalar.clear_provider_id();
-      scalar.clear_allocation_info();
-
-      // We collapse the stack of reservations here to a single `STATIC`
-      // reservation in order to maintain existing behavior of ignoring
-      // the reservation type, and keeping the reservation role.
-      if (Resources::isReserved(scalar)) {
-        Resource::ReservationInfo collapsedReservation;
-        collapsedReservation.set_type(Resource::ReservationInfo::STATIC);
-        collapsedReservation.set_role(Resources::reservationRole(scalar));
-        scalar.clear_reservations();
-        scalar.add_reservations()->CopyFrom(collapsedReservation);
-      }
+      Resource scalar;
+
+      scalar.set_name(resource.name());
+      scalar.set_type(resource.type());
+      scalar.mutable_scalar()->CopyFrom(resource.scalar());
 
-      scalar.clear_disk();
-      scalar.clear_shared();
-      scalar.clear_revocable();
       stripped.add(scalar);
     }
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/f0284a51/src/master/allocator/mesos/hierarchical.cpp
----------------------------------------------------------------------
diff --git a/src/master/allocator/mesos/hierarchical.cpp b/src/master/allocator/mesos/hierarchical.cpp
index 60d8b5d..cbdfb2b 100644
--- a/src/master/allocator/mesos/hierarchical.cpp
+++ b/src/master/allocator/mesos/hierarchical.cpp
@@ -1572,11 +1572,7 @@ void HierarchicalAllocatorProcess::__allocate()
   // NOTE: Revocable resources are excluded in `quotaRoleSorter`.
   auto getQuotaRoleAllocatedScalarQuantities = [this](const string& role) {
     CHECK(quotas.contains(role));
-
-    // NOTE: `allocationScalarQuantities` omits dynamic reservation,
-    // persistent volume info, and allocation info. We additionally
-    // remove the static reservations here via `toUnreserved()`.
-    return quotaRoleSorter->allocationScalarQuantities(role).toUnreserved();
+    return quotaRoleSorter->allocationScalarQuantities(role);
   };
 
   // Returns the result of shrinking the provided resources down to the
@@ -1664,10 +1660,8 @@ void HierarchicalAllocatorProcess::__allocate()
       quotaRoleSorter->allocation(role);
 
     foreachvalue (const Resources& resources, allocations) {
-      // We need to remove the static reservation metadata here via
-      // `toUnreserved()`.
       rolesConsumedQuotaScalarQuantites[role] -=
-        resources.reserved().createStrippedScalarQuantity().toUnreserved();
+        resources.reserved().createStrippedScalarQuantity();
     }
   }
 
@@ -1702,20 +1696,11 @@ void HierarchicalAllocatorProcess::__allocate()
   //                        allocated resources -
   //                        unallocated reservations -
   //                        unallocated revocable resources
-
-  // NOTE: `totalScalarQuantities` omits dynamic reservation,
-  // persistent volume info, and allocation info. We additionally
-  // remove the static reservations here via `toUnreserved()`.
-  Resources availableHeadroom =
-    roleSorter->totalScalarQuantities().toUnreserved();
+  Resources availableHeadroom = roleSorter->totalScalarQuantities();
 
   // Subtract allocated resources from the total.
   foreachkey (const string& role, roles) {
-    // NOTE: `totalScalarQuantities` omits dynamic reservation,
-    // persistent volume info, and allocation info. We additionally
-    // remove the static reservations here via `toUnreserved()`.
-    availableHeadroom -=
-      roleSorter->allocationScalarQuantities(role).toUnreserved();
+    availableHeadroom -= roleSorter->allocationScalarQuantities(role);
   }
 
   // Calculate total allocated reservations. Note that we need to ensure
@@ -1733,11 +1718,8 @@ void HierarchicalAllocatorProcess::__allocate()
     }
 
     foreachvalue (const Resources& resources, allocations) {
-      // NOTE: `totalScalarQuantities` omits dynamic reservation,
-      // persistent volume info, and allocation info. We additionally
-      // remove the static reservations here via `toUnreserved()`.
       totalAllocatedReservationScalarQuantities +=
-        resources.reserved().createStrippedScalarQuantity().toUnreserved();
+        resources.reserved().createStrippedScalarQuantity();
     }
   }
 
@@ -1748,11 +1730,8 @@ void HierarchicalAllocatorProcess::__allocate()
 
   // Subtract revocable resources.
   foreachvalue (const Slave& slave, slaves) {
-    // NOTE: `totalScalarQuantities` omits dynamic reservation,
-    // persistent volume info, and allocation info. We additionally
-    // remove the static reservations here via `toUnreserved()`.
-    availableHeadroom -= slave.getAvailable().revocable()
-      .createStrippedScalarQuantity().toUnreserved();
+    availableHeadroom -=
+      slave.getAvailable().revocable().createStrippedScalarQuantity();
   }
 
   // Due to the two stages in the allocation algorithm and the nature of
@@ -2637,9 +2616,8 @@ void HierarchicalAllocatorProcess::trackReservations(
 {
   foreachpair (const string& role,
                const Resources& resources, reservations) {
-    // We remove the static reservation metadata here via `toUnreserved()`.
     const Resources scalarQuantitesToTrack =
-        resources.createStrippedScalarQuantity().toUnreserved();
+      resources.createStrippedScalarQuantity();
 
     reservationScalarQuantities[role] += scalarQuantitesToTrack;
   }
@@ -2655,9 +2633,8 @@ void HierarchicalAllocatorProcess::untrackReservations(
     Resources& currentReservationQuantity =
         reservationScalarQuantities.at(role);
 
-    // We remove the static reservation metadata here via `toUnreserved()`.
     const Resources scalarQuantitesToUntrack =
-        resources.createStrippedScalarQuantity().toUnreserved();
+      resources.createStrippedScalarQuantity();
     CHECK(currentReservationQuantity.contains(scalarQuantitesToUntrack));
     currentReservationQuantity -= scalarQuantitesToUntrack;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f0284a51/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index 1410423..0095b94 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -2637,19 +2637,14 @@ TEST(ResourcesOperationTest, StrippedResourcesVolume)
   Resources stripped = volume.createStrippedScalarQuantity();
 
   EXPECT_TRUE(stripped.persistentVolumes().empty());
+  EXPECT_TRUE(stripped.reserved().empty());
   EXPECT_EQ(Megabytes(200), stripped.disk().get());
 
-  // `createStrippedScalarQuantity` doesn't remove the `role` from a
-  // reserved resource.
-  EXPECT_FALSE(stripped.reserved("role").empty());
-
   Resource strippedVolume = *(stripped.begin());
 
   ASSERT_EQ(Value::SCALAR, strippedVolume.type());
   EXPECT_DOUBLE_EQ(200, strippedVolume.scalar().value());
-  EXPECT_EQ("role", Resources::reservationRole(strippedVolume));
   EXPECT_EQ("disk", strippedVolume.name());
-  EXPECT_EQ(1, strippedVolume.reservations_size());
   EXPECT_FALSE(strippedVolume.has_disk());
   EXPECT_FALSE(Resources::isPersistentVolume(strippedVolume));
 }
@@ -2678,13 +2673,11 @@ TEST(ResourcesOperationTest, StrippedResourcesReserved)
 
   Resources stripped = dynamicallyReserved.createStrippedScalarQuantity();
 
-  EXPECT_FALSE(stripped.reserved("role").empty());
+  EXPECT_TRUE(stripped.reserved("role").empty());
 
   foreach (const Resource& resource, stripped) {
-    EXPECT_EQ("role", Resources::reservationRole(resource));
-    EXPECT_FALSE(resource.reservations().empty());
     EXPECT_FALSE(Resources::isDynamicallyReserved(resource));
-    EXPECT_FALSE(Resources::isUnreserved(resource));
+    EXPECT_TRUE(Resources::isUnreserved(resource));
   }
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f0284a51/src/tests/sorter_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/sorter_tests.cpp b/src/tests/sorter_tests.cpp
index 9cdffd7..266a9e7 100644
--- a/src/tests/sorter_tests.cpp
+++ b/src/tests/sorter_tests.cpp
@@ -1594,7 +1594,7 @@ TYPED_TEST(CommonSorterTest, RemoveSharedResources)
   sorter.add(slaveId, sharedDisk);
   Resources quantity2 = sorter.totalScalarQuantities();
 
-  EXPECT_EQ(Resources::parse("disk(role1):100").get(), quantity2 - quantity1);
+  EXPECT_EQ(Resources::parse("disk:100").get(), quantity2 - quantity1);
 
   sorter.add(slaveId, sharedDisk);
   Resources quantity3 = sorter.totalScalarQuantities();

http://git-wip-us.apache.org/repos/asf/mesos/blob/f0284a51/src/v1/resources.cpp
----------------------------------------------------------------------
diff --git a/src/v1/resources.cpp b/src/v1/resources.cpp
index 3d06fc6..ab8fc3e 100644
--- a/src/v1/resources.cpp
+++ b/src/v1/resources.cpp
@@ -1658,24 +1658,12 @@ Resources Resources::createStrippedScalarQuantity() const
 
   foreach (const Resource& resource, resources) {
     if (resource.type() == Value::SCALAR) {
-      Resource scalar = resource;
-      scalar.clear_provider_id();
-      scalar.clear_allocation_info();
-
-      // We collapse the stack of reservations here to a single `STATIC`
-      // reservation in order to maintain existing behavior of ignoring
-      // the reservation type, and keeping the reservation role.
-      if (Resources::isReserved(scalar)) {
-        Resource::ReservationInfo collapsedReservation;
-        collapsedReservation.set_type(Resource::ReservationInfo::STATIC);
-        collapsedReservation.set_role(Resources::reservationRole(scalar));
-        scalar.clear_reservations();
-        scalar.add_reservations()->CopyFrom(collapsedReservation);
-      }
+      Resource scalar;
+
+      scalar.set_name(resource.name());
+      scalar.set_type(resource.type());
+      scalar.mutable_scalar()->CopyFrom(resource.scalar());
 
-      scalar.clear_disk();
-      scalar.clear_shared();
-      scalar.clear_revocable();
       stripped.add(scalar);
     }
   }