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/09/19 02:08:10 UTC

svn commit: r1387408 - in /incubator/mesos/trunk: src/common/ src/files/ src/local/ src/master/ src/slave/ src/tests/ src/webui/master/static/ third_party/libprocess/ third_party/libprocess/include/stout/

Author: benh
Date: Wed Sep 19 00:08:08 2012
New Revision: 1387408

URL: http://svn.apache.org/viewvc?rev=1387408&view=rev
Log:
Added more webui stats, as well as table sorting (contributed by Ben
Mahler, https://reviews.apache.org/r/6739).

Added:
    incubator/mesos/trunk/third_party/libprocess/include/stout/fs.hpp
Modified:
    incubator/mesos/trunk/src/common/attributes.cpp
    incubator/mesos/trunk/src/common/attributes.hpp
    incubator/mesos/trunk/src/common/resources.cpp
    incubator/mesos/trunk/src/common/resources.hpp
    incubator/mesos/trunk/src/common/values.cpp
    incubator/mesos/trunk/src/common/values.hpp
    incubator/mesos/trunk/src/files/files.cpp
    incubator/mesos/trunk/src/local/local.cpp
    incubator/mesos/trunk/src/local/local.hpp
    incubator/mesos/trunk/src/master/http.cpp
    incubator/mesos/trunk/src/slave/http.cpp
    incubator/mesos/trunk/src/slave/slave.cpp
    incubator/mesos/trunk/src/tests/exception_tests.cpp
    incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp
    incubator/mesos/trunk/src/tests/resource_offers_tests.cpp
    incubator/mesos/trunk/src/webui/master/static/controllers.js
    incubator/mesos/trunk/src/webui/master/static/dashboard.html
    incubator/mesos/trunk/src/webui/master/static/framework.html
    incubator/mesos/trunk/src/webui/master/static/frameworks.html
    incubator/mesos/trunk/src/webui/master/static/home.html
    incubator/mesos/trunk/src/webui/master/static/index.html
    incubator/mesos/trunk/src/webui/master/static/mesos.css
    incubator/mesos/trunk/src/webui/master/static/pailer.html
    incubator/mesos/trunk/src/webui/master/static/slave.html
    incubator/mesos/trunk/src/webui/master/static/slaves.html
    incubator/mesos/trunk/third_party/libprocess/Makefile.am
    incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp
    incubator/mesos/trunk/third_party/libprocess/include/stout/strings.hpp

Modified: incubator/mesos/trunk/src/common/attributes.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/attributes.cpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/attributes.cpp (original)
+++ incubator/mesos/trunk/src/common/attributes.cpp Wed Sep 19 00:08:08 2012
@@ -32,6 +32,25 @@ using std::vector;
 
 
 namespace mesos {
+
+
+std::ostream& operator << (std::ostream& stream, const Attribute& attribute)
+{
+  stream << attribute.name() << "=";
+  switch (attribute.type()) {
+    case Value::SCALAR: stream << attribute.scalar(); break;
+    case Value::RANGES: stream << attribute.ranges(); break;
+    case Value::SET:    stream << attribute.set();    break;
+    case Value::TEXT:   stream << attribute.text();   break;
+    default:
+      LOG(FATAL) << "Unexpected Value type: " << attribute.type();
+      break;
+  }
+
+  return stream;
+}
+
+
 namespace internal {
 
 Attribute Attributes::parse(const std::string& name, const std::string& text)

Modified: incubator/mesos/trunk/src/common/attributes.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/attributes.hpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/attributes.hpp (original)
+++ incubator/mesos/trunk/src/common/attributes.hpp Wed Sep 19 00:08:08 2012
@@ -32,6 +32,11 @@
 #include "logging/logging.hpp"
 
 namespace mesos {
+
+
+std::ostream& operator << (std::ostream& stream, const Attribute& attribute);
+
+
 namespace internal {
 
 

Modified: incubator/mesos/trunk/src/common/resources.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/resources.cpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/resources.cpp (original)
+++ incubator/mesos/trunk/src/common/resources.cpp Wed Sep 19 00:08:08 2012
@@ -147,28 +147,14 @@ Resource& operator -= (Resource& left, c
 ostream& operator << (ostream& stream, const Resource& resource)
 {
   stream << resource.name() << "=";
-  if (resource.type() == Value::SCALAR) {
-    stream << resource.scalar().value();
-  } else if (resource.type() == Value::RANGES) {
-    stream << "[";
-    for (int i = 0; i < resource.ranges().range_size(); i++) {
-      stream << resource.ranges().range(i).begin()
-             << "-"
-             << resource.ranges().range(i).end();
-      if (i + 1 < resource.ranges().range_size()) {
-        stream << ", ";
-      }
-    }
-    stream << "]";
-  } else if (resource.type() == Value::SET) {
-    stream << "{";
-    for (int i = 0; i < resource.set().item_size(); i++) {
-      stream << resource.set().item(i);
-      if (i + 1 < resource.set().item_size()) {
-        stream << ", ";
-      }
-    }
-    stream << "}";
+
+  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 stream;

Modified: incubator/mesos/trunk/src/common/resources.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/resources.hpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/resources.hpp (original)
+++ incubator/mesos/trunk/src/common/resources.hpp Wed Sep 19 00:08:08 2012
@@ -300,7 +300,7 @@ public:
     } else if (resource.type() == Value::RANGES) {
       return resource.has_ranges();
     } else if (resource.type() == Value::SET) {
-      return resource.has_ranges();
+      return resource.has_set();
     } else if (resource.type() == Value::TEXT) {
       // Resources doesn't support text.
       return false;

Modified: incubator/mesos/trunk/src/common/values.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/values.cpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/values.cpp (original)
+++ incubator/mesos/trunk/src/common/values.cpp Wed Sep 19 00:08:08 2012
@@ -134,6 +134,12 @@ Try<Value> parse(const std::string& text
 } // namespace internal {
 
 
+ostream& operator << (ostream& stream, const Value::Scalar& scalar)
+{
+  return stream << scalar.value();
+}
+
+
 bool operator == (const Value::Scalar& left, const Value::Scalar& right)
 {
   return left.value() == right.value();
@@ -287,6 +293,20 @@ static void remove(Value::Ranges* ranges
 }
 
 
+ostream& operator << (ostream& stream, const Value::Ranges& ranges)
+{
+  stream << "[";
+  for (int i = 0; i < ranges.range_size(); i++) {
+    stream << ranges.range(i).begin() << "-" << ranges.range(i).end();
+    if (i + 1 < ranges.range_size()) {
+      stream << ", ";
+    }
+  }
+  stream << "]";
+  return stream;
+}
+
+
 bool operator == (const Value::Ranges& _left, const Value::Ranges& _right)
 {
   Value::Ranges left;
@@ -403,6 +423,20 @@ Value::Ranges& operator -= (Value::Range
 }
 
 
+ostream& operator << (ostream& stream, const Value::Set& set)
+{
+  stream << "{";
+  for (int i = 0; i < set.item_size(); i++) {
+    stream << set.item(i);
+    if (i + 1 < set.item_size()) {
+      stream << ", ";
+    }
+  }
+  stream << "}";
+  return stream;
+}
+
+
 bool operator == (const Value::Set& left, const Value::Set& right)
 {
   if (left.item_size() == right.item_size()) {
@@ -544,6 +578,13 @@ Value::Set& operator -= (Value::Set& lef
   return left;
 }
 
+
+ostream& operator << (ostream& stream, const Value::Text& value)
+{
+  return stream << value.value();
+}
+
+
 bool operator == (const Value::Text& left, const Value::Text& right)
 {
   return left.value() == right.value();

Modified: incubator/mesos/trunk/src/common/values.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/values.hpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/values.hpp (original)
+++ incubator/mesos/trunk/src/common/values.hpp Wed Sep 19 00:08:08 2012
@@ -25,6 +25,7 @@
 
 namespace mesos {
 
+std::ostream& operator << (std::ostream& stream, const Value::Scalar& scalar);
 bool operator == (const Value::Scalar& left, const Value::Scalar& right);
 bool operator <= (const Value::Scalar& left, const Value::Scalar& right);
 Value::Scalar operator + (const Value::Scalar& left, const Value::Scalar& right);
@@ -32,6 +33,7 @@ Value::Scalar operator - (const Value::S
 Value::Scalar& operator += (Value::Scalar& left, const Value::Scalar& right);
 Value::Scalar& operator -= (Value::Scalar& left, const Value::Scalar& right);
 
+std::ostream& operator << (std::ostream& stream, const Value::Ranges& ranges);
 bool operator == (const Value::Ranges& left, const Value::Ranges& right);
 bool operator <= (const Value::Ranges& left, const Value::Ranges& right);
 Value::Ranges operator + (const Value::Ranges& left, const Value::Ranges& right);
@@ -39,6 +41,7 @@ Value::Ranges operator - (const Value::R
 Value::Ranges& operator += (Value::Ranges& left, const Value::Ranges& right);
 Value::Ranges& operator -= (Value::Ranges& left, const Value::Ranges& right);
 
+std::ostream& operator << (std::ostream& stream, const Value::Set& set);
 bool operator == (const Value::Set& left, const Value::Set& right);
 bool operator <= (const Value::Set& left, const Value::Set& right);
 Value::Set operator + (const Value::Set& left, const Value::Set& right);
@@ -46,6 +49,7 @@ Value::Set operator - (const Value::Set&
 Value::Set& operator += (Value::Set& left, const Value::Set& right);
 Value::Set& operator -= (Value::Set& left, const Value::Set& right);
 
+std::ostream& operator << (std::ostream& stream, const Value::Text& value);
 bool operator == (const Value::Text& left, const Value::Text& right);
 
 namespace internal {

Modified: incubator/mesos/trunk/src/files/files.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/files/files.cpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/files/files.cpp (original)
+++ incubator/mesos/trunk/src/files/files.cpp Wed Sep 19 00:08:08 2012
@@ -139,7 +139,7 @@ Future<Response> FilesProcess::browse(co
   // The result will be a sorted (on path) array of files and dirs:
   // [{"name": "README", "path": "dir/README" "dir":False, "size":42}, ...]
   map<string, JSON::Object> files;
-  foreach(const string& filename, os::ls(resolvedPath.get())) {
+  foreach (const string& filename, os::ls(resolvedPath.get())) {
     struct stat s;
     string fullPath = path::join(resolvedPath.get(), filename);
 

Modified: incubator/mesos/trunk/src/local/local.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/local/local.cpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/local/local.cpp (original)
+++ incubator/mesos/trunk/src/local/local.cpp Wed Sep 19 00:08:08 2012
@@ -68,8 +68,9 @@ static Files* files = NULL;
 
 
 PID<Master> launch(int numSlaves,
-                   int32_t cpus,
-                   int64_t mem,
+                   double cpus,
+                   uint64_t mem,
+                   uint64_t disk,
                    bool quiet,
                    AllocatorProcess* _allocator)
 {
@@ -79,7 +80,7 @@ PID<Master> launch(int numSlaves,
   configuration.set("quiet", quiet);
 
   stringstream out;
-  out << "cpus:" << cpus << ";" << "mem:" << mem;
+  out << "cpus:" << cpus << ";" << "mem:" << mem << ";" << "disk:" << disk;
   configuration.set("resources", out.str());
 
   return launch(configuration, _allocator);

Modified: incubator/mesos/trunk/src/local/local.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/local/local.hpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/local/local.hpp (original)
+++ incubator/mesos/trunk/src/local/local.hpp Wed Sep 19 00:08:08 2012
@@ -34,8 +34,9 @@ namespace local {
 // of CPUs and memory per slave. Additionally one can also toggle whether
 // to initialize Google Logging and whether to log quietly.
 process::PID<master::Master> launch(int numSlaves,
-                                    int32_t cpus,
-                                    int64_t mem,
+                                    double cpus,
+                                    uint64_t mem,
+                                    uint64_t disk,
                                     bool quiet,
                                     master::AllocatorProcess* _allocator = NULL);
 

Modified: incubator/mesos/trunk/src/master/http.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/http.cpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/http.cpp (original)
+++ incubator/mesos/trunk/src/master/http.cpp Wed Sep 19 00:08:08 2012
@@ -30,6 +30,7 @@
 #include <stout/result.hpp>
 #include <stout/strings.hpp>
 
+#include "common/attributes.hpp"
 #include "common/build.hpp"
 #include "common/resources.hpp"
 #include "common/type_utils.hpp"
@@ -64,14 +65,52 @@ using std::vector;
 // Returns a JSON object modeled on a Resources.
 JSON::Object model(const Resources& resources)
 {
-  // TODO(benh): Add all of the resources.
-  Value::Scalar none;
-  Value::Scalar cpus = resources.get("cpus", none);
-  Value::Scalar mem = resources.get("mem", none);
+  JSON::Object object;
+
+  foreach (const Resource& resource, resources) {
+    switch (resource.type()) {
+      case Value::SCALAR:
+        object.values[resource.name()] = resource.scalar().value();
+        break;
+      case Value::RANGES:
+        object.values[resource.name()] = stringify(resource.ranges());
+        break;
+      case Value::SET:
+        object.values[resource.name()] = stringify(resource.set());
+        break;
+      default:
+        LOG(FATAL) << "Unexpected Value type: " << resource.type();
+        break;
+    }
+  }
+
+  return object;
+}
+
 
+JSON::Object model(const Attributes& attributes)
+{
   JSON::Object object;
-  object.values["cpus"] = cpus.value();
-  object.values["mem"] = mem.value();
+
+  foreach (const Attribute& attribute, attributes) {
+    switch (attribute.type()) {
+      case Value::SCALAR:
+        object.values[attribute.name()] = attribute.scalar().value();
+        break;
+      case Value::RANGES:
+        object.values[attribute.name()] = stringify(attribute.ranges());
+        break;
+      case Value::SET:
+        object.values[attribute.name()] = stringify(attribute.set());
+        break;
+      case Value::TEXT:
+        object.values[attribute.name()] = attribute.text().value();
+        break;
+      default:
+        LOG(FATAL) << "Unexpected Value type: " << attribute.type();
+        break;
+    }
+  }
 
   return object;
 }
@@ -165,6 +204,7 @@ JSON::Object model(const Slave& slave)
   object.values["webui_port"] = slave.info.webui_port();
   object.values["registered_time"] = slave.registeredTime;
   object.values["resources"] = model(slave.info.resources());
+  object.values["attributes"] = model(slave.info.attributes());
   return object;
 }
 
@@ -276,6 +316,14 @@ Future<Response> state(
   object.values["start_time"] = master.startTime;
   object.values["id"] = master.info.id();
   object.values["pid"] = string(master.self());
+  object.values["activated_slaves"] = master.slaveHostnamePorts.size();
+  object.values["connected_slaves"] = master.slaves.size();
+  object.values["staged_tasks"] = master.stats.tasks[TASK_STAGING];
+  object.values["started_tasks"] = master.stats.tasks[TASK_STARTING];
+  object.values["finished_tasks"] = master.stats.tasks[TASK_FINISHED];
+  object.values["killed_tasks"] = master.stats.tasks[TASK_KILLED];
+  object.values["failed_tasks"] = master.stats.tasks[TASK_FAILED];
+  object.values["lost_tasks"] = master.stats.tasks[TASK_LOST];
 
   if (master.flags.cluster.isSome()) {
     object.values["cluster"] = master.flags.cluster.get();

Modified: incubator/mesos/trunk/src/slave/http.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/http.cpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/http.cpp (original)
+++ incubator/mesos/trunk/src/slave/http.cpp Wed Sep 19 00:08:08 2012
@@ -23,9 +23,11 @@
 
 #include <stout/foreach.hpp>
 #include <stout/json.hpp>
+#include <stout/numify.hpp>
 #include <stout/stringify.hpp>
 #include <stout/strings.hpp>
 
+#include "common/attributes.hpp"
 #include "common/build.hpp"
 #include "common/resources.hpp"
 #include "common/type_utils.hpp"
@@ -55,14 +57,52 @@ using std::vector;
 // Returns a JSON object modeled on a Resources.
 JSON::Object model(const Resources& resources)
 {
-  // TODO(benh): Add all of the resources.
-  Value::Scalar none;
-  Value::Scalar cpus = resources.get("cpus", none);
-  Value::Scalar mem = resources.get("mem", none);
+  JSON::Object object;
+
+  foreach (const Resource& resource, resources) {
+    switch (resource.type()) {
+      case Value::SCALAR:
+        object.values[resource.name()] = resource.scalar().value();
+        break;
+      case Value::RANGES:
+        object.values[resource.name()] = stringify(resource.ranges());
+        break;
+      case Value::SET:
+        object.values[resource.name()] = stringify(resource.set());
+        break;
+      default:
+        LOG(FATAL) << "Unexpected Value type: " << resource.type();
+        break;
+    }
+  }
+
+  return object;
+}
+
 
+JSON::Object model(const Attributes& attributes)
+{
   JSON::Object object;
-  object.values["cpus"] = cpus.value();
-  object.values["mem"] = mem.value();
+
+  foreach (const Attribute& attribute, attributes) {
+    switch (attribute.type()) {
+      case Value::SCALAR:
+        object.values[attribute.name()] = attribute.scalar().value();
+        break;
+      case Value::RANGES:
+        object.values[attribute.name()] = stringify(attribute.ranges());
+        break;
+      case Value::SET:
+        object.values[attribute.name()] = stringify(attribute.set());
+        break;
+      case Value::TEXT:
+        object.values[attribute.name()] = attribute.text().value();
+        break;
+      default:
+        LOG(FATAL) << "Unexpected Value type: " << attribute.type();
+        break;
+    }
+  }
 
   return object;
 }
@@ -179,6 +219,7 @@ Future<Response> state(
   object.values["id"] = slave.id.value();
   object.values["pid"] = string(slave.self());
   object.values["resources"] = model(slave.resources);
+  object.values["attributes"] = model(slave.attributes);
 
   if (slave.flags.log_dir.isSome()) {
     object.values["log_dir"] = slave.flags.log_dir.get();

Modified: incubator/mesos/trunk/src/slave/slave.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.cpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.cpp (original)
+++ incubator/mesos/trunk/src/slave/slave.cpp Wed Sep 19 00:08:08 2012
@@ -28,6 +28,7 @@
 #include <process/id.hpp>
 
 #include <stout/duration.hpp>
+#include <stout/fs.hpp>
 #include <stout/option.hpp>
 #include <stout/os.hpp>
 #include <stout/path.hpp>
@@ -115,9 +116,10 @@ Slave::Slave(const flags::Flags<logging:
     files(_files)
 {
   if (flags.resources.isNone()) {
-    // TODO(benh): Move this compuation into Flags as the "default".
+    // TODO(benh): Move this computation into Flags as the "default".
     Try<long> cpus = os::cpus();
-    Try<long> mem = os::memory();
+    Try<uint64_t> mem = os::memory();
+    Try<uint64_t> disk = fs::available();
 
     if (!cpus.isSome()) {
       LOG(WARNING) << "Failed to auto-detect the number of cpus to use,"
@@ -128,7 +130,7 @@ Slave::Slave(const flags::Flags<logging:
     if (!mem.isSome()) {
       LOG(WARNING) << "Failed to auto-detect the size of main memory,"
                    << " defaulting to 1024 MB";
-      mem = Try<long>::some(1024);
+      mem = Try<uint64_t>::some(1024);
     } else {
       // Convert to MB.
       mem = mem.get() / 1048576;
@@ -137,12 +139,28 @@ Slave::Slave(const flags::Flags<logging:
       // TODO(benh): Have better default scheme (e.g., % of mem not
       // greater than 1 GB?)
       if (mem.get() > 1024) {
-        mem = Try<long>::some(mem.get() - 1024);
+        mem = Try<uint64_t>::some(mem.get() - 1024);
       }
     }
 
-    Try<string> defaults =
-      strings::format("cpus:%d;mem:%d", cpus.get(), mem.get());
+    if (!disk.isSome()) {
+      LOG(WARNING) << "Failed to auto-detect the free disk space,"
+                   << " defaulting to 10 GB";
+      disk = Try<uint64_t>::some(1024 * 10);
+    } else {
+      // Convert to MB.
+      disk = disk.get() / 1048576;
+
+      // Leave 5 GB free if we have more than 10 GB, otherwise, use all!
+      // TODO(benh): Have better default scheme (e.g., % of disk not
+      // greater than 10 GB?)
+      if (disk.get() > 1024 * 10) {
+        disk = Try<uint64_t>::some(disk.get() - (1024 * 5));
+      }
+    }
+
+    Try<string> defaults = strings::format(
+        "cpus:%d;mem:%d;disk:%d", cpus.get(), mem.get(), disk.get());
 
     CHECK(defaults.isSome());
 

Modified: incubator/mesos/trunk/src/tests/exception_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/exception_tests.cpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/exception_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/exception_tests.cpp Wed Sep 19 00:08:08 2012
@@ -67,7 +67,7 @@ TEST(ExceptionTest, DeactiveFrameworkOnA
   EXPECT_MESSAGE(filter, _, _, _)
     .WillRepeatedly(Return(false));
 
-  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, false);
+  PID<Master> master = local::launch(1, 2, 1 * Gigabyte,1 * Gigabyte,  false);
 
   MockScheduler sched;
 
@@ -108,7 +108,7 @@ TEST(ExceptionTest, DisallowSchedulerAct
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, false);
+  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched;
 
@@ -148,7 +148,7 @@ TEST(ExceptionTest, DisallowSchedulerCal
   EXPECT_MESSAGE(filter, _, _, _)
     .WillRepeatedly(Return(false));
 
-  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, false);
+  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched;
 

Modified: incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp Wed Sep 19 00:08:08 2012
@@ -149,7 +149,7 @@ TEST(FaultToleranceTest, SlavePartitione
   EXPECT_MESSAGE(filter, Eq("PONG"), _, _)
     .WillRepeatedly(Return(true));
 
-  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, false);
+  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(&sched, DEFAULT_FRAMEWORK_INFO, master);
@@ -201,7 +201,7 @@ TEST(FaultToleranceTest, SchedulerFailov
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, false);
+  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   // Launch the first (i.e., failing) scheduler and wait until
   // registered gets called to launch the second (i.e., failover)
@@ -279,7 +279,7 @@ TEST(FaultToleranceTest, FrameworkReliab
   EXPECT_MESSAGE(filter, _, _, _)
     .WillRepeatedly(Return(false));
 
-  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, false);
+  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(&sched, DEFAULT_FRAMEWORK_INFO, master);
@@ -332,7 +332,7 @@ TEST(FaultToleranceTest, FrameworkReregi
   EXPECT_MESSAGE(filter, _, _, _)
     .WillRepeatedly(Return(false));
 
-  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, false);
+  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(&sched, DEFAULT_FRAMEWORK_INFO, master);

Modified: incubator/mesos/trunk/src/tests/resource_offers_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/resource_offers_tests.cpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/resource_offers_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/resource_offers_tests.cpp Wed Sep 19 00:08:08 2012
@@ -61,7 +61,7 @@ TEST(ResourceOffersTest, ResourceOfferWi
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  PID<Master> master = local::launch(10, 2, 1 * Gigabyte, false);
+  PID<Master> master = local::launch(10, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(&sched, DEFAULT_FRAMEWORK_INFO, master);
@@ -103,7 +103,7 @@ TEST(ResourceOffersTest, TaskUsesNoResou
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, false);
+  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(&sched, DEFAULT_FRAMEWORK_INFO, master);
@@ -163,7 +163,7 @@ TEST(ResourceOffersTest, TaskUsesInvalid
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, false);
+  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(&sched, DEFAULT_FRAMEWORK_INFO, master);
@@ -228,7 +228,7 @@ TEST(ResourceOffersTest, TaskUsesMoreRes
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, false);
+  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(&sched, DEFAULT_FRAMEWORK_INFO, master);
@@ -293,7 +293,7 @@ TEST(ResourceOffersTest, ResourcesGetReo
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, false);
+  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched1;
   MesosSchedulerDriver driver1(&sched1, DEFAULT_FRAMEWORK_INFO, master);
@@ -353,7 +353,7 @@ TEST(ResourceOffersTest, ResourcesGetReo
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, false);
+  PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched1;
   MesosSchedulerDriver driver1(&sched1, DEFAULT_FRAMEWORK_INFO, master);
@@ -474,7 +474,7 @@ TEST(ResourceOffersTest, Request)
     .WillRepeatedly(Return());
 
   PID<Master> master = local::launch(
-      1, 2, 1 * Gigabyte, false, &allocator);
+      1, 2, 1 * Gigabyte, 1 * Gigabyte, false, &allocator);
 
   MesosSchedulerDriver driver(&sched, DEFAULT_FRAMEWORK_INFO, master);
 

Modified: incubator/mesos/trunk/src/webui/master/static/controllers.js
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/static/controllers.js?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/controllers.js (original)
+++ incubator/mesos/trunk/src/webui/master/static/controllers.js Wed Sep 19 00:08:08 2012
@@ -1,5 +1,21 @@
 'use strict';
 
+var slaves = [];
+
+
+// Table Object.
+//   selected_column: column predicate for the selected column.
+//   reverse:         boolean indicating sort order.
+function Table(selected_column) {
+  if (this instanceof Table) {
+    this.selected_column = selected_column;
+    this.reverse = true;
+  } else {
+    return new Table(selected_column);
+  }
+}
+
+
 function hasSelectedText () {
   if (window.getSelection) {  // All browsers except IE before version 9.
     var range = window.getSelection();
@@ -9,6 +25,42 @@ function hasSelectedText () {
 }
 
 
+// Returns a curried function for returning the HTML 'class=' tag
+// attribute value for sorting table columns in the provided scope.
+function columnClass($scope) {
+  // For the given table column, this behaves as follows:
+  // Column unselected            : ''
+  // Column selected / descending : 'descending'
+  // Column selected / ascending  : 'ascending'
+  return function(table, column) {
+    if ($scope.tables[table].selected_column === column) {
+      if ($scope.tables[table].reverse) {
+        return 'descending';
+      } else {
+        return 'ascending';
+      }
+    }
+    return '';
+  }
+}
+
+
+// Returns a curried function to be called when a table column is clicked
+// in the provided scope.
+function selectColumn($scope) {
+  // Assigns the given table column as the sort column, flipping the
+  // sort order if the sort column has not changed.
+  return function(table, column) {
+    if ($scope.tables[table].selected_column === column) {
+      $scope.tables[table].reverse = !$scope.tables[table].reverse;
+    } else {
+      $scope.tables[table].reverse = true;
+    }
+    $scope.tables[table].selected_column = column;
+  }
+}
+
+
 // Update the outermost scope with the new state.
 function update($scope, $defer, data) {
   // Don't do anything if the data hasn't changed.
@@ -86,6 +138,16 @@ function update($scope, $defer, data) {
   $scope.offered_cpus = 0;
   $scope.offered_mem = 0;
 
+  $scope.staged_tasks = $scope.state.staged_tasks;
+  $scope.started_tasks = $scope.state.started_tasks;
+  $scope.finished_tasks = $scope.state.finished_tasks;
+  $scope.killed_tasks = $scope.state.killed_tasks;
+  $scope.failed_tasks = $scope.state.failed_tasks;
+  $scope.lost_tasks = $scope.state.lost_tasks;
+
+  $scope.activated_slaves = $scope.state.activated_slaves;
+  $scope.connected_slaves = $scope.state.connected_slaves;
+
   _.each($scope.slaves, function(slave) {
     $scope.total_cpus += slave.resources.cpus;
     $scope.total_mem += slave.resources.mem;
@@ -94,6 +156,8 @@ function update($scope, $defer, data) {
   _.each($scope.frameworks, function(framework) {
       $scope.used_cpus += framework.resources.cpus;
       $scope.used_mem += framework.resources.mem;
+      $scope.active_tasks += framework.tasks.length;
+      $scope.completed_tasks += framework.completed_tasks.length;
 
       framework.cpus_share = 0;
       if ($scope.total_cpus > 0) {
@@ -111,6 +175,8 @@ function update($scope, $defer, data) {
   _.each($scope.offers, function(offer) {
     $scope.offered_cpus += offer.resources.cpus;
     $scope.offered_mem += offer.resources.mem;
+    offer.framework_name = $scope.frameworks[offer.framework_id].name;
+    offer.hostname = $scope.slaves[offer.slave_id].hostname;
   });
 
   $scope.used_cpus -= $scope.offered_cpus;
@@ -137,6 +203,11 @@ function MainCntl($scope, $http, $route,
   $("#loading").hide();
   $("#navbar").show();
 
+  // Adding bindings into scope so that they can be used from within
+  // AngularJS expressions.
+  $scope._ = _;
+  $scope.stringify = JSON.stringify;
+
   // Initialize popovers and bind the function used to show a popover.
   Popovers.initialize();
   $scope.popover = Popovers.show;
@@ -197,6 +268,15 @@ function MainCntl($scope, $http, $route,
 function HomeCtrl($scope) {
   setNavbarActiveTab('home');
 
+  $scope.tables = {};
+  $scope.tables['frameworks'] = new Table('id');
+  $scope.tables['slaves'] = new Table('id');
+  $scope.tables['offers'] = new Table('id');
+  $scope.tables['completed_frameworks'] = new Table('id');
+
+  $scope.columnClass = columnClass($scope);
+  $scope.selectColumn = selectColumn($scope);
+
   $scope.log = function($event) {
     if (!$scope.state.log_dir) {
       $('#no-log-dir-modal').modal('show');
@@ -217,7 +297,7 @@ function HomeCtrl($scope) {
 
 function DashboardCtrl($scope) {
   setNavbarActiveTab('dashboard');
-  
+
   var context = cubism.context()
     .step(1000)
     .size(1440);
@@ -235,12 +315,25 @@ function DashboardCtrl($scope) {
 
 function FrameworksCtrl($scope) {
   setNavbarActiveTab('frameworks');
+
+  $scope.tables = {};
+  $scope.tables['frameworks'] = new Table('id');
+
+  $scope.columnClass = columnClass($scope);
+  $scope.selectColumn = selectColumn($scope);
 }
 
 
 function FrameworkCtrl($scope, $routeParams) {
   setNavbarActiveTab('frameworks');
 
+  $scope.tables = {};
+  $scope.tables['active_tasks'] = new Table('id');
+  $scope.tables['completed_tasks'] = new Table('id');
+
+  $scope.columnClass = columnClass($scope);
+  $scope.selectColumn = selectColumn($scope);
+
   var update = function() {
     if ($routeParams.id in $scope.completed_frameworks) {
       $scope.framework = $scope.completed_frameworks[$routeParams.id];
@@ -267,6 +360,12 @@ function FrameworkCtrl($scope, $routePar
 
 function SlavesCtrl($scope) {
   setNavbarActiveTab('slaves');
+
+  $scope.tables = {};
+  $scope.tables['slaves'] = new Table('id');
+
+  $scope.columnClass = columnClass($scope);
+  $scope.selectColumn = selectColumn($scope);
 }
 
 

Modified: incubator/mesos/trunk/src/webui/master/static/dashboard.html
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/static/dashboard.html?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/dashboard.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/dashboard.html Wed Sep 19 00:08:08 2012
@@ -11,4 +11,4 @@
 
 <!-- <div class="well" id="graph"></div> -->
 
-<div id="graphs"></div>
+<div id="graphs"></div>
\ No newline at end of file

Modified: incubator/mesos/trunk/src/webui/master/static/framework.html
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/static/framework.html?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/framework.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/framework.html Wed Sep 19 00:08:08 2012
@@ -62,14 +62,26 @@
     <table class="table table-striped table-bordered table-condensed">
       <thead>
         <tr>
-          <th>ID</th>
-          <th>Name</th>
-          <th>State</th>
-          <th>Host</th>
+          <th ng-class="columnClass('active_tasks', 'id')"
+              ng-click="selectColumn('active_tasks', 'id')">
+            <a href="">ID</a>
+          </th>
+          <th ng-class="columnClass('active_tasks', 'name')"
+              ng-click="selectColumn('active_tasks', 'name')">
+            <a href="">Name</a>
+          </th>
+          <th ng-class="columnClass('active_tasks', 'state')"
+              ng-click="selectColumn('active_tasks', 'state')">
+            <a href="">State</a>
+          </th>
+          <th ng-class="columnClass('active_tasks', 'host')"
+              ng-click="selectColumn('active_tasks', 'host')">
+            <a href="">Host</a>
+          </th>
         </tr>
       </thead>
       <tbody>
-        <tr ng-repeat="task in framework.tasks">
+        <tr ng-repeat="task in framework.tasks | orderBy:tables['active_tasks'].selected_column:tables['active_tasks'].reverse">
           <td>{{task.id}}</td>
           <td>{{task.name}}</td>
           <td>{{task.state | truncateMesosState}}</td>
@@ -82,18 +94,30 @@
       </tbody>
     </table>
 
-    <h2 id="frameworks">Terminated Tasks</h2>
+    <h2 id="frameworks">Completed Tasks</h2>
     <table class="table table-striped table-bordered table-condensed">
       <thead>
         <tr>
-          <th>ID</th>
-          <th>Name</th>
-          <th>State</th>
-          <th>Host</th>
+          <th ng-class="columnClass('completed_tasks', 'id')"
+              ng-click="selectColumn('completed_tasks', 'id')">
+            <a href="">ID</a>
+          </th>
+          <th ng-class="columnClass('completed_tasks', 'name')"
+              ng-click="selectColumn('completed_tasks', 'name')">
+            <a href="">Name</a>
+          </th>
+          <th ng-class="columnClass('completed_tasks', 'state')"
+              ng-click="selectColumn('completed_tasks', 'state')">
+            <a href="">State</a>
+          </th>
+          <th ng-class="columnClass('completed_tasks', 'host')"
+              ng-click="selectColumn('completed_tasks', 'host')">
+            <a href="">Host</a>
+          </th>
         </tr>
       </thead>
       <tbody>
-        <tr ng-repeat="task in framework.completed_tasks">
+        <tr ng-repeat="task in framework.completed_tasks | orderBy:tables['completed_tasks'].selected_column:tables['completed_tasks'].reverse">
           <td>{{task.id}}</td>
           <td>{{task.name}}</td>
           <td>{{task.state | truncateMesosState}}</td>
@@ -106,4 +130,4 @@
       </tbody>
     </table>
   </div>
-</div>
+</div>
\ No newline at end of file

Modified: incubator/mesos/trunk/src/webui/master/static/frameworks.html
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/static/frameworks.html?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/frameworks.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/frameworks.html Wed Sep 19 00:08:08 2012
@@ -13,19 +13,46 @@
     <table class="table table-striped table-bordered table-condensed">
       <thead>
         <tr>
-          <th>ID</th>
-          <th>User</th>
-          <th>Name</th>
-          <th>Active Tasks</th>
-          <th>CPUs</th>
-          <th>Mem</th>
-          <th>Max Share</th>
-          <th>Registered</th>
-          <th>Reregistered</th>
+          <th ng-class="columnClass('frameworks', 'id')"
+              ng-click="selectColumn('frameworks', 'id')">
+            <a href="">ID</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'user')"
+              ng-click="selectColumn('frameworks', 'user')">
+            <a href="">User</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'name')"
+              ng-click="selectColumn('frameworks', 'name')">
+            <a href="">Name</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'tasks.length')"
+              ng-click="selectColumn('frameworks', 'tasks.length')">
+            <a href="">Active Tasks</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'resources.cpus')"
+              ng-click="selectColumn('frameworks', 'resources.cpus')">
+            <a href="">CPUs</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'resources.mem')"
+              ng-click="selectColumn('frameworks', 'resources.mem')">
+            <a href="">Mem</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'max_share')"
+              ng-click="selectColumn('frameworks', 'max_share')">
+            <a href="">Max Share</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'registered_time')"
+              ng-click="selectColumn('frameworks', 'registered_time')">
+            <a href="">Registered</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'reregistered_time')"
+              ng-click="selectColumn('frameworks', 'reregistered_time')">
+            <a href="">Re-Registered</a>
+          </th>
         </tr>
       </thead>
       <tbody>
-        <tr ng-repeat="framework in frameworks">
+        <tr ng-repeat="framework in _.values(frameworks) | orderBy:tables['frameworks'].selected_column:tables['frameworks'].reverse">
           <td>
             <a href="" rel="popover" ng-click="popover($event, 'bottom')" data-content="{{framework.id}}" data-original-title="ID">
               {{framework.id | truncateMesosID}}
@@ -51,4 +78,4 @@
       </tbody>
     </table>
   </div>
-</div>
+</div>
\ No newline at end of file

Modified: incubator/mesos/trunk/src/webui/master/static/home.html
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/static/home.html?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/home.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/home.html Wed Sep 19 00:08:08 2012
@@ -40,13 +40,58 @@
 
       <p><a href="" ng-click="log($event)">LOG</a></p>
 
+      <h2>Slaves</h2>
+      <table class="table table-condensed">
+        <tbody>
+          <tr>
+            <th>Connected</th>
+            <td>{{connected_slaves | number}}</td>
+          </tr>
+          <tr>
+            <th>Activated</th>
+            <td>{{activated_slaves | number}}</td>
+          </tr>
+        </tbody>
+      </table>
+
+      <h2>Tasks</h2>
+      <table class="table table-condensed">
+
+        <tbody>
+          <tr>
+            <th>Staged</th>
+            <td>{{staged_tasks | number}}</td>
+          </tr>
+          <tr>
+            <th>Started</th>
+            <td>{{started_tasks | number}}</td>
+          </tr>
+          <tr>
+            <th>Finished</th>
+            <td>{{finished_tasks | number}}</td>
+          </tr>
+          <tr>
+            <th>Killed</th>
+            <td>{{killed_tasks | number}}</td>
+          </tr>
+          <tr>
+            <th>Failed</th>
+            <td>{{failed_tasks | number}}</td>
+          </tr>
+          <tr>
+            <th>Lost</th>
+            <td>{{lost_tasks | number}}</td>
+          </tr>
+        </tbody>
+      </table>
+
       <h2>Resources</h2>
       <table class="table table-condensed">
         <thead>
           <tr>
             <th></th>
             <th>CPUs</th>
-            <th>Memory</th>
+            <th>Mem</th>
           </tr>
         </thead>
         <tbody>
@@ -72,9 +117,10 @@
           </tr>
         </tbody>
       </table>
+
     </div>
   </div>
-  
+
   <div class="span9">
     <h2>
       Active Frameworks
@@ -83,19 +129,46 @@
     <table class="table table-striped table-bordered table-condensed">
       <thead>
         <tr>
-          <th>ID</th>
-          <th>User</th>
-          <th>Name</th>
-          <th>Active Tasks</th>
-          <th>CPUs</th>
-          <th>Mem</th>
-          <th>Max Share</th>
-          <th>Registered</th>
-          <th>Reregistered</th>
+          <th ng-class="columnClass('frameworks', 'id')"
+              ng-click="selectColumn('frameworks', 'id')">
+            <a href="">ID</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'user')"
+              ng-click="selectColumn('frameworks', 'user')">
+            <a href="">User</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'name')"
+              ng-click="selectColumn('frameworks', 'name')">
+            <a href="">Name</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'tasks.length')"
+              ng-click="selectColumn('frameworks', 'tasks.length')">
+            <a href="">Active Tasks</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'resources.cpus')"
+              ng-click="selectColumn('frameworks', 'resources.cpus')">
+            <a href="">CPUs</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'resources.mem')"
+              ng-click="selectColumn('frameworks', 'resources.mem')">
+            <a href="">Mem</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'max_share')"
+              ng-click="selectColumn('frameworks', 'max_share')">
+            <a href="">Max Share</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'registered_time')"
+              ng-click="selectColumn('frameworks', 'registered_time')">
+            <a href="">Registered</a>
+          </th>
+          <th ng-class="columnClass('frameworks', 'reregistered_time')"
+              ng-click="selectColumn('frameworks', 'reregistered_time')">
+            <a href="">Re-Registered</a>
+          </th>
         </tr>
       </thead>
       <tbody>
-        <tr ng-repeat="framework in frameworks">
+        <tr ng-repeat="framework in _.values(frameworks) | orderBy:tables['frameworks'].selected_column:tables['frameworks'].reverse">
           <td>
             <a href="" rel="popover" ng-click="popover($event, 'bottom')" data-content="{{framework.id}}" data-original-title="ID">
               {{framework.id | truncateMesosID}}
@@ -121,7 +194,6 @@
       </tbody>
     </table>
 
-    
     <h2>
       Active Slaves
       <small>(<a href="#/slaves">see all</a>)</small>
@@ -129,14 +201,33 @@
     <table class="table table-striped table-bordered table-condensed">
       <thead>
         <tr>
-          <th>ID</th>
-          <th>Host</th>
-          <th>CPUs</th>
-          <th>Mem</th>
-          <th>Registered</th>
+          <th ng-class="columnClass('slaves', 'id')"
+              ng-click="selectColumn('slaves', 'id')">
+            <a href="">ID</a>
+          </th>
+          <th ng-class="columnClass('slaves', 'hostname')"
+              ng-click="selectColumn('slaves', 'hostname')">
+            <a href="">Host</a>
+          </th>
+          <th ng-class="columnClass('slaves', 'resources.cpus')"
+              ng-click="selectColumn('slaves', 'resources.cpus')">
+            <a href="">CPUs</a>
+          </th>
+          <th ng-class="columnClass('slaves', 'resources.mem')"
+              ng-click="selectColumn('slaves', 'resources.mem')">
+            <a href="">Mem</a>
+          </th>
+          <th ng-class="columnClass('slaves', 'resources.disk')"
+              ng-click="selectColumn('slaves', 'resources.disk')">
+            <a href="">Disk</a>
+          </th>
+          <th ng-class="columnClass('slaves', 'registered_time')"
+              ng-click="selectColumn('slaves', 'registered_time')">
+            <a href="">Registered</a>
+          </th>
         </tr>
       </thead>
-      <tr ng-repeat="slave in slaves">
+      <tr ng-repeat="slave in _.values(slaves) | orderBy:tables['slaves'].selected_column:tables['slaves'].reverse">
         <td>
           <a href="" rel="popover" ng-click="popover($event, 'bottom')" data-content="{{slave.id}}" data-original-title="ID">
             {{slave.id | truncateMesosID}}
@@ -148,6 +239,7 @@
           </a>
         <td>{{slave.resources.cpus | number}}</td>
         <td>{{slave.resources.mem / 1024 | number}} GB</td>
+        <td>{{slave.resources.disk / 1024 | number}} GB</td>
         <td>
           <a href="" rel="popover" ng-click="popover($event, 'bottom')" data-content="{{slave.registered_time * 1000 | mesosDate}}" data-original-title="Registered">
             {{slave.registered_time * 1000 | relativeDate}}
@@ -155,27 +247,42 @@
         </td>
       </tr>
     </table>
-    
+
     <h2>Offers</h2>
     <table class="table table-striped table-bordered table-condensed">
       <thead>
         <tr>
-          <th>ID</th>
-          <th>Framework</th>
-          <th>Host</th>
-          <th>CPUs</th>
-          <th>Mem</th>
+          <th ng-class="columnClass('offers', 'id')"
+              ng-click="selectColumn('offers', 'id')">
+            <a href="">ID</a>
+          </th>
+          <th ng-class="columnClass('offers', 'framework_name')"
+              ng-click="selectColumn('offers', 'framework_name')">
+            <a href="">Framework</a>
+          </th>
+          <th ng-class="columnClass('offers', 'hostname')"
+              ng-click="selectColumn('offers', 'hostname')">
+            <a href="">Host</a>
+          </th>
+          <th ng-class="columnClass('offers', 'resources.cpus')"
+              ng-click="selectColumn('offers', 'resources.cpus')">
+            <a href="">CPUs</a>
+          </th>
+          <th ng-class="columnClass('offers', 'resources.mem')"
+              ng-click="selectColumn('offers', 'resources.mem')">
+            <a href="">Mem</a>
+          </th>
         </tr>
       </thead>
       <tbody>
-        <tr ng-repeat="offer in offers">
+        <tr ng-repeat="offer in _.values(offers) | orderBy:tables['offers'].selected_column:tables['offers'].reverse">
           <td>
             <abbr title="{{offer.id}}">{{offer.id | truncateMesosID}}</abbr>
           </td>
-          <td><a href="{{'#/framework/' + offer.framework_id}}">{{frameworks[offer.framework_id].name}}</a></td>
+          <td><a href="{{'#/framework/' + offer.framework_id}}">{{offer.framework_name}}</a></td>
           <td>
             <a href="http://{{slaves[offer.slave_id].webui_hostname}}:{{slaves[offer.slave_id].webui_port}}/">
-              {{slaves[offer.slave_id].hostname}}
+              {{offer.hostname}}
             </a>
           </td>
           <td>{{offer.resources.cpus | number}}</td>
@@ -188,15 +295,30 @@
     <table class="table table-striped table-bordered table-condensed">
       <thead>
         <tr>
-          <th>ID</th>
-          <th>User</th>
-          <th>Name</th>
-          <th>Registered</th>
-          <th>Unregistered</th>
+          <th ng-class="columnClass('completed_frameworks', 'id')"
+              ng-click="selectColumn('completed_frameworks', 'id')">
+            <a href="">ID</a>
+          </th>
+          <th ng-class="columnClass('completed_frameworks', 'user')"
+              ng-click="selectColumn('completed_frameworks', 'user')">
+            <a href="">User</a>
+          </th>
+          <th ng-class="columnClass('completed_frameworks', 'name')"
+              ng-click="selectColumn('completed_frameworks', 'name')">
+            <a href="">Name</a>
+          </th>
+          <th ng-class="columnClass('completed_frameworks', 'registered_time')"
+              ng-click="selectColumn('completed_frameworks', 'registered_time')">
+            <a href="">Registered</a>
+          </th>
+          <th ng-class="columnClass('completed_frameworks', 'unregistered_time')"
+              ng-click="selectColumn('completed_frameworks', 'unregistered_time')">
+            <a href="">Unregistered</a>
+          </th>
         </tr>
       </thead>
       <tbody>
-        <tr ng-repeat="framework in completed_frameworks">
+        <tr ng-repeat="framework in _.values(completed_frameworks) | orderBy:tables['completed_frameworks'].selected_column:tables['completed_frameworks'].reverse">
           <td>
             <abbr title="{{framework.id}}">{{framework.id | truncateMesosID}}</abbr>
           </td>
@@ -217,4 +339,4 @@
     </table>
 
   </div>
-</div>
+</div>
\ No newline at end of file

Modified: incubator/mesos/trunk/src/webui/master/static/index.html
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/static/index.html?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/index.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/index.html Wed Sep 19 00:08:08 2012
@@ -111,4 +111,4 @@
     <script src="/static/dashboard.js"></script>
     <script src="/static/popovers.js"></script>
   </body>
-</html>
+</html>
\ No newline at end of file

Modified: incubator/mesos/trunk/src/webui/master/static/mesos.css
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/static/mesos.css?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/mesos.css (original)
+++ incubator/mesos/trunk/src/webui/master/static/mesos.css Wed Sep 19 00:08:08 2012
@@ -16,4 +16,14 @@ dl.inline dd + dd {
 
 dl.inline dt {
   font-weight: bold
-}
\ No newline at end of file
+}
+
+th.descending:after {
+  padding-left: 5px;
+  content: "▼";
+}
+
+th.ascending:after {
+  padding-left: 5px;
+  content: "▲";
+}

Modified: incubator/mesos/trunk/src/webui/master/static/pailer.html
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/static/pailer.html?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/pailer.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/pailer.html Wed Sep 19 00:08:08 2012
@@ -72,4 +72,4 @@
       });
     </script>
   </body>
-</html>
+</html>
\ No newline at end of file

Modified: incubator/mesos/trunk/src/webui/master/static/slave.html
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/static/slave.html?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/slave.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/slave.html Wed Sep 19 00:08:08 2012
@@ -47,4 +47,4 @@
       <p><a href="" ng-click="log($event)">LOG</a></p>
     </div>
   </div>
-</div>
+</div>
\ No newline at end of file

Modified: incubator/mesos/trunk/src/webui/master/static/slaves.html
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/static/slaves.html?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/slaves.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/slaves.html Wed Sep 19 00:08:08 2012
@@ -11,14 +11,33 @@
 <table class="table table-striped table-bordered table-condensed">
   <thead>
     <tr>
-      <th>ID</th>
-      <th>Host</th>
-      <th>CPUs</th>
-      <th>Mem</th>
-      <th>Registered</th>
+      <th ng-class="columnClass('slaves', 'id')"
+          ng-click="selectColumn('slaves', 'id')">
+        <a href="">ID</a>
+      </th>
+      <th ng-class="columnClass('slaves', 'hostname')"
+          ng-click="selectColumn('slaves', 'hostname')">
+        <a href="">Host</a>
+      </th>
+      <th ng-class="columnClass('slaves', 'resources.cpus')"
+          ng-click="selectColumn('slaves', 'resources.cpus')">
+        <a href="">CPUs</a>
+      </th>
+          <th ng-class="columnClass('slaves', 'resources.mem')"
+              ng-click="selectColumn('slaves', 'resources.mem')">
+        <a href="">Mem</a>
+      </th>
+      <th ng-class="columnClass('slaves', 'resources.disk')"
+          ng-click="selectColumn('slaves', 'resources.disk')">
+        <a href="">Disk</a>
+      </th>
+      <th ng-class="columnClass('slaves', 'registered_time')"
+          ng-click="selectColumn('slaves', 'registered_time')">
+        <a href="">Registered</a>
+      </th>
     </tr>
   </thead>
-  <tr ng-repeat="slave in state.slaves">
+  <tr ng-repeat="slave in _.values(slaves) | orderBy:tables['slaves'].selected_column:tables['slaves'].reverse">
     <td>
       <a href="" rel="popover" ng-click="popover($event, 'bottom')" data-content="{{slave.id}}" data-original-title="ID">
         {{slave.id | truncateMesosID}}
@@ -27,10 +46,11 @@
     <td><a href="#/slave/{{slave.id}}">{{slave.hostname}}</a></td>
     <td>{{slave.resources.cpus | number}}</td>
     <td>{{slave.resources.mem / 1024 | number}} GB</td>
+    <td>{{slave.resources.disk / 1024 | number}} GB</td>
     <td>
       <a href="" rel="popover" ng-click="popover($event, 'bottom')" data-content="{{slave.registered_time * 1000 | mesosDate}}" data-original-title="Registered">
         {{slave.registered_time * 1000 | relativeDate}}
       </a>
     </td>
   </tr>
-</table>
+</table>
\ No newline at end of file

Modified: incubator/mesos/trunk/third_party/libprocess/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/Makefile.am?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/Makefile.am (original)
+++ incubator/mesos/trunk/third_party/libprocess/Makefile.am Wed Sep 19 00:08:08 2012
@@ -70,6 +70,7 @@ libprocess_la_SOURCES += $(top_srcdir)/i
 	$(top_srcdir)/include/stout/fatal.hpp				\
 	$(top_srcdir)/include/stout/foreach.hpp				\
 	$(top_srcdir)/include/stout/format.hpp				\
+	$(top_srcdir)/include/stout/fs.hpp				\
 	$(top_srcdir)/include/stout/hashmap.hpp				\
 	$(top_srcdir)/include/stout/hashset.hpp				\
 	$(top_srcdir)/include/stout/json.hpp				\

Added: incubator/mesos/trunk/third_party/libprocess/include/stout/fs.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/stout/fs.hpp?rev=1387408&view=auto
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/stout/fs.hpp (added)
+++ incubator/mesos/trunk/third_party/libprocess/include/stout/fs.hpp Wed Sep 19 00:08:08 2012
@@ -0,0 +1,30 @@
+#ifndef __STOUT_FS_HPP__
+#define __STOUT_FS_HPP__
+
+#include <errno.h>
+
+#include <sys/statvfs.h>
+
+#include <cstring>
+#include <string>
+
+#include "try.hpp"
+
+// TODO(bmahler): Migrate the appropriate 'os' namespace funtions here.
+namespace fs {
+
+
+inline Try<uint64_t> available(const std::string& path = "/")
+{
+  struct statvfs buf;
+
+  if (statvfs(path.c_str(), &buf) < 0) {
+    return Try<uint64_t>::error(strerror(errno));
+  }
+  return buf.f_bavail * buf.f_frsize;
+}
+
+
+} // namespace fs {
+
+#endif // __STOUT_FS_HPP__

Modified: incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/stout/os.hpp Wed Sep 19 00:08:08 2012
@@ -734,16 +734,16 @@ inline Try<long> cpus()
 
 
 // Returns the total size of main memory in bytes.
-inline Try<long> memory()
+inline Try<uint64_t> memory()
 {
 #ifdef __linux__
   struct sysinfo info;
   if (sysinfo(&info) != 0) {
-    return Try<long>::error(strerror(errno));
+    return Try<uint64_t>::error(strerror(errno));
   }
   return info.totalram;
 #else
-  return Try<long>::error("Cannot determine the size of main memory");
+  return Try<uint64_t>::error("Cannot determine the size of main memory");
 #endif
 }
 

Modified: incubator/mesos/trunk/third_party/libprocess/include/stout/strings.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/stout/strings.hpp?rev=1387408&r1=1387407&r2=1387408&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/stout/strings.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/stout/strings.hpp Wed Sep 19 00:08:08 2012
@@ -140,6 +140,23 @@ inline std::map<std::string, std::vector
 }
 
 
+// Returns a string which is the concatenation of the strings in
+// items, the separator is inserted between elements.
+inline std::string join(const std::vector<std::string>& items,
+                        const std::string& separator = "") {
+  if (items.empty()) {
+    return "";
+  }
+
+  std::string result = items[0];
+  for (size_t i = 1; i < items.size(); ++i) {
+    result.append(separator);
+    result.append(items[i]);
+  }
+  return result;
+}
+
+
 inline bool checkBracketsMatching(
     const std::string& s,
     const char openBracket,