You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by je...@apache.org on 2015/11/14 23:57:33 UTC

tez git commit: TEZ-2923. Tez Live UI counters view empty for vertices, tasks, attempts (jeagles)

Repository: tez
Updated Branches:
  refs/heads/master e1424bce7 -> abfc8bfb0


TEZ-2923. Tez Live UI counters view empty for vertices, tasks, attempts (jeagles)


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

Branch: refs/heads/master
Commit: abfc8bfb0a8620d31697a31ad516674a8d3f9f7c
Parents: e1424bc
Author: Jonathan Eagles <je...@yahoo-inc.com>
Authored: Sat Nov 14 16:56:08 2015 -0600
Committer: Jonathan Eagles <je...@yahoo-inc.com>
Committed: Sat Nov 14 16:56:08 2015 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../java/org/apache/tez/dag/app/dag/DAG.java    |  3 +-
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    | 43 +++++++++++++++
 .../apache/tez/dag/app/web/AMWebController.java | 33 +++++++++---
 .../tez/dag/app/web/TestAMWebController.java    | 23 +++++++-
 tez-ui/src/main/webapp/app/scripts/app.js       |  5 +-
 .../controllers/dag_counters_controller.js      | 51 ++++++++++++++++++
 .../scripts/controllers/shared-controllers.js   | 23 --------
 .../controllers/task-counters-controller.js     | 40 +++++++++++++-
 .../task_attempt_counters_controller.js         | 57 ++++++++++++++++++++
 .../controllers/vertex_counters_controller.js   | 52 ++++++++++++++++++
 .../main/webapp/app/scripts/helpers/em-data.js  | 20 +++----
 .../src/main/webapp/app/scripts/helpers/misc.js |  4 +-
 .../src/main/webapp/app/scripts/models/dag.js   |  1 +
 14 files changed, 309 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/abfc8bfb/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 436203d..0e14fa2 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES
   TEZ-2679. Admin forms of launch env settings
 
 ALL CHANGES:
+  TEZ-2923. Tez Live UI counters view empty for vertices, tasks, attempts
   TEZ-2924. Framework for Hadoop shims.
   TEZ-2935. Add MR slow start translation for ShuffleVertexManager
   TEZ-2918. Make progress notifications in IOs
@@ -251,6 +252,7 @@ INCOMPATIBLE CHANGES
   TEZ-2679. Admin forms of launch env settings
 
 ALL CHANGES
+  TEZ-2923. Tez Live UI counters view empty for vertices, tasks, attempts
   TEZ-2935. Add MR slow start translation for ShuffleVertexManager
   TEZ-2940. Invalid shuffle max slow start setting causes vertex to hang indefinitely
   TEZ-1670. Add tests for all converter functions in HistoryEventTimelineConversion.

http://git-wip-us.apache.org/repos/asf/tez/blob/abfc8bfb/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
index 15349a0..640359d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
@@ -56,7 +56,8 @@ public interface DAG {
    * @return job-counters and aggregate task-counters
    */
   TezCounters getAllCounters();
-  
+  TezCounters getCachedCounters();
+
   @SuppressWarnings("rawtypes")
   EventHandler getEventHandler();
 

http://git-wip-us.apache.org/repos/asf/tez/blob/abfc8bfb/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
index 5b69f0e..4dfba84 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
@@ -190,6 +190,8 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
   private Object fullCountersLock = new Object();
   @VisibleForTesting
   TezCounters fullCounters = null;
+  private TezCounters cachedCounters = null;
+  private long cachedCountersTimestamp = 0;
   private Set<TezVertexID> reRunningVertices = new HashSet<TezVertexID>();
 
   private final Configuration dagConf;
@@ -750,6 +752,47 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     }
   }
 
