You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2012/11/26 22:17:23 UTC

svn commit: r1413852 - in /incubator/mesos/trunk: src/linux/ src/slave/ src/tests/ third_party/libprocess/include/stout/

Author: benh
Date: Mon Nov 26 21:17:21 2012
New Revision: 1413852

URL: http://svn.apache.org/viewvc?rev=1413852&view=rev
Log:
Added cpuset isolation to cgroups isolation module.

From: Ben Mahler <be...@gmail.com>
Review: https://reviews.apache.org/r/8108

Modified:
    incubator/mesos/trunk/src/linux/proc.hpp
    incubator/mesos/trunk/src/slave/cgroups_isolation_module.cpp
    incubator/mesos/trunk/src/slave/cgroups_isolation_module.hpp
    incubator/mesos/trunk/src/slave/isolation_module.hpp
    incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp
    incubator/mesos/trunk/src/slave/lxc_isolation_module.hpp
    incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp
    incubator/mesos/trunk/src/slave/process_based_isolation_module.hpp
    incubator/mesos/trunk/src/slave/slave.cpp
    incubator/mesos/trunk/src/tests/utils.hpp
    incubator/mesos/trunk/third_party/libprocess/include/stout/stringify.hpp

Modified: incubator/mesos/trunk/src/linux/proc.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/linux/proc.hpp?rev=1413852&r1=1413851&r2=1413852&view=diff
==============================================================================
--- incubator/mesos/trunk/src/linux/proc.hpp (original)
+++ incubator/mesos/trunk/src/linux/proc.hpp Mon Nov 26 21:17:21 2012
@@ -57,12 +57,42 @@ struct CPU
   CPU(unsigned int _id, unsigned int _core, unsigned int _socket)
     : id(_id), core(_core), socket(_socket) {}
 
-  const unsigned int id; // "processor"
-  const unsigned int core; // "core id"
-  const unsigned int socket; // "physical id"
+  bool operator == (const CPU& that) const
+  {
+    return (id == that.id) && (core == that.core) && (socket == that.socket);
+  }
+
+  bool operator < (const CPU& that) const
+  {
+    // Sort by (socket, core, id).
+    if (socket != that.socket) {
+      return socket < that.socket;
+    }
+
+    // On the same socket.
+    if (core != that.core) {
+      return core < that.core;
+    }
+
+    // On the same core.
+    return id < that.id;
+  }
+
+  // These are non-const because we need the default assignment operator.
+  unsigned int id; // "processor"
+  unsigned int core; // "core id"
+  unsigned int socket; // "physical id"
 };
 
 
