You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by ss...@apache.org on 2015/02/18 23:41:01 UTC

[14/23] tez git commit: TEZ-2106. TEZ UI: Display data load time, and add a refresh button for items that can be refreshed. (Sreenath Somarajapuram via hitesh)

TEZ-2106. TEZ UI: Display data load time, and add a refresh button for items that can be refreshed. (Sreenath Somarajapuram via hitesh)


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

Branch: refs/heads/TEZ-2003
Commit: dd44bdec1394975e5e3dd0dd97532dd6ca3c10ce
Parents: ec7ad20
Author: Hitesh Shah <hi...@apache.org>
Authored: Wed Feb 18 10:03:56 2015 -0800
Committer: Hitesh Shah <hi...@apache.org>
Committed: Wed Feb 18 10:03:56 2015 -0800

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../app/scripts/components/counter-table.js     |  2 +-
 .../scripts/components/load-time-component.js   | 34 +++++++++
 .../controllers/dag-task-attempts-controller.js | 49 ++++++++++--
 .../scripts/controllers/dag-view-controller.js  | 18 +++++
 .../app/scripts/controllers/dag_controller.js   |  4 +
 .../scripts/controllers/dag_index_controller.js | 73 +++++++++++-------
 .../webapp/app/scripts/controllers/dag_tasks.js | 20 +++++
 .../app/scripts/controllers/dag_vertices.js     | 21 +++++-
 .../app/scripts/controllers/dags_controller.js  | 32 +++++---
 .../scripts/controllers/shared-controllers.js   | 24 ++++++
 .../app/scripts/controllers/task_controller.js  |  2 +-
 .../controllers/task_index_controller.js        |  2 +-
 .../task_task_attempts_controller.js            |  9 ++-
 .../scripts/controllers/tez-app-controller.js   |  2 +-
 .../controllers/tez-app-dags-controller.js      | 79 ++++++++++++--------
 .../controllers/tez-app-index-controller.js     | 47 ++++++++++++
 .../scripts/controllers/vertex_controller.js    |  4 +-
 .../controllers/vertex_index_controller.js      | 18 ++++-
 .../vertex_task_attempts_controller.js          | 50 +++++++++++--
 .../controllers/vertex_tasks_controller.js      | 19 +++++
 .../src/main/webapp/app/scripts/helpers/misc.js | 13 ++++
 .../app/scripts/mixins/model-refresh-mixin.js   | 34 +++++++++
 .../app/scripts/mixins/paginated_content.js     | 20 +++--
 .../app/scripts/models/abstract_entity.js       | 21 ++++--
 .../src/main/webapp/app/scripts/models/dag.js   | 10 +--
 tez-ui/src/main/webapp/app/styles/main.less     | 35 ++++++++-
 tez-ui/src/main/webapp/app/styles/shared.less   |  4 +
 .../webapp/app/templates/common/counters.hbs    | 12 ++-
 .../webapp/app/templates/common/swimlanes.hbs   |  5 ++
 .../app/templates/common/table-with-spinner.hbs |  7 +-
 .../app/templates/components/counter-table.hbs  |  6 +-
 .../app/templates/components/load-time.hbs      | 37 +++++++++
 .../src/main/webapp/app/templates/dag/index.hbs | 10 ++-
 .../src/main/webapp/app/templates/dag/view.hbs  |  7 ++
 .../webapp/app/templates/partials/configs.hbs   |  7 +-
 .../app/templates/partials/table-controls.hbs   |  2 +-
 .../main/webapp/app/templates/task/index.hbs    | 10 ++-
 .../webapp/app/templates/taskAttempt/index.hbs  | 10 ++-
 .../main/webapp/app/templates/tez-app/index.hbs | 11 ++-
 .../webapp/app/templates/vertex/additionals.hbs |  7 ++
 .../main/webapp/app/templates/vertex/index.hbs  |  7 ++
 42 files changed, 658 insertions(+), 127 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index cc481c2..a566a37 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -64,6 +64,7 @@ Release 0.6.1: Unreleased
 INCOMPATIBLE CHANGES
 
 ALL CHANGES:
+  TEZ-2106. TEZ UI: Display data load time, and add a refresh button for items that can be refreshed.
   TEZ-2114. Tez UI: task/task attempt status is not available when its running.
   TEZ-2112. Tez UI: fix offset calculation, add home button to breadcrumbs.
   TEZ-2038. TEZ-UI DAG is always running in tez-ui when the app is failed but no DAGFinishedEvent is logged.

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/components/counter-table.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/components/counter-table.js b/tez-ui/src/main/webapp/app/scripts/components/counter-table.js
index ca58f52..781df32 100644
--- a/tez-ui/src/main/webapp/app/scripts/components/counter-table.js
+++ b/tez-ui/src/main/webapp/app/scripts/components/counter-table.js
@@ -49,7 +49,7 @@ App.CounterTableComponent = Em.Component.extend({
     })
 
     return filtered;
-  }.property('data', 'nameFilter')
+  }.property('data', 'nameFilter', 'timeStamp')
 });
 
 Em.Handlebars.helper('counter-table-component', App.CounterTableComponent);

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/components/load-time-component.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/components/load-time-component.js b/tez-ui/src/main/webapp/app/scripts/components/load-time-component.js
new file mode 100644
index 0000000..648a395
--- /dev/null
+++ b/tez-ui/src/main/webapp/app/scripts/components/load-time-component.js
@@ -0,0 +1,34 @@
+/**
+ * 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.
+ */
+
+App.LoadTimeComponent = Em.Component.extend({
+  layoutName: 'components/load-time',
+
+  actions: {
+    refresh: function() {
+      this.sendAction('refresh');
+    }
+  },
+
+  displayTime: function() {
+    var time = this.get('time');
+    return time ? App.Helpers.date.dateFormat(time.getTime(), true) : null;
+  }.property('time')
+});
+
+Em.Handlebars.helper('load-time-component', App.LoadTimeComponent);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js
index 50df0bf..e2f5fad 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag-task-attempts-controller.js
@@ -40,6 +40,24 @@ App.DagTaskAttemptsController = Em.ObjectController.extend(App.PaginatedContentM
     this.setFiltersAndLoadEntities(filters);
   },
 
