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/10/27 01:41:07 UTC

svn commit: r1402718 - in /incubator/mesos/trunk/src: master/ messages/ slave/ webui/master/static/

Author: benh
Date: Fri Oct 26 23:41:06 2012
New Revision: 1402718

URL: http://svn.apache.org/viewvc?rev=1402718&view=rev
Log:
Exposing completed frameworks, executors, tasks, and their sandboxes,
in the webui.

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

Modified:
    incubator/mesos/trunk/src/master/constants.hpp
    incubator/mesos/trunk/src/master/http.cpp
    incubator/mesos/trunk/src/master/master.cpp
    incubator/mesos/trunk/src/master/master.hpp
    incubator/mesos/trunk/src/messages/messages.proto
    incubator/mesos/trunk/src/slave/constants.hpp
    incubator/mesos/trunk/src/slave/http.cpp
    incubator/mesos/trunk/src/slave/slave.cpp
    incubator/mesos/trunk/src/slave/slave.hpp
    incubator/mesos/trunk/src/webui/master/static/browse.html
    incubator/mesos/trunk/src/webui/master/static/controllers.js
    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/slave.html
    incubator/mesos/trunk/src/webui/master/static/slave_executor.html
    incubator/mesos/trunk/src/webui/master/static/slave_framework.html

Modified: incubator/mesos/trunk/src/master/constants.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/constants.hpp?rev=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/constants.hpp (original)
+++ incubator/mesos/trunk/src/master/constants.hpp Fri Oct 26 23:41:06 2012
@@ -51,11 +51,11 @@ const uint32_t MAX_SLAVE_PING_TIMEOUTS =
 
 // Maximum number of completed frameworks to store in the cache.
 // TODO(thomasm): Make configurable.
-const uint32_t MAX_COMPLETED_FRAMEWORKS = 100;
+const uint32_t MAX_COMPLETED_FRAMEWORKS = 50;
 
 // Maximum number of completed tasks per framework to store in the
 // cache.  TODO(thomasm): Make configurable.
-const uint32_t MAX_COMPLETED_TASKS_PER_FRAMEWORK = 500;
+const uint32_t MAX_COMPLETED_TASKS_PER_FRAMEWORK = 1000;
 
 // Time interval to check for updated watchers list.
 const Duration WHITELIST_WATCH_INTERVAL = Seconds(5.0);

Modified: incubator/mesos/trunk/src/master/http.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/http.cpp?rev=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/http.cpp (original)
+++ incubator/mesos/trunk/src/master/http.cpp Fri Oct 26 23:41:06 2012
@@ -123,6 +123,7 @@ JSON::Object model(const Task& task)
   object.values["id"] = task.task_id().value();
   object.values["name"] = task.name();
   object.values["framework_id"] = task.framework_id().value();
+  object.values["executor_id"] = task.executor_id().value();
   object.values["slave_id"] = task.slave_id().value();
   object.values["state"] = TaskState_Name(task.state());
   object.values["resources"] = model(task.resources());

Modified: incubator/mesos/trunk/src/master/master.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/master.cpp?rev=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/master.cpp (original)
+++ incubator/mesos/trunk/src/master/master.cpp Fri Oct 26 23:41:06 2012
@@ -1722,7 +1722,6 @@ void Master::removeFramework(Framework* 
   framework->unregisteredTime = Clock::now();
 
   completedFrameworks.push_back(*framework);
-
   if (completedFrameworks.size() > MAX_COMPLETED_FRAMEWORKS) {
     completedFrameworks.pop_front();
   }

Modified: incubator/mesos/trunk/src/master/master.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/master.hpp?rev=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/master.hpp (original)
+++ incubator/mesos/trunk/src/master/master.hpp Fri Oct 26 23:41:06 2012
@@ -19,6 +19,7 @@
 #ifndef __MASTER_HPP__
 #define __MASTER_HPP__
 
+#include <list>
 #include <string>
 #include <vector>
 
@@ -373,7 +374,7 @@ struct Slave
 };
 
 
