You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2014/11/19 09:54:06 UTC

[1/9] mesos git commit: Always store validated and combined Resource objects in C++ Resources.

Repository: mesos
Updated Branches:
  refs/heads/master 7ff4920e1 -> 0966e6ec4


Always store validated and combined Resource objects in C++ Resources.

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


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

Branch: refs/heads/master
Commit: a8d0244b20313e3deb2d3b8483353555a8147ac3
Parents: 85728f8
Author: Jie Yu <yu...@gmail.com>
Authored: Fri Nov 14 14:45:05 2014 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Nov 19 00:14:25 2014 -0800

----------------------------------------------------------------------
 include/mesos/resources.hpp                     | 101 ++--
 src/common/resources.cpp                        | 536 ++++++++-----------
 src/examples/low_level_scheduler_libprocess.cpp |   2 +-
 src/examples/low_level_scheduler_pthread.cpp    |   2 +-
 src/examples/test_framework.cpp                 |   4 +-
 src/master/drf_sorter.cpp                       |  11 +-
 src/master/hierarchical_allocator_process.hpp   |   4 +-
 src/master/http.cpp                             |  10 +-
 src/master/master.cpp                           |  18 +-
 src/slave/containerizer/containerizer.cpp       |  16 +-
 src/tests/gc_tests.cpp                          |  20 +-
 src/tests/mesos.hpp                             |   5 +-
 src/tests/resource_offers_tests.cpp             |  10 +-
 src/tests/resources_tests.cpp                   | 157 +++---
 14 files changed, 414 insertions(+), 482 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/include/mesos/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/resources.hpp b/include/mesos/resources.hpp
index 86b70df..3009395 100644
--- a/include/mesos/resources.hpp
+++ b/include/mesos/resources.hpp
@@ -43,8 +43,13 @@
 
 namespace mesos {
 
-// TODO(bmahler): Ensure that the underlying resources are kept
-// in a flattened state: MESOS-1714.
+// NOTE: Resource objects stored in the class are always valid and
+// kept combined if possible. It is the caller's responsibility to
+// validate any Resource object or repeated Resource protobufs before
+// constructing a Resources object. Otherwise, invalid Resource
+// objects will be silently stripped. Invalid Resource objects will
+// also be silently ignored when used in arithmetic operations (e.g.,
+// +=, -=, etc.).
 class Resources
 {
 public:
@@ -62,49 +67,38 @@ public:
       const std::string& text,
       const std::string& defaultRole = "*");
 
-  // Returns true iff this resource has a name, a valid type, i.e.
-  // scalar, range, or set, and has the appropriate value set for its
-  // type.
-  static bool isValid(const Resource& resource);
+  // Validates the given Resource object. Returns Error if it is not
+  // valid. A Resource object is valid if it has a name, a valid type,
+  // i.e. scalar, range, or set, and has the appropriate value set.
+  static Option<Error> validate(const Resource& resource);
 
-  // Returns true iff this resource is valid and allocatable. In
-  // particular, a scalar is allocatable if it's value is greater than
-  // zero, a ranges is allocatable if there is at least one valid
-  // range in it, and a set is allocatable if it has at least one
-  // item.
-  static bool isAllocatable(const Resource& resource);
+  // Validates the given protobufs.
+  // TODO(jieyu): Right now, it's the same as checking each individual
+  // Resource object in the protobufs. In the future, we could add
+  // more checks that are not possible if checking each Resource
+  // object individually. For example, we could check multiple usage
+  // of an item in a set or a ranges, etc.
+  static Option<Error> validate(
+      const google::protobuf::RepeatedPtrField<Resource>& resources);
 
-  // Returns true iff this resource is zero valued, i.e. is zero for
-  // scalars, has a range size of zero for ranges, and has no items
-  // for sets.
-  static bool isZero(const Resource& resource);
+  // Tests if the given Resource object is empty.
+  static bool empty(const Resource& resource);
 
   Resources() {}
 
   // TODO(jieyu): Consider using C++11 initializer list.
-  /*implicit*/ Resources(const Resource& resource)
-  {
-    resources.Add()->CopyFrom(resource);
-  }
+  /*implicit*/ Resources(const Resource& resource);
 
   /*implicit*/
-  Resources(const google::protobuf::RepeatedPtrField<Resource>& _resources)
-  {
-    resources.MergeFrom(_resources);
-  }
+  Resources(const google::protobuf::RepeatedPtrField<Resource>& _resources);
 
-  Resources(const Resources& that)
-  {
-    resources.MergeFrom(that.resources);
-  }
+  Resources(const Resources& that) : resources(that.resources) {}
 
   Resources& operator = (const Resources& that)
   {
     if (this != &that) {
-      resources.Clear();
-      resources.MergeFrom(that.resources);
+      resources = that.resources;
     }
-
     return *this;
   }
 
@@ -118,31 +112,25 @@ public:
   Resources extract(const std::string& role) const;
 
   // Returns a Resources object with the same amount of each resource
-  // type as these Resources, but with only one Resource object per
-  // type and all Resource object marked as the specified role.
+  // type as these Resources, but with all Resource objects marked as
+  // the specified role.
   Resources flatten(const std::string& role = "*") const;
 
-  // Finds a number of resources equal to toFind in these Resources
-  // and returns them marked with appropriate roles. For each resource
-  // type, resources are first taken from the specified role, then
-  // from '*', then from any other role.
-  Option<Resources> find(
-      const Resources& toFind,
-      const std::string& role = "*") const;
-
-  // Returns the Resource from these Resources that matches the
-  // argument in name, type, and role, if it exists.
-  Option<Resource> get(const Resource& r) const;
-
-  // Returns all Resources from these Resources that match the
-  // argument in name and type, regardless of role.
-  Option<Resources> getAll(const Resource& r) const;
-
+  // Finds a Resources object with the same amount of each resource
+  // type as "targets" from these Resources. The roles specified in
+  // "targets" set the preference order. For each resource type,
+  // resources are first taken from the specified role, then from '*',
+  // then from any other role.
+  // TODO(jieyu): 'find' contains some allocation logic for scalars and
+  // fixed set / range elements. However, this is not sufficient for
+  // schedulers that want, say, any N available ports. We should
+  // consider moving this to an internal "allocation" library for our
+  // example frameworks to leverage.
+  Option<Resources> find(const Resources& targets) const;
+
+  // Helpers to get resource values. We consider all roles here.
   template <typename T>
-  T get(const std::string& name, const T& t) const;
-
-  // Returns a Resources object with only the allocatable resources.
-  Resources allocatable() const;
+  Option<T> get(const std::string& name) const;
 
   // Helpers to get known resource types.
   // TODO(vinod): Fix this when we make these types as first class
@@ -195,6 +183,13 @@ public:
   Resources& operator -= (const Resources& that);
 
 private:
+  bool contains(const Resource& that) const;
+
+  // Similar to the public 'find', but only for a single Resource
+  // object. The target resource may span multiple roles, so this
+  // returns Resources.
+  Option<Resources> find(const Resource& target) const;
+
   google::protobuf::RepeatedPtrField<Resource> resources;
 };
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index 8474cdd..3a3c6a6 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -251,83 +251,92 @@ Try<Resources> Resources::parse(
 }
 
 