+  load: function () {
+    var dag = this.get('controllers.dag.model'),
+        controller = this.get('controllers.dag'),
+        t = this;
+    t.set('loading', true);
+    dag.reload().then(function () {
+      return controller.loadAdditional(dag);
+    }).then(function () {
+      t.resetNavigation();
+      t.loadEntities();
+    }).catch(function(error){
+      Em.Logger.error(error);
+      var err = App.Helpers.misc.formatError(error, defaultErrMsg);
+      var msg = 'error code: %@, message: %@'.fmt(err.errCode, err.msg);
+      App.Helpers.ErrorBar.getInstance().show(msg, err.details);
+    });
+  }.observes('count'),
+
   actions : {
     filterUpdated: function(filterID, value) {
       // any validations required goes here.
@@ -52,9 +70,32 @@ App.DagTaskAttemptsController = Em.ObjectController.extend(App.PaginatedContentM
     }
   },
 
+  updateLoading: function () {
+    var dagController = this.get('controllers.dag'),
+        model = this.get('controllers.dag.model'),
+        that = this,
+        dagStatus = that.get('controllers.dag.status');
+
+    dagController.loadAdditional(model).then(function () {
+      that.get('entities').forEach(function (attempt) {
+
+        var attemptStatus = App.Helpers.misc
+          .getFixedupDisplayStatus(attempt.get('status'));
+        if (attemptStatus == 'RUNNING' &&
+          App.Helpers.misc.isStatusInUnsuccessful(dagStatus)) {
+          attemptStatus = 'KILLED'
+        }
+        if (attemptStatus != attempt.get('status')) {
+          attempt.set('status', attemptStatus);
+        }
+      });
+
+      that.set('loading', false);
+    });
+  },
+
   defaultColumnConfigs: function() {
     var that = this;
-    var dagStatus = this.get('controllers.dag.status');
     return [
       {
         id: 'taskId',
@@ -126,11 +167,7 @@ App.DagTaskAttemptsController = Em.ObjectController.extend(App.PaginatedContentM
             &nbsp;&nbsp;{{view.cellContent.status}}</span>')
         }),
         getCellContent: function(row) {
-          var status = App.Helpers.misc.getFixedupDisplayStatus(row.get('status'));
-          if (status == 'RUNNING' &&
-            App.Helpers.misc.isStatusInUnsuccessful(dagStatus)) {
-            status = 'KILLED';
-          }
+          var status = row.get('status');
           return {
             status: status,
             statusIcon: App.Helpers.misc.getStatusClassForEntity(status)

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/dag-view-controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag-view-controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag-view-controller.js
index 31753d1..a748cdf 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag-view-controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag-view-controller.js
@@ -37,6 +37,24 @@ App.DagViewController = Em.ObjectController.extend(App.PaginatedContentMixin, Ap
     this.setFiltersAndLoadEntities(filters);
   },
 
+  load: function () {
+    var dag = this.get('controllers.dag.model'),
+        controller = this.get('controllers.dag'),
+        t = this;
+    t.set('loading', true);
+    dag.reload().then(function () {
+      return controller.loadAdditional(dag);
+    }).then(function () {
+      t.resetNavigation();
+      t.loadEntities();
+    }).catch(function(error){
+      Em.Logger.error(error);
+      var err = App.Helpers.misc.formatError(error, defaultErrMsg);
+      var msg = 'error code: %@, message: %@'.fmt(err.errCode, err.msg);
+      App.Helpers.ErrorBar.getInstance().show(msg, err.details);
+    });
+  },
+
   actions: {
     entityClicked: function (details) {
       switch(details.type) {

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js
index c8d9c87..b3f15be 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_controller.js
@@ -25,8 +25,10 @@ App.DagController = Em.ObjectController.extend(App.Helpers.DisplayHelper, {
     var that = this;
     var loaders = [];
     var applicationId = dag.get('applicationId');
+
     if (dag.get('status') === 'RUNNING') {
       // update the progress info if available. this need not block the UI
+      App.Helpers.misc.removeRecord(this.store, 'dagProgress', dag.get('id'));
       var aminfoLoader = that.store.find('dagProgress', dag.get('id'), {
         appId: applicationId,
         dagIdx: dag.get('idx')
@@ -37,6 +39,7 @@ App.DagController = Em.ObjectController.extend(App.Helpers.DisplayHelper, {
       });
       loaders.push(aminfoLoader);
     }
+    App.Helpers.misc.removeRecord(this.store, 'appDetail', applicationId);
     var appDetailLoader = this.store.find('appDetail', applicationId)
       .then(function(app){
         dag.set('appDetail', app);
@@ -46,6 +49,7 @@ App.DagController = Em.ObjectController.extend(App.Helpers.DisplayHelper, {
         }
         dag.set('status', App.Helpers.misc.getRealStatus(dag.get('status'), app.get('appState'), app.get('finalAppStatus')));
       }).catch(function(){});
+    App.Helpers.misc.removeRecord(this.store, 'tezApp', 'tez_' + applicationId);
     var tezAppLoader = this.store.find('tezApp', 'tez_' + applicationId)
       .then(function(app){
         dag.set('tezApp', app);

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js
index 3261787..7576149 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_index_controller.js
@@ -17,12 +17,29 @@
  */
 
  //TODO: watch individual counters.
-App.DagIndexController = Em.ObjectController.extend({
-	controllerName: 'DagIndexController',
+App.DagIndexController = Em.ObjectController.extend(App.ModelRefreshMixin, {
+  controllerName: 'DagIndexController',
 
-	taskIconStatus: function() {
-		return App.Helpers.misc.getStatusClassForEntity(this.get('model.status'));
-	}.property('id', 'status', 'counterGroups'),
+  needs: 'dag',
+
+  load: function () {
+    var dag = this.get('controllers.dag.model'),
+        controller = this.get('controllers.dag'),
+        t = this;
+    t.set('loading', true);
+    dag.reload().then(function () {
+      return controller.loadAdditional(dag);
+    }).catch(function(error){
+      Em.Logger.error(error);
+      var err = App.Helpers.misc.formatError(error, defaultErrMsg);
+      var msg = 'error code: %@, message: %@'.fmt(err.errCode, err.msg);
+      App.Helpers.ErrorBar.getInstance().show(msg, err.details);
+    });
+  },
+
+  taskIconStatus: function() {
+    return App.Helpers.misc.getStatusClassForEntity(this.get('model.status'));
+  }.property('id', 'status', 'counterGroups'),
 
   progressStr: function() {
     var pct;
@@ -32,29 +49,29 @@ App.DagIndexController = Em.ObjectController.extend({
     return pct;
   }.property('id', 'status', 'progress'),
 
-	totalTasks: function() {
-		return App.Helpers.misc.getCounterValueForDag(this.get('counterGroups'), 
-			this.get('id'), 'org.apache.tez.common.counters.DAGCounter', 'TOTAL_LAUNCHED_TASKS')
-	}.property('id', 'counterGroups'),
-
-	sucessfulTasks: function() {
-		return App.Helpers.misc.getCounterValueForDag(this.get('counterGroups'), this.get('id'),
-			'org.apache.tez.common.counters.DAGCounter', 'NUM_SUCCEEDED_TASKS')
-	}.property('id', 'counterGroups'),
-
-	failedTasks: function() {
-		return App.Helpers.misc.getCounterValueForDag(this.get('counterGroups'), this.get('id'),
-			'org.apache.tez.common.counters.DAGCounter', 'NUM_FAILED_TASKS')
-	}.property('id', 'counterGroups'),
-
-	killedTasks: function() {
-		return App.Helpers.misc.getCounterValueForDag(this.get('counterGroups'), this.get('id'),
-			'org.apache.tez.common.counters.DAGCounter', 'NUM_KILLED_TASKS')
-	}.property('id', 'counterGroups'),
-
-	hasFailedTasks: function() {
-		return this.get('failedTasks') > 0;
-	}.property('id', 'counterGroups'),
+  totalTasks: function() {
+    return App.Helpers.misc.getCounterValueForDag(this.get('counterGroups'),
+      this.get('id'), 'org.apache.tez.common.counters.DAGCounter', 'TOTAL_LAUNCHED_TASKS')
+  }.property('id', 'counterGroups'),
+
+  sucessfulTasks: function() {
+    return App.Helpers.misc.getCounterValueForDag(this.get('counterGroups'), this.get('id'),
+      'org.apache.tez.common.counters.DAGCounter', 'NUM_SUCCEEDED_TASKS')
+  }.property('id', 'counterGroups'),
+
+  failedTasks: function() {
+    return App.Helpers.misc.getCounterValueForDag(this.get('counterGroups'), this.get('id'),
+      'org.apache.tez.common.counters.DAGCounter', 'NUM_FAILED_TASKS')
+  }.property('id', 'counterGroups'),
+
+  killedTasks: function() {
+    return App.Helpers.misc.getCounterValueForDag(this.get('counterGroups'), this.get('id'),
+      'org.apache.tez.common.counters.DAGCounter', 'NUM_KILLED_TASKS')
+  }.property('id', 'counterGroups'),
+
+  hasFailedTasks: function() {
+    return this.get('failedTasks') > 0;
+  }.property('id', 'counterGroups'),
 
   failedTasksLink: function() {
     return '#tasks?status=FAILED&parentType=TEZ_DAG_ID&parentID=' + this.get('id');

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
index 93b9234..2b1304d 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_tasks.js
@@ -48,6 +48,24 @@ App.DagTasksController = Em.ObjectController.extend(App.PaginatedContentMixin, A
     this.setFiltersAndLoadEntities(filters);
   },
 
+  load: function () {
+    var dag = this.get('controllers.dag.model'),
+        controller = this.get('controllers.dag'),
+        t = this;
+    t.set('loading', true);
+    dag.reload().then(function () {
+      return controller.loadAdditional(dag);
+    }).then(function () {
+      t.resetNavigation();
+      t.loadEntities();
+    }).catch(function(error){
+      Em.Logger.error(error);
+      var err = App.Helpers.misc.formatError(error, defaultErrMsg);
+      var msg = 'error code: %@, message: %@'.fmt(err.errCode, err.msg);
+      App.Helpers.ErrorBar.getInstance().show(msg, err.details);
+    });
+  }.observes('count'),
+
   loadEntities: function() {
     var that = this,
     store = this.get('store'),
@@ -56,6 +74,7 @@ App.DagTasksController = Em.ObjectController.extend(App.PaginatedContentMixin, A
     var defaultErrMsg = 'Error while loading tasks. could not connect to %@'
       .fmt(App.env.timelineBaseUrl);
 
+    that.set('loading', true);
     store.unloadAll(childEntityType);
     store.findQuery(childEntityType, this.getFilterProperties())
       .then(function(entities){
@@ -76,6 +95,7 @@ App.DagTasksController = Em.ObjectController.extend(App.PaginatedContentMixin, A
             task.get('attempts.lastObject');
         if (!!taskAttemptId) {
           // Pivot attempt selection logic
+          App.Helpers.misc.removeRecord(store, 'taskAttempt', taskAttemptId);
           fetcher = store.find('taskAttempt', taskAttemptId);
           fetcher.then(function (attempt) {
             task.set('pivotAttempt', attempt);

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
index f63524a..44ffd1a 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_vertices.js
@@ -44,10 +44,28 @@ App.DagVerticesController = Em.ObjectController.extend(App.PaginatedContentMixin
     this.setFiltersAndLoadEntities(filters);
   },
 
+  load: function () {
+    var dag = this.get('controllers.dag.model'),
+        controller = this.get('controllers.dag'),
+        t = this;
+    t.set('loading', true);
+    dag.reload().then(function () {
+      return controller.loadAdditional(dag);
+    }).then(function () {
+      t.resetNavigation();
+      t.loadEntities();
+    }).catch(function(error){
+      Em.Logger.error(error);
+      var err = App.Helpers.misc.formatError(error, defaultErrMsg);
+      var msg = 'error code: %@, message: %@'.fmt(err.errCode, err.msg);
+      App.Helpers.ErrorBar.getInstance().show(msg, err.details);
+    });
+  }.observes('count'),
+
   loadAdditional: function() {
     var defer = Em.RSVP.defer();
 
-    var that = this;
+    var that = this,
         vertices = this.get('entities');
 
     var dagStatus = this.get('controllers.dag.status');
@@ -64,6 +82,7 @@ App.DagVerticesController = Em.ObjectController.extend(App.PaginatedContentMixin
         return item.get('id').split('_').splice(-1).pop();
       });
     if (runningVerticesIdx.length > 0) {
+      this.store.unloadAll('vertexProgress');
       this.store.findQuery('vertexProgress', {
         metadata: {
           appId: that.get('applicationId'),

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js
index 35a0ecc..880378a 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dags_controller.js
@@ -65,33 +65,42 @@ App.DagsController = Em.ObjectController.extend(App.PaginatedContentMixin, App.C
     var that = this,
     store = this.get('store'),
     childEntityType = this.get('childEntityType'),
-    fetcher;
+    fetcher,
+    record;
     var defaultErrMsg = 'Error while loading dag info. could not connect to %@'.fmt(App.env.timelineBaseUrl);
 
+    that.set('loading', true);
     store.unloadAll(childEntityType);
+    store.unloadAll('dagProgress');
+
     store.findQuery(childEntityType, this.getFilterProperties()).then(function(entities){
       var loaders = [];
-      that.set('entities', entities);
       entities.forEach(function (dag) {
         var appId = dag.get('applicationId');
         if(appId) {
           // Pivot attempt selection logic
-          fetcher = store.find('appDetail', appId);
-          fetcher.then(function (app) {
+          record = store.getById('appDetail', appId);
+          if(record && !App.Helpers.misc.isStatusInUnsuccessful(record.get('appState'))) {
+            store.unloadRecord(record);
+          }
+          fetcher = store.find('appDetail', appId).then(function (app) {
             dag.set('appDetail', app);
             if (dag.get('status') === 'RUNNING') {
-              dag.set('status', App.Helpers.misc.getRealStatus(dag.get('status'), app.get('appState'),
-                app.get('finalAppStatus')));
+              dag.set('status', App.Helpers.misc.getRealStatus(
+                dag.get('status'),
+                app.get('appState'),
+                app.get('finalAppStatus')
+              ));
+              App.Helpers.misc.removeRecord(store, 'tezApp', 'tez_' + appId);
             }
+            return store.find('tezApp', 'tez_' + appId).then(function (app) {
+              dag.set('tezApp', app);
+            });
           });
           loaders.push(fetcher);
           //Load tezApp details
-          fetcher = store.find('tezApp', 'tez_' + appId);
-          fetcher.then(function (app) {
-            dag.set('tezApp', app);
-          });
-          loaders.push(fetcher);
           if (dag.get('status') === 'RUNNING') {
+            App.Helpers.misc.removeRecord(store, 'dagProgress', dag.get('id'));
             amInfoFetcher = store.find('dagProgress', dag.get('id'), {
               appId: dag.get('applicationId'),
               dagIdx: dag.get('idx')
@@ -107,6 +116,7 @@ App.DagsController = Em.ObjectController.extend(App.PaginatedContentMixin, App.C
         }
       });
       Em.RSVP.allSettled(loaders).then(function(){
+        that.set('entities', entities);
         that.set('loading', false);
       });
     }).catch(function(error){

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/shared-controllers.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/shared-controllers.js b/tez-ui/src/main/webapp/app/scripts/controllers/shared-controllers.js
new file mode 100644
index 0000000..70d5e8c
--- /dev/null
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/shared-controllers.js
@@ -0,0 +1,24 @@
+/**
+ * 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.
+ */
+
+App.DagCountersController =
+    App.VertexCountersController =
+    App.TaskCountersController =
+    App.TaskAttemptCountersController =
+
+    Em.ObjectController.extend(App.ModelRefreshMixin);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/task_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/task_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/task_controller.js
index ac4b22b..4f33b3b 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/task_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/task_controller.js
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-App.TaskController = Em.ObjectController.extend(App.Helpers.DisplayHelper, {
+App.TaskController = Em.ObjectController.extend(App.Helpers.DisplayHelper, App.ModelRefreshMixin, {
 	controllerName: 'TaskController',
 
 	pageTitle: 'Task',

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/task_index_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/task_index_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/task_index_controller.js
index 7f3eeb3..2c526c4 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/task_index_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/task_index_controller.js
@@ -17,7 +17,7 @@
  */
 
  //TODO: watch individual counters.
-App.TaskIndexController = Em.ObjectController.extend({
+App.TaskIndexController = Em.ObjectController.extend(App.ModelRefreshMixin, {
   controllerName: 'TaskIndexController',
 
   taskStatus: function() {

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js
index 0749e00..b400375 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/task_task_attempts_controller.js
@@ -48,10 +48,8 @@ App.TaskAttemptsController = Em.ObjectController.extend(App.PaginatedContentMixi
       .fmt(childEntityType, App.env.timelineBaseUrl);
 
     that.set('loading', true);
-
     this.get('store').unloadAll(childEntityType);
     this.get('store').findQuery(childEntityType, this.getFilterProperties()).then(function(entities){
-      that.set('entities', entities);
       var loaders = [];
       try {
         var loader = Em.tryInvoke(that, 'loadAdditional');
@@ -62,8 +60,10 @@ App.TaskAttemptsController = Em.ObjectController.extend(App.PaginatedContentMixi
         Em.Logger.error("Exception invoking additional load", error);
       }
 
+      App.Helpers.misc.removeRecord(that.store, 'dag', that.get('controllers.task.dagID'));
       var appDetailFetcher = that.store.find('dag', that.get('controllers.task.dagID')).
         then(function (dag) {
+          App.Helpers.misc.removeRecord(that.store, 'appDetail', dag.get('applicationId'));
           return that.store.find('appDetail', dag.get('applicationId'));
         }).
         then(function(appDetail) {
@@ -74,7 +74,8 @@ App.TaskAttemptsController = Em.ObjectController.extend(App.PaginatedContentMixi
         });
       loaders.push(appDetailFetcher);
       Em.RSVP.allSettled(loaders).then(function(){
-          that.set('loading', false);
+        that.set('entities', entities);
+        that.set('loading', false);
       });
     }).catch(function(error){
       Em.Logger.error(error);
@@ -253,7 +254,7 @@ App.TaskAttemptsController = Em.ObjectController.extend(App.PaginatedContentMixi
 });
 
 
-App.TaskAttemptIndexController = Em.ObjectController.extend({
+App.TaskAttemptIndexController = Em.ObjectController.extend(App.ModelRefreshMixin, {
   controllerName: 'TaskAttemptIndexController',
 
   taskAttemptStatus: function() {

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-controller.js
index 3df83fd..1941b3f 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-controller.js
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-App.TezAppController = Em.ObjectController.extend(App.Helpers.DisplayHelper, {
+App.TezAppController = Em.ObjectController.extend(App.Helpers.DisplayHelper, App.ModelRefreshMixin, {
   controllerName: 'AppController',
 
   pageTitle: 'App',

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js
index 151438e..5d1ae0b 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-dags-controller.js
@@ -48,48 +48,61 @@ App.TezAppDagsController = Em.ObjectController.extend(App.PaginatedContentMixin,
   },
 
   loadEntities: function() {
-    var that = this;
-    var childEntityType = this.get('childEntityType');
-    var defaultErrMsg = 'Error while loading %@. could not connect to %@'
-      .fmt(childEntityType, App.env.timelineBaseUrl);
-
+    var that = this,
+    store = this.get('store'),
+    childEntityType = this.get('childEntityType'),
+    fetcher,
+    record;
+    var defaultErrMsg = 'Error while loading dag info. could not connect to %@'.fmt(App.env.timelineBaseUrl);
 
     that.set('loading', true);
+    store.unloadAll(childEntityType);
+    store.unloadAll('dagProgress');
 
-    this.get('store').unloadAll(childEntityType);
-    this.get('store').findQuery(childEntityType, this.getFilterProperties())
-      .then(function(entities){
-
-      that.set('entities', entities);
-
+    store.findQuery(childEntityType, this.getFilterProperties()).then(function(entities){
       var loaders = [];
       entities.forEach(function (dag) {
-        var applicationId = dag.get('applicationId');
-        if (dag.get('status') === 'RUNNING') {
-          amInfoFetcher = that.store.find('dagProgress', dag.get('id'), {
-            appId: applicationId,
-            dagIdx: dag.get('idx')
-          }).then(function(dagProgressInfo) {
-              dag.set('progress', dagProgressInfo.get('progress'));
-          }).catch(function(error) {
-            Em.Logger.error('Failed to fetch dagProgress' + error);
+        var appId = dag.get('applicationId');
+        if(appId) {
+          // Pivot attempt selection logic
+          record = store.getById('appDetail', appId);
+          if(record && !App.Helpers.misc.isStatusInUnsuccessful(record.get('appState'))) {
+            store.unloadRecord(record);
+          }
+          fetcher = store.find('appDetail', appId).then(function (app) {
+            dag.set('appDetail', app);
+            if (dag.get('status') === 'RUNNING') {
+              dag.set('status', App.Helpers.misc.getRealStatus(
+                dag.get('status'),
+                app.get('appState'),
+                app.get('finalAppStatus')
+              ));
+              App.Helpers.misc.removeRecord(store, 'tezApp', 'tez_' + appId);
+            }
+            return store.find('tezApp', 'tez_' + appId).then(function (app) {
+              dag.set('tezApp', app);
+            });
           });
-          loaders.push(amInfoFetcher);
-        }
-
-        var appDetailLoader = that.store.find('appDetail', applicationId)
-          .then(function(app){
-          dag.set('appDetail', app);
-          var appState = app.get('appState');
-          if (appState) {
-            dag.set('yarnAppState', appState);
+          loaders.push(fetcher);
+          //Load tezApp details
+          if (dag.get('status') === 'RUNNING') {
+            App.Helpers.misc.removeRecord(store, 'dagProgress', dag.get('id'));
+            amInfoFetcher = store.find('dagProgress', dag.get('id'), {
+              appId: dag.get('applicationId'),
+              dagIdx: dag.get('idx')
+            })
+            .then(function(dagProgressInfo) {
+              dag.set('progress', dagProgressInfo.get('progress'));
+            })
+            .catch(function(error) {
+              Em.Logger.error('Failed to fetch dagProgress' + error);
+            });
+            loaders.push(amInfoFetcher);
           }
-          dag.set('status', App.Helpers.misc.getRealStatus(dag.get('status'),
-            app.get('appState'), app.get('finalAppStatus')));
-        });
+        }
       });
-
       Em.RSVP.allSettled(loaders).then(function(){
+        that.set('entities', entities);
         that.set('loading', false);
       });
     }).catch(function(error){

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-index-controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-index-controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-index-controller.js
new file mode 100644
index 0000000..3b11e3a
--- /dev/null
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/tez-app-index-controller.js
@@ -0,0 +1,47 @@
+/**
+ * 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.
+ */
+
+App.TezAppIndexController = Em.ObjectController.extend(App.ModelRefreshMixin, {
+
+  needs: "tezApp",
+  controllerName: 'TezAppIndexController',
+
+  load: function () {
+    var tezApp = this.get('model'),
+      store  = this.get('store');
+
+      tezApp.reload().then(function (tezApp) {
+        var appId = tezApp.get('appId');
+        if(!appId) return tezApp;
+        App.Helpers.misc.removeRecord(store, 'appDetail', appId);
+        return store.find('appDetail', appId).then(function (appDetails){
+          tezApp.set('appDetail', appDetails);
+          return tezApp;
+        });
+      }).catch(function (error) {
+        Em.Logger.error(error);
+        var err = App.Helpers.misc.formatError(error, defaultErrMsg);
+        var msg = 'error code: %@, message: %@'.fmt(err.errCode, err.msg);
+        App.Helpers.ErrorBar.getInstance().show(msg, err.details);
+      });
+  },
+
+  iconStatus: function() {
+    return App.Helpers.misc.getStatusClassForEntity(this.get('model.appDetail.finalAppStatus'));
+  }.property('id', 'appDetail.finalAppStatus'),
+});

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/vertex_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_controller.js
index 89a99a9..b575d83 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_controller.js
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-App.VertexController = Em.ObjectController.extend(App.Helpers.DisplayHelper, {
+App.VertexController = Em.ObjectController.extend(App.Helpers.DisplayHelper, App.ModelRefreshMixin, {
   controllerName: 'VertexController',
 
   pageTitle: 'Vertex',
@@ -30,6 +30,7 @@ App.VertexController = Em.ObjectController.extend(App.Helpers.DisplayHelper, {
 
     if (vertex.get('status') == 'RUNNING') {
       var vertexIdx = vertex.get('id').split('_').splice(-1).pop();
+      App.Helpers.misc.removeRecord(this.store, 'vertexProgress', vertexIdx);
       var progressLoader = this.store.find('vertexProgress', vertexIdx, {
         appId: applicationId,
         dagIdx: vertex.get('dagIdx')
@@ -43,6 +44,7 @@ App.VertexController = Em.ObjectController.extend(App.Helpers.DisplayHelper, {
       loaders.push(progressLoader);
     }
 
+    App.Helpers.misc.removeRecord(that.store, 'appDetail', applicationId);
     var appDetailFetcher = that.store.find('appDetail', applicationId).then(function(appDetail) {
       var appState = appDetail.get('appState');
       if (appState) {

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
index f175ed1..f104c11 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_index_controller.js
@@ -16,9 +16,25 @@
  * limitations under the License.
  */
 
-App.VertexIndexController = Em.ObjectController.extend({
+App.VertexIndexController = Em.ObjectController.extend(App.ModelRefreshMixin, {
   controllerName: 'VertexIndexController',
 
+  needs: 'vertex',
+
+  load: function () {
+    var vertex = this.get('controllers.vertex.model'),
+        controller = this.get('controllers.vertex'),
+        t = this;
+    vertex.reload().then(function () {
+      return controller.loadAdditional(vertex);
+    }).catch(function(error){
+      Em.Logger.error(error);
+      var err = App.Helpers.misc.formatError(error, defaultErrMsg);
+      var msg = 'error code: %@, message: %@'.fmt(err.errCode, err.msg);
+      App.Helpers.ErrorBar.getInstance().show(msg, err.details);
+    });
+  },
+
   //TODO: TEZ-1705 : Create a parent class and move this function there to avoid duplication.
   iconStatus: function() {
     return App.Helpers.misc.getStatusClassForEntity(this.get('model.status'));

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js
index 5f51de4..c335615 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_task_attempts_controller.js
@@ -41,6 +41,23 @@ App.VertexTaskAttemptsController = Em.ObjectController.extend(App.PaginatedConte
     this.setFiltersAndLoadEntities(filters);
   },
 
+  load: function () {
+    var vertex = this.get('controllers.vertex.model'),
+        controller = this.get('controllers.vertex'),
+        t = this;
+    vertex.reload().then(function () {
+      return controller.loadAdditional(vertex);
+    }).then(function () {
+      t.resetNavigation();
+      t.loadEntities();
+    }).catch(function(error){
+      Em.Logger.error(error);
+      var err = App.Helpers.misc.formatError(error, defaultErrMsg);
+      var msg = 'error code: %@, message: %@'.fmt(err.errCode, err.msg);
+      App.Helpers.ErrorBar.getInstance().show(msg, err.details);
+    });
+  }.observes('count'),
+
   actions : {
     filterUpdated: function(filterID, value) {
       // any validations required goes here.
@@ -53,9 +70,32 @@ App.VertexTaskAttemptsController = Em.ObjectController.extend(App.PaginatedConte
     }
   },
 
-  defaultColumnConfigs: function() {
-    var that = this,
+  updateLoading: function () {
+    var controller = this.get('controllers.vertex'),
+        model = this.get('controllers.vertex.model'),
+        that = this,
         vertexStatus = that.get('controllers.vertex.status');
+
+    controller.loadAdditional(model).then(function () {
+      that.get('entities').forEach(function (attempt) {
+
+        var attemptStatus = App.Helpers.misc
+          .getFixedupDisplayStatus(attempt.get('status'));
+        if (attemptStatus == 'RUNNING' &&
+          App.Helpers.misc.isStatusInUnsuccessful(vertexStatus)) {
+          attemptStatus = 'KILLED'
+        }
+        if (attemptStatus != attempt.get('status')) {
+          attempt.set('status', attemptStatus);
+        }
+      });
+
+      that.set('loading', false);
+    });
+  },
+
+  defaultColumnConfigs: function() {
+    var that = this;
     return [
       {
         id: 'taskId',
@@ -127,11 +167,7 @@ App.VertexTaskAttemptsController = Em.ObjectController.extend(App.PaginatedConte
             &nbsp;&nbsp;{{view.cellContent.status}}</span>')
         }),
         getCellContent: function(row) {
-          var status = App.Helpers.misc.getFixedupDisplayStatus(row.get('status'));
-          if (status == 'RUNNING' &&
-            App.Helpers.misc.isStatusInUnsuccessful(vertexStatus)) {
-            status = 'KILLED'
-          }
+          var status = row.get('status');
           return {
             status: status,
             statusIcon: App.Helpers.misc.getStatusClassForEntity(status)

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
index 9b76aee..bba5aa0 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_tasks_controller.js
@@ -41,6 +41,23 @@ App.VertexTasksController = Em.ObjectController.extend(App.PaginatedContentMixin
     this.setFiltersAndLoadEntities(filters);
   },
 
+  load: function () {
+    var vertex = this.get('controllers.vertex.model'),
+        controller = this.get('controllers.vertex'),
+        t = this;
+    vertex.reload().then(function () {
+      return controller.loadAdditional(vertex);
+    }).then(function () {
+      t.resetNavigation();
+      t.loadEntities();
+    }).catch(function(error){
+      Em.Logger.error(error);
+      var err = App.Helpers.misc.formatError(error, defaultErrMsg);
+      var msg = 'error code: %@, message: %@'.fmt(err.errCode, err.msg);
+      App.Helpers.ErrorBar.getInstance().show(msg, err.details);
+    });
+  }.observes('count'),
+
   loadEntities: function() {
     var that = this,
     store = this.get('store'),
@@ -48,6 +65,7 @@ App.VertexTasksController = Em.ObjectController.extend(App.PaginatedContentMixin
     childEntityType = this.get('childEntityType');
     var defaultErrMsg = 'Error while loading tasks. could not connect to %@'.fmt(App.env.timelineBaseUrl);
 
+    that.set('loading', true);
     store.unloadAll(childEntityType);
     store.findQuery(childEntityType, this.getFilterProperties()).then(function(entities){
       var pivotLoaders = [],
@@ -65,6 +83,7 @@ App.VertexTasksController = Em.ObjectController.extend(App.PaginatedContentMixin
         var taskAttemptId = task.get('successfulAttemptId') || task.get('attempts.lastObject');
         if (!!taskAttemptId) {
           // Pivot attempt selection logic
+          App.Helpers.misc.removeRecord(store, 'taskAttempt', taskAttemptId);
           fetcher = store.find('taskAttempt', taskAttemptId);
           fetcher.then(function (attempt) {
             task.set('pivotAttempt', attempt);

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/helpers/misc.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/helpers/misc.js b/tez-ui/src/main/webapp/app/scripts/helpers/misc.js
index 187f800..cebe988 100644
--- a/tez-ui/src/main/webapp/app/scripts/helpers/misc.js
+++ b/tez-ui/src/main/webapp/app/scripts/helpers/misc.js
@@ -231,6 +231,19 @@ App.Helpers.misc = {
     });
   },
 
+  /**
+   * Remove the specific record from store
+   * @param store {DS.Store}
+   * @param type {String}
+   * @param id {String}
+   */
+  removeRecord: function (store, type, id) {
+    var record = store.getById(type, id);
+    if(record) {
+      store.unloadRecord(record);
+    }
+  },
+
   dagStatusUIOptions: [
     { label: 'All', id: null },
     { label: 'Submitted', id: 'SUBMITTED' },

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/mixins/model-refresh-mixin.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/mixins/model-refresh-mixin.js b/tez-ui/src/main/webapp/app/scripts/mixins/model-refresh-mixin.js
new file mode 100644
index 0000000..d11744e
--- /dev/null
+++ b/tez-ui/src/main/webapp/app/scripts/mixins/model-refresh-mixin.js
@@ -0,0 +1,34 @@
+/**
+ * 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.
+ */
+
+App.ModelRefreshMixin = Em.Mixin.create({
+  isRefreshable: true,
+
+  load: function () {
+    var model = this.get('content');
+    if(model && $.isFunction(model.reload)) {
+      model.reload();
+    }
+  },
+
+  actions: {
+    refresh: function () {
+      this.load();
+    }
+  }
+});

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/mixins/paginated_content.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/mixins/paginated_content.js b/tez-ui/src/main/webapp/app/scripts/mixins/paginated_content.js
index 7ce5f5e..7b08036 100644
--- a/tez-ui/src/main/webapp/app/scripts/mixins/paginated_content.js
+++ b/tez-ui/src/main/webapp/app/scripts/mixins/paginated_content.js
@@ -27,6 +27,8 @@ App.PaginatedContentMixin = Em.Mixin.create({
   // The dropdown contents for number of items to show.
   countOptions: [5, 10, 25, 50, 100],
 
+  isRefreshable: true,
+
   /* There is currently no efficient way in ATS to get pagination data, so we fake one.
    * store the first dag id on a page so that we can navigate back and store the last one 
    * (not shown on page to get the id where next page starts)
@@ -45,8 +47,9 @@ App.PaginatedContentMixin = Em.Mixin.create({
   _paginationFilters: {},
   loading: true,
 
-  countUpdated: function() {
-    this.loadData();
+  load: function() {
+    this.resetNavigation();
+    this.loadEntities();
   }.observes('count'),
 
   sortedContent: function() {
@@ -59,6 +62,10 @@ App.PaginatedContentMixin = Em.Mixin.create({
     return sorted.slice(0, this.count);
   }.property('entities', 'numEntities'),
 
+  updateLoading: function () {
+    this.set('loading', false);
+  },
+
   loadEntities: function() {
     var that = this;
     var childEntityType = this.get('childEntityType');
@@ -81,7 +88,7 @@ App.PaginatedContentMixin = Em.Mixin.create({
         Em.Logger.error("Exception invoking additional load", error);
       }
       Em.RSVP.allSettled(loaders).then(function(){
-          that.set('loading', false);
+        that.updateLoading();
       });
     }).catch(function(error){
       Em.Logger.error(error);
@@ -93,8 +100,7 @@ App.PaginatedContentMixin = Em.Mixin.create({
 
   setFiltersAndLoadEntities: function(filters) {
     this._paginationFilters = filters;
-    this.resetNavigation();
-    this.loadEntities();
+    this.load();
   },
 
   resetNavigation: function() {
@@ -135,6 +141,10 @@ App.PaginatedContentMixin = Em.Mixin.create({
       this.loadEntities();
     },
 
+    refresh: function () {
+      this.load();
+    },
+
     // goto first page.
     navigateFirst: function() {
       var firstPageId = this.navIDs.prevIDs[0];

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/models/abstract_entity.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/models/abstract_entity.js b/tez-ui/src/main/webapp/app/scripts/models/abstract_entity.js
index a3e7332..251d680 100644
--- a/tez-ui/src/main/webapp/app/scripts/models/abstract_entity.js
+++ b/tez-ui/src/main/webapp/app/scripts/models/abstract_entity.js
@@ -16,13 +16,24 @@
  */
 
 App.AbstractEntity = DS.Model.extend({
-	// type of the entity. should be one of App.EntityType
-	entityType: DS.attr('string')
+  // type of the entity. should be one of App.EntityType
+  entityType: DS.attr('string'),
+  timeStamp: null,
+
+  didLoad: function () {
+    this.set('timeStamp', new Date());
+  },
+
+  observeReloading: function () {
+    if(!this.get('isReloading')) {
+      this.didLoad();
+    }
+  }.observes('isReloading')
 });
 
 App.EntityType = {
-	DAG: 'dag',
-	VERTEX: 'vertex',
-	TASK: 'task',
+  DAG: 'dag',
+  VERTEX: 'vertex',
+  TASK: 'task',
   TASK_ATTEMPT: 'task_attempt',
 };

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/scripts/models/dag.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/models/dag.js b/tez-ui/src/main/webapp/app/scripts/models/dag.js
index e050037..9b844f7 100644
--- a/tez-ui/src/main/webapp/app/scripts/models/dag.js
+++ b/tez-ui/src/main/webapp/app/scripts/models/dag.js
@@ -95,7 +95,7 @@ App.Edge = DS.Model.extend({
   dag: DS.belongsTo('dag')
 });
 
-App.Vertex = DS.Model.extend({
+App.Vertex = App.AbstractEntity.extend({
   name: DS.attr('string'),
   vertexIdx: function() {
     return this.get('id').split('_').splice(-1).pop();
@@ -257,7 +257,7 @@ App.Vertex = DS.Model.extend({
   }.property('duration')
 });
 
-App.Input = DS.Model.extend({
+App.Input = App.AbstractEntity.extend({
   entity: DS.attr('string'),
 
   inputName: DS.attr('string'),
@@ -267,7 +267,7 @@ App.Input = DS.Model.extend({
   configs: DS.hasMany('kVData', { async: false })
 });
 
-App.Output = DS.Model.extend({
+App.Output = App.AbstractEntity.extend({
   entity: DS.attr('string'),
 
   outputName: DS.attr('string'),
@@ -276,7 +276,7 @@ App.Output = DS.Model.extend({
   configs: DS.hasMany('kVData', { async: false })
 });
 
-App.AppDetail = DS.Model.extend({
+App.AppDetail = App.AbstractEntity.extend({
   attemptId: DS.attr('string'),
 
   user: DS.attr('string'),
@@ -296,7 +296,7 @@ App.AppDetail = DS.Model.extend({
   diagnostics: DS.attr('string'),
 });
 
-App.TezApp = DS.Model.extend({
+App.TezApp = App.AbstractEntity.extend({
   appId: DS.attr('string'),
   entityType: DS.attr('string'),
   domain: DS.attr('string'),

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/styles/main.less
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/styles/main.less b/tez-ui/src/main/webapp/app/styles/main.less
index b29fa6e..be8b228 100644
--- a/tez-ui/src/main/webapp/app/styles/main.less
+++ b/tez-ui/src/main/webapp/app/styles/main.less
@@ -117,6 +117,40 @@ body, html, body > .ember-view {
     }
   }
 
+  .table-control {
+    .align-children-left {
+      vertical-align: top;
+      padding-top: 4px;
+    }
+    .align-children-right {
+      margin-top: -8px;
+    }
+  }
+
+  .load-component {
+    margin-bottom: 5px;
+
+    .panel-body {
+      padding-top: 5px;
+      padding-bottom: 5px;
+    }
+
+    .fa-clock-o {
+      .fa-2x;
+
+      position: relative;
+      top: 2px;
+    }
+
+    .align-children-right {
+      vertical-align: top;
+    }
+
+    .margin-small-horizontal {
+      vertical-align: middle;
+    }
+  }
+
   .navbar {
     margin-bottom: 0px;
   }
@@ -410,7 +444,6 @@ div.indent {
 
 .kv-table {
   width: 100%;
-  margin-top: 10px;
   table-layout: fixed;
 
   th, td {

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/styles/shared.less
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/styles/shared.less b/tez-ui/src/main/webapp/app/styles/shared.less
index 341bb10..617c87f 100644
--- a/tez-ui/src/main/webapp/app/styles/shared.less
+++ b/tez-ui/src/main/webapp/app/styles/shared.less
@@ -33,6 +33,10 @@
   display: none !important;
 }
 
+.no-visible {
+  visibility: hidden !important;
+}
+
 .no-pointer {
   pointer-events: none;
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/common/counters.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/common/counters.hbs b/tez-ui/src/main/webapp/app/templates/common/counters.hbs
index e41e8f4..30419e1 100644
--- a/tez-ui/src/main/webapp/app/templates/common/counters.hbs
+++ b/tez-ui/src/main/webapp/app/templates/common/counters.hbs
@@ -16,6 +16,14 @@
 * limitations under the License.
 }}
 
-<div class='table-container'>
-  {{counter-table-component data=counterGroups}}
+{{load-time-component
+  isRefreshable=isRefreshable
+  time=timeStamp
+  refresh='refresh'
+}}
+<div class='table-container margin-small-vertical'>
+  {{counter-table-component
+    data=counterGroups
+    timeStamp=timeStamp
+  }}
 </div>

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/common/swimlanes.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/common/swimlanes.hbs b/tez-ui/src/main/webapp/app/templates/common/swimlanes.hbs
index 2745e08..4d051b8 100644
--- a/tez-ui/src/main/webapp/app/templates/common/swimlanes.hbs
+++ b/tez-ui/src/main/webapp/app/templates/common/swimlanes.hbs
@@ -16,6 +16,11 @@
 * limitations under the License.
 }}
 
+{{load-time-component
+  isRefreshable=isRefreshable
+  time=controller.model.content.0.timeStamp
+  refresh='refresh'
+}}
 <div id="swimlane">
   {{#view App.SwimlanesView contentBinding="controller.model.content"}}
   {{/view}}

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/common/table-with-spinner.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/common/table-with-spinner.hbs b/tez-ui/src/main/webapp/app/templates/common/table-with-spinner.hbs
index e1c154e..88b5d63 100644
--- a/tez-ui/src/main/webapp/app/templates/common/table-with-spinner.hbs
+++ b/tez-ui/src/main/webapp/app/templates/common/table-with-spinner.hbs
@@ -17,7 +17,12 @@
 }}
 
 {{#unless loading}}
-  <div class='margin-small align-children-right'>
+  {{load-time-component
+    isRefreshable=isRefreshable
+    time=sortedContent.0.timeStamp
+    refresh='refresh'
+  }}
+  <div class='align-children-right'>
     {{partial 'partials/table-controls'}}
   </div>
   {{partial 'partials/table'}}

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/components/counter-table.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/components/counter-table.hbs b/tez-ui/src/main/webapp/app/templates/components/counter-table.hbs
index e8d12b1..e9070f1 100644
--- a/tez-ui/src/main/webapp/app/templates/components/counter-table.hbs
+++ b/tez-ui/src/main/webapp/app/templates/components/counter-table.hbs
@@ -29,12 +29,12 @@
     <tbody>
       {{#each counterGroup in filteredData}}
         <tr class='group-header'>
-          <td colspan='2'>{{unbound counterGroup.displayName}}</td>
+          <td colspan='2'>{{counterGroup.displayName}}</td>
         </tr>
         {{#each counter in counterGroup.counters}}
           <tr>
-            <td>{{unbound counter.displayName}}</td>
-            <td>{{unbound formatNumThousands counter.value}}</td>
+            <td>{{counter.displayName}}</td>
+            <td>{{formatNumThousands counter.value}}</td>
           </tr>
         {{/each}}
       {{/each}}

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/components/load-time.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/components/load-time.hbs b/tez-ui/src/main/webapp/app/templates/components/load-time.hbs
new file mode 100644
index 0000000..3f6881f
--- /dev/null
+++ b/tez-ui/src/main/webapp/app/templates/components/load-time.hbs
@@ -0,0 +1,37 @@
+{{!
+* 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 class="panel panel-default load-component">
+  <div class="panel-body">
+    <div class="horizontal-half">
+      <i class='fa fa-clock-o'></i>
+    </div><div class="horizontal-half align-children-right">
+      <span class="margin-small-horizontal">
+        {{#if displayTime}}
+          Last refreshed at <b>{{displayTime}}</b>
+        {{else}}
+          Load time not available!
+        {{/if}}
+      </span>
+      <button type="button" {{bind-attr class=":btn :btn-success :btn-sm isRefreshable::no-display"}} {{action 'refresh'}}>
+        <i class='fa fa-refresh'></i>
+        Refresh
+      </button>
+    </div>
+  </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/dag/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/dag/index.hbs b/tez-ui/src/main/webapp/app/templates/dag/index.hbs
index 7e9dbcb..bc6b1d3 100644
--- a/tez-ui/src/main/webapp/app/templates/dag/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/dag/index.hbs
@@ -16,6 +16,13 @@
 * limitations under the License.
 }}
 
+<div class="margin-small-vertical">
+  {{load-time-component
+    isRefreshable=isRefreshable
+    time=timeStamp
+    refresh='refresh'
+  }}
+</div>
 <div class='type-table fill-full margin-small-horizontal'>
   <div class='align-left'>
     <table class='detail-list'>
@@ -23,8 +30,7 @@
         <tr>
           <td>{{t 'common.status'}}</td>
           <td>
-            {{status}}
-            <i {{bind-attr class=':task-status taskIconStatus'}}></i>
+            <i {{bind-attr class=':task-status taskIconStatus'}}></i> {{status}}
             {{#if progressStr}} {{bs-badge content=progressStr}}{{/if}}
           </td>
           <td>

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/dag/view.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/dag/view.hbs b/tez-ui/src/main/webapp/app/templates/dag/view.hbs
index 27ee267..699d0c8 100644
--- a/tez-ui/src/main/webapp/app/templates/dag/view.hbs
+++ b/tez-ui/src/main/webapp/app/templates/dag/view.hbs
@@ -16,6 +16,13 @@
 * limitations under the License.
 }}
 
+<div {{bind-attr class=":margin-small-vertical loading:no-visible"}}>
+  {{load-time-component
+    isRefreshable=isRefreshable
+    time=entities.content.0.timeStamp
+    refresh='refresh'
+  }}
+</div>
 <div class="dag-view-component-container">
   {{#unless loading}}
     {{dag-view-component

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/partials/configs.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/partials/configs.hbs b/tez-ui/src/main/webapp/app/templates/partials/configs.hbs
index 0e00ca5..0370890 100644
--- a/tez-ui/src/main/webapp/app/templates/partials/configs.hbs
+++ b/tez-ui/src/main/webapp/app/templates/partials/configs.hbs
@@ -17,7 +17,12 @@
 }}
 
 {{#if configs}}
-  <div class='table-container'>
+  {{load-time-component
+    isRefreshable=isRefreshable
+    time=timeStamp
+    refresh='refresh'
+  }}
+  <div class='table-container margin-small-vertical'>
     {{kv-table-component data=configs filterExp="^tez."}}
   </div>
 {{else}}

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/partials/table-controls.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/partials/table-controls.hbs b/tez-ui/src/main/webapp/app/templates/partials/table-controls.hbs
index 1b5745f..5cc6840 100644
--- a/tez-ui/src/main/webapp/app/templates/partials/table-controls.hbs
+++ b/tez-ui/src/main/webapp/app/templates/partials/table-controls.hbs
@@ -16,7 +16,7 @@
 * limitations under the License.
 }}
 
-<div class='margin-small'>
+<div class='margin-small-horizontal table-control'>
   <div class="horizontal-half align-children-left">
     Number of rows
     {{view Ember.Select

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/task/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/task/index.hbs b/tez-ui/src/main/webapp/app/templates/task/index.hbs
index 6826e7b..833e624 100644
--- a/tez-ui/src/main/webapp/app/templates/task/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/task/index.hbs
@@ -16,6 +16,13 @@
 * limitations under the License.
 }}
 
+<div class="margin-small-vertical">
+  {{load-time-component
+    isRefreshable=isRefreshable
+    time=timeStamp
+    refresh='refresh'
+  }}
+</div>
 <div class='type-table fill-full margin-small-horizontal'>
 	<div class='align-left'>
 		<table class='detail-list'>
@@ -23,8 +30,7 @@
 				<tr>
 					<td>{{t 'common.status'}}</td>
 					<td>
-						{{taskStatus}}
-						<i {{bind-attr class=':task-status taskIconStatus'}}></i>
+						<i {{bind-attr class=':task-status taskIconStatus'}}></i> {{taskStatus}}
 					</td>
 				</tr>
 			</tbody>

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/taskAttempt/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/taskAttempt/index.hbs b/tez-ui/src/main/webapp/app/templates/taskAttempt/index.hbs
index 738b636..23b96b4 100644
--- a/tez-ui/src/main/webapp/app/templates/taskAttempt/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/taskAttempt/index.hbs
@@ -16,6 +16,13 @@
 * limitations under the License.
 }}
 
+<div class="margin-small-vertical">
+  {{load-time-component
+    isRefreshable=isRefreshable
+    time=timeStamp
+    refresh='refresh'
+  }}
+</div>
 <div class='type-table fill-full margin-small-horizontal'>
 	<div class='align-left'>
 		<table class='detail-list'>
@@ -23,8 +30,7 @@
 				<tr>
 					<td>{{t 'common.status'}}</td>
 					<td>
-						{{taskAttemptStatus}}
-						<i {{bind-attr class=':task-status taskAttemptIconStatus'}}></i>
+						<i {{bind-attr class=':task-status taskAttemptIconStatus'}}></i> {{taskAttemptStatus}}
 					</td>
 				</tr>
 				<tr>

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/tez-app/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/tez-app/index.hbs b/tez-ui/src/main/webapp/app/templates/tez-app/index.hbs
index 07777dd..bedb8f2 100644
--- a/tez-ui/src/main/webapp/app/templates/tez-app/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/tez-app/index.hbs
@@ -16,6 +16,13 @@
 * limitations under the License.
 }}
 
+<div class="margin-small-vertical">
+  {{load-time-component
+    isRefreshable=isRefreshable
+    time=timeStamp
+    refresh='refresh'
+  }}
+</div>
 <div class='type-table fill-full margin-small-horizontal'>
   <div class='align-left'>
     <table class='detail-list'>
@@ -48,7 +55,9 @@
         </tr>
         <tr>
           <td>Final App Status</td>
-          <td>{{appDetail.finalAppStatus}}</td>
+          <td>
+            <i {{bind-attr class=':task-status iconStatus'}}></i> {{appDetail.finalAppStatus}}
+          </td>
         </tr>
       </tbody>
     </table>

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/vertex/additionals.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/vertex/additionals.hbs b/tez-ui/src/main/webapp/app/templates/vertex/additionals.hbs
index bb85eb3..f6ee573 100644
--- a/tez-ui/src/main/webapp/app/templates/vertex/additionals.hbs
+++ b/tez-ui/src/main/webapp/app/templates/vertex/additionals.hbs
@@ -16,6 +16,13 @@
 * limitations under the License.
 }}
 
+<div class="margin-small-vertical">
+  {{load-time-component
+    isRefreshable=isRefreshable
+    time=timeStamp
+    refresh='refresh'
+  }}
+</div>
 {{#unless loading}}
   <h3>Sources</h3>
   {{#if inputsAvailable}}

http://git-wip-us.apache.org/repos/asf/tez/blob/dd44bdec/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/templates/vertex/index.hbs b/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
index bc2e7ef..557c644 100644
--- a/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
+++ b/tez-ui/src/main/webapp/app/templates/vertex/index.hbs
@@ -16,6 +16,13 @@
 * limitations under the License.
 }}
 
+<div class="margin-small-vertical">
+  {{load-time-component
+    isRefreshable=isRefreshable
+    time=timeStamp
+    refresh='refresh'
+  }}
+</div>
 <div class='type-table margin-small-horizontal fill-full'>
 
   <div class="horizontal-half align-children-left">