-// An connected framework.
+// Information about a connected or completed framework.
 struct Framework
 {
   Framework(const FrameworkInfo& _info,

Modified: incubator/mesos/trunk/src/messages/messages.proto
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/messages/messages.proto?rev=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/messages/messages.proto (original)
+++ incubator/mesos/trunk/src/messages/messages.proto Fri Oct 26 23:41:06 2012
@@ -34,6 +34,10 @@ package mesos.internal;
 // way). Also, one performance reason why we don't do that now is
 // because storing whatever data is coupled with a TaskInfo
 // could be large and unnecessary.
+// TODO(bmahler): Add executor_uuid here, and send it to the master. This will
+// allow us to expose executor work directories for tasks in the webui when
+// looking from the master level. Currently only the slave knows which run the
+// task belongs to.
 message Task {
   required string name = 1;
   required TaskID task_id = 2;

Modified: incubator/mesos/trunk/src/slave/constants.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/constants.hpp?rev=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/constants.hpp (original)
+++ incubator/mesos/trunk/src/slave/constants.hpp Fri Oct 26 23:41:06 2012
@@ -30,6 +30,15 @@ const Duration STATUS_UPDATE_RETRY_INTER
 const Duration GC_DELAY = Weeks(1.0);
 const Duration DISK_WATCH_INTERVAL = Minutes(1.0);
 
+// Maximum number of completed frameworks to store in memory.
+const uint32_t MAX_COMPLETED_FRAMEWORKS = 50;
+
+// Maximum number of completed executors per framework to store in memory.
+const uint32_t MAX_COMPLETED_EXECUTORS_PER_FRAMEWORK = 150;
+
+// Maximum number of completed tasks per executor to store in memeory.
+const uint32_t MAX_COMPLETED_TASKS_PER_EXECUTOR = 200;
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

Modified: incubator/mesos/trunk/src/slave/http.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/http.cpp?rev=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/http.cpp (original)
+++ incubator/mesos/trunk/src/slave/http.cpp Fri Oct 26 23:41:06 2012
@@ -154,48 +154,66 @@ JSON::Object model(const ExecutorInfo& e
 }
 
 
+JSON::Object model(const Task& task)
+{
+  JSON::Object object;
+  object.values["id"] = task.task_id().value();
+  object.values["name"] = task.name();
+  object.values["executor_id"] = task.executor_id().value();
+  object.values["framework_id"] = task.framework_id().value();
+  object.values["slave_id"] = task.slave_id().value();
+  object.values["state"] = TaskState_Name(task.state());
+  object.values["resources"] = model(task.resources());
+  return object;
+}
+
+
+JSON::Object model(const TaskInfo& task)
+{
+  JSON::Object object;
+  object.values["id"] = task.task_id().value();
+  object.values["name"] = task.name();
+  object.values["slave_id"] = task.slave_id().value();
+  object.values["resources"] = model(task.resources());
+  object.values["data"] = task.data();
+
+  if (task.has_command()) {
+    object.values["command"] = model(task.command());
+  }
+  if (task.has_executor()) {
+    object.values["executor_id"] = model(task.executor());
+  }
+
+  return object;
+}
+
+
 JSON::Object model(const Executor& executor)
 {
   JSON::Object object;
   object.values["id"] = executor.id.value();
+  object.values["uuid"] = executor.uuid.toString();
   object.values["directory"] = executor.directory;
   object.values["resources"] = model(executor.resources);
 
   JSON::Array tasks;
   foreachvalue (Task* task, executor.launchedTasks) {
-    JSON::Object object;
-    object.values["id"] = task->task_id().value();
-    object.values["name"] = task->name();
-    object.values["executor_id"] = task->executor_id().value();
-    object.values["framework_id"] = task->framework_id().value();
-    object.values["slave_id"] = task->slave_id().value();
-    object.values["state"] = TaskState_Name(task->state());
-    object.values["resources"] = model(task->resources());
-
-    tasks.values.push_back(object);
+    tasks.values.push_back(model(*task));
   }
   object.values["tasks"] = tasks;
 
   JSON::Array queued;
   foreachvalue (const TaskInfo& task, executor.queuedTasks) {
-    JSON::Object object;
-    object.values["id"] = task.task_id().value();
-    object.values["name"] = task.name();
-    object.values["slave_id"] = task.slave_id().value();
-    object.values["resources"] = model(task.resources());
-    object.values["data"] = task.data();
-
-    if (task.has_command()) {
-      object.values["command"] = model(task.command());
-    }
-    if (task.has_executor()) {
-      object.values["executor_id"] = model(task.executor());
-    }
-
-    queued.values.push_back(object);
+    queued.values.push_back(model(task));
   }
   object.values["queued_tasks"] = queued;
 
+  JSON::Array completedTasks;
+  foreach (const Task& task, executor.completedTasks) {
+    completedTasks.values.push_back(model(task));
+  }
+  object.values["completed_tasks"] = completedTasks;
+
   return object;
 }
 
@@ -208,14 +226,17 @@ JSON::Object model(const Framework& fram
   object.values["name"] = framework.info.name();
   object.values["user"] = framework.info.user();
 
-  JSON::Array array;
-
-  // Model all of the executors.
+  JSON::Array executors;
   foreachvalue (Executor* executor, framework.executors) {
-    array.values.push_back(model(*executor));
+    executors.values.push_back(model(*executor));
   }
+  object.values["executors"] = executors;
 
-  object.values["executors"] = array;
+  JSON::Array completedExecutors;
+  foreach (const Executor& executor, framework.completedExecutors) {
+    completedExecutors.values.push_back(model(executor));
+  }
+  object.values["completed_executors"] = completedExecutors;
 
   return object;
 }
@@ -302,14 +323,18 @@ Future<Response> state(
     object.values["log_dir"] = slave.flags.log_dir.get();
   }
 
-  JSON::Array array;
-
-  // Model all of the frameworks.
+  JSON::Array frameworks;
   foreachvalue (Framework* framework, slave.frameworks) {
-    array.values.push_back(model(*framework));
+    frameworks.values.push_back(model(*framework));
+  }
+  object.values["frameworks"] = frameworks;
+
+  JSON::Array completedFrameworks;
+  foreach (const Framework& framework, slave.completedFrameworks) {
+    completedFrameworks.values.push_back(model(framework));
   }
+  object.values["completed_frameworks"] = completedFrameworks;
 
-  object.values["frameworks"] = array;
   return OK(object, request.query.get("jsonp"));
 }
 

Modified: incubator/mesos/trunk/src/slave/slave.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.cpp?rev=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.cpp (original)
+++ incubator/mesos/trunk/src/slave/slave.cpp Fri Oct 26 23:41:06 2012
@@ -540,15 +540,7 @@ void Slave::runTask(const FrameworkInfo&
     // Launch an executor for this task.
     executor = framework->createExecutor(id, executorInfo);
 
-    // NOTE: This constant "virtual path" format is shared with the webui.
-    // TODO(bmahler): Pass this to the webui explicitly via the existing JSON.
-    string attached = strings::format(
-        "/slaves/%s/frameworks/%s/executors/%s",
-        id.value(),
-        framework->id.value(),
-        executorId.value()).get();
-
-    files->attach(executor->directory, attached)
+    files->attach(executor->directory, executor->directory)
       .onAny(defer(self(),
                    &Self::fileAttached,
                    params::_1,
@@ -741,6 +733,12 @@ void Slave::statusUpdateAcknowledgement(
       // Cleanup if this framework has no executors running and no pending updates.
       if (framework->executors.size() == 0 && framework->updates.empty()) {
         frameworks.erase(framework->id);
+
+        completedFrameworks.push_back(*framework);
+        if (completedFrameworks.size() > MAX_COMPLETED_FRAMEWORKS) {
+          completedFrameworks.pop_front();
+        }
+
         delete framework;
       }
     }
@@ -1147,6 +1145,12 @@ void Slave::shutdownExecutorTimeout(cons
   // Cleanup if this framework has no executors running.
   if (framework->executors.size() == 0) {
     frameworks.erase(framework->id);
+
+    completedFrameworks.push_back(*framework);
+    if (completedFrameworks.size() > MAX_COMPLETED_FRAMEWORKS) {
+      completedFrameworks.pop_front();
+    }
+
     delete framework;
   }
 }

Modified: incubator/mesos/trunk/src/slave/slave.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.hpp?rev=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.hpp (original)
+++ incubator/mesos/trunk/src/slave/slave.hpp Fri Oct 26 23:41:06 2012
@@ -19,6 +19,9 @@
 #ifndef __SLAVE_HPP__
 #define __SLAVE_HPP__
 
+#include <list>
+#include <string>
+
 #include <process/http.hpp>
 #include <process/process.hpp>
 #include <process/protobuf.hpp>
@@ -193,6 +196,7 @@ private:
   Attributes attributes;
 
   hashmap<FrameworkID, Framework*> frameworks;
+  std::list<Framework> completedFrameworks;
 
   IsolationModule* isolationModule;
   Files* files;
@@ -216,7 +220,7 @@ private:
 };
 
 
-// Information describing an executor (goes away if executor crashes).
+// Information describing an executor.
 struct Executor
 {
   Executor(const FrameworkID& _frameworkId,
@@ -275,6 +279,12 @@ struct Executor
         resources -= resource;
       }
       launchedTasks.erase(taskId);
+
+      completedTasks.push_back(*task);
+      if (completedTasks.size() > MAX_COMPLETED_TASKS_PER_EXECUTOR) {
+        completedTasks.pop_front();
+      }
+
       delete task;
     }
   }
@@ -303,6 +313,8 @@ struct Executor
 
   hashmap<TaskID, TaskInfo> queuedTasks;
   hashmap<TaskID, Task*> launchedTasks;
+
+  std::list<Task> completedTasks;
 };
 
 
@@ -393,6 +405,12 @@ struct Framework
     if (executors.contains(executorId)) {
       Executor* executor = executors[executorId];
       executors.erase(executorId);
+
+      completedExecutors.push_back(*executor);
+      if (completedExecutors.size() > MAX_COMPLETED_EXECUTORS_PER_FRAMEWORK) {
+        completedExecutors.pop_front();
+      }
+
       delete executor;
     }
   }
@@ -428,6 +446,9 @@ struct Framework
   // Current running executors.
   hashmap<ExecutorID, Executor*> executors;
 
+  // Up to MAX_COMPLETED_EXECUTORS_PER_FRAMEWORK completed executors.
+  std::list<Executor> completedExecutors;
+
   // Status updates keyed by uuid.
   hashmap<UUID, StatusUpdate> updates;
 };