-bool Resources::isValid(const Resource& resource)
+Option<Error> Resources::validate(const Resource& resource)
 {
-  if (!resource.has_name() ||
-      resource.name() == "" ||
-      !resource.has_type() ||
-      !Value::Type_IsValid(resource.type())) {
-    return false;
+  if (resource.name().empty()) {
+    return Error("Empty resource name");
   }
 
-  if (resource.type() == Value::SCALAR) {
-    return resource.has_scalar();
-  } else if (resource.type() == Value::RANGES) {
-    return resource.has_ranges();
-  } else if (resource.type() == Value::SET) {
-    return resource.has_set();
-  } else if (resource.type() == Value::TEXT) {
-    // Resources doesn't support text.
-    return false;
+  if (!Value::Type_IsValid(resource.type())) {
+    return Error("Invalid resource type");
   }
 
-  return false;
-}
+  if (resource.type() == Value::SCALAR) {
+    if (!resource.has_scalar() ||
+        resource.has_ranges() ||
+        resource.has_set()) {
+      return Error("Invalid scalar resource");
+    }
 
+    if (resource.scalar().value() < 0) {
+      return Error("Invalid scalar resource: value < 0");
+    }
+  } else if (resource.type() == Value::RANGES) {
+    if (resource.has_scalar() ||
+        !resource.has_ranges() ||
+        resource.has_set()) {
+      return Error("Invalid ranges resource");
+    }
 
-bool Resources::isAllocatable(const Resource& resource)
-{
-  if (isValid(resource)) {
-    if (resource.type() == Value::SCALAR) {
-      if (resource.scalar().value() <= 0) {
-        return false;
+    for (int i = 0; i < resource.ranges().range_size(); i++) {
+      const Value::Range& range = resource.ranges().range(i);
+
+      // Ensure the range make sense (isn't inverted).
+      if (range.begin() > range.end()) {
+        return Error("Invalid ranges resource: begin > end");
       }
-    } else if (resource.type() == Value::RANGES) {
-      if (resource.ranges().range_size() == 0) {
-        return false;
-      } else {
-        for (int i = 0; i < resource.ranges().range_size(); i++) {
-          const Value::Range& range = resource.ranges().range(i);
-
-          // Ensure the range make sense (isn't inverted).
-          if (range.begin() > range.end()) {
-            return false;
-          }
-
-          // Ensure ranges don't overlap (but not necessarily coalesced).
-          for (int j = i + 1; j < resource.ranges().range_size(); j++) {
-            if (range.begin() <= resource.ranges().range(j).begin() &&
-                resource.ranges().range(j).begin() <= range.end()) {
-              return false;
-            }
-          }
+
+      // Ensure ranges don't overlap (but not necessarily coalesced).
+      for (int j = i + 1; j < resource.ranges().range_size(); j++) {
+        if (range.begin() <= resource.ranges().range(j).begin() &&
+            resource.ranges().range(j).begin() <= range.end()) {
+          return Error("Invalid ranges resource: overlapping ranges");
         }
       }
-    } else if (resource.type() == Value::SET) {
-      if (resource.set().item_size() == 0) {
-        return false;
-      } else {
-        for (int i = 0; i < resource.set().item_size(); i++) {
-          const string& item = resource.set().item(i);
-
-          // Ensure no duplicates.
-          for (int j = i + 1; j < resource.set().item_size(); j++) {
-            if (item == resource.set().item(j)) {
-              return false;
-            }
-          }
+    }
+  } else if (resource.type() == Value::SET) {
+    if (resource.has_scalar() ||
+        resource.has_ranges() ||
+        !resource.has_set()) {
+      return Error("Invalid set resource");
+    }
+
+    for (int i = 0; i < resource.set().item_size(); i++) {
+      const string& item = resource.set().item(i);
+
+      // Ensure no duplicates.
+      for (int j = i + 1; j < resource.set().item_size(); j++) {
+        if (item == resource.set().item(j)) {
+          return Error("Invalid set resource: duplicated elements");
         }
       }
     }
+  } else {
+    // Resource doesn't support TEXT or other value types.
+    return Error("Unsupported resource type");
+  }
 
-    return true;
+  return None();
+}
+
+
+Option<Error> Resources::validate(
+    const google::protobuf::RepeatedPtrField<Resource>& resources)
+{
+  foreach (const Resource& resource, resources) {
+    Option<Error> error = validate(resource);
+    if (error.isSome()) {
+      return Error(
+          "Resource '" + stringify(resource) +
+          "' is invalid: " + error.get().message);
+    }
   }
 
-  return false;
+  return None();
 }
 
 
-bool Resources::isZero(const Resource& resource)
+bool Resources::empty(const Resource& resource)
 {
   if (resource.type() == Value::SCALAR) {
     return resource.scalar().value() == 0;
@@ -335,9 +344,9 @@ bool Resources::isZero(const Resource& resource)
     return resource.ranges().range_size() == 0;
   } else if (resource.type() == Value::SET) {
     return resource.set().item_size() == 0;
+  } else {
+    return false;
   }
-
-  return false;
 }
 
 
@@ -346,6 +355,23 @@ bool Resources::isZero(const Resource& resource)
 /////////////////////////////////////////////////
 
 
+Resources::Resources(const Resource& resource)
+{
+  // NOTE: Invalid and zero Resource object will be ignored.
+  *this += resource;
+}
+
+
+Resources::Resources(
+    const google::protobuf::RepeatedPtrField<Resource>& resources)
+{
+  foreach (const Resource& resource, resources) {
+    // NOTE: Invalid and zero Resource objects will be ignored.
+    *this += resource;
+  }
+}
+
+
 Resources Resources::extract(const string& role) const
 {
   Resources r;
@@ -364,85 +390,61 @@ Resources Resources::flatten(const string& role) const
 {
   Resources flattened;
 
-  foreach (const Resource& r, resources) {
-    Resource toRemove = r;
-    toRemove.set_role(role);
-
-    bool found = false;
-    for (int i = 0; i < flattened.resources.size(); i++) {
-      Resource removed = flattened.resources.Get(i);
-      if (toRemove.name() == removed.name() &&
-          toRemove.type() == removed.type()) {
-        flattened.resources.Mutable(i)->MergeFrom(toRemove + removed);
-        found = true;
-        break;
-      }
-    }
-
-    if (!found) {
-      flattened.resources.Add()->MergeFrom(toRemove);
-    }
+  foreach (Resource resource, resources) {
+    resource.set_role(role);
+    flattened += resource;
   }
 
   return flattened;
 }
 
 
-Option<Resources> Resources::find(
-    const Resources& toFind,
-    const string& role) const
-{
-  Resources foundResources;
-
-  foreach (const Resource& findResource, toFind) {
-    Resource remaining = findResource;
-    Option<Resources> all = getAll(findResource);
-    bool done = false;
-
-    if (isZero(findResource)) {
-      // Done, as no resources of this type have been requested.
-      done = true;
-    } else if (all.isSome()) {
-      for (int i = 0; i < 3 && !done; i++) {
-        foreach (const Resource& potential, all.get()) {
-          // Ensures that we take resources first from the specified role,
-          // then from the default role, and then from any other role.
-          if ((i == 0 && potential.role() == role) ||
-              (i == 1 && potential.role() == "*" && potential.role() != role) ||
-              (i == 2 && potential.role() != "*" && potential.role() != role)) {
-            // The resources must have the same role for <= to work.
-            Resource potential_ = potential;
-            potential_.set_role(remaining.role());
-            if (remaining <= potential_) {
-              // We can satisfy the remaining requirements for this
-              // resource type.
-              Resource found = remaining;
-              found.set_role(potential.role());
-              foundResources += found;
-              done = true;
-            } else {
-              foundResources += potential;
-              remaining -= potential_;
-            }
-          }
-        }
-      }
-    }
+class RoleFilter
+{
+public:
+  static RoleFilter any() { return RoleFilter(); }
 
-    if (!done) {
-      return None();
-    }
+  RoleFilter() : type(ANY) {}
+  /*implicit*/ RoleFilter(const string& _role) : type(SOME), role(_role) {}
+
+  Resources apply(const Resources& resources) const
+  {
+    return type == ANY? resources : resources.extract(role);
   }
 
-  return foundResources;
-}
+private:
+  enum { ANY, SOME } type;
+  string role;
+};
 
 
-Option<Resource> Resources::get(const Resource& r) const
+Option<Resources> Resources::find(const Resource& target) const
 {
-  foreach (const Resource& resource, resources) {
-    if (addable(resource, r)) {
-      return resource;
+  Resources found;
+  Resources total = *this;
+  Resources remaining = Resources(target).flatten();
+
+  // First look in the target role, then "*", then any remaining role.
+  vector<RoleFilter> filters = {
+    RoleFilter(target.role()),
+    RoleFilter("*"),
+    RoleFilter::any()
+  };
+
+  foreach (const RoleFilter& filter, filters) {
+    foreach (const Resource& resource, filter.apply(total)) {
+      // Need to flatten to ignore the roles in contains().
+      Resources flattened = Resources(resource).flatten();
+
+      if (remaining <= flattened) {
+        // Done!
+        return found + remaining.flatten(resource.role());
+      } else if (flattened <= remaining) {
+        found += resource;
+        total -= resource;
+        remaining -= flattened;
+        break;
+      }
     }
   }
 
@@ -450,29 +452,27 @@ Option<Resource> Resources::get(const Resource& r) const
 }
 
 
-Option<Resources> Resources::getAll(const Resource& r) const
+Option<Resources> Resources::find(const Resources& targets) const
 {
   Resources total;
 
-  foreach (const Resource& resource, resources) {
-    if (r.name() == resource.name() &&
-        r.type() == resource.type()) {
-      total += resource;
+  foreach (const Resource& target, targets) {
+    Option<Resources> found = find(target);
+
+    // Each target needs to be found!
+    if (found.isNone()) {
+      return None();
     }
-  }
 
-  if (total.size() > 0) {
-    return total;
+    total += found.get();
   }
 
-  return None();
+  return total;
 }
 
 
 template <>
-Value::Scalar Resources::get(
-    const string& name,
-    const Value::Scalar& scalar) const
+Option<Value::Scalar> Resources::get(const string& name) const
 {
   Value::Scalar total;
   bool found = false;
@@ -489,22 +489,20 @@ Value::Scalar Resources::get(
     return total;
   }
 
-  return scalar;
+  return None();
 }
 
 
 template <>
-Value::Ranges Resources::get(
-    const string& name,
-    const Value::Ranges& ranges) const
+Option<Value::Set> Resources::get(const string& name) const
 {
-  Value::Ranges total;
+  Value::Set total;
   bool found = false;
 
   foreach (const Resource& resource, resources) {
     if (resource.name() == name &&
-        resource.type() == Value::RANGES) {
-      total += resource.ranges();
+        resource.type() == Value::SET) {
+      total += resource.set();
       found = true;
     }
   }
@@ -513,22 +511,20 @@ Value::Ranges Resources::get(
     return total;
   }
 
-  return ranges;
+  return None();
 }
 
 
 template <>
-Value::Set Resources::get(
-    const string& name,
-    const Value::Set& set) const
+Option<Value::Ranges> Resources::get(const string& name) const
 {
-  Value::Set total;
+  Value::Ranges total;
   bool found = false;
 
   foreach (const Resource& resource, resources) {
     if (resource.name() == name &&
-        resource.type() == Value::SET) {
-      total += resource.set();
+        resource.type() == Value::RANGES) {
+      total += resource.ranges();
       found = true;
     }
   }
@@ -537,125 +533,74 @@ Value::Set Resources::get(
     return total;
   }
 
-  return set;
-}
-
-
-Resources Resources::allocatable() const
-{
-  Resources result;
-
-  foreach (const Resource& resource, resources) {
-    if (isAllocatable(resource)) {
-      result.resources.Add()->MergeFrom(resource);
-    }
-  }
-
-  return result;
+  return None();
 }
 
 
 Option<double> Resources::cpus() const
 {
-  double total= 0;
-  bool found = false;
-
-  foreach (const Resource& resource, resources) {
-    if (resource.name() == "cpus" && resource.type() == Value::SCALAR) {
-      total += resource.scalar().value();
-      found = true;
-    }
-  }
-
-  if (found) {
-    return total;
+  Option<Value::Scalar> value = get<Value::Scalar>("cpus");
+  if (value.isSome()) {
+    return value.get().value();
+  } else {
+    return None();
   }
-
-  return None();
 }
 
 
 Option<Bytes> Resources::mem() const
 {
-  double total = 0;
-  bool found = false;
-
-  foreach (const Resource& resource, resources) {
-    if (resource.name() == "mem" &&
-        resource.type() == Value::SCALAR) {
-      total += resource.scalar().value();
-      found = true;
-    }
-  }
-
-  if (found) {
-    return Megabytes(static_cast<uint64_t>(total));
+  Option<Value::Scalar> value = get<Value::Scalar>("mem");
+  if (value.isSome()) {
+    return Megabytes(static_cast<uint64_t>(value.get().value()));
+  } else {
+    return None();
   }
-
-  return None();
 }
 
 
 Option<Bytes> Resources::disk() const
 {
-  double total = 0;
-  bool found = false;
-
-  foreach (const Resource& resource, resources) {
-    if (resource.name() == "disk" &&
-        resource.type() == Value::SCALAR) {
-      total += resource.scalar().value();
-      found = true;
-    }
-  }
-
-  if (found) {
-    return Megabytes(static_cast<uint64_t>(total));
+  Option<Value::Scalar> value = get<Value::Scalar>("disk");
+  if (value.isSome()) {
+    return Megabytes(static_cast<uint64_t>(value.get().value()));
+  } else {
+    return None();
   }
-
-  return None();
 }
 
 
 Option<Value::Ranges> Resources::ports() const
 {
-  Value::Ranges total;
-  bool found = false;
-
-  foreach (const Resource& resource, resources) {
-    if (resource.name() == "ports" &&
-        resource.type() == Value::RANGES) {
-      total += resource.ranges();
-      found = true;
-    }
-  }
-
-  if (found) {
-    return total;
+  Option<Value::Ranges> value = get<Value::Ranges>("ports");
+  if (value.isSome()) {
+    return value.get();
+  } else {
+    return None();
   }
-
-  return None();
 }
 
 
 Option<Value::Ranges> Resources::ephemeral_ports() const
 {
-  Value::Ranges total;
-  bool found = false;
+  Option<Value::Ranges> value = get<Value::Ranges>("ephemeral_ports");
+  if (value.isSome()) {
+    return value.get();
+  } else {
+    return None();
+  }
+}
+
 
+bool Resources::contains(const Resource& that) const
+{
   foreach (const Resource& resource, resources) {
-    if (resource.name() == "ephemeral_ports" &&
-        resource.type() == Value::RANGES) {
-      total += resource.ranges();
-      found = true;
+    if (mesos::contains(resource, that)) {
+      return true;
     }
   }
 
-  if (found) {
-    return total;
-  }
-
-  return None();
+  return false;
 }
 
 
@@ -672,22 +617,7 @@ Resources::operator const google::protobuf::RepeatedPtrField<Resource>& () const
 
 bool Resources::operator == (const Resources& that) const
 {
-  if (size() != that.size()) {
-    return false;
-  }
-
-  foreach (const Resource& resource, resources) {
-    Option<Resource> option = that.get(resource);
-    if (option.isNone()) {
-      return false;
-      } else {
-      if (!(resource == option.get())) {
-        return false;
-      }
-    }
-  }
-
-    return true;
+  return *this <= that && that <= *this;
 }
 
 
@@ -700,15 +630,8 @@ bool Resources::operator != (const Resources& that) const
 bool Resources::operator <= (const Resources& that) const
 {
   foreach (const Resource& resource, resources) {
-    Option<Resource> option = that.get(resource);
-    if (option.isNone()) {
-      if (!isZero(resource)) {
-        return false;
-      }
-    } else {
-      if (!(resource <= option.get())) {
-        return false;
-      }
+    if (!that.contains(resource)) {
+      return false;
     }
   }
 
@@ -718,42 +641,38 @@ bool Resources::operator <= (const Resources& that) const
 
 Resources Resources::operator + (const Resource& that) const
 {
-  Resources result;
-
-  bool added = false;
-
-  foreach (const Resource& resource, resources) {
-    if (addable(resource, that)) {
-      result.resources.Add()->MergeFrom(resource + that);
-      added = true;
-    } else {
-      result.resources.Add()->MergeFrom(resource);
-    }
-  }
-
-  if (!added) {
-    result.resources.Add()->MergeFrom(that);
-  }
-
+  Resources result = *this;
+  result += that;
   return result;
 }
 
 
 Resources Resources::operator + (const Resources& that) const
 {
-  Resources result(*this);
-
-  foreach (const Resource& resource, that.resources) {
-    result += resource;
-  }
-
+  Resources result = *this;
+  result += that;
   return result;
 }
 
 
 Resources& Resources::operator += (const Resource& that)
 {
-  *this = *this + that;
+  if (validate(that).isNone() && !empty(that)) {
+    bool found = false;
+    foreach (Resource& resource, resources) {
+      if (addable(resource, that)) {
+        resource += that;
+        found = true;
+        break;
+      }
+    }
+
+    // Cannot be combined with any existing Resource object.
+    if (!found) {
+      resources.Add()->CopyFrom(that);
+    }
+  }
+
   return *this;
 }
 
@@ -770,38 +689,41 @@ Resources& Resources::operator += (const Resources& that)
 
 Resources Resources::operator - (const Resource& that) const
 {
-  Resources result;
-
-  foreach (const Resource& resource, resources) {
-    if (subtractable(resource, that)) {
-      Resource r = resource - that;
-      if (!isZero(r)) {
-        result.resources.Add()->MergeFrom(r);
-      }
-    } else {
-      result.resources.Add()->MergeFrom(resource);
-    }
-  }
-
+  Resources result = *this;
+  result -= that;
   return result;
 }
 
 
 Resources Resources::operator - (const Resources& that) const
 {
-  Resources result(*this);
-
-  foreach (const Resource& resource, that.resources) {
-    result -= resource;
-  }
-
+  Resources result = *this;
+  result -= that;
   return result;
 }
 
 
 Resources& Resources::operator -= (const Resource& that)
 {
-  *this = *this - that;
+  if (validate(that).isNone() && !empty(that)) {
+    for (int i = 0; i < resources.size(); i++) {
+      Resource* resource = resources.Mutable(i);
+
+      if (subtractable(*resource, that)) {
+        *resource -= that;
+
+        // Remove the resource if it becomes invalid or zero. We need
+        // to do the validation because we want to strip negative
+        // scalar Resource object.
+        if (validate(*resource).isSome() || empty(*resource)) {
+          resources.DeleteSubrange(i, 1);
+        }
+
+        break;
+      }
+    }
+  }
+
   return *this;
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/src/examples/low_level_scheduler_libprocess.cpp
----------------------------------------------------------------------
diff --git a/src/examples/low_level_scheduler_libprocess.cpp b/src/examples/low_level_scheduler_libprocess.cpp
index 89b4318..7229797 100644
--- a/src/examples/low_level_scheduler_libprocess.cpp
+++ b/src/examples/low_level_scheduler_libprocess.cpp
@@ -238,7 +238,7 @@ private:
         task.mutable_executor()->MergeFrom(executor);
 
         Option<Resources> resources =
-          remaining.find(TASK_RESOURCES, framework.role());
+          remaining.find(TASK_RESOURCES.flatten(framework.role()));
 
         CHECK_SOME(resources);
         task.mutable_resources()->MergeFrom(resources.get());

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/src/examples/low_level_scheduler_pthread.cpp
----------------------------------------------------------------------
diff --git a/src/examples/low_level_scheduler_pthread.cpp b/src/examples/low_level_scheduler_pthread.cpp
index e5cd48a..2b012a8 100644
--- a/src/examples/low_level_scheduler_pthread.cpp
+++ b/src/examples/low_level_scheduler_pthread.cpp
@@ -288,7 +288,7 @@ private:
         task.mutable_executor()->MergeFrom(executor);
 
         Option<Resources> resources =
-          remaining.find(TASK_RESOURCES, framework.role());
+          remaining.find(TASK_RESOURCES.flatten(framework.role()));
 
         CHECK_SOME(resources);
         task.mutable_resources()->MergeFrom(resources.get());

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/src/examples/test_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_framework.cpp b/src/examples/test_framework.cpp
index e87198b..ce1616d 100644
--- a/src/examples/test_framework.cpp
+++ b/src/examples/test_framework.cpp
@@ -104,7 +104,9 @@ public:
         task.mutable_slave_id()->MergeFrom(offer.slave_id());
         task.mutable_executor()->MergeFrom(executor);
 
-        Option<Resources> resources = remaining.find(TASK_RESOURCES, role);
+        Option<Resources> resources =
+          remaining.find(TASK_RESOURCES.flatten(role));
+
         CHECK_SOME(resources);
         task.mutable_resources()->MergeFrom(resources.get());
         remaining -= resources.get();

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/src/master/drf_sorter.cpp
----------------------------------------------------------------------
diff --git a/src/master/drf_sorter.cpp b/src/master/drf_sorter.cpp
index 5464900..0ad6c52 100644
--- a/src/master/drf_sorter.cpp
+++ b/src/master/drf_sorter.cpp
@@ -226,11 +226,14 @@ double DRFSorter::calculateShare(const string& name)
       double total = resource.scalar().value();
 
       if (total > 0) {
-        Value::Scalar none;
-        const Value::Scalar& scalar =
-          allocations[name].get(resource.name(), none);
+        Option<Value::Scalar> scalar =
+          allocations[name].get<Value::Scalar>(resource.name());
 
-        share = std::max(share, scalar.value() / total);
+        if (scalar.isNone()) {
+          scalar = Value::Scalar();
+        }
+
+        share = std::max(share, scalar.get().value() / total);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/src/master/hierarchical_allocator_process.hpp
----------------------------------------------------------------------
diff --git a/src/master/hierarchical_allocator_process.hpp b/src/master/hierarchical_allocator_process.hpp
index 31dfb2c..77f9741 100644
--- a/src/master/hierarchical_allocator_process.hpp
+++ b/src/master/hierarchical_allocator_process.hpp
@@ -537,7 +537,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::resourcesRecovered(
 {
   CHECK(initialized);
 
-  if (resources.allocatable().size() == 0) {
+  if (resources.size() == 0) {
     return;
   }
 
@@ -560,7 +560,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::resourcesRecovered(
   if (slaves.contains(slaveId)) {
     slaves[slaveId].available += resources;
 
-    LOG(INFO) << "Recovered " << resources.allocatable()
+    LOG(INFO) << "Recovered " << resources
               << " (total allocatable: " << slaves[slaveId].available
               << ") on slave " << slaveId
               << " from framework " << frameworkId;

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/src/master/http.cpp
----------------------------------------------------------------------
diff --git a/src/master/http.cpp b/src/master/http.cpp
index 3189933..9157a41 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -426,12 +426,16 @@ Future<Response> Master::Http::stats(const Request& request)
 
   foreach (const Resource& resource, totalResources) {
     CHECK(resource.has_scalar());
+
     double total = resource.scalar().value();
     object.values[resource.name() + "_total"] = total;
-    Option<Resource> option = usedResources.get(resource);
-    CHECK(!option.isSome() || option.get().has_scalar());
-    double used = option.isSome() ? option.get().scalar().value() : 0.0;
+
+    Option<Value::Scalar> _used =
+      usedResources.get<Value::Scalar>(resource.name());
+
+    double used = _used.isSome() ? _used.get().value() : 0.0;
     object.values[resource.name() + "_used"] = used;
+
     double percent = used / total;
     object.values[resource.name() + "_percent"] = percent;
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 9e2d768..4cd9a4d 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -1961,9 +1961,15 @@ struct ResourceUsageChecker : TaskInfoVisitor
       return Error("Task uses no resources");
     }
 
+    Option<Error> error = Resources::validate(task.resources());
+    if (error.isSome()) {
+      return Error("Task uses invalid resources: " + error.get().message);
+    }
+
+    // Ensure no empty resource is used.
     foreach (const Resource& resource, task.resources()) {
-      if (!Resources::isAllocatable(resource)) {
-        return Error("Task uses invalid resources: " + stringify(resource));
+      if (Resources::empty(resource)) {
+        return Error("Task uses empty resources: " + stringify(resource));
       }
     }
 
@@ -1972,7 +1978,8 @@ struct ResourceUsageChecker : TaskInfoVisitor
 
     if (task.has_executor()) {
       foreach (const Resource& resource, task.executor().resources()) {
-        if (!Resources::isAllocatable(resource)) {
+        Option<Error> error = Resources::validate(resource);
+        if (error.isSome() || Resources::empty(resource)) {
           // TODO(benh): Send back the invalid resources?
           return Error(
               "Executor for task " + stringify(task.task_id()) +
@@ -2664,13 +2671,10 @@ void Master::_launchTasks(
     }
   }
 
-  // All used resources should be allocatable, enforced by our validators.
-  CHECK_EQ(usedResources, usedResources.allocatable());
-
   // Calculate unused resources.
   Resources unusedResources = totalResources - usedResources;
 
-  if (unusedResources.allocatable().size() > 0) {
+  if (unusedResources.size() > 0) {
     // Tell the allocator about the unused (e.g., refused) resources.
     allocator->resourcesRecovered(
         frameworkId, slaveId, unusedResources, filters);

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/src/slave/containerizer/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.cpp b/src/slave/containerizer/containerizer.cpp
index f234835..1448bea 100644
--- a/src/slave/containerizer/containerizer.cpp
+++ b/src/slave/containerizer/containerizer.cpp
@@ -26,6 +26,7 @@
 #include <stout/hashmap.hpp>
 #include <stout/net.hpp>
 #include <stout/stringify.hpp>
+#include <stout/strings.hpp>
 #include <stout/uuid.hpp>
 
 #include "slave/flags.hpp"
@@ -67,8 +68,13 @@ Try<Resources> Containerizer::resources(const Flags& flags)
 
   Resources resources = parsed.get();
 
-  // CPU resource.
-  if (!resources.cpus().isSome()) {
+  // NOTE: We need to check for the "cpus" string within the flag
+  // because once Resources are parsed, we cannot distinguish between
+  //  (1) "cpus:0", and
+  //  (2) no cpus specified.
+  // We only auto-detect cpus in case (2).
+  // The same logic applies for the other resources!
+  if (!strings::contains(flags.resources.get(""), "cpus")) {
     // No CPU specified so probe OS or resort to DEFAULT_CPUS.
     double cpus;
     Try<long> cpus_ = os::cpus();
@@ -88,7 +94,7 @@ Try<Resources> Containerizer::resources(const Flags& flags)
   }
 
   // Memory resource.
-  if (!resources.mem().isSome()) {
+  if (!strings::contains(flags.resources.get(""), "mem")) {
     // No memory specified so probe OS or resort to DEFAULT_MEM.
     Bytes mem;
     Try<os::Memory> mem_ = os::memory();
@@ -113,7 +119,7 @@ Try<Resources> Containerizer::resources(const Flags& flags)
   }
 
   // Disk resource.
-  if (!resources.disk().isSome()) {
+  if (!strings::contains(flags.resources.get(""), "disk")) {
     // No disk specified so probe OS or resort to DEFAULT_DISK.
     Bytes disk;
 
@@ -141,7 +147,7 @@ Try<Resources> Containerizer::resources(const Flags& flags)
   }
 
   // Network resource.
-  if (!resources.ports().isSome()) {
+  if (!strings::contains(flags.resources.get(""), "ports")) {
     // No ports specified so resort to DEFAULT_PORTS.
     resources += Resources::parse(
         "ports",

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/src/tests/gc_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/gc_tests.cpp b/src/tests/gc_tests.cpp
index 8618ae1..cb5dc5f 100644
--- a/src/tests/gc_tests.cpp
+++ b/src/tests/gc_tests.cpp
@@ -276,8 +276,8 @@ TEST_F(GarbageCollectorIntegrationTest, Restart)
     .Times(1);
 
   Resources resources = Resources::parse(flags.resources.get()).get();
-  double cpus = resources.get("cpus", Value::Scalar()).value();
-  double mem = resources.get("mem", Value::Scalar()).value();
+  double cpus = resources.get<Value::Scalar>("cpus").get().value();
+  double mem = resources.get<Value::Scalar>("mem").get().value();
 
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, cpus, mem, "*"))
@@ -383,8 +383,8 @@ TEST_F(GarbageCollectorIntegrationTest, ExitedFramework)
     .WillOnce(SaveArg<1>(&frameworkId));
 
   Resources resources = Resources::parse(flags.resources.get()).get();
-  double cpus = resources.get("cpus", Value::Scalar()).value();
-  double mem = resources.get("mem", Value::Scalar()).value();
+  double cpus = resources.get<Value::Scalar>("cpus").get().value();
+  double mem = resources.get<Value::Scalar>("mem").get().value();
 
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, cpus, mem, "*"))
@@ -498,8 +498,8 @@ TEST_F(GarbageCollectorIntegrationTest, ExitedExecutor)
     .WillOnce(FutureArg<1>(&frameworkId));
 
   Resources resources = Resources::parse(flags.resources.get()).get();
-  double cpus = resources.get("cpus", Value::Scalar()).value();
-  double mem = resources.get("mem", Value::Scalar()).value();
+  double cpus = resources.get<Value::Scalar>("cpus").get().value();
+  double mem = resources.get<Value::Scalar>("mem").get().value();
 
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, cpus, mem, "*"))
@@ -602,8 +602,8 @@ TEST_F(GarbageCollectorIntegrationTest, DiskUsage)
     .WillOnce(FutureArg<1>(&frameworkId));
 
   Resources resources = Resources::parse(flags.resources.get()).get();
-  double cpus = resources.get("cpus", Value::Scalar()).value();
-  double mem = resources.get("mem", Value::Scalar()).value();
+  double cpus = resources.get<Value::Scalar>("cpus").get().value();
+  double mem = resources.get<Value::Scalar>("mem").get().value();
 
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillOnce(LaunchTasks(DEFAULT_EXECUTOR_INFO, 1, cpus, mem, "*"))
@@ -730,8 +730,8 @@ TEST_F(GarbageCollectorIntegrationTest, Unschedule)
     .WillOnce(FutureArg<1>(&frameworkId));
 
   Resources resources = Resources::parse(flags.resources.get()).get();
-  double cpus = resources.get("cpus", Value::Scalar()).value();
-  double mem = resources.get("mem", Value::Scalar()).value();
+  double cpus = resources.get<Value::Scalar>("cpus").get().value();
+  double mem = resources.get<Value::Scalar>("mem").get().value();
 
   EXPECT_CALL(sched, resourceOffers(_, _))
     .WillOnce(LaunchTasks(executor1, 1, cpus, mem, "*"));

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index c1d64a7..0a5e8ec 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -413,8 +413,11 @@ ACTION_P5(LaunchTasks, executor, tasks, cpus, mem, role)
       task.mutable_slave_id()->MergeFrom(offer.slave_id());
       task.mutable_executor()->MergeFrom(executor);
 
-      Option<Resources> resources = remaining.find(TASK_RESOURCES, role);
+      Option<Resources> resources =
+        remaining.find(TASK_RESOURCES.flatten(role));
+
       CHECK_SOME(resources);
+
       task.mutable_resources()->MergeFrom(resources.get());
       remaining -= resources.get();
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/src/tests/resource_offers_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resource_offers_tests.cpp b/src/tests/resource_offers_tests.cpp
index 43820b0..7e09319 100644
--- a/src/tests/resource_offers_tests.cpp
+++ b/src/tests/resource_offers_tests.cpp
@@ -200,7 +200,7 @@ TEST_F(TaskValidationTest, TaskUsesNoResources)
 }
 
 
-TEST_F(TaskValidationTest, TaskUsesInvalidResources)
+TEST_F(TaskValidationTest, TaskUsesEmptyResources)
 {
   Try<PID<Master>> master = StartMaster();
   ASSERT_SOME(master);
@@ -250,7 +250,7 @@ TEST_F(TaskValidationTest, TaskUsesInvalidResources)
   EXPECT_EQ(TASK_ERROR, status.get().state());
   EXPECT_EQ(TaskStatus::REASON_TASK_INVALID, status.get().reason());
   EXPECT_TRUE(status.get().has_message());
-  EXPECT_EQ("Task uses invalid resources: cpus(*):0", status.get().message());
+  EXPECT_EQ("Task uses empty resources: cpus(*):0", status.get().message());
 
   driver.stop();
   driver.join();
@@ -653,8 +653,8 @@ TEST_F(ResourceOffersTest, ResourceOfferWithMultipleSlaves)
   EXPECT_GE(10u, offers.get().size());
 
   Resources resources(offers.get()[0].resources());
-  EXPECT_EQ(2, resources.get("cpus", Value::Scalar()).value());
-  EXPECT_EQ(1024, resources.get("mem", Value::Scalar()).value());
+  EXPECT_EQ(2, resources.get<Value::Scalar>("cpus").get().value());
+  EXPECT_EQ(1024, resources.get<Value::Scalar>("mem").get().value());
 
   driver.stop();
   driver.join();
@@ -818,7 +818,7 @@ TEST_F(ResourceOffersTest, ResourcesGetReofferedAfterTaskInfoError)
   EXPECT_EQ(TASK_ERROR, status.get().state());
   EXPECT_EQ(TaskStatus::REASON_TASK_INVALID, status.get().reason());
   EXPECT_TRUE(status.get().has_message());
-  EXPECT_EQ("Task uses invalid resources: cpus(*):0", status.get().message());
+  EXPECT_EQ("Task uses empty resources: cpus(*):0", status.get().message());
 
   MockScheduler sched2;
   MesosSchedulerDriver driver2(

http://git-wip-us.apache.org/repos/asf/mesos/blob/a8d0244b/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index 8d1e86a..9387a29 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -192,14 +192,14 @@ TEST(ResourcesTest, BadResourcesNotAllocatable)
   Resources r;
   r += cpus;
 
-  EXPECT_EQ(0u, r.allocatable().size());
+  EXPECT_EQ(0u, r.size());
 
   cpus.set_name("cpus");
   cpus.mutable_scalar()->set_value(0);
 
   r += cpus;
 
-  EXPECT_EQ(0u, r.allocatable().size());
+  EXPECT_EQ(0u, r.size());
 }
 
 
@@ -293,15 +293,15 @@ TEST(ResourcesTest, ScalarAddition)
   Resources sum = r1 + r2;
 
   EXPECT_EQ(2u, sum.size());
-  EXPECT_EQ(3, sum.get("cpus", Value::Scalar()).value());
-  EXPECT_EQ(15, sum.get("mem", Value::Scalar()).value());
+  EXPECT_EQ(3, sum.get<Value::Scalar>("cpus").get().value());
+  EXPECT_EQ(15, sum.get<Value::Scalar>("mem").get().value());
 
   Resources r = r1;
   r += r2;
 
   EXPECT_EQ(2u, r.size());
-  EXPECT_EQ(3, r.get("cpus", Value::Scalar()).value());
-  EXPECT_EQ(15, r.get("mem", Value::Scalar()).value());
+  EXPECT_EQ(3, r.get<Value::Scalar>("cpus").get().value());
+  EXPECT_EQ(15, r.get<Value::Scalar>("mem").get().value());
 }
 
 
@@ -345,18 +345,19 @@ TEST(ResourcesTest, ScalarSubtraction)
   Resources diff = r1 - r2;
 
   EXPECT_EQ(2u, diff.size());
-  EXPECT_EQ(49.5, diff.get("cpus", Value::Scalar()).value());
-  EXPECT_EQ(3072, diff.get("mem", Value::Scalar()).value());
+  EXPECT_EQ(49.5, diff.get<Value::Scalar>("cpus").get().value());
+  EXPECT_EQ(3072, diff.get<Value::Scalar>("mem").get().value());
 
   Resources r = r1;
   r -= r2;
 
-  EXPECT_EQ(49.5, diff.get("cpus", Value::Scalar()).value());
-  EXPECT_EQ(3072, diff.get("mem", Value::Scalar()).value());
+  EXPECT_EQ(49.5, diff.get<Value::Scalar>("cpus").get().value());
+  EXPECT_EQ(3072, diff.get<Value::Scalar>("mem").get().value());
 
   r = r1;
   r -= r1;
-  EXPECT_EQ(0u, r.allocatable().size());
+
+  EXPECT_EQ(0u, r.size());
 }
 
 
@@ -444,7 +445,7 @@ TEST(ResourcesTest, RangesSubset)
 TEST(ResourcesTest, RangesAddition)
 {
   Resource ports1 = Resources::parse(
-      "ports", "[20000-40000, 21000-38000]", "*").get();
+      "ports", "[20000-40000]", "*").get();
 
   Resource ports2 = Resources::parse(
       "ports", "[30000-50000, 10000-20000]", "*").get();
@@ -455,9 +456,9 @@ TEST(ResourcesTest, RangesAddition)
 
   EXPECT_EQ(1u, r.size());
 
-  const Value::Ranges& ranges = r.get("ports", Value::Ranges());
-
-  EXPECT_EQ(values::parse("[10000-50000]").get().ranges(), ranges);
+  EXPECT_SOME_EQ(
+      values::parse("[10000-50000]").get().ranges(),
+      r.get<Value::Ranges>("ports"));
 }
 
 
@@ -472,9 +473,9 @@ TEST(ResourcesTest, RangesAddition2)
 
   EXPECT_EQ(1u, r.size());
 
-  const Value::Ranges& ranges = r.get("ports", Value::Ranges());
-
-  EXPECT_EQ(values::parse("[1-65, 70-80]").get().ranges(), ranges);
+  EXPECT_SOME_EQ(
+      values::parse("[1-65, 70-80]").get().ranges(),
+      r.get<Value::Ranges>("ports"));
 }
 
 
@@ -491,9 +492,9 @@ TEST(ResourcesTest, RangesAdditon3)
 
   EXPECT_EQ(1u, r1.size());
 
-  const Value::Ranges& ranges = r1.get("ports", Value::Ranges());
-
-  EXPECT_EQ(values::parse("[1-4]").get().ranges(), ranges);
+  EXPECT_SOME_EQ(
+      values::parse("[1-4]").get().ranges(),
+      r1.get<Value::Ranges>("ports"));
 
   Resources r2;
   r2 += ports3;
@@ -501,17 +502,17 @@ TEST(ResourcesTest, RangesAdditon3)
 
   EXPECT_EQ(1u, r2.size());
 
-  const Value::Ranges& ranges2 = r2.get("ports", Value::Ranges());
-
-  EXPECT_EQ(values::parse("[5-8]").get().ranges(), ranges2);
+  EXPECT_SOME_EQ(
+      values::parse("[5-8]").get().ranges(),
+      r2.get<Value::Ranges>("ports"));
 
   r2 += r1;
 
   EXPECT_EQ(1u, r2.size());
 
-  const Value::Ranges& ranges3 = r2.get("ports", Value::Ranges());
-
-  EXPECT_EQ(values::parse("[1-8]").get().ranges(), ranges3);
+  EXPECT_SOME_EQ(
+      values::parse("[1-8]").get().ranges(),
+      r2.get<Value::Ranges>("ports"));
 }
 
 
@@ -529,9 +530,9 @@ TEST(ResourcesTest, RangesAddition4)
 
   EXPECT_EQ(1u, r.size());
 
-  const Value::Ranges& ranges = r.get("ports", Value::Ranges());
-
-  EXPECT_EQ(values::parse("[1-10, 20-30]").get().ranges(), ranges);
+  EXPECT_SOME_EQ(
+      values::parse("[1-10, 20-30]").get().ranges(),
+      r.get<Value::Ranges>("ports"));
 }
 
 
@@ -549,9 +550,9 @@ TEST(ResourcesTest, RangesSubtraction)
 
   EXPECT_EQ(1u, r.size());
 
-  const Value::Ranges& ranges = r.get("ports", Value::Ranges());
-
-  EXPECT_EQ(values::parse("[20001-29999]").get().ranges(), ranges);
+  EXPECT_SOME_EQ(
+      values::parse("[20001-29999]").get().ranges(),
+      r.get<Value::Ranges>("ports"));
 }
 
 
@@ -566,9 +567,9 @@ TEST(ResourcesTest, RangesSubtraction1)
 
   EXPECT_EQ(1u, r.size());
 
-  const Value::Ranges& ranges = r.get("ports", Value::Ranges());
-
-  EXPECT_EQ(values::parse("[50002-60000]").get().ranges(), ranges);
+  EXPECT_SOME_EQ(
+      values::parse("[50002-60000]").get().ranges(),
+      r.get<Value::Ranges>("ports"));
 }
 
 
@@ -583,9 +584,9 @@ TEST(ResourcesTest, RangesSubtraction2)
 
   EXPECT_EQ(1u, r.size());
 
-  const Value::Ranges& ranges = r.get("ports", Value::Ranges());
-
-  EXPECT_EQ(values::parse("[50001-60000]").get().ranges(), ranges);
+  EXPECT_SOME_EQ(
+      values::parse("[50001-60000]").get().ranges(),
+      r.get<Value::Ranges>("ports"));
 }
 
 
@@ -597,13 +598,12 @@ TEST(ResourcesTest, RangesSubtraction3)
   Resources resourcesInUse = Resources::parse("ports:[50000-50001]").get();
 
   Resources resourcesFree = resources - (resourcesOffered + resourcesInUse);
-  resourcesFree = resourcesFree.allocatable();
 
   EXPECT_EQ(1u, resourcesFree.size());
 
-  const Value::Ranges& ranges = resourcesFree.get("ports", Value::Ranges());
-
-  EXPECT_EQ(values::parse("[50002-60000]").get().ranges(), ranges);
+  EXPECT_SOME_EQ(
+      values::parse("[50002-60000]").get().ranges(),
+      resourcesFree.get<Value::Ranges>("ports"));
 }
 
 
@@ -616,10 +616,7 @@ TEST(ResourcesTest, RangesSubtraction4)
   resourcesOffered -= resources;
 
   EXPECT_EQ(0u, resourcesOffered.size());
-
-  const Value::Ranges& ranges = resourcesOffered.get("ports", Value::Ranges());
-
-  EXPECT_EQ(0, ranges.range_size());
+  EXPECT_NONE(resourcesOffered.get<Value::Ranges>("ports"));
 }
 
 
@@ -637,9 +634,9 @@ TEST(ResourcesTest, RangesSubtraction5)
 
   EXPECT_EQ(1u, r.size());
 
-  const Value::Ranges& ranges = r.get("ports", Value::Ranges());
-
-  EXPECT_EQ(values::parse("[1-1, 10-10, 46-47]").get().ranges(), ranges);
+  EXPECT_SOME_EQ(
+      values::parse("[1-1, 10-10, 46-47]").get().ranges(),
+      r.get<Value::Ranges>("ports"));
 }
 
 
@@ -654,9 +651,9 @@ TEST(ResourcesTest, RangesSubtraction6)
 
   EXPECT_EQ(1u, r.size());
 
-  const Value::Ranges& ranges = r.get("ports", Value::Ranges());
-
-  EXPECT_EQ(values::parse("[1-10]").get().ranges(), ranges);
+  EXPECT_SOME_EQ(
+      values::parse("[1-10]").get().ranges(),
+      r.get<Value::Ranges>("ports"));
 }
 
 
@@ -709,9 +706,10 @@ TEST(ResourcesTest, SetAddition)
 
   EXPECT_EQ(1u, r.size());
 
-  const Value::Set& set = r.get("disks", Value::Set());
+  Option<Value::Set> set = r.get<Value::Set>("disks");
 
-  EXPECT_EQ(4, set.item_size());
+  ASSERT_SOME(set);
+  EXPECT_EQ(4, set.get().item_size());
 }
 
 
@@ -729,10 +727,11 @@ TEST(ResourcesTest, SetSubtraction)
 
   EXPECT_EQ(1u, r.size());
 
-  const Value::Set& set = r.get("disks", Value::Set());
+  Option<Value::Set> set = r.get<Value::Set>("disks");
 
-  EXPECT_EQ(1, set.item_size());
-  EXPECT_EQ("sda1", set.item(0));
+  ASSERT_SOME(set);
+  EXPECT_EQ(1, set.get().item_size());
+  EXPECT_EQ("sda1", set.get().item(0));
 }
 
 
@@ -765,43 +764,37 @@ TEST(ResourcesTest, Find)
   Resources resources1 = Resources::parse(
       "cpus(role1):2;mem(role1):10;cpus:4;mem:20").get();
 
-  Resources toFind1 = Resources::parse("cpus:3;mem:15").get();
+  Resources targets1 = Resources::parse(
+      "cpus(role1):3;mem(role1):15").get();
 
-  Resources found1 = resources1.find(toFind1, "role1").get();
-
-  Resources expected1 = Resources::parse(
-      "cpus(role1):2;mem(role1):10;cpus:1;mem:5").get();
-
-  EXPECT_EQ(found1, expected1);
+  EXPECT_SOME_EQ(
+      Resources::parse("cpus(role1):2;mem(role1):10;cpus:1;mem:5").get(),
+      resources1.find(targets1));
 
   Resources resources2 = Resources::parse(
       "cpus(role1):1;mem(role1):5;cpus(role2):2;"
       "mem(role2):8;cpus:1;mem:7").get();
 
-  Resources toFind2 = Resources::parse("cpus:3;mem:15").get();
-
-  Resources found2 = resources2.find(toFind2, "role1").get();
+  Resources targets2 = Resources::parse(
+      "cpus(role1):3;mem(role1):15").get();
 
-  Resources expected2 = Resources::parse(
-      "cpus(role1):1;mem(role1):5;cpus:1;mem:7;"
-      "cpus(role2):1;mem(role2):3").get();
-
-  EXPECT_EQ(found2, expected2);
+  EXPECT_SOME_EQ(
+      Resources::parse(
+        "cpus(role1):1;mem(role1):5;cpus:1;mem:7;"
+        "cpus(role2):1;mem(role2):3").get(),
+      resources2.find(targets2));
 
   Resources resources3 = Resources::parse(
       "cpus(role1):5;mem(role1):5;cpus:5;mem:5").get();
 
-  Resources toFind3 = Resources::parse("cpus:6;mem:6").get();
-
-  Resources found3 = resources3.find(toFind3).get();
-
-  Resources expected3 = Resources::parse(
-      "cpus:5;mem:5;cpus(role1):1;mem(role1):1").get();
+  Resources targets3 = Resources::parse("cpus:6;mem:6").get();
 
-  EXPECT_EQ(found3, expected3);
+  EXPECT_SOME_EQ(
+      Resources::parse("cpus:5;mem:5;cpus(role1):1;mem(role1):1").get(),
+      resources3.find(targets3));
 
   Resources resources4 = Resources::parse("cpus(role1):1;mem(role1):1").get();
-  Resources toFind4 = Resources::parse("cpus:2;mem:2").get();
+  Resources targets4 = Resources::parse("cpus(role1):2;mem(role1):2").get();
 
-  EXPECT_NONE(resources4.find(toFind1, "role1"));
+  EXPECT_NONE(resources4.find(targets4));
 }


[9/9] mesos git commit: Eliminated the copying in Resource addition and subtraction.

Posted by ji...@apache.org.
Eliminated the copying in Resource addition and subtraction.

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


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

Branch: refs/heads/master
Commit: 0966e6ec4a101702388c1331f7af5ce35f26e4da
Parents: 550f5fc
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Nov 17 17:33:14 2014 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Nov 19 00:38:48 2014 -0800

----------------------------------------------------------------------
 src/common/resources.cpp | 14 ++++++--------
 1 file changed, 6 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0966e6ec/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index 23fd6a3..61d16a8 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -115,13 +115,12 @@ bool operator != (const Resource& left, const Resource& right)
 
 Resource& operator += (Resource& left, const Resource& right)
 {
-  // TODO(jieyu): Leverage += for Value to avoid copying.
   if (left.type() == Value::SCALAR) {
-    left.mutable_scalar()->CopyFrom(left.scalar() + right.scalar());
+    *left.mutable_scalar() += right.scalar();
   } else if (left.type() == Value::RANGES) {
-    left.mutable_ranges()->CopyFrom(left.ranges() + right.ranges());
+    *left.mutable_ranges() += right.ranges();
   } else if (left.type() == Value::SET) {
-    left.mutable_set()->CopyFrom(left.set() + right.set());
+    *left.mutable_set() += right.set();
   }
 
   return left;
@@ -138,13 +137,12 @@ Resource operator + (const Resource& left, const Resource& right)
 
 Resource& operator -= (Resource& left, const Resource& right)
 {
-  // TODO(jieyu): Leverage -= for Value to avoid copying.
   if (left.type() == Value::SCALAR) {
-    left.mutable_scalar()->CopyFrom(left.scalar() - right.scalar());
+    *left.mutable_scalar() -= right.scalar();
   } else if (left.type() == Value::RANGES) {
-    left.mutable_ranges()->CopyFrom(left.ranges() - right.ranges());
+    *left.mutable_ranges() -= right.ranges();
   } else if (left.type() == Value::SET) {
-    left.mutable_set()->CopyFrom(left.set() - right.set());
+    *left.mutable_set() -= right.set();
   }
 
   return left;


[5/9] mesos git commit: Re-organized the functions in C++ Resources.

Posted by ji...@apache.org.
Re-organized the functions in C++ Resources.

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


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

Branch: refs/heads/master
Commit: bdf2d9d410aeda6733a3d1ccdcd61ae7e89dac4a
Parents: 7ff4920
Author: Jie Yu <yu...@gmail.com>
Authored: Fri Nov 14 10:14:24 2014 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Nov 19 00:14:25 2014 -0800

----------------------------------------------------------------------
 include/mesos/resources.hpp | 154 ++++-----
 src/common/resources.cpp    | 697 ++++++++++++++++++++-------------------
 2 files changed, 428 insertions(+), 423 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/bdf2d9d4/include/mesos/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/resources.hpp b/include/mesos/resources.hpp
index 0e37170..c2ba597 100644
--- a/include/mesos/resources.hpp
+++ b/include/mesos/resources.hpp
@@ -19,6 +19,7 @@
 #ifndef __RESOURCES_HPP__
 #define __RESOURCES_HPP__
 
+#include <iostream>
 #include <string>
 
 #include <mesos/mesos.hpp>
@@ -26,6 +27,7 @@
 
 #include <stout/bytes.hpp>
 #include <stout/option.hpp>
+#include <stout/try.hpp>
 
 
 /**
@@ -50,28 +52,66 @@
  * names is a no-op.
  */
 
-
 namespace mesos {
 
-
 bool operator == (const Resource& left, const Resource& right);
 bool operator != (const Resource& left, const Resource& right);
+
+
 bool operator <= (const Resource& left, const Resource& right);
-Resource operator + (const Resource& left, const Resource& right);
-Resource operator - (const Resource& left, const Resource& right);
+
+
 Resource& operator += (Resource& left, const Resource& right);
+Resource operator + (const Resource& left, const Resource& right);
 Resource& operator -= (Resource& left, const Resource& right);
+Resource operator - (const Resource& left, const Resource& right);
+
+
 // Return true iff both Resources have the same name, type, and role.
 bool matches(const Resource& left, const Resource& right);
 
-std::ostream& operator << (std::ostream& stream, const Resource& resource);
-
 
 // TODO(bmahler): Ensure that the underlying resources are kept
 // in a flattened state: MESOS-1714.
 class Resources
 {
 public:
+  /**
+   * Parses the value and returns a Resource with the given name and role.
+   */
+  static Try<Resource> parse(
+      const std::string& name,
+      const std::string& value,
+      const std::string& role);
+
+  /**
+   * Parses resources in the form "name:value (role);name:value...".
+   * Any name/value pair that doesn't specify a role is assigned to defaultRole.
+   */
+  static Try<Resources> parse(
+      const std::string& s,
+      const std::string& defaultRole = "*");
+
+  /**
+   * Returns true iff this resource has a name, a valid type, i.e. scalar,
+   * range, or set, and has the appropriate value set for its type.
+   */
+  static bool isValid(const Resource& resource);
+
+  /**
+   * Returns true iff this resource is valid and allocatable. In particular,
+   * a scalar is allocatable if it's value is greater than zero, a ranges
+   * is allocatable if there is at least one valid range in it, and a set
+   * is allocatable if it has at least one item.
+   */
+  static bool isAllocatable(const Resource& resource);
+
+  /**
+   * Returns true iff this resource is zero valued, i.e. is zero for scalars,
+   * has a range size of zero for ranges, and has no items for sets.
+   */
+  static bool isZero(const Resource& resource);
+
   Resources() {}
 
   /*implicit*/
@@ -95,57 +135,16 @@ public:
     return *this;
   }
 
-  /**
-   * Returns a Resources object with only the allocatable resources.
-   */
-  Resources allocatable() const;
-
   size_t size() const
   {
     return resources.size();
   }
 
   /**
-   * Using this operator makes it easy to copy a resources object into
-   * a protocol buffer field.
+   * Returns all resources in this object that are marked with the
+   * specified role.
    */
-  operator const google::protobuf::RepeatedPtrField<Resource>& () const
-  {
-    return resources;
-  }
-
-  bool operator == (const Resources& that) const;
-
-  bool operator != (const Resources& that) const
-  {
-    return !(*this == that);
-  }
-
-  bool operator <= (const Resources& that) const;
-
-  Resources operator + (const Resources& that) const;
-
-  Resources operator - (const Resources& that) const;
-
-  Resources& operator += (const Resources& that);
-
-  Resources& operator -= (const Resources& that);
-
-  Resources operator + (const Resource& that) const;
-
-  Resources operator - (const Resource& that) const;
-
-  Resources& operator += (const Resource& that)
-  {
-    *this = *this + that;
-    return *this;
-  }
-
-  Resources& operator -= (const Resource& that)
-  {
-    *this = *this - that;
-    return *this;
-  }
+  Resources extract(const std::string& role) const;
 
   /**
    * Returns a Resources object with the same amount of each resource
@@ -155,12 +154,6 @@ public:
   Resources flatten(const std::string& role = "*") const;
 
   /**
-   * Returns all resources in this object that are marked with the
-   * specified role.
-   */
-  Resources extract(const std::string& role) const;
-
-  /**
    * Finds a number of resources equal to toFind in these Resources
    * and returns them marked with appropriate roles. For each resource
    * type, resources are first taken from the specified role, then
@@ -185,6 +178,11 @@ public:
   template <typename T>
   T get(const std::string& name, const T& t) const;
 
+  /**
+   * Returns a Resources object with only the allocatable resources.
+   */
+  Resources allocatable() const;
+
   // Helpers to get known resource types.
   // TODO(vinod): Fix this when we make these types as first class protobufs.
   Option<double> cpus() const;
@@ -215,46 +213,32 @@ public:
   const_iterator end() const { return resources.end(); }
 
   /**
-   * Parses the value and returns a Resource with the given name and role.
+   * Using this operator makes it easy to copy a resources object into
+   * a protocol buffer field.
    */
-  static Try<Resource> parse(
-      const std::string& name,
-      const std::string& value,
-      const std::string& role);
+  operator const google::protobuf::RepeatedPtrField<Resource>& () const;
 
-  /**
-   * Parses resources in the form "name:value (role);name:value...".
-   * Any name/value pair that doesn't specify a role is assigned to defaultRole.
-   */
-  static Try<Resources> parse(
-      const std::string& s,
-      const std::string& defaultRole = "*");
+  bool operator == (const Resources& that) const;
+  bool operator != (const Resources& that) const;
 
-  /**
-   * Returns true iff this resource has a name, a valid type, i.e. scalar,
-   * range, or set, and has the appropriate value set for its type.
-   */
-  static bool isValid(const Resource& resource);
+  bool operator <= (const Resources& that) const;
 
-  /**
-   * Returns true iff this resource is valid and allocatable. In particular,
-   * a scalar is allocatable if it's value is greater than zero, a ranges
-   * is allocatable if there is at least one valid range in it, and a set
-   * is allocatable if it has at least one item.
-   */
-  static bool isAllocatable(const Resource& resource);
+  Resources operator + (const Resource& that) const;
+  Resources operator + (const Resources& that) const;
+  Resources& operator += (const Resource& that);
+  Resources& operator += (const Resources& that);
 
-  /**
-   * Returns true iff this resource is zero valued, i.e. is zero for scalars,
-   * has a range size of zero for ranges, and has no items for sets.
-   */
-  static bool isZero(const Resource& resource);
+  Resources operator - (const Resource& that) const;
+  Resources operator - (const Resources& that) const;
+  Resources& operator -= (const Resource& that);
+  Resources& operator -= (const Resources& that);
 
 private:
   google::protobuf::RepeatedPtrField<Resource> resources;
 };
 
 
+std::ostream& operator << (std::ostream& stream, const Resource& resource);
 std::ostream& operator << (std::ostream& stream, const Resources& resources);
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/bdf2d9d4/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index e9a0c85..1ffebbb 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -18,7 +18,6 @@
 
 #include <stdint.h>
 
-#include <iostream>
 #include <vector>
 
 #include <glog/logging.h>
@@ -28,7 +27,6 @@
 
 #include <stout/foreach.hpp>
 #include <stout/strings.hpp>
-#include <stout/try.hpp>
 
 using std::ostream;
 using std::string;
@@ -59,14 +57,6 @@ bool operator != (const Resource& left, const Resource& right)
 }
 
 
-bool matches(const Resource& left, const Resource& right)
-{
-  return left.name() == right.name() &&
-    left.type() == right.type() &&
-    left.role() == right.role();
-}
-
-
 bool operator <= (const Resource& left, const Resource& right)
 {
   if (matches(left, right)) {
@@ -83,39 +73,37 @@ bool operator <= (const Resource& left, const Resource& right)
 }
 
 
-Resource operator + (const Resource& left, const Resource& right)
+Resource& operator += (Resource& left, const Resource& right)
 {
-  Resource result = left;
-
   if (matches(left, right)) {
     if (left.type() == Value::SCALAR) {
-      result.mutable_scalar()->MergeFrom(left.scalar() + right.scalar());
+      left.mutable_scalar()->MergeFrom(left.scalar() + right.scalar());
     } else if (left.type() == Value::RANGES) {
-      result.mutable_ranges()->Clear();
-      result.mutable_ranges()->MergeFrom(left.ranges() + right.ranges());
+      left.mutable_ranges()->Clear();
+      left.mutable_ranges()->MergeFrom(left.ranges() + right.ranges());
     } else if (left.type() == Value::SET) {
-      result.mutable_set()->Clear();
-      result.mutable_set()->MergeFrom(left.set() + right.set());
+      left.mutable_set()->Clear();
+      left.mutable_set()->MergeFrom(left.set() + right.set());
     }
   }
 
-  return result;
+  return left;
 }
 
 
-Resource operator - (const Resource& left, const Resource& right)
+Resource operator + (const Resource& left, const Resource& right)
 {
   Resource result = left;
 
   if (matches(left, right)) {
     if (left.type() == Value::SCALAR) {
-      result.mutable_scalar()->MergeFrom(left.scalar() - right.scalar());
+      result.mutable_scalar()->MergeFrom(left.scalar() + right.scalar());
     } else if (left.type() == Value::RANGES) {
       result.mutable_ranges()->Clear();
-      result.mutable_ranges()->MergeFrom(left.ranges() - right.ranges());
+      result.mutable_ranges()->MergeFrom(left.ranges() + right.ranges());
     } else if (left.type() == Value::SET) {
       result.mutable_set()->Clear();
-      result.mutable_set()->MergeFrom(left.set() - right.set());
+      result.mutable_set()->MergeFrom(left.set() + right.set());
     }
   }
 
@@ -123,17 +111,17 @@ Resource operator - (const Resource& left, const Resource& right)
 }
 
 
-Resource& operator += (Resource& left, const Resource& right)
+Resource& operator -= (Resource& left, const Resource& right)
 {
   if (matches(left, right)) {
     if (left.type() == Value::SCALAR) {
-      left.mutable_scalar()->MergeFrom(left.scalar() + right.scalar());
+      left.mutable_scalar()->MergeFrom(left.scalar() - right.scalar());
     } else if (left.type() == Value::RANGES) {
       left.mutable_ranges()->Clear();
-      left.mutable_ranges()->MergeFrom(left.ranges() + right.ranges());
+      left.mutable_ranges()->MergeFrom(left.ranges() - right.ranges());
     } else if (left.type() == Value::SET) {
       left.mutable_set()->Clear();
-      left.mutable_set()->MergeFrom(left.set() + right.set());
+      left.mutable_set()->MergeFrom(left.set() - right.set());
     }
   }
 
@@ -141,188 +129,223 @@ Resource& operator += (Resource& left, const Resource& right)
 }
 
 
-Resource& operator -= (Resource& left, const Resource& right)
+Resource operator - (const Resource& left, const Resource& right)
 {
+  Resource result = left;
+
   if (matches(left, right)) {
     if (left.type() == Value::SCALAR) {
-      left.mutable_scalar()->MergeFrom(left.scalar() - right.scalar());
+      result.mutable_scalar()->MergeFrom(left.scalar() - right.scalar());
     } else if (left.type() == Value::RANGES) {
-      left.mutable_ranges()->Clear();
-      left.mutable_ranges()->MergeFrom(left.ranges() - right.ranges());
+      result.mutable_ranges()->Clear();
+      result.mutable_ranges()->MergeFrom(left.ranges() - right.ranges());
     } else if (left.type() == Value::SET) {
-      left.mutable_set()->Clear();
-      left.mutable_set()->MergeFrom(left.set() - right.set());
+      result.mutable_set()->Clear();
+      result.mutable_set()->MergeFrom(left.set() - right.set());
     }
   }
 
-  return left;
+  return result;
 }
 
 
-ostream& operator << (ostream& stream, const Resource& resource)
+bool matches(const Resource& left, const Resource& right)
 {
-  stream << resource.name() << "(" << resource.role() << "):";
+  return left.name() == right.name() &&
+    left.type() == right.type() &&
+    left.role() == right.role();
+}
 
-  switch (resource.type()) {
-    case Value::SCALAR: stream << resource.scalar(); break;
-    case Value::RANGES: stream << resource.ranges(); break;
-    case Value::SET:    stream << resource.set();    break;
-    default:
-      LOG(FATAL) << "Unexpected Value type: " << resource.type();
-      break;
+
+Try<Resource> Resources::parse(
+    const string& name,
+    const string& text,
+    const string& role)
+{
+  Resource resource;
+  Try<Value> result = internal::values::parse(text);
+
+  if (result.isError()) {
+    return Error("Failed to parse resource " + name +
+                 " text " + text +
+                 " error " + result.error());
+  } else{
+    Value value = result.get();
+    resource.set_name(name);
+    resource.set_role(role);
+
+    if (value.type() == Value::RANGES) {
+      resource.set_type(Value::RANGES);
+      resource.mutable_ranges()->MergeFrom(value.ranges());
+    } else if (value.type() == Value::SET) {
+      resource.set_type(Value::SET);
+      resource.mutable_set()->MergeFrom(value.set());
+    } else if (value.type() == Value::SCALAR) {
+      resource.set_type(Value::SCALAR);
+      resource.mutable_scalar()->MergeFrom(value.scalar());
+    } else {
+      return Error("Bad type for resource " + name +
+                   " text " + text +
+                   " type " + Value::Type_Name(value.type()));
+    }
   }
 
-  return stream;
+  return resource;
 }
 
 
-Resources Resources::allocatable() const
+Try<Resources> Resources::parse(const string& s, const string& defaultRole)
 {
-  Resources result;
+  Resources resources;
 
-  foreach (const Resource& resource, resources) {
-    if (isAllocatable(resource)) {
-      result.resources.Add()->MergeFrom(resource);
+  vector<string> tokens = strings::tokenize(s, ";");
+
+  foreach (const string& token, tokens) {
+    vector<string> pair = strings::tokenize(token, ":");
+    if (pair.size() != 2) {
+      return Error("Bad value for resources, missing or extra ':' in " + token);
+    }
+
+    string name;
+    string role;
+    size_t openParen = pair[0].find("(");
+    if (openParen == string::npos) {
+      name = strings::trim(pair[0]);
+      role = defaultRole;
+    } else {
+      size_t closeParen = pair[0].find(")");
+      if (closeParen == string::npos || closeParen < openParen) {
+        return Error("Bad value for resources, mismatched parentheses in " +
+                     token);
+      }
+
+      name = strings::trim(pair[0].substr(0, openParen));
+      role = strings::trim(pair[0].substr(openParen + 1,
+                                          closeParen - openParen - 1));
+    }
+
+    Try<Resource> resource = Resources::parse(name, pair[1], role);
+    if (resource.isError()) {
+      return Error(resource.error());
     }
+    resources += resource.get();
   }
 
-  return result;
+  return resources;
 }
 
 
-bool Resources::operator == (const Resources& that) const
+bool Resources::isValid(const Resource& resource)
 {
-  if (size() != that.size()) {
+  if (!resource.has_name() ||
+      resource.name() == "" ||
+      !resource.has_type() ||
+      !Value::Type_IsValid(resource.type())) {
     return false;
   }
 
-  foreach (const Resource& resource, resources) {
-    Option<Resource> option = that.get(resource);
-    if (option.isNone()) {
-      return false;
-      } else {
-      if (!(resource == option.get())) {
-        return false;
-      }
-    }
+  if (resource.type() == Value::SCALAR) {
+    return resource.has_scalar();
+  } else if (resource.type() == Value::RANGES) {
+    return resource.has_ranges();
+  } else if (resource.type() == Value::SET) {
+    return resource.has_set();
+  } else if (resource.type() == Value::TEXT) {
+    // Resources doesn't support text.
+    return false;
   }
 
-    return true;
+  return false;
 }
 
 
-bool Resources::operator <= (const Resources& that) const
+bool Resources::isAllocatable(const Resource& resource)
 {
-  foreach (const Resource& resource, resources) {
-    Option<Resource> option = that.get(resource);
-    if (option.isNone()) {
-      if (!isZero(resource)) {
+  if (isValid(resource)) {
+    if (resource.type() == Value::SCALAR) {
+      if (resource.scalar().value() <= 0) {
         return false;
       }
-    } else {
-      if (!(resource <= option.get())) {
+    } else if (resource.type() == Value::RANGES) {
+      if (resource.ranges().range_size() == 0) {
+        return false;
+      } else {
+        for (int i = 0; i < resource.ranges().range_size(); i++) {
+          const Value::Range& range = resource.ranges().range(i);
+
+          // Ensure the range make sense (isn't inverted).
+          if (range.begin() > range.end()) {
+            return false;
+          }
+
+          // Ensure ranges don't overlap (but not necessarily coalesced).
+          for (int j = i + 1; j < resource.ranges().range_size(); j++) {
+            if (range.begin() <= resource.ranges().range(j).begin() &&
+                resource.ranges().range(j).begin() <= range.end()) {
+              return false;
+            }
+          }
+        }
+      }
+    } else if (resource.type() == Value::SET) {
+      if (resource.set().item_size() == 0) {
         return false;
+      } else {
+        for (int i = 0; i < resource.set().item_size(); i++) {
+          const string& item = resource.set().item(i);
+
+          // Ensure no duplicates.
+          for (int j = i + 1; j < resource.set().item_size(); j++) {
+            if (item == resource.set().item(j)) {
+              return false;
+            }
+          }
+        }
       }
     }
+
+    return true;
   }
 
-  return true;
+  return false;
 }
 
 
-Resources Resources::operator + (const Resources& that) const
+bool Resources::isZero(const Resource& resource)
 {
-  Resources result(*this);
-
-  foreach (const Resource& resource, that.resources) {
-    result += resource;
+  if (resource.type() == Value::SCALAR) {
+    return resource.scalar().value() == 0;
+  } else if (resource.type() == Value::RANGES) {
+    return resource.ranges().range_size() == 0;
+  } else if (resource.type() == Value::SET) {
+    return resource.set().item_size() == 0;
   }
 
-  return result;
+  return false;
 }
 
 
-Resources Resources::operator - (const Resources& that) const
+Resources Resources::extract(const string& role) const
 {
-  Resources result(*this);
+  Resources r;
 
-  foreach (const Resource& resource, that.resources) {
-    result -= resource;
+  foreach (const Resource& resource, resources) {
+    if (resource.role() == role) {
+      r += resource;
+    }
   }
 
-  return result;
+  return r;
 }
 
 
-Resources& Resources::operator += (const Resources& that)
+Resources Resources::flatten(const string& role) const
 {
-  foreach (const Resource& resource, that.resources) {
-    *this += resource;
-  }
-
-  return *this;
-}
+  Resources flattened;
 
-
-Resources& Resources::operator -= (const Resources& that)
-{
-  foreach (const Resource& resource, that.resources) {
-    *this -= resource;
-  }
-
-  return *this;
-}
-
-
-Resources Resources::operator + (const Resource& that) const
-{
-  Resources result;
-
-  bool added = false;
-
-  foreach (const Resource& resource, resources) {
-    if (matches(resource, that)) {
-      result.resources.Add()->MergeFrom(resource + that);
-      added = true;
-    } else {
-      result.resources.Add()->MergeFrom(resource);
-    }
-  }
-
-  if (!added) {
-    result.resources.Add()->MergeFrom(that);
-  }
-
-  return result;
-}
-
-
-Resources Resources::operator - (const Resource& that) const
-{
-  Resources result;
-
-  foreach (const Resource& resource, resources) {
-    if (matches(resource, that)) {
-      Resource r = resource - that;
-      if (!isZero(r)) {
-        result.resources.Add()->MergeFrom(r);
-      }
-    } else {
-      result.resources.Add()->MergeFrom(resource);
-    }
-  }
-
-  return result;
-}
-
-
-Resources Resources::flatten(const string& role) const
-{
-  Resources flattened;
-
-  foreach (const Resource& r, resources) {
-    Resource toRemove = r;
-    toRemove.set_role(role);
+  foreach (const Resource& r, resources) {
+    Resource toRemove = r;
+    toRemove.set_role(role);
 
     bool found = false;
     for (int i = 0; i < flattened.resources.size(); i++) {
@@ -344,20 +367,6 @@ Resources Resources::flatten(const string& role) const
 }
 
 
-Resources Resources::extract(const string& role) const
-{
-  Resources r;
-
-  foreach (const Resource& resource, resources) {
-    if (resource.role() == role) {
-      r += resource;
-    }
-  }
-
-  return r;
-}
-
-
 Option<Resources> Resources::find(
     const Resources& toFind,
     const string& role) const
@@ -439,6 +448,92 @@ Option<Resources> Resources::getAll(const Resource& r) const
 }
 
 
+template <>
+Value::Scalar Resources::get(
+    const string& name,
+    const Value::Scalar& scalar) const
+{
+  Value::Scalar total;
+  bool found = false;
+
+  foreach (const Resource& resource, resources) {
+    if (resource.name() == name &&
+        resource.type() == Value::SCALAR) {
+      total += resource.scalar();
+      found = true;
+    }
+  }
+
+  if (found) {
+    return total;
+  }
+
+  return scalar;
+}
+
+
+template <>
+Value::Ranges Resources::get(
+    const string& name,
+    const Value::Ranges& ranges) const
+{
+  Value::Ranges total;
+  bool found = false;
+
+  foreach (const Resource& resource, resources) {
+    if (resource.name() == name &&
+        resource.type() == Value::RANGES) {
+      total += resource.ranges();
+      found = true;
+    }
+  }
+
+  if (found) {
+    return total;
+  }
+
+  return ranges;
+}
+
+
+template <>
+Value::Set Resources::get(
+    const string& name,
+    const Value::Set& set) const
+{
+  Value::Set total;
+  bool found = false;
+
+  foreach (const Resource& resource, resources) {
+    if (resource.name() == name &&
+        resource.type() == Value::SET) {
+      total += resource.set();
+      found = true;
+    }
+  }
+
+  if (found) {
+    return total;
+  }
+
+  return set;
+}
+
+
+Resources Resources::allocatable() const
+{
+  Resources result;
+
+  foreach (const Resource& resource, resources) {
+    if (isAllocatable(resource)) {
+      result.resources.Add()->MergeFrom(resource);
+    }
+  }
+
+  return result;
+}
+
+
 Option<double> Resources::cpus() const
 {
   double total= 0;
@@ -573,249 +668,176 @@ Option<Value::Ranges> Resources::ephemeral_ports() const
 }
 
 
-Try<Resource> Resources::parse(
-    const string& name,
-    const string& text,
-    const string& role)
+Resources::operator const google::protobuf::RepeatedPtrField<Resource>& () const
 {
-  Resource resource;
-  Try<Value> result = internal::values::parse(text);
+  return resources;
+}
 
-  if (result.isError()) {
-    return Error("Failed to parse resource " + name +
-                 " text " + text +
-                 " error " + result.error());
-  } else{
-    Value value = result.get();
-    resource.set_name(name);
-    resource.set_role(role);
 
-    if (value.type() == Value::RANGES) {
-      resource.set_type(Value::RANGES);
-      resource.mutable_ranges()->MergeFrom(value.ranges());
-    } else if (value.type() == Value::SET) {
-      resource.set_type(Value::SET);
-      resource.mutable_set()->MergeFrom(value.set());
-    } else if (value.type() == Value::SCALAR) {
-      resource.set_type(Value::SCALAR);
-      resource.mutable_scalar()->MergeFrom(value.scalar());
-    } else {
-      return Error("Bad type for resource " + name +
-                   " text " + text +
-                   " type " + Value::Type_Name(value.type()));
+bool Resources::operator == (const Resources& that) const
+{
+  if (size() != that.size()) {
+    return false;
+  }
+
+  foreach (const Resource& resource, resources) {
+    Option<Resource> option = that.get(resource);
+    if (option.isNone()) {
+      return false;
+      } else {
+      if (!(resource == option.get())) {
+        return false;
+      }
     }
   }
 
-  return resource;
+    return true;
 }
 
 
-Try<Resources> Resources::parse(const string& s, const string& defaultRole)
+bool Resources::operator != (const Resources& that) const
 {
-  Resources resources;
+  return !(*this == that);
+}
 
-  vector<string> tokens = strings::tokenize(s, ";");
 
-  foreach (const string& token, tokens) {
-    vector<string> pair = strings::tokenize(token, ":");
-    if (pair.size() != 2) {
-      return Error("Bad value for resources, missing or extra ':' in " + token);
-    }
-
-    string name;
-    string role;
-    size_t openParen = pair[0].find("(");
-    if (openParen == string::npos) {
-      name = strings::trim(pair[0]);
-      role = defaultRole;
+bool Resources::operator <= (const Resources& that) const
+{
+  foreach (const Resource& resource, resources) {
+    Option<Resource> option = that.get(resource);
+    if (option.isNone()) {
+      if (!isZero(resource)) {
+        return false;
+      }
     } else {
-      size_t closeParen = pair[0].find(")");
-      if (closeParen == string::npos || closeParen < openParen) {
-        return Error("Bad value for resources, mismatched parentheses in " +
-                     token);
+      if (!(resource <= option.get())) {
+        return false;
       }
-
-      name = strings::trim(pair[0].substr(0, openParen));
-      role = strings::trim(pair[0].substr(openParen + 1,
-                                          closeParen - openParen - 1));
     }
-
-    Try<Resource> resource = Resources::parse(name, pair[1], role);
-    if (resource.isError()) {
-      return Error(resource.error());
-    }
-    resources += resource.get();
   }
 
-  return resources;
+  return true;
 }
 
 
-bool Resources::isValid(const Resource& resource)
+Resources Resources::operator + (const Resource& that) const
 {
-  if (!resource.has_name() ||
-      resource.name() == "" ||
-      !resource.has_type() ||
-      !Value::Type_IsValid(resource.type())) {
-    return false;
+  Resources result;
+
+  bool added = false;
+
+  foreach (const Resource& resource, resources) {
+    if (matches(resource, that)) {
+      result.resources.Add()->MergeFrom(resource + that);
+      added = true;
+    } else {
+      result.resources.Add()->MergeFrom(resource);
+    }
   }
 
-  if (resource.type() == Value::SCALAR) {
-    return resource.has_scalar();
-  } else if (resource.type() == Value::RANGES) {
-    return resource.has_ranges();
-  } else if (resource.type() == Value::SET) {
-    return resource.has_set();
-  } else if (resource.type() == Value::TEXT) {
-    // Resources doesn't support text.
-    return false;
+  if (!added) {
+    result.resources.Add()->MergeFrom(that);
   }
 
-  return false;
+  return result;
 }
 
 
-bool Resources::isAllocatable(const Resource& resource)
+Resources Resources::operator + (const Resources& that) const
 {
-  if (isValid(resource)) {
-    if (resource.type() == Value::SCALAR) {
-      if (resource.scalar().value() <= 0) {
-        return false;
-      }
-    } else if (resource.type() == Value::RANGES) {
-      if (resource.ranges().range_size() == 0) {
-        return false;
-      } else {
-        for (int i = 0; i < resource.ranges().range_size(); i++) {
-          const Value::Range& range = resource.ranges().range(i);
-
-          // Ensure the range make sense (isn't inverted).
-          if (range.begin() > range.end()) {
-            return false;
-          }
+  Resources result(*this);
 
-          // Ensure ranges don't overlap (but not necessarily coalesced).
-          for (int j = i + 1; j < resource.ranges().range_size(); j++) {
-            if (range.begin() <= resource.ranges().range(j).begin() &&
-                resource.ranges().range(j).begin() <= range.end()) {
-              return false;
-            }
-          }
-        }
-      }
-    } else if (resource.type() == Value::SET) {
-      if (resource.set().item_size() == 0) {
-        return false;
-      } else {
-        for (int i = 0; i < resource.set().item_size(); i++) {
-          const std::string& item = resource.set().item(i);
+  foreach (const Resource& resource, that.resources) {
+    result += resource;
+  }
 
-          // Ensure no duplicates.
-          for (int j = i + 1; j < resource.set().item_size(); j++) {
-            if (item == resource.set().item(j)) {
-              return false;
-            }
-          }
-        }
-      }
-    }
+  return result;
+}
 
-    return true;
-  }
 
-  return false;
+Resources& Resources::operator += (const Resource& that)
+{
+  *this = *this + that;
+  return *this;
 }
 
 
-bool Resources::isZero(const Resource& resource)
+Resources& Resources::operator += (const Resources& that)
 {
-  if (resource.type() == Value::SCALAR) {
-    return resource.scalar().value() == 0;
-  } else if (resource.type() == Value::RANGES) {
-    return resource.ranges().range_size() == 0;
-  } else if (resource.type() == Value::SET) {
-    return resource.set().item_size() == 0;
+  foreach (const Resource& resource, that.resources) {
+    *this += resource;
   }
 
-  return false;
+  return *this;
 }
 
 
-template <>
-Value::Scalar Resources::get(
-    const std::string& name,
-    const Value::Scalar& scalar) const
+Resources Resources::operator - (const Resource& that) const
 {
-  Value::Scalar total;
-  bool found = false;
+  Resources result;
 
   foreach (const Resource& resource, resources) {
-    if (resource.name() == name &&
-        resource.type() == Value::SCALAR) {
-      total += resource.scalar();
-      found = true;
+    if (matches(resource, that)) {
+      Resource r = resource - that;
+      if (!isZero(r)) {
+        result.resources.Add()->MergeFrom(r);
+      }
+    } else {
+      result.resources.Add()->MergeFrom(resource);
     }
   }
 
-  if (found) {
-    return total;
+  return result;
+}
+
+
+Resources Resources::operator - (const Resources& that) const
+{
+  Resources result(*this);
+
+  foreach (const Resource& resource, that.resources) {
+    result -= resource;
   }
 
-  return scalar;
+  return result;
 }
 
 
-template <>
-Value::Ranges Resources::get(
-    const std::string& name,
-    const Value::Ranges& ranges) const
+Resources& Resources::operator -= (const Resource& that)
 {
-  Value::Ranges total;
-  bool found = false;
+  *this = *this - that;
+  return *this;
+}
 
-  foreach (const Resource& resource, resources) {
-    if (resource.name() == name &&
-        resource.type() == Value::RANGES) {
-      total += resource.ranges();
-      found = true;
-    }
-  }
 
-  if (found) {
-    return total;
+Resources& Resources::operator -= (const Resources& that)
+{
+  foreach (const Resource& resource, that.resources) {
+    *this -= resource;
   }
 
-  return ranges;
+  return *this;
 }
 
 
-template <>
-Value::Set Resources::get(
-    const std::string& name,
-    const Value::Set& set) const
+ostream& operator << (ostream& stream, const Resource& resource)
 {
-  Value::Set total;
-  bool found = false;
-
-  foreach (const Resource& resource, resources) {
-    if (resource.name() == name &&
-        resource.type() == Value::SET) {
-      total += resource.set();
-      found = true;
-    }
-  }
+  stream << resource.name() << "(" << resource.role() << "):";
 
-  if (found) {
-    return total;
+  switch (resource.type()) {
+    case Value::SCALAR: stream << resource.scalar(); break;
+    case Value::RANGES: stream << resource.ranges(); break;
+    case Value::SET:    stream << resource.set();    break;
+    default:
+      LOG(FATAL) << "Unexpected Value type: " << resource.type();
+      break;
   }
 
-  return set;
+  return stream;
 }
 
 
-std::ostream& operator << (
-    std::ostream& stream,
-    const Resources& resources)
+ostream& operator << (ostream& stream, const Resources& resources)
 {
   mesos::Resources::const_iterator it = resources.begin();
 
@@ -829,5 +851,4 @@ std::ostream& operator << (
   return stream;
 }
 
-
 } // namespace mesos {


[8/9] mesos git commit: Fixed a bug in operator -= for Value::Set.

Posted by ji...@apache.org.
Fixed a bug in operator -= for Value::Set.

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


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

Branch: refs/heads/master
Commit: 550f5fcd9579edc1546bcd342b170d333b2b8a44
Parents: 5fbc626
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Nov 17 14:26:38 2014 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Nov 19 00:38:43 2014 -0800

----------------------------------------------------------------------
 src/common/values.cpp      |  9 ++-------
 src/tests/values_tests.cpp | 16 ++++++++++++++++
 2 files changed, 18 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/550f5fcd/src/common/values.cpp
----------------------------------------------------------------------
diff --git a/src/common/values.cpp b/src/common/values.cpp
index c524459..597c452 100644
--- a/src/common/values.cpp
+++ b/src/common/values.cpp
@@ -558,19 +558,14 @@ Value::Set& operator += (Value::Set& left, const Value::Set& right)
 
 Value::Set& operator -= (Value::Set& left, const Value::Set& right)
 {
-  // For each item in right check if it's in left and add it if not.
+  // For each item in right, remove it if it's in left.
   for (int i = 0; i < right.item_size(); i++) {
-    bool found = false;
     for (int j = 0; j < left.item_size(); j++) {
       if (right.item(i) == left.item(j)) {
-        found = true;
+        left.mutable_item()->DeleteSubrange(j, 1);
         break;
       }
     }
-
-    if (!found) {
-      left.add_item(right.item(i));
-    }
   }
 
   return left;

http://git-wip-us.apache.org/repos/asf/mesos/blob/550f5fcd/src/tests/values_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/values_tests.cpp b/src/tests/values_tests.cpp
index 976ba4d..35d3b3e 100644
--- a/src/tests/values_tests.cpp
+++ b/src/tests/values_tests.cpp
@@ -83,3 +83,19 @@ TEST(ValuesTest, InvalidInput)
   // Test when giving empty string.
   EXPECT_ERROR(parse("  "));
 }
+
+
+TEST(ValuesTest, SetSubtraction)
+{
+  Value::Set set1 = parse("{sda1, sda2, sda3}").get().set();
+  Value::Set set2 = parse("{sda2, sda3}").get().set();
+  Value::Set set3 = parse("{sda4}").get().set();
+
+  set1 -= set2;
+
+  EXPECT_EQ(set1, parse("{sda1}").get().set());
+
+  set3 -= set1;
+
+  EXPECT_EQ(set3, parse("{sda4}").get().set());
+}


[6/9] mesos git commit: Replaced <= with contains() in C++ Resources.

Posted by ji...@apache.org.
Replaced <= with contains() in C++ Resources.

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


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

Branch: refs/heads/master
Commit: 5fbc6264aab8370925793aecb29d6d1f54d37cba
Parents: 7fa0217
Author: Jie Yu <yu...@gmail.com>
Authored: Fri Nov 14 22:33:28 2014 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Nov 19 00:14:26 2014 -0800

----------------------------------------------------------------------
 include/mesos/resources.hpp                     |  5 ++-
 src/cli/execute.cpp                             |  3 +-
 src/common/resources.cpp                        | 36 ++++++++----------
 src/examples/low_level_scheduler_libprocess.cpp |  2 +-
 src/examples/low_level_scheduler_pthread.cpp    |  2 +-
 src/examples/no_executor_framework.cpp          |  2 +-
 src/examples/test_framework.cpp                 |  2 +-
 src/master/hierarchical_allocator_process.hpp   |  6 +--
 src/master/master.cpp                           |  2 +-
 src/tests/mesos.hpp                             |  3 +-
 src/tests/resources_tests.cpp                   | 40 ++++++++++----------
 src/tests/slave_recovery_tests.cpp              |  5 ++-
 12 files changed, 53 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/5fbc6264/include/mesos/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/resources.hpp b/include/mesos/resources.hpp
index 13a81e2..10777a6 100644
--- a/include/mesos/resources.hpp
+++ b/include/mesos/resources.hpp
@@ -104,6 +104,9 @@ public:
 
   bool empty() const { return resources.size() == 0; }
 
+  // Checks if this Resources is a superset of the given Resources.
+  bool contains(const Resources& that) const;
+
   // Returns all resources in this object that are marked with the
   // specified role.
   Resources extract(const std::string& role) const;
@@ -162,8 +165,6 @@ public:
   bool operator == (const Resources& that) const;
   bool operator != (const Resources& that) const;
 
-  bool operator <= (const Resources& that) const;
-
   // NOTE: If any error occurs (e.g., input Resource is not valid or
   // the first operand is not a superset of the second oprand while
   // doing subtraction), the semantics is as though the second operand

http://git-wip-us.apache.org/repos/asf/mesos/blob/5fbc6264/src/cli/execute.cpp
----------------------------------------------------------------------
diff --git a/src/cli/execute.cpp b/src/cli/execute.cpp
index ddaa20d..77deec9 100644
--- a/src/cli/execute.cpp
+++ b/src/cli/execute.cpp
@@ -170,7 +170,8 @@ public:
     }
 
     foreach (const Offer& offer, offers) {
-      if (!launched && TASK_RESOURCES.get() <= offer.resources()) {
+      if (!launched &&
+          Resources(offer.resources()).contains(TASK_RESOURCES.get())) {
         TaskInfo task;
         task.set_name(name);
         task.mutable_task_id()->set_value(name);

http://git-wip-us.apache.org/repos/asf/mesos/blob/5fbc6264/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index 3a3c6a6..23fd6a3 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -113,12 +113,6 @@ bool operator != (const Resource& left, const Resource& right)
 }
 
 
-bool operator <= (const Resource& left, const Resource& right)
-{
-  return contains(right, left);
-}
-
-
 Resource& operator += (Resource& left, const Resource& right)
 {
   // TODO(jieyu): Leverage += for Value to avoid copying.
@@ -372,6 +366,18 @@ Resources::Resources(
 }
 
 
+bool Resources::contains(const Resources& that) const
+{
+  foreach (const Resource& resource, that.resources) {
+    if (!contains(resource)) {
+      return false;
+    }
+  }
+
+  return true;
+}
+
+
 Resources Resources::extract(const string& role) const
 {
   Resources r;
@@ -436,10 +442,10 @@ Option<Resources> Resources::find(const Resource& target) const
       // Need to flatten to ignore the roles in contains().
       Resources flattened = Resources(resource).flatten();
 
-      if (remaining <= flattened) {
+      if (flattened.contains(remaining)) {
         // Done!
         return found + remaining.flatten(resource.role());
-      } else if (flattened <= remaining) {
+      } else if (remaining.contains(flattened)) {
         found += resource;
         total -= resource;
         remaining -= flattened;
@@ -617,7 +623,7 @@ Resources::operator const google::protobuf::RepeatedPtrField<Resource>& () const
 
 bool Resources::operator == (const Resources& that) const
 {
-  return *this <= that && that <= *this;
+  return this->contains(that) && that.contains(*this);
 }
 
 
@@ -627,18 +633,6 @@ bool Resources::operator != (const Resources& that) const
 }
 
 
-bool Resources::operator <= (const Resources& that) const
-{
-  foreach (const Resource& resource, resources) {
-    if (!that.contains(resource)) {
-      return false;
-    }
-  }
-
-  return true;
-}
-
-
 Resources Resources::operator + (const Resource& that) const
 {
   Resources result = *this;

http://git-wip-us.apache.org/repos/asf/mesos/blob/5fbc6264/src/examples/low_level_scheduler_libprocess.cpp
----------------------------------------------------------------------
diff --git a/src/examples/low_level_scheduler_libprocess.cpp b/src/examples/low_level_scheduler_libprocess.cpp
index 7229797..e5267f7 100644
--- a/src/examples/low_level_scheduler_libprocess.cpp
+++ b/src/examples/low_level_scheduler_libprocess.cpp
@@ -224,7 +224,7 @@ private:
       // Launch tasks.
       vector<TaskInfo> tasks;
       while (tasksLaunched < totalTasks &&
-             TASK_RESOURCES <= remaining.flatten()) {
+             remaining.flatten().contains(TASK_RESOURCES)) {
         int taskId = tasksLaunched++;
 
         cout << "Launching task " << taskId << " using offer "

http://git-wip-us.apache.org/repos/asf/mesos/blob/5fbc6264/src/examples/low_level_scheduler_pthread.cpp
----------------------------------------------------------------------
diff --git a/src/examples/low_level_scheduler_pthread.cpp b/src/examples/low_level_scheduler_pthread.cpp
index 2b012a8..4f9a594 100644
--- a/src/examples/low_level_scheduler_pthread.cpp
+++ b/src/examples/low_level_scheduler_pthread.cpp
@@ -275,7 +275,7 @@ private:
       // Launch tasks.
       vector<TaskInfo> tasks;
       while (tasksLaunched < totalTasks &&
-             TASK_RESOURCES <= remaining.flatten()) {
+             remaining.flatten().contains(TASK_RESOURCES)) {
         int taskId = tasksLaunched++;
 
         cout << "Launching task " << taskId << " using offer "

http://git-wip-us.apache.org/repos/asf/mesos/blob/5fbc6264/src/examples/no_executor_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/no_executor_framework.cpp b/src/examples/no_executor_framework.cpp
index 9c84e03..5e47cfc 100644
--- a/src/examples/no_executor_framework.cpp
+++ b/src/examples/no_executor_framework.cpp
@@ -85,7 +85,7 @@ public:
       // Launch tasks.
       vector<TaskInfo> tasks;
       while (tasksLaunched < totalTasks &&
-             TASK_RESOURCES <= remaining.flatten()) {
+             remaining.flatten().contains(TASK_RESOURCES)) {
         int taskId = tasksLaunched++;
 
         cout << "Launching task " << taskId << " using offer "

http://git-wip-us.apache.org/repos/asf/mesos/blob/5fbc6264/src/examples/test_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_framework.cpp b/src/examples/test_framework.cpp
index ce1616d..e5ec3b9 100644
--- a/src/examples/test_framework.cpp
+++ b/src/examples/test_framework.cpp
@@ -92,7 +92,7 @@ public:
       // Launch tasks.
       vector<TaskInfo> tasks;
       while (tasksLaunched < totalTasks &&
-             TASK_RESOURCES <= remaining.flatten()) {
+             remaining.flatten().contains(TASK_RESOURCES)) {
         int taskId = tasksLaunched++;
 
         cout << "Launching task " << taskId << " using offer "

http://git-wip-us.apache.org/repos/asf/mesos/blob/5fbc6264/src/master/hierarchical_allocator_process.hpp
----------------------------------------------------------------------
diff --git a/src/master/hierarchical_allocator_process.hpp b/src/master/hierarchical_allocator_process.hpp
index 45d45b8..e631b60 100644
--- a/src/master/hierarchical_allocator_process.hpp
+++ b/src/master/hierarchical_allocator_process.hpp
@@ -246,10 +246,10 @@ public:
       const process::Timeout& _timeout)
     : slaveId(_slaveId), resources(_resources), timeout(_timeout) {}
 
-  virtual bool filter(const SlaveID& slaveId, const Resources& resources)
+  virtual bool filter(const SlaveID& _slaveId, const Resources& _resources)
   {
-    return slaveId == this->slaveId &&
-           resources <= this->resources && // Refused resources are superset.
+    return slaveId == _slaveId &&
+           resources.contains(_resources) && // Refused resources are superset.
            timeout.remaining() > Seconds(0);
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5fbc6264/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 7ab25b3..de42f8e 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -2026,7 +2026,7 @@ struct ResourceUsageChecker : TaskInfoVisitor
       resources += task.executor().resources();
     }
 
-    if (!(resources + usedResources <= totalResources)) {
+    if (!totalResources.contains(resources + usedResources)) {
       return Error(
           "Task uses more resources " + stringify(resources) +
           " than available " + stringify(totalResources - usedResources));

http://git-wip-us.apache.org/repos/asf/mesos/blob/5fbc6264/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 0a5e8ec..f132c6c 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -406,7 +406,8 @@ ACTION_P5(LaunchTasks, executor, tasks, cpus, mem, role)
     std::vector<TaskInfo> tasks;
     Resources remaining = offer.resources();
 
-    while (TASK_RESOURCES <= remaining.flatten() && launched < numTasks) {
+    while (remaining.flatten().contains(TASK_RESOURCES) &&
+           launched < numTasks) {
       TaskInfo task;
       task.set_name("TestTask");
       task.mutable_task_id()->set_value(stringify(nextTaskId++));

http://git-wip-us.apache.org/repos/asf/mesos/blob/5fbc6264/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index c0486f4..73f50ba 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -243,8 +243,8 @@ TEST(ResourcesTest, ScalarSubset)
   r2 += cpus2;
   r2 += mem2;
 
-  EXPECT_TRUE(r1 <= r2);
-  EXPECT_FALSE(r2 <= r1);
+  EXPECT_TRUE(r2.contains(r1));
+  EXPECT_FALSE(r1.contains(r2));
 }
 
 
@@ -259,18 +259,18 @@ TEST(ResourcesTest, ScalarSubset2)
   Resources r2;
   r2 += cpus2;
 
-  EXPECT_FALSE(r1 <= r2);
-  EXPECT_FALSE(r2 <= r1);
+  EXPECT_FALSE(r2.contains(r1));
+  EXPECT_FALSE(r1.contains(r2));
 
   Resource cpus3 = Resources::parse("cpus", "3", "role1").get();
 
   Resources r3;
   r3 += cpus3;
 
-  EXPECT_FALSE(r3 <= r1);
-  EXPECT_FALSE(r3 <= r2);
-  EXPECT_FALSE(r2 <= r3);
-  EXPECT_LE(r1, r3);
+  EXPECT_FALSE(r1.contains(r3));
+  EXPECT_FALSE(r2.contains(r3));
+  EXPECT_FALSE(r3.contains(r2));
+  EXPECT_TRUE(r3.contains(r1));
 }
 
 
@@ -429,16 +429,16 @@ TEST(ResourcesTest, RangesSubset)
   Resources r5;
   r5 += ports5;
 
-  EXPECT_TRUE(r1 <= r2);
-  EXPECT_FALSE(r2 <= r1);
-  EXPECT_FALSE(r1 <= r3);
-  EXPECT_FALSE(r3 <= r1);
-  EXPECT_TRUE(r3 <= r2);
-  EXPECT_FALSE(r2 <= r3);
-  EXPECT_TRUE(r1 <= r4);
-  EXPECT_TRUE(r4 <= r2);
-  EXPECT_TRUE(r1 <= r5);
-  EXPECT_FALSE(r5 <= r1);
+  EXPECT_TRUE(r2.contains(r1));
+  EXPECT_FALSE(r1.contains(r2));
+  EXPECT_FALSE(r3.contains(r1));
+  EXPECT_FALSE(r1.contains(r3));
+  EXPECT_TRUE(r2.contains(r3));
+  EXPECT_FALSE(r3.contains(r2));
+  EXPECT_TRUE(r4.contains(r1));
+  EXPECT_TRUE(r2.contains(r4));
+  EXPECT_TRUE(r5.contains(r1));
+  EXPECT_FALSE(r1.contains(r5));
 }
 
 
@@ -687,8 +687,8 @@ TEST(ResourcesTest, SetSubset)
 
   EXPECT_FALSE(r1.empty());
   EXPECT_FALSE(r2.empty());
-  EXPECT_TRUE(r1 <= r2);
-  EXPECT_FALSE(r2 <= r1);
+  EXPECT_TRUE(r2.contains(r1));
+  EXPECT_FALSE(r1.contains(r2));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5fbc6264/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index 98e059f..782f57a 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -1061,8 +1061,9 @@ TYPED_TEST(SlaveRecoveryTest, RemoveNonCheckpointingFramework)
       Resources::parse("cpus:1;mem:512").get());
   tasks.push_back(createTask(offer2, "sleep 1000")); // Long-running task,
 
-  ASSERT_LE(Resources(offer1.resources()) + Resources(offer2.resources()),
-            Resources(offer.resources()));
+  ASSERT_TRUE(Resources(offer.resources()).contains(
+        Resources(offer1.resources()) +
+        Resources(offer2.resources())));
 
   Future<Nothing> update1;
   Future<Nothing> update2;


[2/9] mesos git commit: Killed ports allocation in C++ Resources.

Posted by ji...@apache.org.
Killed ports allocation in C++ Resources.

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


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

Branch: refs/heads/master
Commit: 85728f85afaf6db0fd2e55cd8d7b2135a4941310
Parents: a01773b
Author: Jie Yu <yu...@gmail.com>
Authored: Tue Nov 18 21:35:54 2014 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Nov 19 00:14:25 2014 -0800

----------------------------------------------------------------------
 include/mesos/resources.hpp   |  4 ----
 src/common/resources.cpp      | 30 ------------------------------
 src/tests/resources_tests.cpp | 28 ----------------------------
 3 files changed, 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/85728f85/include/mesos/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/resources.hpp b/include/mesos/resources.hpp
index c8a261f..86b70df 100644
--- a/include/mesos/resources.hpp
+++ b/include/mesos/resources.hpp
@@ -154,10 +154,6 @@ public:
   // TODO(vinod): Provide a Ranges abstraction.
   Option<Value::Ranges> ports() const;
 
-  // Helper function to extract the given number of ports from the
-  // "ports" resource.
-  Option<Value::Ranges> ports(size_t numPorts) const;
-
   // TODO(jieyu): Consider returning an EphemeralPorts abstraction
   // which holds the ephemeral ports allocation logic.
   Option<Value::Ranges> ephemeral_ports() const;

http://git-wip-us.apache.org/repos/asf/mesos/blob/85728f85/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index 458c1cd..8474cdd 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -637,36 +637,6 @@ Option<Value::Ranges> Resources::ports() const
   return None();
 }
 
-Option<Value::Ranges> Resources::ports(size_t numPorts) const
-{
-  Value::Ranges total;
-
-  foreach (const Resource& resource, resources) {
-    if (resource.name() == "ports" &&
-        resource.type() == Value::RANGES &&
-        isAllocatable(resource)) {
-      foreach (const Value::Range& range, resource.ranges().range()) {
-        size_t interval = range.end() - range.begin() + 1;
-        if (numPorts < interval) {
-          Value::Range* lastRange = total.add_range();
-          lastRange->set_begin(range.begin());
-          lastRange->set_end(range.begin() + numPorts - 1);
-
-          return total;
-        } else {
-          total.add_range()->CopyFrom(range);
-          numPorts -= interval;
-
-          if (numPorts == 0) {
-            return total;
-          }
-        }
-      }
-    }
-  }
-
-  return None();
-}
 
 Option<Value::Ranges> Resources::ephemeral_ports() const
 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/85728f85/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index c8d069a..8d1e86a 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -156,34 +156,6 @@ TEST(ResourcesTest, Resources)
 }
 
 
-TEST(ResourcesTest, Ports)
-{
-  // Extract one Value::Range.
-  Resources r = Resources::parse("ports:[10000-20000, 30000-50000]").get();
-  Option<Value::Ranges> ports = r.ports(5);
-  EXPECT_SOME(ports);
-  EXPECT_EQ("[10000-10004]", stringify(ports.get()));
-
-  // Extract two Value::Ranges.
-  r = Resources::parse("ports:[10000-10000, 20000-50000]").get();
-  ports = r.ports(5);
-  EXPECT_SOME(ports);
-  EXPECT_EQ("[10000-10000, 20000-20003]", stringify(ports.get()));
-
-  // Extract mutiple Value::Ranges.
-  r = Resources::parse("ports:[10000-10001, 10003-10004, 10007-10009,"
-                       "10020-20000]").get();
-  ports = r.ports(10);
-  EXPECT_SOME(ports);
-  EXPECT_EQ("[10000-10001, 10003-10004, 10007-10009, 10020-10022]",
-            stringify(ports.get()));
-
-  // Not enough ports.
-  r = Resources::parse("ports:[10000-10004]").get();
-  EXPECT_TRUE(r.ports(10).isNone());
-}
-
-
 TEST(ResourcesTest, Printing)
 {
   Resources r = Resources::parse(


[4/9] mesos git commit: A few style fixes for C++ Resources and tests.

Posted by ji...@apache.org.
A few style fixes for C++ Resources and tests.

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


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

Branch: refs/heads/master
Commit: 05b5ffa43c7e6b5178c1946162cd10d5984acfbe
Parents: bdf2d9d
Author: Jie Yu <yu...@gmail.com>
Authored: Fri Nov 14 11:46:15 2014 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Nov 19 00:14:25 2014 -0800

----------------------------------------------------------------------
 include/mesos/resources.hpp   | 128 +++++++++++++-----------------
 src/common/resources.cpp      |  84 ++++++++++++--------
 src/tests/resources_tests.cpp | 154 ++++++++++++++++++++++---------------
 3 files changed, 194 insertions(+), 172 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/05b5ffa4/include/mesos/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/resources.hpp b/include/mesos/resources.hpp
index c2ba597..4ed8cee 100644
--- a/include/mesos/resources.hpp
+++ b/include/mesos/resources.hpp
@@ -30,27 +30,16 @@
 #include <stout/try.hpp>
 
 
-/**
- * Resources come in three types: scalar, ranges, and sets. These are
- * represented using protocol buffers. To make manipulation of
- * resources easier within the Mesos core and for scheduler writers,
- * we provide generic overloaded opertors (see below) as well as a
- * general Resources class that encapsulates a collection of protocol
- * buffer Resource objects. The Resources class also provides a few
- * static routines to allow parsing resources (e.g., from the command
- * line), as well as determining whether or not a Resource object is
- * valid or allocatable. Note that many of these operations have not
- * been optimized but instead just written for correct semantics.
- *
- * Note! A resource is described by a tuple (name, type, role). Doing
- * "arithmetic" operations (those defined below) on two resources of
- * the same name but different type, or the same name and type but
- * different roles, doesn't make sense, so it's semantics are as
- * though the second operand was actually just an empty resource
- * (as though you didn't do the operation at all). In addition,
- * doing operations on two resources of the same type but different
- * names is a no-op.
- */
+// Resources come in three types: scalar, ranges, and sets. These are
+// represented using protocol buffers. To make manipulation of
+// resources easier within the Mesos core and for scheduler writers,
+// we provide generic overloaded operators (see below) as well as a
+// general Resources class that encapsulates a collection of protocol
+// buffer Resource objects. The Resources class also provides a few
+// static routines to allow parsing resources (e.g., from the command
+// line), as well as determining whether or not a Resource object is
+// valid. Note that many of these operations have not been optimized
+// but instead just written for correct semantics.
 
 namespace mesos {
 
@@ -76,40 +65,35 @@ bool matches(const Resource& left, const Resource& right);
 class Resources
 {
 public:
-  /**
-   * Parses the value and returns a Resource with the given name and role.
-   */
+  // Parses the text and returns a Resource object with the given name
+  // and role. For example, "Resource r = parse("mem", "1024", "*");".
   static Try<Resource> parse(
       const std::string& name,
       const std::string& value,
       const std::string& role);
 
-  /**
-   * Parses resources in the form "name:value (role);name:value...".
-   * Any name/value pair that doesn't specify a role is assigned to defaultRole.
-   */
+  // Parses Resources from text in the form "name:value(role);
+  // name:value;...". Any name/value pair that doesn't specify a role
+  // is assigned to defaultRole.
   static Try<Resources> parse(
-      const std::string& s,
+      const std::string& text,
       const std::string& defaultRole = "*");
 
-  /**
-   * Returns true iff this resource has a name, a valid type, i.e. scalar,
-   * range, or set, and has the appropriate value set for its type.
-   */
+  // Returns true iff this resource has a name, a valid type, i.e.
+  // scalar, range, or set, and has the appropriate value set for its
+  // type.
   static bool isValid(const Resource& resource);
 
-  /**
-   * Returns true iff this resource is valid and allocatable. In particular,
-   * a scalar is allocatable if it's value is greater than zero, a ranges
-   * is allocatable if there is at least one valid range in it, and a set
-   * is allocatable if it has at least one item.
-   */
+  // Returns true iff this resource is valid and allocatable. In
+  // particular, a scalar is allocatable if it's value is greater than
+  // zero, a ranges is allocatable if there is at least one valid
+  // range in it, and a set is allocatable if it has at least one
+  // item.
   static bool isAllocatable(const Resource& resource);
 
-  /**
-   * Returns true iff this resource is zero valued, i.e. is zero for scalars,
-   * has a range size of zero for ranges, and has no items for sets.
-   */
+  // Returns true iff this resource is zero valued, i.e. is zero for
+  // scalars, has a range size of zero for ranges, and has no items
+  // for sets.
   static bool isZero(const Resource& resource);
 
   Resources() {}
@@ -140,51 +124,40 @@ public:
     return resources.size();
   }
 
-  /**
-   * Returns all resources in this object that are marked with the
-   * specified role.
-   */
+  // Returns all resources in this object that are marked with the
+  // specified role.
   Resources extract(const std::string& role) const;
 
-  /**
-   * Returns a Resources object with the same amount of each resource
-   * type as these Resources, but with only one Resource object per
-   * type and all Resource object marked as the specified role.
-   */
+  // Returns a Resources object with the same amount of each resource
+  // type as these Resources, but with only one Resource object per
+  // type and all Resource object marked as the specified role.
   Resources flatten(const std::string& role = "*") const;
 
-  /**
-   * Finds a number of resources equal to toFind in these Resources
-   * and returns them marked with appropriate roles. For each resource
-   * type, resources are first taken from the specified role, then
-   * from '*', then from any other role.
-   */
+  // Finds a number of resources equal to toFind in these Resources
+  // and returns them marked with appropriate roles. For each resource
+  // type, resources are first taken from the specified role, then
+  // from '*', then from any other role.
   Option<Resources> find(
       const Resources& toFind,
       const std::string& role = "*") const;
 
-  /**
-   * Returns the Resource from these Resources that matches the argument
-   * in name, type, and role, if it exists.
-   */
+  // Returns the Resource from these Resources that matches the
+  // argument in name, type, and role, if it exists.
   Option<Resource> get(const Resource& r) const;
 
-  /**
-   * Returns all Resources from these Resources that match the argument
-   * in name and type, regardless of role.
-   */
+  // Returns all Resources from these Resources that match the
+  // argument in name and type, regardless of role.
   Option<Resources> getAll(const Resource& r) const;
 
   template <typename T>
   T get(const std::string& name, const T& t) const;
 
-  /**
-   * Returns a Resources object with only the allocatable resources.
-   */
+  // Returns a Resources object with only the allocatable resources.
   Resources allocatable() const;
 
   // Helpers to get known resource types.
-  // TODO(vinod): Fix this when we make these types as first class protobufs.
+  // TODO(vinod): Fix this when we make these types as first class
+  // protobufs.
   Option<double> cpus() const;
   Option<Bytes> mem() const;
   Option<Bytes> disk() const;
@@ -192,8 +165,8 @@ public:
   // TODO(vinod): Provide a Ranges abstraction.
   Option<Value::Ranges> ports() const;
 
-  // Helper function to extract the given number of ports
-  // from the "ports" resource.
+  // Helper function to extract the given number of ports from the
+  // "ports" resource.
   Option<Value::Ranges> ports(size_t numPorts) const;
 
   // TODO(jieyu): Consider returning an EphemeralPorts abstraction
@@ -212,10 +185,8 @@ public:
   const_iterator begin() const { return resources.begin(); }
   const_iterator end() const { return resources.end(); }
 
-  /**
-   * Using this operator makes it easy to copy a resources object into
-   * a protocol buffer field.
-   */
+  // Using this operator makes it easy to copy a resources object into
+  // a protocol buffer field.
   operator const google::protobuf::RepeatedPtrField<Resource>& () const;
 
   bool operator == (const Resources& that) const;
@@ -223,6 +194,11 @@ public:
 
   bool operator <= (const Resources& that) const;
 
+  // NOTE: If any error occurs (e.g., input Resource is not valid or
+  // the first operand is not a superset of the second oprand while
+  // doing subtraction), the semantics is as though the second operand
+  // was actually just an empty resource (as though you didn't do the
+  // operation at all).
   Resources operator + (const Resource& that) const;
   Resources operator + (const Resources& that) const;
   Resources& operator += (const Resource& that);

http://git-wip-us.apache.org/repos/asf/mesos/blob/05b5ffa4/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index 1ffebbb..810b698 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -157,50 +157,55 @@ bool matches(const Resource& left, const Resource& right)
 }
 
 
+/////////////////////////////////////////////////
+// Public static functions.
+/////////////////////////////////////////////////
+
+
 Try<Resource> Resources::parse(
     const string& name,
-    const string& text,
+    const string& value,
     const string& role)
 {
+  Try<Value> result = internal::values::parse(value);
+  if (result.isError()) {
+    return Error(
+        "Failed to parse resource " + name +
+        " value " + value + " error " + result.error());
+  }
+
   Resource resource;
-  Try<Value> result = internal::values::parse(text);
 
-  if (result.isError()) {
-    return Error("Failed to parse resource " + name +
-                 " text " + text +
-                 " error " + result.error());
-  } else{
-    Value value = result.get();
-    resource.set_name(name);
-    resource.set_role(role);
-
-    if (value.type() == Value::RANGES) {
-      resource.set_type(Value::RANGES);
-      resource.mutable_ranges()->MergeFrom(value.ranges());
-    } else if (value.type() == Value::SET) {
-      resource.set_type(Value::SET);
-      resource.mutable_set()->MergeFrom(value.set());
-    } else if (value.type() == Value::SCALAR) {
-      resource.set_type(Value::SCALAR);
-      resource.mutable_scalar()->MergeFrom(value.scalar());
-    } else {
-      return Error("Bad type for resource " + name +
-                   " text " + text +
-                   " type " + Value::Type_Name(value.type()));
-    }
+  Value _value = result.get();
+  resource.set_name(name);
+  resource.set_role(role);
+
+  if (_value.type() == Value::SCALAR) {
+    resource.set_type(Value::SCALAR);
+    resource.mutable_scalar()->CopyFrom(_value.scalar());
+  } else if (_value.type() == Value::RANGES) {
+    resource.set_type(Value::RANGES);
+    resource.mutable_ranges()->CopyFrom(_value.ranges());
+  } else if (_value.type() == Value::SET) {
+    resource.set_type(Value::SET);
+    resource.mutable_set()->CopyFrom(_value.set());
+  } else {
+    return Error(
+        "Bad type for resource " + name + " value " + value +
+        " type " + Value::Type_Name(_value.type()));
   }
 
   return resource;
 }
 
 
-Try<Resources> Resources::parse(const string& s, const string& defaultRole)
+Try<Resources> Resources::parse(
+    const string& text,
+    const string& defaultRole)
 {
   Resources resources;
 
-  vector<string> tokens = strings::tokenize(s, ";");
-
-  foreach (const string& token, tokens) {
+  foreach (const string& token, strings::tokenize(text, ";")) {
     vector<string> pair = strings::tokenize(token, ":");
     if (pair.size() != 2) {
       return Error("Bad value for resources, missing or extra ':' in " + token);
@@ -215,19 +220,22 @@ Try<Resources> Resources::parse(const string& s, const string& defaultRole)
     } else {
       size_t closeParen = pair[0].find(")");
       if (closeParen == string::npos || closeParen < openParen) {
-        return Error("Bad value for resources, mismatched parentheses in " +
-                     token);
+        return Error(
+            "Bad value for resources, mismatched parentheses in " + token);
       }
 
       name = strings::trim(pair[0].substr(0, openParen));
-      role = strings::trim(pair[0].substr(openParen + 1,
-                                          closeParen - openParen - 1));
+
+      role = strings::trim(pair[0].substr(
+          openParen + 1,
+          closeParen - openParen - 1));
     }
 
     Try<Resource> resource = Resources::parse(name, pair[1], role);
     if (resource.isError()) {
       return Error(resource.error());
     }
+
     resources += resource.get();
   }
 
@@ -325,6 +333,11 @@ bool Resources::isZero(const Resource& resource)
 }
 
 
+/////////////////////////////////////////////////
+// Public member functions.
+/////////////////////////////////////////////////
+
+
 Resources Resources::extract(const string& role) const
 {
   Resources r;
@@ -668,6 +681,11 @@ Option<Value::Ranges> Resources::ephemeral_ports() const
 }
 
 
+/////////////////////////////////////////////////
+// Overloaded operators.
+/////////////////////////////////////////////////
+
+
 Resources::operator const google::protobuf::RepeatedPtrField<Resource>& () const
 {
   return resources;

http://git-wip-us.apache.org/repos/asf/mesos/blob/05b5ffa4/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index 3e50889..9fe3c3a 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -38,24 +38,24 @@ using std::string;
 TEST(ResourcesTest, Parsing)
 {
   Resource cpus = Resources::parse("cpus", "45.55", "*").get();
+
   ASSERT_EQ(Value::SCALAR, cpus.type());
   EXPECT_EQ(45.55, cpus.scalar().value());
 
-  Resource ports = Resources::parse("ports",
-                                    "[10000-20000, 30000-50000]",
-                                    "*").get();
+  Resource ports = Resources::parse(
+      "ports", "[10000-20000, 30000-50000]", "*").get();
 
   ASSERT_EQ(Value::RANGES, ports.type());
   EXPECT_EQ(2, ports.ranges().range_size());
 
   Resource disks = Resources::parse("disks", "{sda1}", "*").get();
+
   ASSERT_EQ(Value::SET, disks.type());
   ASSERT_EQ(1, disks.set().item_size());
   EXPECT_EQ("sda1", disks.set().item(0));
 
-  Resources r1 = Resources::parse("cpus:45.55;"
-                                  "ports:[10000-20000, 30000-50000];"
-                                  "disks:{sda1}").get();
+  Resources r1 = Resources::parse(
+      "cpus:45.55;ports:[10000-20000, 30000-50000];disks:{sda1}").get();
 
   Resources r2;
   r2 += cpus;
@@ -87,11 +87,10 @@ TEST(ResourcesTest, ParsingWithRoles)
   resources1 += mem;
 
   EXPECT_EQ(parse1, resources1);
-
   EXPECT_EQ(resources1, Resources::parse(stringify(resources1)).get());
 
-  Resources parse2 =
-    Resources::parse("cpus(role1):2.5;ports(role2):[0-100]").get();
+  Resources parse2 = Resources::parse(
+      "cpus(role1):2.5;ports(role2):[0-100]").get();
 
   Resource cpus2;
   cpus2.set_name("cpus");
@@ -112,11 +111,10 @@ TEST(ResourcesTest, ParsingWithRoles)
   resources2 += cpus2;
 
   EXPECT_EQ(parse2, resources2);
-
   EXPECT_EQ(resources2, Resources::parse(stringify(resources2)).get());
 
-  Resources parse3 =
-    Resources::parse("cpus:2.5;ports(role2):[0-100]", "role1").get();
+  Resources parse3 = Resources::parse(
+      "cpus:2.5;ports(role2):[0-100]", "role1").get();
 
   EXPECT_EQ(parse2, parse3);
 }
@@ -129,29 +127,28 @@ TEST(ResourcesTest, ParseError)
 
   // Mismatched parentheses.
   EXPECT_ERROR(Resources::parse("cpus(role1:1"));
-
   EXPECT_ERROR(Resources::parse("cpus)(role1:1"));
 }
 
 
 TEST(ResourcesTest, Resources)
 {
-  Resources r = Resources::parse("cpus:45.55;"
-                                 "mem:1024;"
-                                 "ports:[10000-20000, 30000-50000];"
-                                 "disk:512").get();
+  Resources r = Resources::parse(
+      "cpus:45.55;mem:1024;ports:[10000-20000, 30000-50000];disk:512").get();
 
   EXPECT_SOME_EQ(45.55, r.cpus());
   EXPECT_SOME_EQ(Megabytes(1024), r.mem());
   EXPECT_SOME_EQ(Megabytes(512), r.disk());
 
-  EXPECT_SOME(r.ports());
+  ASSERT_SOME(r.ports());
+
   ostringstream ports;
   ports << r.ports().get();
 
   EXPECT_EQ("[10000-20000, 30000-50000]", ports.str());
 
   r = Resources::parse("cpus:45.55;disk:512").get();
+
   EXPECT_SOME_EQ(45.55, r.cpus());
   EXPECT_SOME_EQ(Megabytes(512), r.disk());
   EXPECT_TRUE(r.mem().isNone());
@@ -189,9 +186,8 @@ TEST(ResourcesTest, Ports)
 
 TEST(ResourcesTest, Printing)
 {
-  Resources r = Resources::parse("cpus:45.55;"
-                                 "ports:[10000-20000, 30000-50000];"
-                                 "disks:{sda1}").get();
+  Resources r = Resources::parse(
+      "cpus:45.55;ports:[10000-20000, 30000-50000];disks:{sda1}").get();
 
   string output =
     "cpus(*):45.55; ports(*):[10000-20000, 30000-50000]; disks(*):{sda1}";
@@ -220,12 +216,17 @@ TEST(ResourcesTest, BadResourcesNotAllocatable)
   Resource cpus;
   cpus.set_type(Value::SCALAR);
   cpus.mutable_scalar()->set_value(1);
+
   Resources r;
   r += cpus;
+
   EXPECT_EQ(0u, r.allocatable().size());
+
   cpus.set_name("cpus");
   cpus.mutable_scalar()->set_value(0);
+
   r += cpus;
+
   EXPECT_EQ(0u, r.allocatable().size());
 }
 
@@ -249,6 +250,7 @@ TEST(ResourcesTest, ScalarEquals)
 
   Resource cpus1 = Resources::parse("cpus", "3", "role1").get();
   Resource cpus2 = Resources::parse("cpus", "3", "role2").get();
+
   EXPECT_NE(cpus1, cpus2);
 }
 
@@ -317,12 +319,14 @@ TEST(ResourcesTest, ScalarAddition)
   r2 += mem2;
 
   Resources sum = r1 + r2;
+
   EXPECT_EQ(2u, sum.size());
   EXPECT_EQ(3, sum.get("cpus", Value::Scalar()).value());
   EXPECT_EQ(15, sum.get("mem", Value::Scalar()).value());
 
   Resources r = r1;
   r += r2;
+
   EXPECT_EQ(2u, r.size());
   EXPECT_EQ(3, r.get("cpus", Value::Scalar()).value());
   EXPECT_EQ(15, r.get("mem", Value::Scalar()).value());
@@ -343,6 +347,7 @@ TEST(ResourcesTest, ScalarAddition2)
   r2 += cpus3;
 
   Resources sum = r1 + r2;
+
   EXPECT_EQ(2u, sum.size());
   EXPECT_EQ(9, sum.cpus().get());
   EXPECT_EQ(sum, Resources::parse("cpus(role1):6;cpus(role2):3").get());
@@ -366,12 +371,14 @@ TEST(ResourcesTest, ScalarSubtraction)
   r2 += mem2;
 
   Resources diff = r1 - r2;
+
   EXPECT_EQ(2u, diff.size());
   EXPECT_EQ(49.5, diff.get("cpus", Value::Scalar()).value());
   EXPECT_EQ(3072, diff.get("mem", Value::Scalar()).value());
 
   Resources r = r1;
   r -= r2;
+
   EXPECT_EQ(49.5, diff.get("cpus", Value::Scalar()).value());
   EXPECT_EQ(3072, diff.get("mem", Value::Scalar()).value());
 
@@ -395,6 +402,7 @@ TEST(ResourcesTest, ScalarSubtraction2)
   r2 += cpus3;
 
   Resources diff = r1 - r2;
+
   EXPECT_EQ(2u, diff.size());
   EXPECT_EQ(7, diff.cpus().get());
   EXPECT_EQ(diff, Resources::parse("cpus(role1):4;cpus(role2):3").get());
@@ -403,9 +411,11 @@ TEST(ResourcesTest, ScalarSubtraction2)
 
 TEST(ResourcesTest, RangesEquals)
 {
-  Resource ports1 = Resources::parse("ports", "[20-40]", "*").get();
-  Resource ports2 =
-    Resources::parse("ports", "[20-30, 31-39, 40-40]", "*").get();
+  Resource ports1 = Resources::parse(
+      "ports", "[20-40]", "*").get();
+
+  Resource ports2 = Resources::parse(
+      "ports", "[20-30, 31-39, 40-40]", "*").get();
 
   Resources r1;
   r1 += ports1;
@@ -461,10 +471,11 @@ TEST(ResourcesTest, RangesSubset)
 
 TEST(ResourcesTest, RangesAddition)
 {
-  Resource ports1 =
-    Resources::parse("ports", "[20000-40000, 21000-38000]", "*").get();
-  Resource ports2 =
-    Resources::parse("ports", "[30000-50000, 10000-20000]", "*").get();
+  Resource ports1 = Resources::parse(
+      "ports", "[20000-40000, 21000-38000]", "*").get();
+
+  Resource ports2 = Resources::parse(
+      "ports", "[30000-50000, 10000-20000]", "*").get();
 
   Resources r;
   r += ports1;
@@ -534,9 +545,11 @@ TEST(ResourcesTest, RangesAdditon3)
 
 TEST(ResourcesTest, RangesAddition4)
 {
-  Resource ports1 =
-    Resources::parse("ports", "[1-4, 9-10, 20-22, 26-30]", "*").get();
-  Resource ports2 = Resources::parse("ports", "[5-8, 23-25]", "*").get();
+  Resource ports1 = Resources::parse(
+      "ports", "[1-4, 9-10, 20-22, 26-30]", "*").get();
+
+  Resource ports2 = Resources::parse(
+      "ports", "[5-8, 23-25]", "*").get();
 
   Resources r;
   r += ports1;
@@ -552,9 +565,11 @@ TEST(ResourcesTest, RangesAddition4)
 
 TEST(ResourcesTest, RangesSubtraction)
 {
-  Resource ports1 = Resources::parse("ports", "[20000-40000]", "*").get();
-  Resource ports2 =
-    Resources::parse("ports", "[10000-20000, 30000-50000]", "*").get();
+  Resource ports1 = Resources::parse(
+      "ports", "[20000-40000]", "*").get();
+
+  Resource ports2 = Resources::parse(
+      "ports", "[10000-20000, 30000-50000]", "*").get();
 
   Resources r;
   r += ports1;
@@ -610,7 +625,6 @@ TEST(ResourcesTest, RangesSubtraction3)
   Resources resourcesInUse = Resources::parse("ports:[50000-50001]").get();
 
   Resources resourcesFree = resources - (resourcesOffered + resourcesInUse);
-
   resourcesFree = resourcesFree.allocatable();
 
   EXPECT_EQ(1u, resourcesFree.size());
@@ -626,9 +640,7 @@ TEST(ResourcesTest, RangesSubtraction4)
   Resources resources = Resources::parse("ports:[50000-60000]").get();
 
   Resources resourcesOffered;
-
   resourcesOffered += resources;
-
   resourcesOffered -= resources;
 
   EXPECT_EQ(0u, resourcesOffered.size());
@@ -641,9 +653,11 @@ TEST(ResourcesTest, RangesSubtraction4)
 
 TEST(ResourcesTest, RangesSubtraction5)
 {
-  Resource ports1 =
-    Resources::parse("ports", "[1-10, 20-30, 40-50]", "*").get();
-  Resource ports2 = Resources::parse("ports", "[2-9, 15-45, 48-50]", "*").get();
+  Resource ports1 = Resources::parse(
+      "ports", "[1-10, 20-30, 40-50]", "*").get();
+
+  Resource ports2 = Resources::parse(
+      "ports", "[2-9, 15-45, 48-50]", "*").get();
 
   Resources r;
   r += ports1;
@@ -690,9 +704,11 @@ TEST(ResourcesTest, SetEquals)
 
 TEST(ResourcesTest, SetSubset)
 {
-  Resource disks1 = Resources::parse("disks", "{sda1,sda2}", "*").get();
-  Resource disks2 =
-    Resources::parse("disks", "{sda1,sda3,sda4,sda2}", "*").get();
+  Resource disks1 = Resources::parse(
+      "disks", "{sda1,sda2}", "*").get();
+
+  Resource disks2 = Resources::parse(
+      "disks", "{sda1,sda3,sda4,sda2}", "*").get();
 
   Resources r1;
   r1 += disks1;
@@ -709,9 +725,11 @@ TEST(ResourcesTest, SetSubset)
 
 TEST(ResourcesTest, SetAddition)
 {
-  Resource disks1 = Resources::parse("disks", "{sda1,sda2,sda3}", "*").get();
-  Resource disks2 =
-    Resources::parse("disks", "{sda1,sda2,sda3,sda4}", "*").get();
+  Resource disks1 = Resources::parse(
+      "disks", "{sda1,sda2,sda3}", "*").get();
+
+  Resource disks2 = Resources::parse(
+      "disks", "{sda1,sda2,sda3,sda4}", "*").get();
 
   Resources r;
   r += disks1;
@@ -727,9 +745,11 @@ TEST(ResourcesTest, SetAddition)
 
 TEST(ResourcesTest, SetSubtraction)
 {
-  Resource disks1 =
-    Resources::parse("disks", "{sda1,sda2,sda3,sda4}", "*").get();
-  Resource disks2 = Resources::parse("disks", "{sda2,sda3,sda4}", "*").get();
+  Resource disks1 = Resources::parse(
+      "disks", "{sda1,sda2,sda3,sda4}", "*").get();
+
+  Resource disks2 = Resources::parse(
+      "disks", "{sda2,sda3,sda4}", "*").get();
 
   Resources r;
   r += disks1;
@@ -770,33 +790,41 @@ TEST(ResourcesTest, FlattenRoles)
 
 TEST(ResourcesTest, Find)
 {
-  Resources resources1 =
-    Resources::parse("cpus(role1):2;mem(role1):10;cpus:4;mem:20").get();
+  Resources resources1 = Resources::parse(
+      "cpus(role1):2;mem(role1):10;cpus:4;mem:20").get();
+
   Resources toFind1 = Resources::parse("cpus:3;mem:15").get();
 
   Resources found1 = resources1.find(toFind1, "role1").get();
-  Resources expected1 =
-    Resources::parse("cpus(role1):2;mem(role1):10;cpus:1;mem:5").get();
+
+  Resources expected1 = Resources::parse(
+      "cpus(role1):2;mem(role1):10;cpus:1;mem:5").get();
+
   EXPECT_EQ(found1, expected1);
 
-  Resources resources2 =
-    Resources::parse("cpus(role1):1;mem(role1):5;cpus(role2):2;"
-                     "mem(role2):8;cpus:1;mem:7").get();
+  Resources resources2 = Resources::parse(
+      "cpus(role1):1;mem(role1):5;cpus(role2):2;"
+      "mem(role2):8;cpus:1;mem:7").get();
+
   Resources toFind2 = Resources::parse("cpus:3;mem:15").get();
 
   Resources found2 = resources2.find(toFind2, "role1").get();
-  Resources expected2 =
-    Resources::parse("cpus(role1):1;mem(role1):5;cpus:1;mem:7;"
-                     "cpus(role2):1;mem(role2):3").get();
+
+  Resources expected2 = Resources::parse(
+      "cpus(role1):1;mem(role1):5;cpus:1;mem:7;"
+      "cpus(role2):1;mem(role2):3").get();
+
   EXPECT_EQ(found2, expected2);
 
-  Resources resources3 =
-    Resources::parse("cpus(role1):5;mem(role1):5;cpus:5;mem:5").get();
+  Resources resources3 = Resources::parse(
+      "cpus(role1):5;mem(role1):5;cpus:5;mem:5").get();
+
   Resources toFind3 = Resources::parse("cpus:6;mem:6").get();
 
   Resources found3 = resources3.find(toFind3).get();
-  Resources expected3 =
-    Resources::parse("cpus:5;mem:5;cpus(role1):1;mem(role1):1").get();
+
+  Resources expected3 = Resources::parse(
+      "cpus:5;mem:5;cpus(role1):1;mem(role1):1").get();
 
   EXPECT_EQ(found3, expected3);
 


[7/9] mesos git commit: Replaced size() with empty() in C++ Resources.

Posted by ji...@apache.org.
Replaced size() with empty() in C++ Resources.

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


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

Branch: refs/heads/master
Commit: 7fa02175c5dd4c0a72193ea4089bf9a20633e543
Parents: a8d0244
Author: Jie Yu <yu...@gmail.com>
Authored: Fri Nov 14 21:59:16 2014 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Nov 19 00:14:26 2014 -0800

----------------------------------------------------------------------
 include/mesos/resources.hpp                   |  5 +-
 src/master/hierarchical_allocator_process.hpp |  2 +-
 src/master/master.cpp                         |  2 +-
 src/tests/resources_tests.cpp                 | 56 +++++++++++-----------
 4 files changed, 31 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7fa02175/include/mesos/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/resources.hpp b/include/mesos/resources.hpp
index 3009395..13a81e2 100644
--- a/include/mesos/resources.hpp
+++ b/include/mesos/resources.hpp
@@ -102,10 +102,7 @@ public:
     return *this;
   }
 
-  size_t size() const
-  {
-    return resources.size();
-  }
+  bool empty() const { return resources.size() == 0; }
 
   // Returns all resources in this object that are marked with the
   // specified role.

http://git-wip-us.apache.org/repos/asf/mesos/blob/7fa02175/src/master/hierarchical_allocator_process.hpp
----------------------------------------------------------------------
diff --git a/src/master/hierarchical_allocator_process.hpp b/src/master/hierarchical_allocator_process.hpp
index 77f9741..45d45b8 100644
--- a/src/master/hierarchical_allocator_process.hpp
+++ b/src/master/hierarchical_allocator_process.hpp
@@ -537,7 +537,7 @@ HierarchicalAllocatorProcess<RoleSorter, FrameworkSorter>::resourcesRecovered(
 {
   CHECK(initialized);
 
-  if (resources.size() == 0) {
+  if (resources.empty()) {
     return;
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/7fa02175/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 4cd9a4d..7ab25b3 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -2674,7 +2674,7 @@ void Master::_launchTasks(
   // Calculate unused resources.
   Resources unusedResources = totalResources - usedResources;
 
-  if (unusedResources.size() > 0) {
+  if (!unusedResources.empty()) {
     // Tell the allocator about the unused (e.g., refused) resources.
     allocator->resourcesRecovered(
         frameworkId, slaveId, unusedResources, filters);

http://git-wip-us.apache.org/repos/asf/mesos/blob/7fa02175/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index 9387a29..c0486f4 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -179,7 +179,7 @@ TEST(ResourcesTest, Printing)
 TEST(ResourcesTest, InitializedIsEmpty)
 {
   Resources r;
-  EXPECT_EQ(0u, r.size());
+  EXPECT_TRUE(r.empty());
 }
 
 
@@ -192,14 +192,14 @@ TEST(ResourcesTest, BadResourcesNotAllocatable)
   Resources r;
   r += cpus;
 
-  EXPECT_EQ(0u, r.size());
+  EXPECT_TRUE(r.empty());
 
   cpus.set_name("cpus");
   cpus.mutable_scalar()->set_value(0);
 
   r += cpus;
 
-  EXPECT_EQ(0u, r.size());
+  EXPECT_TRUE(r.empty());
 }
 
 
@@ -216,8 +216,8 @@ TEST(ResourcesTest, ScalarEquals)
   r2 += cpus;
   r2 += mem;
 
-  EXPECT_EQ(2u, r1.size());
-  EXPECT_EQ(2u, r2.size());
+  EXPECT_FALSE(r1.empty());
+  EXPECT_FALSE(r2.empty());
   EXPECT_EQ(r1, r2);
 
   Resources cpus1 = Resources::parse("cpus", "3", "role1").get();
@@ -292,14 +292,14 @@ TEST(ResourcesTest, ScalarAddition)
 
   Resources sum = r1 + r2;
 
-  EXPECT_EQ(2u, sum.size());
+  EXPECT_FALSE(sum.empty());
   EXPECT_EQ(3, sum.get<Value::Scalar>("cpus").get().value());
   EXPECT_EQ(15, sum.get<Value::Scalar>("mem").get().value());
 
   Resources r = r1;
   r += r2;
 
-  EXPECT_EQ(2u, r.size());
+  EXPECT_FALSE(r.empty());
   EXPECT_EQ(3, r.get<Value::Scalar>("cpus").get().value());
   EXPECT_EQ(15, r.get<Value::Scalar>("mem").get().value());
 }
@@ -320,7 +320,7 @@ TEST(ResourcesTest, ScalarAddition2)
 
   Resources sum = r1 + r2;
 
-  EXPECT_EQ(2u, sum.size());
+  EXPECT_FALSE(sum.empty());
   EXPECT_EQ(9, sum.cpus().get());
   EXPECT_EQ(sum, Resources::parse("cpus(role1):6;cpus(role2):3").get());
 }
@@ -344,7 +344,7 @@ TEST(ResourcesTest, ScalarSubtraction)
 
   Resources diff = r1 - r2;
 
-  EXPECT_EQ(2u, diff.size());
+  EXPECT_FALSE(diff.empty());
   EXPECT_EQ(49.5, diff.get<Value::Scalar>("cpus").get().value());
   EXPECT_EQ(3072, diff.get<Value::Scalar>("mem").get().value());
 
@@ -357,7 +357,7 @@ TEST(ResourcesTest, ScalarSubtraction)
   r = r1;
   r -= r1;
 
-  EXPECT_EQ(0u, r.size());
+  EXPECT_TRUE(r.empty());
 }
 
 
@@ -376,7 +376,7 @@ TEST(ResourcesTest, ScalarSubtraction2)
 
   Resources diff = r1 - r2;
 
-  EXPECT_EQ(2u, diff.size());
+  EXPECT_FALSE(diff.empty());
   EXPECT_EQ(7, diff.cpus().get());
   EXPECT_EQ(diff, Resources::parse("cpus(role1):4;cpus(role2):3").get());
 }
@@ -454,7 +454,7 @@ TEST(ResourcesTest, RangesAddition)
   r += ports1;
   r += ports2;
 
-  EXPECT_EQ(1u, r.size());
+  EXPECT_FALSE(r.empty());
 
   EXPECT_SOME_EQ(
       values::parse("[10000-50000]").get().ranges(),
@@ -471,7 +471,7 @@ TEST(ResourcesTest, RangesAddition2)
   r += ports1;
   r += ports2;
 
-  EXPECT_EQ(1u, r.size());
+  EXPECT_FALSE(r.empty());
 
   EXPECT_SOME_EQ(
       values::parse("[1-65, 70-80]").get().ranges(),
@@ -490,7 +490,7 @@ TEST(ResourcesTest, RangesAdditon3)
   r1 += ports1;
   r1 += ports2;
 
-  EXPECT_EQ(1u, r1.size());
+  EXPECT_FALSE(r1.empty());
 
   EXPECT_SOME_EQ(
       values::parse("[1-4]").get().ranges(),
@@ -500,7 +500,7 @@ TEST(ResourcesTest, RangesAdditon3)
   r2 += ports3;
   r2 += ports4;
 
-  EXPECT_EQ(1u, r2.size());
+  EXPECT_FALSE(r2.empty());
 
   EXPECT_SOME_EQ(
       values::parse("[5-8]").get().ranges(),
@@ -508,7 +508,7 @@ TEST(ResourcesTest, RangesAdditon3)
 
   r2 += r1;
 
-  EXPECT_EQ(1u, r2.size());
+  EXPECT_FALSE(r2.empty());
 
   EXPECT_SOME_EQ(
       values::parse("[1-8]").get().ranges(),
@@ -528,7 +528,7 @@ TEST(ResourcesTest, RangesAddition4)
   r += ports1;
   r += ports2;
 
-  EXPECT_EQ(1u, r.size());
+  EXPECT_FALSE(r.empty());
 
   EXPECT_SOME_EQ(
       values::parse("[1-10, 20-30]").get().ranges(),
@@ -548,7 +548,7 @@ TEST(ResourcesTest, RangesSubtraction)
   r += ports1;
   r -= ports2;
 
-  EXPECT_EQ(1u, r.size());
+  EXPECT_FALSE(r.empty());
 
   EXPECT_SOME_EQ(
       values::parse("[20001-29999]").get().ranges(),
@@ -565,7 +565,7 @@ TEST(ResourcesTest, RangesSubtraction1)
   r += ports1;
   r -= ports2;
 
-  EXPECT_EQ(1u, r.size());
+  EXPECT_FALSE(r.empty());
 
   EXPECT_SOME_EQ(
       values::parse("[50002-60000]").get().ranges(),
@@ -582,7 +582,7 @@ TEST(ResourcesTest, RangesSubtraction2)
   r += ports1;
   r -= ports2;
 
-  EXPECT_EQ(1u, r.size());
+  EXPECT_FALSE(r.empty());
 
   EXPECT_SOME_EQ(
       values::parse("[50001-60000]").get().ranges(),
@@ -599,7 +599,7 @@ TEST(ResourcesTest, RangesSubtraction3)
 
   Resources resourcesFree = resources - (resourcesOffered + resourcesInUse);
 
-  EXPECT_EQ(1u, resourcesFree.size());
+  EXPECT_FALSE(resourcesFree.empty());
 
   EXPECT_SOME_EQ(
       values::parse("[50002-60000]").get().ranges(),
@@ -615,7 +615,7 @@ TEST(ResourcesTest, RangesSubtraction4)
   resourcesOffered += resources;
   resourcesOffered -= resources;
 
-  EXPECT_EQ(0u, resourcesOffered.size());
+  EXPECT_TRUE(resourcesOffered.empty());
   EXPECT_NONE(resourcesOffered.get<Value::Ranges>("ports"));
 }
 
@@ -632,7 +632,7 @@ TEST(ResourcesTest, RangesSubtraction5)
   r += ports1;
   r -= ports2;
 
-  EXPECT_EQ(1u, r.size());
+  EXPECT_FALSE(r.empty());
 
   EXPECT_SOME_EQ(
       values::parse("[1-1, 10-10, 46-47]").get().ranges(),
@@ -649,7 +649,7 @@ TEST(ResourcesTest, RangesSubtraction6)
   r += ports1;
   r -= ports2;
 
-  EXPECT_EQ(1u, r.size());
+  EXPECT_FALSE(r.empty());
 
   EXPECT_SOME_EQ(
       values::parse("[1-10]").get().ranges(),
@@ -685,8 +685,8 @@ TEST(ResourcesTest, SetSubset)
   Resources r2;
   r2 += disks2;
 
-  EXPECT_EQ(1u, r1.size());
-  EXPECT_EQ(1u, r2.size());
+  EXPECT_FALSE(r1.empty());
+  EXPECT_FALSE(r2.empty());
   EXPECT_TRUE(r1 <= r2);
   EXPECT_FALSE(r2 <= r1);
 }
@@ -704,7 +704,7 @@ TEST(ResourcesTest, SetAddition)
   r += disks1;
   r += disks2;
 
-  EXPECT_EQ(1u, r.size());
+  EXPECT_FALSE(r.empty());
 
   Option<Value::Set> set = r.get<Value::Set>("disks");
 
@@ -725,7 +725,7 @@ TEST(ResourcesTest, SetSubtraction)
   r += disks1;
   r -= disks2;
 
-  EXPECT_EQ(1u, r.size());
+  EXPECT_FALSE(r.empty());
 
   Option<Value::Set> set = r.get<Value::Set>("disks");
 


[3/9] mesos git commit: Refactored operators for Resource object and made them private.

Posted by ji...@apache.org.
Refactored operators for Resource object and made them private.

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


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

Branch: refs/heads/master
Commit: a01773b5fae7a434d508e04217aa3285e71fdc0a
Parents: 05b5ffa
Author: Jie Yu <yu...@gmail.com>
Authored: Fri Nov 14 14:20:45 2014 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Wed Nov 19 00:14:25 2014 -0800

----------------------------------------------------------------------
 include/mesos/resources.hpp   |  23 ++----
 src/common/resources.cpp      | 162 +++++++++++++++++++------------------
 src/tests/resources_tests.cpp |   8 +-
 3 files changed, 95 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a01773b5/include/mesos/resources.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/resources.hpp b/include/mesos/resources.hpp
index 4ed8cee..c8a261f 100644
--- a/include/mesos/resources.hpp
+++ b/include/mesos/resources.hpp
@@ -43,23 +43,6 @@
 
 namespace mesos {
 
-bool operator == (const Resource& left, const Resource& right);
-bool operator != (const Resource& left, const Resource& right);
-
-
-bool operator <= (const Resource& left, const Resource& right);
-
-
-Resource& operator += (Resource& left, const Resource& right);
-Resource operator + (const Resource& left, const Resource& right);
-Resource& operator -= (Resource& left, const Resource& right);
-Resource operator - (const Resource& left, const Resource& right);
-
-
-// Return true iff both Resources have the same name, type, and role.
-bool matches(const Resource& left, const Resource& right);
-
-
 // TODO(bmahler): Ensure that the underlying resources are kept
 // in a flattened state: MESOS-1714.
 class Resources
@@ -98,6 +81,12 @@ public:
 
   Resources() {}
 
+  // TODO(jieyu): Consider using C++11 initializer list.
+  /*implicit*/ Resources(const Resource& resource)
+  {
+    resources.Add()->CopyFrom(resource);
+  }
+
   /*implicit*/
   Resources(const google::protobuf::RepeatedPtrField<Resource>& _resources)
   {

http://git-wip-us.apache.org/repos/asf/mesos/blob/a01773b5/src/common/resources.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources.cpp b/src/common/resources.cpp
index 810b698..458c1cd 100644
--- a/src/common/resources.cpp
+++ b/src/common/resources.cpp
@@ -35,19 +35,75 @@ using std::vector;
 
 namespace mesos {
 
+/////////////////////////////////////////////////
+// Helper functions.
+/////////////////////////////////////////////////
+
+// Tests if we can add two Resource objects together resulting in one
+// valid Resource object. For example, two Resource objects with
+// different name, type or role are not addable.
+static bool addable(const Resource& left, const Resource& right)
+{
+  return left.name() == right.name() &&
+    left.type() == right.type() &&
+    left.role() == right.role();
+}
+
+
+// Tests if we can subtract "right" from "left" resulting in one valid
+// Resource object. For example, two Resource objects with different
+// name, type or role are not subtractable.
+// NOTE: Set substraction is always well defined, it does not require
+// 'right' to be contained within 'left'. For example, assuming that
+// "left = {1, 2}" and "right = {2, 3}", "left" and "right" are
+// subtractable because "left - right = {1}". However, "left" does not
+// contains "right".
+static bool subtractable(const Resource& left, const Resource& right)
+{
+  return left.name() == right.name() &&
+    left.type() == right.type() &&
+    left.role() == right.role();
+}
+
+
+// Tests if "right" is contained in "left".
+static bool contains(const Resource& left, const Resource& right)
+{
+  if (left.name() != right.name() ||
+      left.type() != right.type() ||
+      left.role() != right.role()) {
+    return false;
+  }
+
+  if (left.type() == Value::SCALAR) {
+    return right.scalar() <= left.scalar();
+  } else if (left.type() == Value::RANGES) {
+    return right.ranges() <= left.ranges();
+  } else if (left.type() == Value::SET) {
+    return right.set() <= left.set();
+  } else {
+    return false;
+  }
+}
+
+
 bool operator == (const Resource& left, const Resource& right)
 {
-  if (matches(left, right)) {
-    if (left.type() == Value::SCALAR) {
-      return left.scalar() == right.scalar();
-    } else if (left.type() == Value::RANGES) {
-      return left.ranges() == right.ranges();
-    } else if (left.type() == Value::SET) {
-      return left.set() == right.set();
-    }
+  if (left.name() != right.name() ||
+      left.type() != right.type() ||
+      left.role() != right.role()) {
+    return false;
   }
 
-  return false;
+  if (left.type() == Value::SCALAR) {
+    return left.scalar() == right.scalar();
+  } else if (left.type() == Value::RANGES) {
+    return left.ranges() == right.ranges();
+  } else if (left.type() == Value::SET) {
+    return left.set() == right.set();
+  } else {
+    return false;
+  }
 }
 
 
@@ -59,32 +115,19 @@ bool operator != (const Resource& left, const Resource& right)
 
 bool operator <= (const Resource& left, const Resource& right)
 {
-  if (matches(left, right)) {
-    if (left.type() == Value::SCALAR) {
-      return left.scalar() <= right.scalar();
-    } else if (left.type() == Value::RANGES) {
-      return left.ranges() <= right.ranges();
-    } else if (left.type() == Value::SET) {
-      return left.set() <= right.set();
-    }
-  }
-
-  return false;
+  return contains(right, left);
 }
 
 
 Resource& operator += (Resource& left, const Resource& right)
 {
-  if (matches(left, right)) {
-    if (left.type() == Value::SCALAR) {
-      left.mutable_scalar()->MergeFrom(left.scalar() + right.scalar());
-    } else if (left.type() == Value::RANGES) {
-      left.mutable_ranges()->Clear();
-      left.mutable_ranges()->MergeFrom(left.ranges() + right.ranges());
-    } else if (left.type() == Value::SET) {
-      left.mutable_set()->Clear();
-      left.mutable_set()->MergeFrom(left.set() + right.set());
-    }
+  // TODO(jieyu): Leverage += for Value to avoid copying.
+  if (left.type() == Value::SCALAR) {
+    left.mutable_scalar()->CopyFrom(left.scalar() + right.scalar());
+  } else if (left.type() == Value::RANGES) {
+    left.mutable_ranges()->CopyFrom(left.ranges() + right.ranges());
+  } else if (left.type() == Value::SET) {
+    left.mutable_set()->CopyFrom(left.set() + right.set());
   }
 
   return left;
@@ -94,35 +137,20 @@ Resource& operator += (Resource& left, const Resource& right)
 Resource operator + (const Resource& left, const Resource& right)
 {
   Resource result = left;
-
-  if (matches(left, right)) {
-    if (left.type() == Value::SCALAR) {
-      result.mutable_scalar()->MergeFrom(left.scalar() + right.scalar());
-    } else if (left.type() == Value::RANGES) {
-      result.mutable_ranges()->Clear();
-      result.mutable_ranges()->MergeFrom(left.ranges() + right.ranges());
-    } else if (left.type() == Value::SET) {
-      result.mutable_set()->Clear();
-      result.mutable_set()->MergeFrom(left.set() + right.set());
-    }
-  }
-
+  result += right;
   return result;
 }
 
 
 Resource& operator -= (Resource& left, const Resource& right)
 {
-  if (matches(left, right)) {
-    if (left.type() == Value::SCALAR) {
-      left.mutable_scalar()->MergeFrom(left.scalar() - right.scalar());
-    } else if (left.type() == Value::RANGES) {
-      left.mutable_ranges()->Clear();
-      left.mutable_ranges()->MergeFrom(left.ranges() - right.ranges());
-    } else if (left.type() == Value::SET) {
-      left.mutable_set()->Clear();
-      left.mutable_set()->MergeFrom(left.set() - right.set());
-    }
+  // TODO(jieyu): Leverage -= for Value to avoid copying.
+  if (left.type() == Value::SCALAR) {
+    left.mutable_scalar()->CopyFrom(left.scalar() - right.scalar());
+  } else if (left.type() == Value::RANGES) {
+    left.mutable_ranges()->CopyFrom(left.ranges() - right.ranges());
+  } else if (left.type() == Value::SET) {
+    left.mutable_set()->CopyFrom(left.set() - right.set());
   }
 
   return left;
@@ -132,31 +160,11 @@ Resource& operator -= (Resource& left, const Resource& right)
 Resource operator - (const Resource& left, const Resource& right)
 {
   Resource result = left;
-
-  if (matches(left, right)) {
-    if (left.type() == Value::SCALAR) {
-      result.mutable_scalar()->MergeFrom(left.scalar() - right.scalar());
-    } else if (left.type() == Value::RANGES) {
-      result.mutable_ranges()->Clear();
-      result.mutable_ranges()->MergeFrom(left.ranges() - right.ranges());
-    } else if (left.type() == Value::SET) {
-      result.mutable_set()->Clear();
-      result.mutable_set()->MergeFrom(left.set() - right.set());
-    }
-  }
-
+  result -= right;
   return result;
 }
 
 
-bool matches(const Resource& left, const Resource& right)
-{
-  return left.name() == right.name() &&
-    left.type() == right.type() &&
-    left.role() == right.role();
-}
-
-
 /////////////////////////////////////////////////
 // Public static functions.
 /////////////////////////////////////////////////
@@ -433,7 +441,7 @@ Option<Resources> Resources::find(
 Option<Resource> Resources::get(const Resource& r) const
 {
   foreach (const Resource& resource, resources) {
-    if (matches(resource, r)) {
+    if (addable(resource, r)) {
       return resource;
     }
   }
@@ -745,7 +753,7 @@ Resources Resources::operator + (const Resource& that) const
   bool added = false;
 
   foreach (const Resource& resource, resources) {
-    if (matches(resource, that)) {
+    if (addable(resource, that)) {
       result.resources.Add()->MergeFrom(resource + that);
       added = true;
     } else {
@@ -795,7 +803,7 @@ Resources Resources::operator - (const Resource& that) const
   Resources result;
 
   foreach (const Resource& resource, resources) {
-    if (matches(resource, that)) {
+    if (subtractable(resource, that)) {
       Resource r = resource - that;
       if (!isZero(r)) {
         result.resources.Add()->MergeFrom(r);

http://git-wip-us.apache.org/repos/asf/mesos/blob/a01773b5/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index 9fe3c3a..c8d069a 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -248,8 +248,8 @@ TEST(ResourcesTest, ScalarEquals)
   EXPECT_EQ(2u, r2.size());
   EXPECT_EQ(r1, r2);
 
-  Resource cpus1 = Resources::parse("cpus", "3", "role1").get();
-  Resource cpus2 = Resources::parse("cpus", "3", "role2").get();
+  Resources cpus1 = Resources::parse("cpus", "3", "role1").get();
+  Resources cpus2 = Resources::parse("cpus", "3", "role2").get();
 
   EXPECT_NE(cpus1, cpus2);
 }
@@ -287,8 +287,8 @@ TEST(ResourcesTest, ScalarSubset2)
   Resources r2;
   r2 += cpus2;
 
-  EXPECT_FALSE(cpus1 <= cpus2);
-  EXPECT_FALSE(cpus2 <= cpus1);
+  EXPECT_FALSE(r1 <= r2);
+  EXPECT_FALSE(r2 <= r1);
 
   Resource cpus3 = Resources::parse("cpus", "3", "role1").get();