You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by mx...@apache.org on 2015/10/09 19:09:35 UTC

[6/6] flink git commit: [FLINK-2730][webfrontend] Add cpu and memory usage graphs

[FLINK-2730][webfrontend] Add cpu and memory usage graphs

This closes #1236.


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

Branch: refs/heads/master
Commit: 58ab14b55e7f1d5835bd4bb660ae9bb676f8dbb2
Parents: 3b77faa
Author: Sachin Goel <sa...@gmail.com>
Authored: Sun Oct 4 11:29:06 2015 +0530
Committer: Maximilian Michels <mx...@apache.org>
Committed: Fri Oct 9 18:24:55 2015 +0200

----------------------------------------------------------------------
 .../runtime/webmonitor/WebRuntimeMonitor.java   |    1 +
 .../handlers/TaskManagersHandler.java           |   41 +-
 flink-runtime-web/web-dashboard/app/index.jade  |    2 +-
 .../app/partials/taskmanager/index.jade         |   53 +
 .../app/partials/taskmanager/taskmanager.jade   |   43 +
 .../taskmanager/taskmanager.logfile.jade        |   17 +
 .../taskmanager/taskmanager.metrics.jade        |  137 ++
 .../taskmanager/taskmanager.stdout.jade         |   17 +
 .../app/partials/taskmanagers/index.jade        |   55 -
 .../app/partials/taskmanagers/metrics.jade      |  163 --
 .../web-dashboard/app/scripts/index.coffee      |  112 +-
 .../modules/taskmanager/taskmanager.ctrl.coffee |   45 +
 .../modules/taskmanager/taskmanager.dir.coffee  |   93 +
 .../modules/taskmanager/taskmanager.svc.coffee  |   44 +
 .../taskmanagers/taskmanagers.ctrl.coffee       |   39 -
 .../taskmanagers/taskmanagers.svc.coffee        |   31 -
 .../web-dashboard/app/styles/index.styl         |   15 +
 flink-runtime-web/web-dashboard/bower.json      |    3 +-
 .../web-dashboard/web/css/index.css             |   14 +
 .../web-dashboard/web/css/vendor.css            |  328 +--
 flink-runtime-web/web-dashboard/web/index.html  |    2 +-
 flink-runtime-web/web-dashboard/web/js/index.js |  302 ++-
 .../web-dashboard/web/js/vendor.js              | 1862 +++++++++++-------
 .../web/partials/taskmanager/index.html         |   57 +
 .../web/partials/taskmanager/taskmanager.html   |   35 +
 .../taskmanager/taskmanager.logfile.html        |   18 +
 .../taskmanager/taskmanager.metrics.html        |  156 ++
 .../taskmanager/taskmanager.stdout.html         |   18 +
 .../web/partials/taskmanagers/index.html        |   62 -
 .../web/partials/taskmanagers/metrics.html      |  197 --
 .../flink/runtime/jobmanager/JobManager.scala   |    5 +
 .../runtime/messages/JobManagerMessages.scala   |   13 +
 32 files changed, 2490 insertions(+), 1490 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
index 946155a..96da0c8 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
@@ -188,6 +188,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 			.GET("/jobs/:jobid/accumulators", handler(new JobAccumulatorsHandler(currentGraphs)))
 
 			.GET("/taskmanagers", handler(new TaskManagersHandler(retriever, DEFAULT_REQUEST_TIMEOUT)))
+			.GET("/taskmanagers/:" + TaskManagersHandler.TASK_MANAGER_ID_KEY, handler(new TaskManagersHandler(retriever, DEFAULT_REQUEST_TIMEOUT)))
 
 			// this handler serves all the static contents
 			.GET("/:*", new StaticFileServerHandler(webRootDir));

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java
index 988531a..dbdb439 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java
@@ -21,9 +21,12 @@ package org.apache.flink.runtime.webmonitor.handlers;
 import com.fasterxml.jackson.core.JsonGenerator;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.webmonitor.JobManagerArchiveRetriever;
 import org.apache.flink.runtime.messages.JobManagerMessages.RegisteredTaskManagers;
+import org.apache.flink.runtime.messages.JobManagerMessages.TaskManagerInstance;
+import org.apache.flink.util.StringUtils;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
@@ -39,6 +42,7 @@ public class TaskManagersHandler implements  RequestHandler, RequestHandler.Json
 
 	private final FiniteDuration timeout;
 