Modified: incubator/mesos/trunk/src/webui/master/static/browse.html
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/static/browse.html?rev=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/browse.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/browse.html Fri Oct 26 23:41:06 2012
@@ -21,7 +21,8 @@
     <em>Path:</em>
   </li>
   <li ng-repeat="dir in path.split('/')">
-    <a href="#/slaves/{{slave_id}}/browse?path={{encodeURIComponent(path.split('/').slice(0, $index+1).join('/'))}}">
+    <a href="#/slaves/{{slave_id}}/browse?path={{
+             encodeURIComponent(path.split('/').slice(0, $index + 1).join('/'))}}">
       {{dir}}
     </a>
     <span class="divider">/</span>

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=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/controllers.js (original)
+++ incubator/mesos/trunk/src/webui/master/static/controllers.js Fri Oct 26 23:41:06 2012
@@ -396,6 +396,8 @@ function SlavesCtrl($scope) {
 }
 
 
+// TODO(bmahler): Pull this apart into:
+// SlaveCtrl, SlaveFrameworkCtrl, SlaveExecutorCtrl.
 function SlaveCtrl($scope, $routeParams, $http) {
   setNavbarActiveTab('slaves');
 
@@ -411,8 +413,12 @@ function SlaveCtrl($scope, $routeParams,
 
   $scope.tables = {};
   $scope.tables['frameworks'] = new Table('id');
+  $scope.tables['completed_frameworks'] = new Table('id');
   $scope.tables['executors'] = new Table('id');
+  $scope.tables['completed_executors'] = new Table('id');
   $scope.tables['tasks'] = new Table('id');
+  $scope.tables['queued_tasks'] = new Table('id');
+  $scope.tables['completed_tasks'] = new Table('id');
 
   $scope.columnClass = columnClass($scope);
   $scope.selectColumn = selectColumn($scope);
@@ -438,19 +444,44 @@ function SlaveCtrl($scope, $routeParams,
 
           $scope.slave = {};
           $scope.slave.frameworks = {};
+          $scope.slave.completed_frameworks = {};
 
           $scope.slave.staging_tasks = 0;
           $scope.slave.starting_tasks = 0;
           $scope.slave.running_tasks = 0;
 
-          // Update the maps.
+          // Update the framework map.
           _.each($scope.state.frameworks, function(framework) {
             $scope.slave.frameworks[framework.id] = framework;
+
             var executors = {};
             _.each(framework.executors, function(executor) {
               executors[executor.id] = executor;
             });
             $scope.slave.frameworks[framework.id].executors = executors;
+
+            var completed_executors = {};
+            _.each(framework.completed_executors, function(executor) {
+              completed_executors[executor.id] = executor;
+            });
+            $scope.slave.frameworks[framework.id].completed_executors = completed_executors;
+          });
+
+          // Update the completed framework map.
+          _.each($scope.state.completed_frameworks, function(framework) {
+            $scope.slave.completed_frameworks[framework.id] = framework;
+
+            var executors = {};
+            _.each(framework.executors, function(executor) {
+              executors[executor.id] = executor;
+            });
+            $scope.slave.completed_frameworks[framework.id].executors = executors;
+
+            var completed_executors = {};
+            _.each(framework.completed_executors, function(executor) {
+              completed_executors[executor.id] = executor;
+            });
+            $scope.slave.completed_frameworks[framework.id].completed_executors = completed_executors;
           });
 
           // Compute the framework stats.
@@ -466,24 +497,47 @@ function SlaveCtrl($scope, $routeParams,
             });
           });
 