+inline std::ostream& operator << (std::ostream& out, const CPU& cpu)
+{
+  return out << "CPU (id:" << cpu.id << ", "
+             << "core:" << cpu.core << ", "
+             << "socket:" << cpu.socket << ")";
+}
+
+
 // Snapshot of a system (modeled after /proc/stat).
 struct SystemStatistics
 {

Modified: incubator/mesos/trunk/src/slave/cgroups_isolation_module.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/cgroups_isolation_module.cpp?rev=1413852&r1=1413851&r2=1413852&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/cgroups_isolation_module.cpp (original)
+++ incubator/mesos/trunk/src/slave/cgroups_isolation_module.cpp Mon Nov 26 21:17:21 2012
@@ -16,12 +16,14 @@
  * limitations under the License.
  */
 
+#include <math.h> // For floor.
 #include <signal.h>
 #include <unistd.h>
 
 #include <sys/file.h> // For flock.
 #include <sys/types.h>
 
+#include <algorithm>
 #include <set>
 #include <sstream>
 #include <string>
@@ -32,6 +34,7 @@
 
 #include <stout/exit.hpp>
 #include <stout/foreach.hpp>
+#include <stout/hashmap.hpp>
 #include <stout/lambda.hpp>
 #include <stout/numify.hpp>
 #include <stout/option.hpp>
@@ -43,12 +46,15 @@
 #include "common/units.hpp"
 
 #include "linux/cgroups.hpp"
+#include "linux/proc.hpp"
 
 #include "slave/cgroups_isolation_module.hpp"
 
 using process::defer;
 using process::Future;
 
+using std::list;
+using std::map;
 using std::set;
 using std::string;
 using std::ostringstream;
@@ -62,6 +68,108 @@ const size_t CPU_SHARES_PER_CPU = 1024;
 const size_t MIN_CPU_SHARES = 10;
 const size_t MIN_MEMORY_MB = 32 * Megabyte;
 
+// This is an approximate double precision equality check.
+// It only considers up to 0.001 precision.
+// This is used so that we can enforce correct arithmetic on "millicpu" units.
+// TODO(bmahler): Banish this to hell when we expose individual cpus as a
+// resource to frameworks, so that we can enforce having no fractions.
+bool almostEqual(double d1, double d2) {
+  return (d1 <= (d2 + 0.001)) && (d1 >= (d2 - 0.001));
+}
+
+
+map<proc::CPU, double> Cpuset::grow(
+    double delta,
+    const map<proc::CPU, double>& usage)
+{
+  // The technique used here is to allocate as much as possible to
+  // each cpu that has availability, until we've allocated the delta.
+  // Note that we examine the cpus in the same order every time, which
+  // means we don't yet consider locality.
+  map<proc::CPU, double> allocation;
+  foreachpair (const proc::CPU& cpu, double used, usage) {
+    // Are we done allocating?
+    if (almostEqual(delta, 0.0)) {
+      break;
+    }
+
+    // Allocate as much as possible to this CPU.
+    if (!almostEqual(used, 1.0)) {
+      double free = 1.0 - used;
+      double allocated = std::min(delta, free);
+      allocation[cpu] = allocated;
+      delta -= allocated;
+      cpus[cpu] += allocated;
+    }
+  }
+
+  CHECK(almostEqual(delta, 0.0))
+    << "Failed to grow the cpuset by " << delta << " cpus\n"
+    << "  cpus: " << stringify(cpus) << "\n"
+    << "  usage: " << stringify(usage);
+
+  return allocation;
+}
+
+
+map<proc::CPU, double> Cpuset::shrink(double delta)
+{
+  // The technique used here is to free as much as possible from the
+  // least allocated cpu. This means we'll avoid fragmenting as we're
+  // constantly trying to remove cpus belonging to this Cpuset.
+  map<proc::CPU, double> deallocation;
+  while (!almostEqual(delta, 0.0)) {
+    // Find the CPU to which we have the least allocated.
+    Option<proc::CPU> least;
+    foreachpair (const proc::CPU& cpu, double used, cpus) {
+      if (least.isNone() || used <= cpus[least.get()]) {
+        least = cpu;
+      }
+    }
+
+    CHECK(least.isSome())
+      << "Failed to shrink the cpuset by " << delta << " cpus\n"
+      << "  cpus: " << stringify(cpus);
+
+    // Deallocate as much as possible from the least allocated CPU.
+    double used = cpus[least.get()];
+    double deallocated = std::min(used, delta);
+    deallocation[least.get()] = deallocated;
+    delta -= deallocated;
+    cpus[least.get()] -= deallocated;
+
+    // Ensure this Cpuset never contains unallocated CPUs.
+    if (almostEqual(cpus[least.get()], 0.0)) {
+      cpus.erase(least.get());
+    }
+  }
+
+  return deallocation;
+}
+
+
+double Cpuset::usage() const
+{
+  double total = 0.0;
+  foreachvalue (double used, cpus) {
+    total += used;
+  }
+  return total;
+}
+
+
+std::ostream& operator << (std::ostream& out, const Cpuset& cpuset)
+{
+  vector<unsigned int> cpus;
+  foreachpair (const proc::CPU& cpu, double used, cpuset.cpus) {
+    CHECK(!almostEqual(used, 0.0));
+    cpus.push_back(cpu.id);
+  }
+  std::sort(cpus.begin(), cpus.end());
+
+  return out << strings::join(",", cpus);
+}
+
 
 CgroupsIsolationModule::CgroupsIsolationModule()
   : ProcessBase(ID::generate("cgroups-isolation-module")),
@@ -87,6 +195,7 @@ CgroupsIsolationModule::~CgroupsIsolatio
 
 void CgroupsIsolationModule::initialize(
     const Flags& _flags,
+    const Resources& _resources,
     bool _local,
     const PID<Slave>& _slave)
 {
@@ -227,12 +336,81 @@ void CgroupsIsolationModule::initialize(
   CHECK(write.isSome())
     << "Failed to disable OOM killer: " << write.error();
 
+  if (subsystems.contains("cpu") && subsystems.contains("cpuset")) {
+    EXIT(1) << "The use of both 'cpu' and 'cpuset' subsystems is not allowed.\n"
+            << "Please use only one of:\n"
+            << "  cpu:    When willing to share cpus for higher efficiency.\n"
+            << "  cpuset: When cpu pinning is desired.";
+  }
+
   // Configure resource changed handlers. We only add handlers for
   // resources that have the appropriate subsystems attached.
   if (subsystems.contains("cpu")) {
     handlers["cpus"] = &CgroupsIsolationModule::cpusChanged;
   }
 
+  if (subsystems.contains("cpuset")) {
+    // TODO(bmahler): Consider making a cgroups primitive helper to perform
+    // cgroups list format -> list of ints / strings conversion.
+    hashset<unsigned int> cgroupCpus;
+    Try<string> cpuset = cgroups::read(hierarchy, "mesos", "cpuset.cpus");
+    CHECK(cpuset.isSome())
+      << "Failed to read cpuset.cpus: " << cpuset.error();
+
+    // Parse from "0-2,7,12-14" to a set(0,1,2,7,12,13,14).
+    foreach (const string& range, strings::tokenize(cpuset.get(), ",")) {
+      if (strings::contains(range, "-")) {
+        // Case startId-endId (e.g. 0-2 in 0-2,7,12-14).
+        vector<string> startEnd = strings::split(range, "-");
+        CHECK(startEnd.size() == 2)
+          << "Failed to parse cpu range '" << range
+          << "' from cpuset.cpus '" << cpuset.get() << "'";
+
+        Try<unsigned int> start = numify<unsigned int>(startEnd[0]);
+        Try<unsigned int> end = numify<unsigned int>(startEnd[1]);
+        CHECK(start.isSome() && end.isSome())
+          << "Failed to parse cpu range '" << range
+          << "' from cpuset.cpus '" << cpuset.get() << "'";
+
+        for (unsigned int i = start.get(); i <= end.get(); i++) {
+          cgroupCpus.insert(i);
+        }
+      } else {
+        // Case id (e.g. 7 in 0-2,7,12-14).
+        Try<unsigned int> cpuId = numify<unsigned int>(range);
+        CHECK(cpuId.isSome())
+          << "Failed to parse cpu '" << range << "' from cpuset.cpus '"
+          << cpuset.get()  << "': " << cpuId.error();
+        cgroupCpus.insert(cpuId.get());
+      }
+    }
+
+    Value::Scalar none;
+    Value::Scalar cpusResource = _resources.get("cpus", none);
+    if (cpusResource.value() > cgroupCpus.size()) {
+      EXIT(1) << "You have specified " << cpusResource.value() << " cpus, but "
+              << "this is more than allowed by the cgroup cpuset.cpus: "
+              << cpuset.get();
+    }
+
+    // Initialize our cpu allocations.
+    Try<list<proc::CPU> > cpus = proc::cpus();
+    CHECK(cpus.isSome())
+      << "Failed to extract CPUs from /proc/cpuinfo: " << cpus.error();
+    foreach (const proc::CPU& cpu, cpus.get()) {
+      if (this->cpus.size() >= cpusResource.value()) {
+        break;
+      }
+
+      if (cgroupCpus.contains(cpu.id)) {
+        LOG(INFO) << "Initializing cpu allocation for " << cpu;
+        this->cpus[cpu] = 0.0;
+      }
+    }
+
+    handlers["cpus"] = &CgroupsIsolationModule::cpusetChanged;
+  }
+
   if (subsystems.contains("memory")) {
     handlers["mem"] = &CgroupsIsolationModule::memChanged;
   }
@@ -441,7 +619,7 @@ void CgroupsIsolationModule::processExit
 
 
 Try<Nothing> CgroupsIsolationModule::cpusChanged(
-    const CgroupInfo* info,
+    CgroupInfo* info,
     const Resource& resource)
 {
   CHECK(resource.name() == "cpus");
@@ -469,8 +647,50 @@ Try<Nothing> CgroupsIsolationModule::cpu
 }
 
 
+Try<Nothing> CgroupsIsolationModule::cpusetChanged(
+    CgroupInfo* info,
+    const Resource& resource)
+{
+  CHECK_NOTNULL(info->cpuset);
+  CHECK(resource.name() == "cpus");
+
+  if (resource.type() != Value::SCALAR) {
+    return Try<Nothing>::error("Expecting resource 'cpus' to be a scalar");
+  }
+
+  double delta = resource.scalar().value() - info->cpuset->usage();
+
+  if (delta < 0) {
+    map<proc::CPU, double> deallocated = info->cpuset->shrink(fabs(delta));
+    foreachpair (const proc::CPU& cpu, double freed, deallocated) {
+      cpus[cpu] -= freed;
+      CHECK(cpus[cpu] > -0.001); // Check approximately >= 0.
+    }
+  } else {
+    map<proc::CPU, double> allocated = info->cpuset->grow(delta, cpus);
+    foreachpair (const proc::CPU& cpu, double used, allocated) {
+      cpus[cpu] += used;
+      CHECK(cpus[cpu] < 1.001); // Check approximately <= 1.
+    }
+  }
+
+  Try<Nothing> write = cgroups::write(
+      hierarchy, info->name(), "cpuset.cpus", stringify(*(info->cpuset)));
+  if (write.isError()) {
+    return Try<Nothing>::error(
+        "Failed to update 'cpuset.cpus': " + write.error());
+  }
+
+  LOG(INFO) << "Updated 'cpuset.cpus' to " << *(info->cpuset)
+            << " for executor " << info->executorId
+            << " of framework " << info->frameworkId;
+
+  return Nothing();
+}
+
+
 Try<Nothing> CgroupsIsolationModule::memChanged(
-    const CgroupInfo* info,
+    CgroupInfo* info,
     const Resource& resource)
 {
   CHECK(resource.name() == "mem");
@@ -668,6 +888,11 @@ CgroupsIsolationModule::CgroupInfo* Cgro
   info->killed = false;
   info->destroyed = false;
   info->reason = "";
+  if (subsystems.contains("cpuset")) {
+    info->cpuset = new Cpuset();
+  } else {
+    info->cpuset = NULL;
+  }
   infos[frameworkId][executorId] = info;
   return info;
 }

Modified: incubator/mesos/trunk/src/slave/cgroups_isolation_module.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/cgroups_isolation_module.hpp?rev=1413852&r1=1413851&r2=1413852&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/cgroups_isolation_module.hpp (original)
+++ incubator/mesos/trunk/src/slave/cgroups_isolation_module.hpp Mon Nov 26 21:17:21 2012
@@ -19,6 +19,7 @@
 #ifndef __CGROUPS_ISOLATION_MODULE_HPP__
 #define __CGROUPS_ISOLATION_MODULE_HPP__
 
+#include <map>
 #include <sstream>
 #include <string>
 
@@ -32,6 +33,8 @@
 
 #include "launcher/launcher.hpp"
 
+#include "linux/proc.hpp"
+
 #include "slave/flags.hpp"
 #include "slave/isolation_module.hpp"
 #include "slave/reaper.hpp"
@@ -41,6 +44,34 @@ namespace mesos {
 namespace internal {
 namespace slave {
 
+// TODO(bmahler): Migrate this into it's own file, along with moving
+// all cgroups code inside of a 'cgroups' directory.
+class Cpuset
+{
+public:
+  // Grows this cpu set by the provided delta.
+  // @param   delta   Amount of cpus to grow by.
+  // @param   usage   Cpu usage, as allocated by the cgroups isolation module.
+  // @return  The new cpu allocations made by this Cpuset.
+  std::map<proc::CPU, double> grow(
+      double delta,
+      const std::map<proc::CPU, double>& usage);
+
+  // Shrinks this cpu set by the provided delta.
+  // @param   delta   Amount of cpus to shrink by.
+  // @return  The new cpu deallocations made by this Cpuset.
+  std::map<proc::CPU, double> shrink(double delta);
+
+  // @return The total cpu usage across all the cpus in this Cpuset.
+  double usage() const;
+
+  friend std::ostream& operator << (std::ostream& out, const Cpuset& cpuset);
+
+private:
+  std::map<proc::CPU, double> cpus; // CPU id -> % allocated.
+};
+
+
 class CgroupsIsolationModule
   : public IsolationModule,
     public ProcessExitedListener
@@ -52,6 +83,7 @@ public:
 
   virtual void initialize(
       const Flags& flags,
+      const Resources& resources,
       bool local,
       const process::PID<Slave>& slave);
 
@@ -81,6 +113,14 @@ private:
   // The cgroup information for each live executor.
   struct CgroupInfo
   {
+    ~CgroupInfo()
+    {
+      if (cpuset != NULL) {
+        delete cpuset;
+        cpuset = NULL;
+      }
+    }
+
     // Returns the canonicalized name of the cgroup in the filesystem.
     std::string name() const
     {
@@ -113,24 +153,34 @@ private:
 
     // Used to cancel the OOM listening.
     process::Future<uint64_t> oomNotifier;
+
+    // CPUs allocated if using 'cpuset' subsystem.
+    Cpuset* cpuset;
   };
 
   // The callback which will be invoked when "cpus" resource has changed.
-  // @param   frameworkId   The id of the given framework.
-  // @param   executorId    The id of the given executor.
+  // @param   info          The Cgroup information.
   // @param   resources     The handle for the resources.
-  // @return  Whether the operation successes.
+  // @return  Whether the operation succeeds.
   Try<Nothing> cpusChanged(
-      const CgroupInfo* info,
+      CgroupInfo* info,
+      const Resource& resource);
+
+  // The callback which will be invoked when "cpus" resource has changed.
+  // This is only invoked when we are using the cpuset subsystem.
+  // @param   info          The Cgroup information.
+  // @param   resources     The handle for the resources.
+  // @return  Whether the operation succeeds.
+  Try<Nothing> cpusetChanged(
+      CgroupInfo* info,
       const Resource& resource);
 
   // The callback which will be invoked when "mem" resource has changed.
-  // @param   frameworkId   The id of the given framework.
-  // @param   executorId    The id of the given executor.
+  // @param   info          The Cgroup information.
   // @param   resources     The handle for the resources.
-  // @return  Whether the operation successes.
+  // @return  Whether the operation succeeds.
   Try<Nothing> memChanged(
-      const CgroupInfo* info,
+      CgroupInfo* info,
       const Resource& resource);
 
   // Start listening on OOM events. This function will create an eventfd and
@@ -210,10 +260,13 @@ private:
   // The cgroups subsystems being used.
   hashset<std::string> subsystems;
 
+  // Allocated cpus (if using cpuset subsystem).
+  std::map<proc::CPU, double> cpus;
+
   // Handlers for each resource name, used for resource changes.
   hashmap<std::string,
           Try<Nothing>(CgroupsIsolationModule::*)(
-              const CgroupInfo*,
+              CgroupInfo*,
               const Resource&)> handlers;
 };
 

Modified: incubator/mesos/trunk/src/slave/isolation_module.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/isolation_module.hpp?rev=1413852&r1=1413851&r2=1413852&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/isolation_module.hpp (original)
+++ incubator/mesos/trunk/src/slave/isolation_module.hpp Mon Nov 26 21:17:21 2012
@@ -47,6 +47,7 @@ public:
 
   // Called during slave initialization.
   virtual void initialize(const Flags& flags,
+                          const Resources& resources,
                           bool local,
                           const process::PID<Slave>& slave) = 0;
 

Modified: incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp?rev=1413852&r1=1413851&r2=1413852&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp (original)
+++ incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp Mon Nov 26 21:17:21 2012
@@ -84,6 +84,7 @@ LxcIsolationModule::~LxcIsolationModule(
 
 void LxcIsolationModule::initialize(
     const Flags& _flags,
+    const Resources& _,
     bool _local,
     const PID<Slave>& _slave)
 {

Modified: incubator/mesos/trunk/src/slave/lxc_isolation_module.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/lxc_isolation_module.hpp?rev=1413852&r1=1413851&r2=1413852&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/lxc_isolation_module.hpp (original)
+++ incubator/mesos/trunk/src/slave/lxc_isolation_module.hpp Mon Nov 26 21:17:21 2012
@@ -41,6 +41,7 @@ public:
   virtual ~LxcIsolationModule();
 
   virtual void initialize(const Flags& flags,
+                          const Resources& resources,
                           bool local,
                           const process::PID<Slave>& slave);
 

Modified: incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp?rev=1413852&r1=1413851&r2=1413852&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp (original)
+++ incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp Mon Nov 26 21:17:21 2012
@@ -73,6 +73,7 @@ ProcessBasedIsolationModule::~ProcessBas
 
 void ProcessBasedIsolationModule::initialize(
     const Flags& _flags,
+    const Resources& _,
     bool _local,
     const PID<Slave>& _slave)
 {

Modified: incubator/mesos/trunk/src/slave/process_based_isolation_module.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/process_based_isolation_module.hpp?rev=1413852&r1=1413851&r2=1413852&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/process_based_isolation_module.hpp (original)
+++ incubator/mesos/trunk/src/slave/process_based_isolation_module.hpp Mon Nov 26 21:17:21 2012
@@ -45,6 +45,7 @@ public:
   virtual ~ProcessBasedIsolationModule();
 
   virtual void initialize(const Flags& flags,
+                          const Resources& resources,
                           bool local,
                           const process::PID<Slave>& slave);
 

Modified: incubator/mesos/trunk/src/slave/slave.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.cpp?rev=1413852&r1=1413851&r2=1413852&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.cpp (original)
+++ incubator/mesos/trunk/src/slave/slave.cpp Mon Nov 26 21:17:21 2012
@@ -196,7 +196,10 @@ void Slave::initialize()
   spawn(isolationModule);
   dispatch(isolationModule,
            &IsolationModule::initialize,
-           flags, local, self());
+           flags,
+           resources,
+           local,
+           self());
 
   // Start disk monitoring.
   // NOTE: We send a delayed message here instead of directly calling

Modified: incubator/mesos/trunk/src/tests/utils.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/utils.hpp?rev=1413852&r1=1413851&r2=1413852&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/utils.hpp (original)
+++ incubator/mesos/trunk/src/tests/utils.hpp Mon Nov 26 21:17:21 2012
@@ -40,6 +40,7 @@
 #include <stout/stringify.hpp>
 #include <stout/try.hpp>
 
+#include "common/resources.hpp"
 #include "common/type_utils.hpp"
 
 #include "logging/logging.hpp"
@@ -610,6 +611,7 @@ public:
   virtual ~TestingIsolationModule() {}
 
   virtual void initialize(const slave::Flags& flags,
+                          const Resources& resources,
                           bool local,
                           const process::PID<slave::Slave>& _slave)
   {

Modified: incubator/mesos/trunk/third_party/libprocess/include/stout/stringify.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/stout/stringify.hpp?rev=1413852&r1=1413851&r2=1413852&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/stout/stringify.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/stout/stringify.hpp Mon Nov 26 21:17:21 2012
@@ -11,6 +11,7 @@
 #include <string>
 #include <vector>
 
+#include "hashmap.hpp"
 
 template <typename T>
 std::string stringify(T t)
@@ -90,9 +91,28 @@ std::string stringify(const std::map<K, 
   out << "{ ";
   typename std::map<K, V>::const_iterator iterator = map.begin();
   while (iterator != map.end()) {
-    out << stringify(*iterator.first);
+    out << stringify(iterator->first);
     out << ": ";
-    out << stringify(*iterator.second);
+    out << stringify(iterator->second);
+    if (++iterator != map.end()) {
+      out << ", ";
+    }
+  }
+  out << " }";
+  return out.str();
+}
+
+
+template <typename K, typename V>
+std::string stringify(const hashmap<K, V>& map)
+{
+  std::ostringstream out;
+  out << "{ ";
+  typename hashmap<K, V>::const_iterator iterator = map.begin();
+  while (iterator != map.end()) {
+    out << stringify(iterator->first);
+    out << ": ";
+    out << stringify(iterator->second);
     if (++iterator != map.end()) {
       out << ", ";
     }