+	public static final String TASK_MANAGER_ID_KEY = "taskmanagerid";
 
 	public TaskManagersHandler(JobManagerArchiveRetriever retriever, FiniteDuration timeout) {
 		if (retriever == null || timeout == null) {
@@ -54,10 +58,28 @@ public class TaskManagersHandler implements  RequestHandler, RequestHandler.Json
 			ActorGateway jobManager = retriever.getJobManagerGateway();
 
 			if (jobManager != null) {
-				Future<Object> future = jobManager.ask(JobManagerMessages.getRequestRegisteredTaskManagers(), timeout);
-				RegisteredTaskManagers taskManagers = (RegisteredTaskManagers) Await.result(future, timeout);
-
-				final List<Instance> instances = new ArrayList<Instance>(taskManagers.asJavaCollection());
+				// whether one task manager's metrics are requested, or all task manager, we
+				// return them in an array. This avoids unnecessary code complexity.
+				// If only one task manager is requested, we only fetch one task manager metrics.
+				final List<Instance> instances = new ArrayList<>();
+				if (params.containsKey(TASK_MANAGER_ID_KEY)) {
+					try {
+						InstanceID instanceID = new InstanceID(StringUtils.hexStringToByte(params.get(TASK_MANAGER_ID_KEY)));
+						Future<Object> future = jobManager.ask(new JobManagerMessages.RequestTaskManagerInstance(instanceID), timeout);
+						TaskManagerInstance instance = (TaskManagerInstance) Await.result(future, timeout);
+						if (instance.instance().nonEmpty()) {
+							instances.add(instance.instance().get());
+						}
+					}
+					// this means the id string was invalid. Keep the list empty.
+					catch (IllegalArgumentException e){
+						// do nothing.
+					}
+				} else {
+					Future<Object> future = jobManager.ask(JobManagerMessages.getRequestRegisteredTaskManagers(), timeout);
+					RegisteredTaskManagers taskManagers = (RegisteredTaskManagers) Await.result(future, timeout);
+					instances.addAll(taskManagers.asJavaCollection());
+				}
 
 				StringWriter writer = new StringWriter();
 				JsonGenerator gen = JsonFactory.jacksonFactory.createJsonGenerator(writer);
@@ -78,10 +100,13 @@ public class TaskManagersHandler implements  RequestHandler, RequestHandler.Json
 					gen.writeNumberField("freeMemory", instance.getResources().getSizeOfJvmHeap());
 					gen.writeNumberField("managedMemory", instance.getResources().getSizeOfManagedMemory());
 
-					byte[] report = instance.getLastMetricsReport();
-					if (report != null) {
-						gen.writeFieldName("metrics");
-						gen.writeRawValue(new String(report, "utf-8"));
+					// only send metrics when only one task manager requests them.
+					if (params.containsKey(TASK_MANAGER_ID_KEY)) {
+						byte[] report = instance.getLastMetricsReport();
+						if (report != null) {
+							gen.writeFieldName("metrics");
+							gen.writeRawValue(new String(report, "utf-8"));
+						}
 					}
 
 					gen.writeEndObject();

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/index.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/index.jade b/flink-runtime-web/web-dashboard/app/index.jade
index a5a28ac..a87faed 100644
--- a/flink-runtime-web/web-dashboard/app/index.jade
+++ b/flink-runtime-web/web-dashboard/app/index.jade
@@ -57,7 +57,7 @@ html(lang='en')
               | 
               | Completed Jobs
           li
-            a(ui-sref="taskmanagers" ui-sref-active='active')
+            a(ui-sref="all-manager" ui-sref-active='active')
               i.fa.fa-sitemap.fa-fw
               | 
               | Task Managers

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/partials/taskmanager/index.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/taskmanager/index.jade b/flink-runtime-web/web-dashboard/app/partials/taskmanager/index.jade
new file mode 100644
index 0000000..b6c98b2
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/app/partials/taskmanager/index.jade
@@ -0,0 +1,53 @@
+//
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+ 
+      http://www.apache.org/licenses/LICENSE-2.0
+ 
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+
+nav.navbar.navbar-default.navbar-fixed-top.navbar-main
+  #fold-button.btn.btn-default.navbar-btn.pull-left(ng-click='showSidebar()')
+    i.fa.fa-navicon
+
+  .navbar-title
+    | Task Managers
+
+#content-inner
+  table.table.table-clickable.table-hover
+    thead
+      tr
+        th Path, ID
+        th Data Port
+        th Last Heartbeat
+        th All Slots
+        th Free Slots
+        th CPU Cores
+        th Physical Memory
+        th Free Memory
+        th Flink Managed Memory
+
+    tbody
+      tr(ng-repeat="manager in managers" ui-sref="single-manager.metrics({taskmanagerid: manager.id})")
+        td
+          | {{ manager.path }}
+          | 
+          .small-label {{ manager.id }}
+        td {{ manager.dataPort }}
+        td {{ manager.timeSinceLastHeartbeat | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}
+        td {{ manager.slotsNumber }}
+        td {{ manager.freeSlots }}
+        td {{ manager.cpuCores }}
+        td {{ manager.physicalMemory | bytes:MB }}
+        td {{ manager.freeMemory | bytes:MB }}
+        td {{ manager.managedMemory | bytes:MB }}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.jade b/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.jade
new file mode 100644
index 0000000..6eed3b2
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.jade
@@ -0,0 +1,43 @@
+//
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+ 
+      http://www.apache.org/licenses/LICENSE-2.0
+ 
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+
+nav.navbar.navbar-default.navbar-fixed-top.navbar-main(ng-if="metrics.id")
+  #fold-button.btn.btn-default.navbar-btn.pull-left(ng-click='showSidebar()')
+    i.fa.fa-navicon
+
+  .navbar-title
+    | Task Manager
+  .navbar-info.first.last
+    | Last Heartbeat: {{ metrics.timeSinceLastHeartbeat | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}
+  .navbar-info.last.first.hidden-xs.hidden-sm
+    | {{metrics.path}}
+
+nav.navbar.navbar-default.navbar-fixed-top.navbar-main-additional(ng-if="metrics.id")
+  ul.nav.nav-tabs
+    li(ui-sref-active='active')
+      a(ui-sref=".metrics") Metrics
+
+    li(ui-sref-active='active')
+      a(ui-sref=".log") Logs
+
+    li(ui-sref-active='active')
+      a(ui-sref=".stdout") Stdout
+
+
+#content-inner.has-navbar-main-additional
+  div(ui-view="details")
+

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.logfile.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.logfile.jade b/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.logfile.jade
new file mode 100644
index 0000000..dacca9d
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.logfile.jade
@@ -0,0 +1,17 @@
+//
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+ 
+      http://www.apache.org/licenses/LICENSE-2.0
+ 
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.metrics.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.metrics.jade b/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.metrics.jade
new file mode 100644
index 0000000..cc3c555
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.metrics.jade
@@ -0,0 +1,137 @@
+//
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+ 
+      http://www.apache.org/licenses/LICENSE-2.0
+ 
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+
+table.table(ng-if="metrics.id")
+  thead
+    tr
+      th CPU Usage
+      th Memory - Used
+
+  tbody
+    tr
+      td
+        livechart(data="metrics" key="cpuLoad")
+      td
+        livechart(data="metrics" key="used")
+        .row.text-center
+          span
+            i.fa.fa-square-o.fa-lg(id="total-mem")
+            | &nbsp;Total Memory&nbsp;
+          span
+            i.fa.fa-square-o.fa-lg(id="heap-mem")
+            | &nbsp;Heap Memory&nbsp;
+          span
+            i.fa.fa-square-o.fa-lg(id="non-heap-mem")
+            | &nbsp;Non-heap Memory&nbsp;
+
+table.table.table-properties(ng-if="metrics.id")
+  thead
+    tr
+      th Memory type / State
+      th Memory - Committed
+      th Memory - Initial
+      th Memory - Maximum
+  tbody
+    tr
+      td Heap
+      td(table-property value="metrics.metrics.gauges['memory.heap.committed'].value | bytes:MB")
+      td(table-property value="metrics.metrics.gauges['memory.heap.init'].value | bytes:MB")
+      td(table-property value="metrics.metrics.gauges['memory.heap.max'].value | bytes:MB")
+    tr
+      td Non-Heap
+      td(table-property value="metrics.metrics.gauges['memory.non-heap.committed'].value | bytes:MB")
+      td(table-property value="metrics.metrics.gauges['memory.non-heap.init'].value | bytes:MB")
+      td(table-property value="metrics.metrics.gauges['memory.non-heap.max'].value | bytes:MB")
+    tr
+      td Total
+      td(table-property value="metrics.metrics.gauges['memory.total.committed'].value | bytes:MB")
+      td(table-property value="metrics.metrics.gauges['memory.total.init'].value | bytes:MB")
+      td(table-property value="metrics.metrics.gauges['memory.total.max'].value | bytes:MB")
+
+table.table(ng-if="metrics.id")
+  thead
+    tr
+      th Data Port
+      th All Slots
+      th Free Slots
+      th CPU Cores
+      th Physical Memory
+      th Free Memory
+      th Flink Managed Memory
+
+  tbody
+    tr
+      td {{ metrics.dataPort }}
+      td {{ metrics.slotsNumber }}
+      td {{ metrics.freeSlots }}
+      td {{ metrics.cpuCores }}
+      td {{ metrics.physicalMemory | bytes:MB }}
+      td {{ metrics.freeMemory | bytes:MB }}
+      td {{ metrics.managedMemory | bytes:MB }}
+
+
+.row(ng-if="metrics.id")
+  .col-md-6
+    table.table.table-properties
+      thead
+        tr
+          th(colspan="2")
+            | Memory - Pools
+
+      tbody
+        tr
+          td Code Cache
+          td(table-property value="metrics.metrics.gauges['memory.pools.Code-Cache.usage'].value | number:2")
+        tr
+          td Compressed Class Space
+          td(table-property value="metrics.metrics.gauges['memory.pools.Compressed-Class-Space.usage'].value | number:2")
+        tr
+          td Metaspace
+          td(table-property value="metrics.metrics.gauges['memory.pools.Metaspace.usage'].value | number:2")
+        tr
+          td PS Eden Space
+          td(table-property value="metrics.metrics.gauges['memory.pools.PS-Eden-Space.usage'].value | number:2")
+        tr
+          td PS Old Gen
+          td(table-property value="metrics.metrics.gauges['memory.pools.PS-Old-Gen.usage'].value | number:2")
+        tr
+          td PS Survivor Space
+          td(table-property value="metrics.metrics.gauges['memory.pools.PS-Survivor-Space.usage'].value | number:2")
+  .col-md-6
+    table.table.table-properties
+      thead
+        tr
+          th(colspan="2")
+            | Garbage Collection
+
+      tbody
+        tr
+          td PS-MarkSweep Count
+          td(table-property value="metrics.metrics.gauges['gc.PS-MarkSweep.count'].value")
+
+        tr
+          td PS-MarkSweep Time (ms)
+          td(table-property value="metrics.metrics.gauges['gc.PS-MarkSweep.time'].value")
+
+        tr
+          td PS-Scavenge Count
+          td(table-property value="metrics.metrics.gauges['gc.PS-Scavenge.count'].value")
+
+        tr
+          td PS-Scavenge Time (ms)
+          td(table-property value="manager.metrics.gauges['gc.PS-Scavenge.time'].value")
+

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.stdout.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.stdout.jade b/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.stdout.jade
new file mode 100644
index 0000000..dacca9d
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/app/partials/taskmanager/taskmanager.stdout.jade
@@ -0,0 +1,17 @@
+//
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+ 
+      http://www.apache.org/licenses/LICENSE-2.0
+ 
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/partials/taskmanagers/index.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/taskmanagers/index.jade b/flink-runtime-web/web-dashboard/app/partials/taskmanagers/index.jade
deleted file mode 100644
index 03a6186..0000000
--- a/flink-runtime-web/web-dashboard/app/partials/taskmanagers/index.jade
+++ /dev/null
@@ -1,55 +0,0 @@
-//
-  Licensed to the Apache Software Foundation (ASF) under one
-  or more contributor license agreements.  See the NOTICE file
-  distributed with this work for additional information
-  regarding copyright ownership.  The ASF licenses this file
-  to you under the Apache License, Version 2.0 (the
-  "License"); you may not use this file except in compliance
-  with the License.  You may obtain a copy of the License at
- 
-      http://www.apache.org/licenses/LICENSE-2.0
- 
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
-
-nav.navbar.navbar-default.navbar-fixed-top.navbar-main
-  #fold-button.btn.btn-default.navbar-btn.pull-left(ng-click='showSidebar()')
-    i.fa.fa-navicon
-
-  .navbar-title
-    | Task Managers
-
-#content-inner
-  table.table.table-body-hover.table-clickable.table-activable
-    thead
-      tr
-        th Path, ID
-        th Data Port
-        th Last Heartbeat
-        th All Slots
-        th Free Slots
-        th CPU Cores
-        th Physical Memory
-        th Free Memory
-        th Flink Managed Memory
-
-    tbody(ng-repeat="manager in managers" ng-class="{ active: manager.id == managerId }" ng-click="changeManager(manager.id)")
-      tr
-        td
-          | {{ manager.path }}
-          | 
-          .small-label {{ manager.id }}
-        td {{ manager.dataPort }}
-        td {{ manager.timeSinceLastHeartbeat | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}
-        td {{ manager.slotsNumber }}
-        td {{ manager.freeSlots }}
-        td {{ manager.cpuCores }}
-        td {{ manager.physicalMemory | bytes:MB }}
-        td {{ manager.freeMemory | bytes:MB }}
-        td {{ manager.managedMemory | bytes:MB }}
-      tr(ng-if="managerId && manager.id == managerId")
-        td(colspan="9")
-          div(ng-include=" 'partials/taskmanagers/metrics.html' ")

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/partials/taskmanagers/metrics.jade
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/partials/taskmanagers/metrics.jade b/flink-runtime-web/web-dashboard/app/partials/taskmanagers/metrics.jade
deleted file mode 100644
index 228d782..0000000
--- a/flink-runtime-web/web-dashboard/app/partials/taskmanagers/metrics.jade
+++ /dev/null
@@ -1,163 +0,0 @@
-//
-  Licensed to the Apache Software Foundation (ASF) under one
-  or more contributor license agreements.  See the NOTICE file
-  distributed with this work for additional information
-  regarding copyright ownership.  The ASF licenses this file
-  to you under the Apache License, Version 2.0 (the
-  "License"); you may not use this file except in compliance
-  with the License.  You may obtain a copy of the License at
- 
-      http://www.apache.org/licenses/LICENSE-2.0
- 
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
-
-div(ng-if="!manager.metrics")
-  p
-    i No metrics
-
-.row(ng-if="manager.metrics && manager.metrics.gauges")
- .col-sm-4
-   table.table.table-properties
-     thead
-       tr
-         th(colspan="2")
-           | CPU Load
-
-     tbody
-       tr
-         td Current (%)
-         td(table-property value="manager.metrics.gauges.cpuLoad.value * 100 | number:2")
-
-   table.table.table-properties
-     thead
-       tr
-         th(colspan="2")
-           | Load
-
-     tbody
-       tr
-         td Current
-         td(table-property value="manager.metrics.gauges.load.value | number:2")
-
-   table.table.table-properties
-     thead
-       tr
-         th(colspan="2")
-           | Memory - Total
-
-     tbody
-       tr
-         td Committed
-         td(table-property value="manager.metrics.gauges['memory.total.committed'].value | bytes:MB")
-       tr
-         td Initial
-         td(table-property value="manager.metrics.gauges['memory.total.init'].value | bytes:MB")
-       tr
-         td Max
-         td(table-property value="manager.metrics.gauges['memory.total.max'].value | bytes:MB")
-       tr
-         td Used
-         td(table-property value="manager.metrics.gauges['memory.total.used'].value | bytes:MB")
-
- .col-sm-4
-   table.table.table-properties
-     thead
-       tr
-         th(colspan="2")
-           | Memory - Heap
-
-     tbody
-       tr
-         td Committed
-         td(table-property value="manager.metrics.gauges['memory.heap.committed'].value | bytes:MB")
-       tr
-         td Initial
-         td(table-property value="manager.metrics.gauges['memory.heap.init'].value | bytes:MB")
-       tr
-         td Max
-         td(table-property value="manager.metrics.gauges['memory.heap.max'].value | bytes:MB")
-       tr
-         td Usage (%)
-         td(table-property value="manager.metrics.gauges['memory.heap.usage'].value | number:2")
-       tr
-         td Used
-         td(table-property value="manager.metrics.gauges['memory.heap.used'].value | bytes:MB")
-
-
-   table.table.table-properties
-     thead
-       tr
-         th(colspan="2")
-           | Memory - Non Heap
-
-     tbody
-       tr
-         td Committed
-         td(table-property value="manager.metrics.gauges['memory.non-heap.committed'].value | bytes:MB")
-       tr
-         td Init
-         td(table-property value="manager.metrics.gauges['memory.non-heap.init'].value | bytes:MB")
-       tr
-         td Max
-         td(table-property value="manager.metrics.gauges['memory.non-heap.max'].value | bytes:MB")
-       tr
-         td Usage
-         td(table-property value="manager.metrics.gauges['memory.non-heap.usage'].value | bytes:MB")
-       tr
-         td Used
-         td(table-property value="manager.metrics.gauges['memory.non-heap.used'].value | bytes:MB")
-
-
- .col-sm-4
-   table.table.table-properties
-     thead
-       tr
-         th(colspan="2")
-           | Garbage Collection
-
-     tbody
-       tr
-         td PS-MarkSweep Count
-         td(table-property value="manager.metrics.gauges['gc.PS-MarkSweep.count'].value")
-
-       tr
-         td PS-MarkSweep Time (ms)
-         td(table-property value="manager.metrics.gauges['gc.PS-MarkSweep.time'].value")
-
-       tr
-         td PS-Scavenge Count
-         td(table-property value="manager.metrics.gauges['gc.PS-Scavenge.count'].value")
-
-       tr
-         td PS-Scavenge Time (ms)
-         td(table-property value="manager.metrics.gauges['gc.PS-Scavenge.time'].value")
-
-   table.table.table-properties
-     thead
-       tr
-         th(colspan="2")
-           | Memory - Pools
-
-     tbody
-       tr
-         td Code Cache
-         td(table-property value="manager.metrics.gauges['memory.pools.Code-Cache.usage'].value | number:2")
-       tr
-         td Compressed Class Space
-         td(table-property value="manager.metrics.gauges['memory.pools.Compressed-Class-Space.usage'].value | number:2")
-       tr
-         td Metaspace
-         td(table-property value="manager.metrics.gauges['memory.pools.Metaspace.usage'].value | number:2")
-       tr
-         td PS Eden Space
-         td(table-property value="manager.metrics.gauges['memory.pools.PS-Eden-Space.usage'].value | number:2")
-       tr
-         td PS Old Gen
-         td(table-property value="manager.metrics.gauges['memory.pools.PS-Old-Gen.usage'].value | number:2")
-       tr
-         td PS Survivor Space
-         td(table-property value="manager.metrics.gauges['memory.pools.PS-Survivor-Space.usage'].value | number:2")

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/scripts/index.coffee
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/scripts/index.coffee b/flink-runtime-web/web-dashboard/app/scripts/index.coffee
index eaa27f1..0fc4cb8 100644
--- a/flink-runtime-web/web-dashboard/app/scripts/index.coffee
+++ b/flink-runtime-web/web-dashboard/app/scripts/index.coffee
@@ -44,6 +44,86 @@ angular.module('flinkApp', ['ui.router', 'angularMoment'])
       JobsService.listJobs()
     , flinkConfig["refresh-interval"]
 
+  Highcharts.setOptions({
+    global: {
+      useUTC: false
+    }
+  })
+
+  #
+  # Grid-light theme for Highcharts JS
+  # @author Torstein Honsi
+  #
+  # Taken from https://github.com/highslide-software/highcharts.com
+  #
+
+
+  Highcharts.createElement('link', {
+  	href: '//fonts.googleapis.com/css?family=Dosis:400,600',
+  	rel: 'stylesheet',
+  	type: 'text/css'
+  }, null, document.getElementsByTagName('head')[0]);
+
+  Highcharts.theme = {
+  	colors: ["#7cb5ec", "#f7a35c", "#90ee7e", "#7798BF", "#aaeeee", "#ff0066", "#eeaaee",
+  		"#55BF3B", "#DF5353", "#7798BF", "#aaeeee"],
+  	chart: {
+  		backgroundColor: null,
+  		style: {
+  			fontFamily: "Dosis, sans-serif"
+  		}
+  	},
+  	title: {
+  		style: {
+  			fontSize: '16px',
+  			fontWeight: 'bold',
+  			textTransform: 'uppercase'
+  		}
+  	},
+  	tooltip: {
+  		borderWidth: 0,
+  		backgroundColor: 'rgba(219,219,216,0.8)',
+  		shadow: false
+  	},
+  	legend: {
+  		itemStyle: {
+  			fontWeight: 'bold',
+  			fontSize: '13px'
+  		}
+  	},
+  	xAxis: {
+  		gridLineWidth: 1,
+  		labels: {
+  			style: {
+  				fontSize: '12px'
+  			}
+  		}
+  	},
+  	yAxis: {
+  		minorTickInterval: 'auto',
+  		title: {
+  			style: {
+  				textTransform: 'uppercase'
+  			}
+  		},
+  		labels: {
+  			style: {
+  				fontSize: '12px'
+  			}
+  		}
+  	},
+  	plotOptions: {
+  		candlestick: {
+  			lineColor: '#404048'
+  		}
+  	},
+
+  	background2: '#F0F0EA'
+
+  };
+
+  Highcharts.setOptions(Highcharts.theme);
+
 
 # --------------------------------------
 
@@ -143,12 +223,37 @@ angular.module('flinkApp', ['ui.router', 'angularMoment'])
       details:
         templateUrl: "partials/jobs/job.config.html"
 
-  .state "taskmanagers",
+  .state "all-manager",
     url: "/taskmanagers"
     views:
       main:
-        templateUrl: "partials/taskmanagers/index.html"
-        controller: 'TaskManagersController'
+        templateUrl: "partials/taskmanager/index.html"
+        controller: 'AllTaskManagersController'
+
+  .state "single-manager",
+      url: "/taskmanager/{taskmanagerid}"
+      views:
+        main:
+          templateUrl: "partials/taskmanager/taskmanager.html"
+          controller: 'SingleTaskManagerController'
+
+  .state "single-manager.metrics",
+    url: "/metrics"
+    views:
+      details:
+        templateUrl: "partials/taskmanager/taskmanager.metrics.html"
+
+  .state "single-manager.log",
+    url: "/logfile"
+    views:
+      details:
+        templateUrl: "partials/taskmanager/taskmanager.logfile.html"
+
+  .state "single-manager.stdout",
+    url: "/stdout"
+    views:
+      details:
+        templateUrl: "partials/taskmanager/taskmanager.stdout.html"
 
   .state "jobmanager",
       url: "/jobmanager"
@@ -176,3 +281,4 @@ angular.module('flinkApp', ['ui.router', 'angularMoment'])
         templateUrl: "partials/jobmanager/logfile.html"
 
   $urlRouterProvider.otherwise "/overview"
+

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.ctrl.coffee
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.ctrl.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.ctrl.coffee
new file mode 100644
index 0000000..16adab6
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.ctrl.coffee
@@ -0,0 +1,45 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+angular.module('flinkApp')
+
+.controller 'AllTaskManagersController', ($scope, TaskManagersService, $interval, flinkConfig) ->
+  TaskManagersService.loadManagers().then (data) ->
+    $scope.managers = data
+
+  refresh = $interval ->
+    TaskManagersService.loadManagers().then (data) ->
+      $scope.managers = data
+  , flinkConfig["refresh-interval"]
+
+  $scope.$on '$destroy', ->
+    $interval.cancel(refresh)
+
+.controller 'SingleTaskManagerController', ($scope, $stateParams, SingleTaskManagerService, $interval, flinkConfig) ->
+  $scope.metrics = {}
+  SingleTaskManagerService.loadMetrics($stateParams.taskmanagerid).then (data) ->
+      $scope.metrics = data[0]
+
+    refresh = $interval ->
+      SingleTaskManagerService.loadMetrics($stateParams.taskmanagerid).then (data) ->
+        $scope.metrics = data[0]
+    , flinkConfig["refresh-interval"]
+
+    $scope.$on '$destroy', ->
+      $interval.cancel(refresh)
+

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.dir.coffee
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.dir.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.dir.coffee
new file mode 100644
index 0000000..f4587b1
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.dir.coffee
@@ -0,0 +1,93 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+angular.module('flinkApp')
+
+.directive 'livechart', () ->
+  {
+    link: (scope, element, attrs) ->
+      getChartType = () ->
+        if attrs.key == "cpuLoad"
+          "spline"
+        else
+          "area"
+
+      getYAxisTitle = () ->
+        if attrs.key == "cpuLoad"
+          "CPU Usage(%)"
+        else
+          "Memory(MB)"
+
+      getKey1 = () ->
+        "memory.total." + attrs.key
+      getKey2 = () ->
+        "memory.heap." + attrs.key
+      getKey3 = () ->
+        "memory.non-heap." + attrs.key
+      getKey4 = () ->
+        "cpuLoad"
+
+      getChartOptions = () -> {
+        title: {text: ' '},
+        chart: {type: getChartType(), zoomType: 'x'},
+        xAxis: {type: 'datetime'},
+        yAxis: {
+          title: {text: getYAxisTitle() }
+          min: 0 if attrs.key == "cpuLoad",
+          max: 100 if attrs.key == "cpuLoad"
+        },
+        series: [
+          {name: "Memory: Total", id: getKey1(), data: [], color: "#7cb5ec"},
+          {name: "Memory: Heap", id: getKey2(), data: [], color: "#434348"},
+          {name: "Memory: Non-Heap", id: getKey3(), data: [], color: "#90ed7d"},
+          {name: "CPU Usage", id: getKey4(), data: [], color: "#f7a35c", showInLegend: false}
+        ],
+        legend: {enabled: false},
+        tooltip: {shared: true},
+        exporting: {enabled: false},
+        credits: {enabled: false}
+      }
+
+      if !element.highcharts()?
+        element.highcharts(getChartOptions())
+
+      scope.$watch(attrs.data, (value) ->
+        updateCharts(value)
+      )
+
+      updateCharts = (value) ->
+        do(value) ->
+          heartbeat = value.timeSinceLastHeartbeat
+          chart = element.highcharts()
+          if attrs.key == "cpuLoad"
+            chart.get(getKey4()).addPoint([
+              heartbeat, +((value.metrics.gauges[getKey4()].value * 100).toFixed(2))
+            ], true, false)
+          else
+            divider = 1048576
+            chart.get(getKey1()).addPoint([
+              heartbeat, +((value.metrics.gauges[getKey1()].value / divider).toFixed(2))
+            ], true, false)
+            chart.get(getKey2()).addPoint([
+              heartbeat, +((value.metrics.gauges[getKey2()].value / divider).toFixed(2))
+            ], true, false)
+            chart.get(getKey3()).addPoint([
+              heartbeat, +((value.metrics.gauges[getKey3()].value / divider).toFixed(2))
+            ], true, false)
+  }
+

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.svc.coffee
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.svc.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.svc.coffee
new file mode 100644
index 0000000..d76ea9c
--- /dev/null
+++ b/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanager/taskmanager.svc.coffee
@@ -0,0 +1,44 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+angular.module('flinkApp')
+
+.service 'TaskManagersService', ($http, flinkConfig, $q) ->
+  @loadManagers = () ->
+    deferred = $q.defer()
+
+    $http.get("/taskmanagers")
+    .success (data, status, headers, config) ->
+      deferred.resolve(data['taskmanagers'])
+
+    deferred.promise
+
+  @
+
+.service 'SingleTaskManagerService', ($http, flinkConfig, $q) ->
+  @loadMetrics = (taskmanagerid) ->
+    deferred = $q.defer()
+
+    $http.get("/taskmanagers/" + taskmanagerid)
+    .success (data, status, headers, config) ->
+      deferred.resolve(data['taskmanagers'])
+
+    deferred.promise
+
+  @
+

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanagers/taskmanagers.ctrl.coffee
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanagers/taskmanagers.ctrl.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanagers/taskmanagers.ctrl.coffee
deleted file mode 100644
index 6e98feb..0000000
--- a/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanagers/taskmanagers.ctrl.coffee
+++ /dev/null
@@ -1,39 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-angular.module('flinkApp')
-
-.controller 'TaskManagersController', ($scope, TaskManagersService, $interval, flinkConfig) ->
-  TaskManagersService.loadManagers().then (data) ->
-    $scope.managers = data
-
-  refresh = $interval ->
-    TaskManagersService.loadManagers().then (data) ->
-      $scope.managers = data
-  , flinkConfig["refresh-interval"]
-
-  $scope.$on '$destroy', ->
-    $interval.cancel(refresh)
-
-  $scope.managerId = null
-
-  $scope.changeManager = (managerId) ->
-    if managerId != $scope.managerId
-      $scope.managerId = managerId
-    else
-      $scope.managerId = null

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanagers/taskmanagers.svc.coffee
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanagers/taskmanagers.svc.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanagers/taskmanagers.svc.coffee
deleted file mode 100644
index fcdf2f9..0000000
--- a/flink-runtime-web/web-dashboard/app/scripts/modules/taskmanagers/taskmanagers.svc.coffee
+++ /dev/null
@@ -1,31 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-angular.module('flinkApp')
-
-.service 'TaskManagersService', ($http, flinkConfig, $q) ->
-  @loadManagers = ->
-    deferred = $q.defer()
-
-    $http.get("/taskmanagers")
-    .success (data, status, headers, config) ->
-      deferred.resolve(data['taskmanagers'])
-
-    deferred.promise
-
-  @

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/app/styles/index.styl
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/app/styles/index.styl b/flink-runtime-web/web-dashboard/app/styles/index.styl
index 597022e..d2702e7 100644
--- a/flink-runtime-web/web-dashboard/app/styles/index.styl
+++ b/flink-runtime-web/web-dashboard/app/styles/index.styl
@@ -363,6 +363,11 @@ pre.exception
   .navbar-title
     padding: 12px 20px 13px 20px
 
+livechart
+  width: 30%
+  height: 30%
+  text-align: center
+
 @import './job'
 @import './graph'
 @import './timeline'
@@ -405,3 +410,13 @@ pre.exception
   .table
     td.td-long
       width: 30%
+
+#total-mem
+  background-color: #7cb5ec
+
+#heap-mem
+  background-color: #434348
+
+#non-heap-mem
+  background-color: #90ed7d
+

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/bower.json
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/bower.json b/flink-runtime-web/web-dashboard/bower.json
index 6aaf4e4..b7e7ec8 100644
--- a/flink-runtime-web/web-dashboard/bower.json
+++ b/flink-runtime-web/web-dashboard/bower.json
@@ -23,7 +23,8 @@
     "dagre-d3": "~0.4.10",
     "font-awesome": "~4.3.0",
     "moment-duration-format": "~1.3.0",
-    "qtip2": "~2.2.1"
+    "qtip2": "~2.2.1",
+    "highcharts-release": "~4.1.8"
   },
   "overrides": {
     "dagre-d3": {

http://git-wip-us.apache.org/repos/asf/flink/blob/58ab14b5/flink-runtime-web/web-dashboard/web/css/index.css
----------------------------------------------------------------------
diff --git a/flink-runtime-web/web-dashboard/web/css/index.css b/flink-runtime-web/web-dashboard/web/css/index.css
index 5a89193..394e640 100644
--- a/flink-runtime-web/web-dashboard/web/css/index.css
+++ b/flink-runtime-web/web-dashboard/web/css/index.css
@@ -424,6 +424,11 @@ pre.exception {
 .navbar-secondary-additional .navbar-title {
   padding: 12px 20px 13px 20px;
 }
+livechart {
+  width: 30%;
+  height: 30%;
+  text-align: center;
+}
 .canvas-wrapper {
   border: 1px solid #ddd;
   position: relative;
@@ -577,3 +582,12 @@ svg.graph .node-label {
     width: 30%;
   }
 }
+#total-mem {
+  background-color: #7cb5ec;
+}
+#heap-mem {
+  background-color: #434348;
+}
+#non-heap-mem {
+  background-color: #90ed7d;
+}