+          // Compute the completed framework stats.
+          _.each($scope.slave.completed_frameworks, function(framework) {
+            framework.num_tasks = 0;
+            framework.cpus = 0;
+            framework.mem = 0;
+
+            _.each(framework.executors, function(executor) {
+              framework.num_tasks += _.size(executor.tasks);
+              framework.cpus += executor.resources.cpus;
+              framework.mem += executor.resources.mem;
+            });
+          });
+
           // Look for the framework / executor if present in the request.
-          if ($scope.framework_id &&
-              !_.has($scope.slave.frameworks, $scope.framework_id)) {
-            $scope.alert_message = 'No framework found with ID: ' + $scope.framework_id;
-            $('#alert').show();
-          } else if ($scope.framework_id && $scope.executor_id &&
-              !_.has($scope.slave.frameworks[$scope.framework_id].executors, $scope.executor_id)) {
-            $scope.alert_message = 'No executor found with ID: ' + $scope.executor_id;
-            $('#alert').show();
-          } else {
-            // Set active framework / executor if present in the request.
-            if ($routeParams.framework_id) {
+          if ($scope.framework_id) {
+            // Look for the framework.
+            if (_.has($scope.slave.frameworks, $scope.framework_id)) {
               $scope.framework = $scope.slave.frameworks[$scope.framework_id];
+            } else if (_.has($scope.slave.completed_frameworks, $scope.framework_id)) {
+              $scope.framework = $scope.slave.completed_frameworks[$scope.framework_id];
+            } else {
+              $scope.alert_message = 'No framework found with ID: ' + $scope.framework_id;
+              $('#alert').show();
             }
-            if ($routeParams.executor_id) {
-              $scope.executor = $scope.framework.executors[$scope.executor_id];
+
+            if ($scope.framework && $scope.executor_id) {
+              // Look for the executor.
+              if (_.has($scope.framework.executors, $scope.executor_id)) {
+                $scope.executor = $scope.framework.executors[$scope.executor_id];
+              } else if (_.has($scope.framework.completed_executors, $scope.executor_id)) {
+                $scope.executor = $scope.framework.completed_executors[$scope.executor_id];
+              } else {
+                $scope.alert_message = 'No executor found with ID: ' + $scope.executor_id;
+                $('#alert').show();
+              }
             }
+          }
 
+          if (!$scope.framework_id || $scope.framework) {
+            $('#slave').show();
+          } else if ($scope.framework && (!$scope.executor_id || $scope.executor)) {
             $('#slave').show();
           }
         })
@@ -524,6 +578,8 @@ function BrowseCtrl($scope, $routeParams
         pailer(host, path, decodeURIComponent(path));
       }
 
+      // TODO(bmahler): Try to get the error code / body in the error callback.
+      // This wasn't working with the current version of angular.
       $http.jsonp(url, {params: {path: $routeParams.path}})
         .success(function(data) {
           $scope.listing = data;

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=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/framework.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/framework.html Fri Oct 26 23:41:06 2012
@@ -80,7 +80,7 @@
           <td>{{task.name}}</td>
           <td>{{task.state | truncateMesosState}}</td>
           <td>
-            <a href="#/slaves/{{task.slave_id}}">
+            <a href="#/slaves/{{task.slave_id}}/frameworks/{{task.framework_id}}/executors/{{task.executor_id}}">
               {{slaves[task.slave_id].hostname}}
             </a>
           </td>
@@ -116,7 +116,7 @@
           <td>{{task.name}}</td>
           <td>{{task.state | truncateMesosState}}</td>
           <td>
-            <a href="#/slaves/{{task.slave_id}}">
+            <a href="#/slaves/{{task.slave_id}}/frameworks/{{task.framework_id}}/executors/{{task.executor_id}}">
               {{slaves[task.slave_id].hostname}}
             </a>
           </td>

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=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/frameworks.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/frameworks.html Fri Oct 26 23:41:06 2012
@@ -9,7 +9,7 @@
 
 <div class="row">
   <div class="span12">
-    <h2 id="frameworks">Frameworks</h2>
+    <h2 id="frameworks">Active Frameworks</h2>
     <table class="table table-striped table-bordered table-condensed">
       <thead>
         <tr>

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=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/home.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/home.html Fri Oct 26 23:41:06 2012
@@ -323,7 +323,7 @@
             <abbr title="{{framework.id}}">{{framework.id | truncateMesosID}}</abbr>
           </td>
           <td>{{framework.user}}</td>
-          <td><a href="{{'#/framework/' + framework.id}}">{{framework.name}}</a></td>
+          <td><a href="{{'#/frameworks/' + framework.id}}">{{framework.name}}</a></td>
           <td>
             <a href="" rel="popover" ng-click="popover($event, 'bottom')" data-content="{{framework.registered_time * 1000 | mesosDate}}" data-original-title="Registered">
               {{framework.registered_time * 1000 | relativeDate}}

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=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/slave.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/slave.html Fri Oct 26 23:41:06 2012
@@ -160,5 +160,55 @@
         </tr>
       </tbody>
     </table>
+
+    <h2>Completed Frameworks</h2>
+    <table class="table table-striped table-bordered table-condensed">
+      <thead>
+        <tr>
+          <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', 'num_tasks')"
+              ng-click="selectColumn('completed_frameworks', 'num_tasks')">
+            <a href="">Active Tasks</a>
+          </th>
+          <th ng-class="columnClass('completed_frameworks', 'cpus')"
+              ng-click="selectColumn('completed_frameworks', 'cpus')">
+            <a href="">CPUs</a>
+          </th>
+          <th ng-class="columnClass('completed_frameworks', 'mem')"
+              ng-click="selectColumn('completed_frameworks', 'mem')">
+            <a href="">Mem</a>
+          </th>
+        </tr>
+      </thead>
+      <tbody>
+        <tr ng-repeat="completed_framework in _.values(slave.completed_frameworks) | orderBy:tables['completed_frameworks'].selected_column:tables['completed_frameworks'].reverse">
+          <td>
+            <a href="" rel="popover" ng-click="popover($event, 'bottom')" data-content="{{completed_framework.id}}" data-original-title="ID">
+              {{completed_framework.id | truncateMesosID}}
+            </a>
+          </td>
+          <td>{{completed_framework.user}}</td>
+          <td>
+            <a href="{{'#/slaves/' + slave_id + '/frameworks/' + completed_framework.id}}">
+              {{completed_framework.name}}
+            </a>
+          </td>
+          <td>{{completed_framework.num_tasks | number}}</td>
+          <td>{{completed_framework.cpus | number}}</td>
+          <td>{{completed_framework.mem / 1024 | number}} GB</td>
+        </tr>
+      </tbody>
+    </table>
   </div>
 </div>

Modified: incubator/mesos/trunk/src/webui/master/static/slave_executor.html
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/static/slave_executor.html?rev=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/slave_executor.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/slave_executor.html Fri Oct 26 23:41:06 2012
@@ -73,8 +73,39 @@
   </div>
 
   <div class="span9">
-    <h2>Tasks</h2>
+    <h2>Queued Tasks</h2>
+    <table class="table table-striped table-bordered table-condensed">
+      <thead>
+        <tr>
+          <th ng-class="columnClass('queued_tasks', 'id')"
+              ng-click="selectColumn('queued_tasks', 'id')">
+            <a href="">ID</a>
+          </th>
+          <th ng-class="columnClass('queued_tasks', 'name')"
+              ng-click="selectColumn('queued_tasks', 'name')">
+            <a href="">Name</a>
+          </th>
+          <th ng-class="columnClass('queued_tasks', 'resources.cpus')"
+              ng-click="selectColumn('queued_tasks', 'resources.cpus')">
+            <a href="">CPUs</a>
+          </th>
+          <th ng-class="columnClass('queued_tasks', 'resources.mem')"
+              ng-click="selectColumn('queued_tasks', 'resources.mem')">
+            <a href="">Mem</a>
+          </th>
+        </tr>
+      </thead>
+      <tbody>
+        <tr ng-repeat="queued_task in _.values(executor.queued_tasks) | orderBy:tables['queued_tasks'].selected_column:tables['queued_tasks'].reverse">
+          <td>{{queued_task.id}}</td>
+          <td>{{queued_task.name}}</td>
+          <td>{{queued_task.resources.cpus | number}}</td>
+          <td>{{queued_task.resources.mem / 1024 | number}} GB</td>
+        </tr>
+      </tbody>
+    </table>
 
+    <h2>Tasks</h2>
     <table class="table table-striped table-bordered table-condensed">
       <thead>
         <tr>
@@ -109,9 +140,52 @@
           <td>{{task.resources.cpus | number}}</td>
           <td>{{task.resources.mem / 1024 | number}} GB</td>
           <td>
-            <a href="{{'#/slaves/' + slave_id + '/browse?path=' + encodeURIComponent(
-                       '/slaves/' + slave_id + '/frameworks/' + framework_id +
-                       '/executors/' + executor.id)}}">
+            <a href="{{'#/slaves/' + slave_id + '/browse?path=' +
+                       encodeURIComponent(executor.directory)}}">
+              browse
+            </a>
+          </td>
+        </tr>
+      </tbody>
+    </table>
+
+    <h2>Completed Tasks</h2>
+    <table class="table table-striped table-bordered table-condensed">
+      <thead>
+        <tr>
+          <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', 'resources.cpus')"
+              ng-click="selectColumn('completed_tasks', 'resources.cpus')">
+            <a href="">CPUs</a>
+          </th>
+          <th ng-class="columnClass('completed_tasks', 'resources.mem')"
+              ng-click="selectColumn('completed_tasks', 'resources.mem')">
+            <a href="">Mem</a>
+          </th>
+          <th>Executor Sandbox</th>
+        </tr>
+      </thead>
+      <tbody>
+        <tr ng-repeat="completed_task in _.values(executor.completed_tasks) | orderBy:tables['completed_tasks'].selected_column:tables['completed_tasks'].reverse">
+          <td>{{completed_task.id}}</td>
+          <td>{{completed_task.name}}</td>
+          <td>{{completed_task.state}}</td>
+          <td>{{completed_task.resources.cpus | number}}</td>
+          <td>{{completed_task.resources.mem / 1024 | number}} GB</td>
+          <td>
+            <a href="{{'#/slaves/' + slave_id + '/browse?path=' +
+                       encodeURIComponent(executor.directory)}}">
               browse
             </a>
           </td>