+  @Override
+  public TezCounters getCachedCounters() {
+
+    readLock.lock();
+
+    try {
+      // FIXME a better lightweight approach for counters is needed
+      if (fullCounters == null && cachedCounters != null
+          && ((cachedCountersTimestamp+10000) > System.currentTimeMillis())) {
+        LOG.info("Asked for counters"
+            + ", cachedCountersTimestamp=" + cachedCountersTimestamp
+            + ", currentTime=" + System.currentTimeMillis());
+        return cachedCounters;
+      }
+
+      cachedCountersTimestamp = System.currentTimeMillis();
+      if (inTerminalState()) {
+        this.mayBeConstructFinalFullCounters();
+        return fullCounters;
+      }
+
+      // dag not yet finished. update cpu time counters
+      updateCpuCounters();
+      TezCounters counters = new TezCounters();
+      counters.incrAllCounters(dagCounters);
+      return incrTaskCounters(counters, vertices.values());
+
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  boolean inTerminalState() {
+    DAGState state = getInternalState();
+    if (state == DAGState.ERROR || state == DAGState.FAILED
+        || state == DAGState.KILLED || state == DAGState.SUCCEEDED) {
+      return true;
+    }
+    return false;
+  }
+
   public static TezCounters incrTaskCounters(
       TezCounters counters, Collection<Vertex> vertices) {
     for (Vertex vertex : vertices) {

http://git-wip-us.apache.org/repos/asf/tez/blob/abfc8bfb/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java
index 0161a7e..f91bc42 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/web/AMWebController.java
@@ -504,11 +504,20 @@ public class AMWebController extends Controller {
       return;
     }
 
-    Map<String, String> dagInfo = new HashMap<String, String>();
+    Map<String, Set<String>> counterNames = getCounterListFromRequest();
+
+    Map<String, Object> dagInfo = new HashMap<String, Object>();
     dagInfo.put("id", dag.getID().toString());
     dagInfo.put("progress", Float.toString(dag.getCompletedTaskProgress()));
     dagInfo.put("status", dag.getState().toString());
 
+    if (counterNames != null && !counterNames.isEmpty()) {
+      TezCounters counters = dag.getCachedCounters();
+      Map<String, Map<String, Long>> counterMap = constructCounterMapInfo(counters, counterNames);
+      if (counterMap != null && !counterMap.isEmpty()) {
+        dagInfo.put("counters", counterMap);
+      }
+    }
     renderJSON(ImmutableMap.of(
         "dag", dagInfo
     ));
@@ -526,15 +535,25 @@ public class AMWebController extends Controller {
 
     Map<String, Map<String, Long>> counterInfo = new TreeMap<String, Map<String, Long>>();
 
-    for (Entry<String, Set<String>> entry : counterNames.entrySet()) {
-      Map<String, Long> matchedCounters = new HashMap<String, Long>();
-      CounterGroup grpCounters = counters.getGroup(entry.getKey());
-      for (TezCounter counter : grpCounters) {
-        if (entry.getValue().isEmpty() || entry.getValue().contains(counter.getName())) {
+    if (counterNames.containsKey("*")) {
+      for (CounterGroup grpCounters : counters) {
+        Map<String, Long> matchedCounters = new HashMap<String, Long>();
+        for (TezCounter counter : grpCounters) {
           matchedCounters.put(counter.getName(), counter.getValue());
         }
+        counterInfo.put(grpCounters.getName(), matchedCounters);
+      }
+    } else {
+      for (Entry<String, Set<String>> entry : counterNames.entrySet()) {
+        Map<String, Long> matchedCounters = new HashMap<String, Long>();
+        CounterGroup grpCounters = counters.getGroup(entry.getKey());
+        for (TezCounter counter : grpCounters) {
+          if (entry.getValue().isEmpty() || entry.getValue().contains(counter.getName())) {
+            matchedCounters.put(counter.getName(), counter.getValue());
+          }
+        }
+        counterInfo.put(entry.getKey(), matchedCounters);
       }
-      counterInfo.put(entry.getKey(), matchedCounters);
     }
 
     return counterInfo;

http://git-wip-us.apache.org/repos/asf/tez/blob/abfc8bfb/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java
index 5a37c04..1b35913 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/web/TestAMWebController.java
@@ -36,6 +36,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -242,11 +243,29 @@ public class TestAMWebController {
     doReturn(TezDAGID.fromString("dag_1422960590892_0007_42")).when(mockDAG).getID();
     doReturn(66.0f).when(mockDAG).getCompletedTaskProgress();
     doReturn(DAGState.RUNNING).when(mockDAG).getState();
+    TezCounters counters = new TezCounters();
+    counters.addGroup("g1", "g1");
+    counters.addGroup("g2", "g2");
+    counters.addGroup("g3", "g3");
+    counters.addGroup("g4", "g4");
+    counters.findCounter("g1", "g1_c1").setValue(100);
+    counters.findCounter("g1", "g1_c2").setValue(100);
+    counters.findCounter("g2", "g2_c3").setValue(100);
+    counters.findCounter("g2", "g2_c4").setValue(100);
+    counters.findCounter("g3", "g3_c5").setValue(100);
+    counters.findCounter("g3", "g3_c6").setValue(100);
+
+    doReturn(counters).when(mockDAG).getAllCounters();
+    doReturn(counters).when(mockDAG).getCachedCounters();
 
     doReturn(true).when(spy).setupResponse();
     doReturn(mockDAG).when(spy).checkAndGetDAGFromRequest();
     doNothing().when(spy).renderJSON(any());
 
+    Map<String, Set<String>> counterNames = new HashMap<String, Set<String>>();
+    counterNames.put("*", null);
+    doReturn(counterNames).when(spy).getCounterListFromRequest();
+
     spy.getDagInfo();
     verify(spy).renderJSON(returnResultCaptor.capture());
 
@@ -255,10 +274,12 @@ public class TestAMWebController {
     Assert.assertTrue(result.containsKey("dag"));
     Map<String, String> dagInfo = (Map<String, String>) result.get("dag");
 
-    Assert.assertEquals(3, dagInfo.size());
+    Assert.assertEquals(4, dagInfo.size());
     Assert.assertTrue("dag_1422960590892_0007_42".equals(dagInfo.get("id")));
     Assert.assertEquals("66.0", dagInfo.get("progress"));
     Assert.assertEquals("RUNNING", dagInfo.get("status"));
+    Assert.assertNotNull(dagInfo.get("counters"));
+
   }
 
   @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/tez/blob/abfc8bfb/tez-ui/src/main/webapp/app/scripts/app.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/app.js b/tez-ui/src/main/webapp/app/scripts/app.js
index f2c3dd0..22b4151 100644
--- a/tez-ui/src/main/webapp/app/scripts/app.js
+++ b/tez-ui/src/main/webapp/app/scripts/app.js
@@ -203,10 +203,11 @@ App.ready = function () {
     },
     buildURL: function(type, id, record) {
       var url = this._super(type, undefined, record);
-      return url.replace('__app_id__', record.get('appID')).fmt(id);
+      return url.replace('__app_id__', record.get('appID'))
+        .fmt(id, record.get('counters'));
     },
     pathForType: function(typeName) {
-      return 'dagInfo?dagID=%@';
+      return 'dagInfo?dagID=%@&counters=%@';
     }
   });
 

http://git-wip-us.apache.org/repos/asf/tez/blob/abfc8bfb/tez-ui/src/main/webapp/app/scripts/controllers/dag_counters_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/dag_counters_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/dag_counters_controller.js
new file mode 100644
index 0000000..4cd3f80
--- /dev/null
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/dag_counters_controller.js
@@ -0,0 +1,51 @@
+/**
+ * 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.PollingController.extend(App.Helpers.DisplayHelper, App.ModelRefreshMixin, {
+  controllerName: 'DagCountersController',
+
+  pollingType: 'dagInfo',
+
+  pollsterControl: function () {
+    if(this.get('status') == 'RUNNING' &&
+        this.get('amWebServiceVersion') != '1' &&
+        this.get('pollingEnabled') &&
+        this.get('isActive')) {
+      this.get('pollster').start();
+    }
+    else {
+      this.get('pollster').stop();
+    }
+  }.observes('status', 'amWebServiceVersion', 'isActive', 'pollingEnabled'),
+
+  pollsterOptionsObserver: function () {
+    var model = this.get('model');
+
+    this.get('pollster').setProperties( (model && model.get('status') != 'SUCCEEDED') ? {
+      targetRecords: [model],
+      options: {
+        appID: this.get('applicationId'),
+        dagID: App.Helpers.misc.getIndexFromId(this.get('id')),
+        counters: '*'
+      }
+    } : {
+      targetRecords: [],
+      options: null
+    });
+  }.observes('applicationId', 'model', 'model.status', 'id'),
+});

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

http://git-wip-us.apache.org/repos/asf/tez/blob/abfc8bfb/tez-ui/src/main/webapp/app/scripts/controllers/task-counters-controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/task-counters-controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/task-counters-controller.js
index 5d514bd..dd74129 100644
--- a/tez-ui/src/main/webapp/app/scripts/controllers/task-counters-controller.js
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/task-counters-controller.js
@@ -16,7 +16,43 @@
  * limitations under the License.
  */
 
-App.TaskCountersController = Em.ObjectController.extend(App.ModelRefreshMixin, {
+App.TaskCountersController = App.PollingController.extend(App.ModelRefreshMixin, {
+  controllerName: 'TaskCountersController',
+
+  pollingType: 'taskInfo',
+
+  pollsterControl: function () {
+    if(this.get('vertex.dag.status') == 'RUNNING' &&
+        this.get('vertex.dag.amWebServiceVersion') != '1' &&
+        this.get('pollingEnabled') &&
+        this.get('isActive')) {
+      this.get('pollster').start();
+    }
+    else {
+      this.get('pollster').stop();
+    }
+  }.observes('vertex.dag.status', 'vertex.dag.amWebServiceVersion', 'isActive', 'pollingEnabled'),
+
+  pollsterOptionsObserver: function () {
+    var model = this.get('model');
+
+    this.get('pollster').setProperties( (model && model.get('status') != 'SUCCEEDED') ? {
+      targetRecords: [model],
+      options: {
+        appID: this.get('vertex.dag.applicationId'),
+        dagID: App.Helpers.misc.getIndexFromId(this.get('dagID')),
+        taskID: '%@_%@'.fmt(
+          App.Helpers.misc.getIndexFromId(this.get('vertexID')),
+          App.Helpers.misc.getIndexFromId(this.get('id'))
+        ),
+        counters: '*'
+      }
+    } : {
+      targetRecords: [],
+      options: null
+    });
+  }.observes('vertex.dag.applicationId', 'status', 'dagID', 'vertexID', 'id'),
+
   message: function () {
     var status = this.get('content.status');
     if(!this.get('content.counterGroups.length')) {
@@ -25,4 +61,4 @@ App.TaskCountersController = Em.ObjectController.extend(App.ModelRefreshMixin, {
       }
     }
   }.property('content.status', 'content.counterGroups.length')
-});
\ No newline at end of file
+});

http://git-wip-us.apache.org/repos/asf/tez/blob/abfc8bfb/tez-ui/src/main/webapp/app/scripts/controllers/task_attempt_counters_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/task_attempt_counters_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/task_attempt_counters_controller.js
new file mode 100644
index 0000000..591fa43
--- /dev/null
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/task_attempt_counters_controller.js
@@ -0,0 +1,57 @@
+/**
+ * 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.TaskAttemptCountersController = App.PollingController.extend(App.Helpers.DisplayHelper, App.ModelRefreshMixin, {
+  controllerName: 'TaskAttemptCountersController',
+
+  pollingType: 'attemptInfo',
+
+  pollsterControl: function () {
+    if(this.get('task.vertex.dag.status') == 'RUNNING' &&
+        this.get('task.vertex.dag.amWebServiceVersion') != '1' &&
+        this.get('pollingEnabled') &&
+        this.get('isActive')) {
+      this.get('pollster').start();
+    }
+    else {
+      this.get('pollster').stop();
+    }
+  }.observes('task.vertex.dag.status', 'task.vertex.dag.amWebServiceVersion', 'isActive', 'pollingEnabled'),
+
+  pollsterOptionsObserver: function () {
+    var model = this.get('model');
+
+    this.get('pollster').setProperties( (model && model.get('status') != 'SUCCEEDED') ? {
+      targetRecords: [model],
+      options: {
+        appID: this.get('task.vertex.dag.applicationId'),
+        dagID: App.Helpers.misc.getIndexFromId(this.get('dagID')),
+        //ID: App.Helpers.misc.getIndexFromId(this.get('id')),
+        attemptID: '%@_%@_%@'.fmt(
+          App.Helpers.misc.getIndexFromId(this.get('vertexID')),
+          App.Helpers.misc.getIndexFromId(this.get('taskID')),
+          App.Helpers.misc.getIndexFromId(this.get('id'))
+        ),
+        counters: '*'
+      }
+    } : {
+      targetRecords: [],
+      options: null
+    });
+  }.observes('task.vertex.dag.applicationId', 'status', 'dagID', 'vertexID', 'id'),
+});

http://git-wip-us.apache.org/repos/asf/tez/blob/abfc8bfb/tez-ui/src/main/webapp/app/scripts/controllers/vertex_counters_controller.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/controllers/vertex_counters_controller.js b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_counters_controller.js
new file mode 100644
index 0000000..90a7e4a
--- /dev/null
+++ b/tez-ui/src/main/webapp/app/scripts/controllers/vertex_counters_controller.js
@@ -0,0 +1,52 @@
+/**
+ * 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.VertexCountersController = App.PollingController.extend(App.Helpers.DisplayHelper, App.ModelRefreshMixin, {
+  controllerName: 'VertexCountersController',
+
+  pollingType: 'vertexInfo',
+
+  pollsterControl: function () {
+    if(this.get('dag.status') == 'RUNNING' &&
+        this.get('dag.amWebServiceVersion') != '1' &&
+        this.get('pollingEnabled') &&
+        this.get('isActive')) {
+      this.get('pollster').start();
+    }
+    else {
+      this.get('pollster').stop();
+    }
+  }.observes('dag.status', 'dag.amWebServiceVersion', 'isActive', 'pollingEnabled'),
+
+  pollsterOptionsObserver: function () {
+    var model = this.get('model');
+
+    this.get('pollster').setProperties( (model && model.get('status') != 'SUCCEEDED') ? {
+      targetRecords: [model],
+      options: {
+        appID: this.get('applicationId'),
+        dagID: App.Helpers.misc.getIndexFromId(this.get('dagID')),
+        vertexID: App.Helpers.misc.getIndexFromId(this.get('id')),
+        counters: '*'
+      }
+    } : {
+      targetRecords: [],
+      options: null
+    });
+  }.observes('applicationId', 'status', 'dagID', 'id'),
+});

http://git-wip-us.apache.org/repos/asf/tez/blob/abfc8bfb/tez-ui/src/main/webapp/app/scripts/helpers/em-data.js
----------------------------------------------------------------------
diff --git a/tez-ui/src/main/webapp/app/scripts/helpers/em-data.js b/tez-ui/src/main/webapp/app/scripts/helpers/em-data.js
index 5c4dadc..8ad3eb2 100644
--- a/tez-ui/src/main/webapp/app/scripts/helpers/em-data.js
+++ b/tez-ui/src/main/webapp/app/scripts/helpers/em-data.js
@@ -31,6 +31,16 @@ App.Helpers.emData = {
         var info = source.findBy('id', row.get('id')),
             merge = !!info;
 
+        if(info && info.get('counters')) {
+          row.set('counterGroups',
+            App.Helpers.misc.mergeCounterInfo(
+              row.get('counterGroups'),
+              info.get('counters')
+            ).slice(0)
+          );
+          row.didLoad();// To update the record time stamp
+        }
+
         if(merge && row.get('progress') && info.get('progress')) {
           if(row.get('progress') >= info.get('progress')) {
             merge = false;
@@ -39,16 +49,6 @@ App.Helpers.emData = {
 
         if(merge) {
           row.setProperties(info.getProperties.apply(info, mergeProps));
-
-          if(info.get('counters')) {
-            row.set('counterGroups',
-              App.Helpers.misc.mergeCounterInfo(
-                row.get('counterGroups'),
-                info.get('counters')
-              ).slice(0)
-            );
-          }
-
           row.didLoad();// To update the record time stamp
         }
       });

http://git-wip-us.apache.org/repos/asf/tez/blob/abfc8bfb/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 550a758..65480e2 100644
--- a/tez-ui/src/main/webapp/app/scripts/helpers/misc.js
+++ b/tez-ui/src/main/webapp/app/scripts/helpers/misc.js
@@ -221,7 +221,7 @@ App.Helpers.misc = {
         atsCounters = targetATSCounters.findBy('counterGroupName', counterGroupName);
         if(!atsCounters) {
           atsCounters = [];
-          targetATSCounters.push({
+          targetATSCounters.pushObject({
             counterGroupName: counterGroupName,
             counterGroupDisplayName: counterGroupName,
             counters: atsCounters
@@ -236,7 +236,7 @@ App.Helpers.misc = {
             Em.set(atsCounter, 'counterValue', counters[counterName]);
           }
           else {
-            atsCounters.push({
+            atsCounters.pushObject({
               "counterName": counterName,
               "counterDisplayName": counterName,
               "counterValue": counters[counterName]

http://git-wip-us.apache.org/repos/asf/tez/blob/abfc8bfb/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 0457157..3442433 100644
--- a/tez-ui/src/main/webapp/app/scripts/models/dag.js
+++ b/tez-ui/src/main/webapp/app/scripts/models/dag.js
@@ -428,6 +428,7 @@ App.DagInfo = DS.Model.extend({
 
   progress: DS.attr('number'),
   status: DS.attr('string'),
+  counters: DS.attr('object')
 });
 
 App.VertexInfo = DS.Model.extend({