Modified: incubator/mesos/trunk/src/webui/master/static/slave_framework.html
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/static/slave_framework.html?rev=1402718&r1=1402717&r2=1402718&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/static/slave_framework.html (original)
+++ incubator/mesos/trunk/src/webui/master/static/slave_framework.html Fri Oct 26 23:41:06 2012
@@ -106,9 +106,40 @@
           <td>{{executor.resources.cpus | number}}</td>
           <td>{{executor.resources.mem / 1024 | number}} GB</td>
           <td>
-            <a href="{{'#/slaves/' + slave_id + '/browse?path=' + encodeURIComponent(
-                       '/slaves/' + slave_id + '/frameworks/' + framework_id +
-                       '/executors/' + executor.id)}}">
+            <a href="{{'#/slaves/' + slave_id + '/browse?path=' +
+                       encodeURIComponent(executor.directory)}}">
+              browse
+            </a>
+          </td>
+        </tr>
+      </tbody>
+    </table>
+
+    <h2>Completed Executors</h2>
+    <table class="table table-striped table-bordered table-condensed">
+      <thead>
+        <tr>
+          <th ng-class="columnClass('completed_executors', 'id')"
+              ng-click="selectColumn('completed_executors', 'id')">
+            <a href="">ID</a>
+          </th>
+          <th ng-class="columnClass('completed_executors', 'sandbox')"
+              ng-click="selectColumn('completed_executors', 'sandbox')">
+            <a href="">Sandbox</a>
+          </th>
+        </tr>
+      </thead>
+
+      <tbody>
+        <tr ng-repeat="completed_executor in _.values(framework.completed_executors) | orderBy:tables['completed_executors'].selected_column:tables['completed_executors'].reverse">
+          <td>
+            <a href="{{'#/slaves/' + slave_id + '/frameworks/' + framework.id + '/executors/' + completed_executor.id}}">
+              {{completed_executor.id}}
+            </a>
+          </td>
+          <td>
+            <a href="{{'#/slaves/' + slave_id + '/browse?path=' +
+                       encodeURIComponent(completed_executor.directory)}}">
               browse
             </a>
           </td>