You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2018/09/17 20:29:46 UTC

[1/7] storm git commit: Addressed review comments

Repository: storm
Updated Branches:
  refs/heads/master eaed3cbf3 -> 83fea15d7


Addressed review comments


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

Branch: refs/heads/master
Commit: 15a62dde69d1a39993268ba23383dcc87a328d6b
Parents: 9d3feb0
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Mon Sep 17 14:28:00 2018 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Mon Sep 17 14:56:56 2018 -0500

----------------------------------------------------------------------
 .../org/apache/storm/stats/ClientStatsUtil.java |   2 +-
 .../test/clj/org/apache/storm/nimbus_test.clj   |   2 +-
 storm-server/pom.xml                            |   2 +-
 .../storm/daemon/nimbus/HeartbeatCache.java     |  39 +-
 .../org/apache/storm/daemon/nimbus/Nimbus.java  |   4 +-
 .../java/org/apache/storm/stats/StatsUtil.java  | 721 +++++++++----------
 6 files changed, 378 insertions(+), 392 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/15a62dde/storm-client/src/jvm/org/apache/storm/stats/ClientStatsUtil.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/stats/ClientStatsUtil.java b/storm-client/src/jvm/org/apache/storm/stats/ClientStatsUtil.java
index 3f63db3..db2f579 100644
--- a/storm-client/src/jvm/org/apache/storm/stats/ClientStatsUtil.java
+++ b/storm-client/src/jvm/org/apache/storm/stats/ClientStatsUtil.java
@@ -49,7 +49,7 @@ public class ClientStatsUtil {
     }
 
     /**
-     * Make and map of executors to empty stats.
+     * Make an map of executors to empty stats, in preparation for doing a heartbeat.
      * @param executors the executors as keys of the map.
      * @return and empty map of executors to stats.
      */

http://git-wip-us.apache.org/repos/asf/storm/blob/15a62dde/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
index 8f9f061..800113f 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -171,7 +171,7 @@
 
     (.workerHeartbeat state storm-id node port
       (ClientStatsUtil/thriftifyZkWorkerHb (ClientStatsUtil/mkZkWorkerHb storm-id stats (int 10))))
-    (.sendSupervisorWorkerHeartbeat (.getNimbus cluster) (StatsUtil/thriftifyRPCWorkerHb storm-id executor))))
+    (.sendSupervisorWorkerHeartbeat (.getNimbus cluster) (StatsUtil/thriftifyRpcWorkerHb storm-id executor))))
 
 (defn slot-assignments [cluster storm-id]
   (let [state (.getClusterState cluster)

http://git-wip-us.apache.org/repos/asf/storm/blob/15a62dde/storm-server/pom.xml
----------------------------------------------------------------------
diff --git a/storm-server/pom.xml b/storm-server/pom.xml
index b3d74b3..e05e16f 100644
--- a/storm-server/pom.xml
+++ b/storm-server/pom.xml
@@ -171,7 +171,7 @@
                 <artifactId>maven-checkstyle-plugin</artifactId>
                 <!--Note - the version would be inherited-->
                 <configuration>
-                    <maxAllowedViolations>853</maxAllowedViolations>
+                    <maxAllowedViolations>757</maxAllowedViolations>
                 </configuration>
             </plugin>
             <plugin>

http://git-wip-us.apache.org/repos/asf/storm/blob/15a62dde/storm-server/src/main/java/org/apache/storm/daemon/nimbus/HeartbeatCache.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/nimbus/HeartbeatCache.java b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/HeartbeatCache.java
index 320f4fb..4774260 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/nimbus/HeartbeatCache.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/HeartbeatCache.java
@@ -46,49 +46,39 @@ public class HeartbeatCache {
 
     private static class ExecutorCache {
         private Boolean isTimedOut;
-        private Integer nimbusTime;
-        private Integer executorReportedTime;
+        private Integer nimbusTimeSecs;
+        private Integer executorReportedTimeSecs;
 
         public ExecutorCache(Map<String, Object> newBeat) {
             if (newBeat != null) {
-                executorReportedTime = (Integer) newBeat.getOrDefault(ClientStatsUtil.TIME_SECS, 0);
+                executorReportedTimeSecs = (Integer) newBeat.getOrDefault(ClientStatsUtil.TIME_SECS, 0);
             } else {
-                executorReportedTime = 0;
+                executorReportedTimeSecs = 0;
             }
 
-            nimbusTime = Time.currentTimeSecs();
+            nimbusTimeSecs = Time.currentTimeSecs();
             isTimedOut = false;
         }
 
-        public ExecutorCache(boolean isTimedOut, Integer nimbusTime, Integer executorReportedTime) {
-            this.isTimedOut = isTimedOut;
-            this.nimbusTime = nimbusTime;
-            this.executorReportedTime = executorReportedTime;
-        }
-
         public synchronized Boolean isTimedOut() {
             return isTimedOut;
         }
 
-        public synchronized Integer getNimbusTime() {
-            return nimbusTime;
-        }
-
-        public synchronized Integer getExecutorReportedTime() {
-            return executorReportedTime;
+        public synchronized Integer getNimbusTimeSecs() {
+            return nimbusTimeSecs;
         }
 
         public synchronized void updateTimeout(Integer timeout) {
-            isTimedOut = Time.deltaSecs(getNimbusTime()) >= timeout;
+            isTimedOut = Time.deltaSecs(getNimbusTimeSecs()) >= timeout;
         }
 
         public synchronized void updateFromHb(Integer timeout, Map<String,Object> newBeat) {
             if (newBeat != null) {
                 Integer newReportedTime = (Integer) newBeat.getOrDefault(ClientStatsUtil.TIME_SECS, 0);
-                if (!newReportedTime.equals(executorReportedTime)) {
-                    nimbusTime = Time.currentTimeSecs();
+                if (!newReportedTime.equals(executorReportedTimeSecs)) {
+                    nimbusTimeSecs = Time.currentTimeSecs();
                 }
-                executorReportedTime = newReportedTime;
+                executorReportedTimeSecs = newReportedTime;
             }
             updateTimeout(timeout);
         }
@@ -140,13 +130,12 @@ public class HeartbeatCache {
     }
 
     /**
-     * Update the heartbeats for a topology with no heartbeats that came in.
+     * Go through all executors and time them out if needed.
      * @param topoId the id of the topology to look at.
      * @param taskTimeoutSecs the timeout to know if they are too old.
      */
     public void timeoutOldHeartbeats(String topoId, Integer taskTimeoutSecs) {
         Map<List<Integer>, ExecutorCache> topoCache = cache.computeIfAbsent(topoId, MAKE_MAP);
-        //if not executor beats, refresh is-timed-out of the cache which is done by master
         for (ExecutorCache ec : topoCache.values()) {
             ec.updateTimeout(taskTimeoutSecs);
         }
@@ -216,9 +205,9 @@ public class HeartbeatCache {
             Long startTime = execToStartTimes.get(longExec);
             ExecutorCache executorCache = topoCache.get(exec);
             //null isTimedOut means worker never reported any heartbeat
-            Boolean isTimedOut = executorCache == null ? null : executorCache.isTimedOut();
+            boolean isTimedOut = executorCache == null ? true : executorCache.isTimedOut();
             Integer delta = startTime == null ? null : Time.deltaSecs(startTime.intValue());
-            if (startTime != null && ((delta < taskLaunchSecs) || (isTimedOut != null && !isTimedOut))) {
+            if (startTime != null && ((delta < taskLaunchSecs) || !isTimedOut)) {
                 ret.add(exec);
             } else {
                 LOG.info("Executor {}:{} not alive", topoId, exec);

http://git-wip-us.apache.org/repos/asf/storm/blob/15a62dde/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
index a33a4d0..aeab4b6 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
@@ -4620,7 +4620,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
         return true;
     }
 
-    static final class Assoc<K, V> implements UnaryOperator<Map<K, V>> {
+    private static final class Assoc<K, V> implements UnaryOperator<Map<K, V>> {
         private final K key;
         private final V value;
 
@@ -4639,7 +4639,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
 
     // Shutdownable methods
 
-    static final class Dissoc<K, V> implements UnaryOperator<Map<K, V>> {
+    private static final class Dissoc<K, V> implements UnaryOperator<Map<K, V>> {
         private final K key;
 
         public Dissoc(K key) {

http://git-wip-us.apache.org/repos/asf/storm/blob/15a62dde/storm-server/src/main/java/org/apache/storm/stats/StatsUtil.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/stats/StatsUtil.java b/storm-server/src/main/java/org/apache/storm/stats/StatsUtil.java
index 1e7d4fa..f5db87c 100644
--- a/storm-server/src/main/java/org/apache/storm/stats/StatsUtil.java
+++ b/storm-server/src/main/java/org/apache/storm/stats/StatsUtil.java
@@ -124,9 +124,9 @@ public class StatsUtil {
                                                          Map<List<String>, Double> id2procAvg,
                                                          Map<List<String>, Long> id2numExec) {
         Map<String, Number> ret = new HashMap<>();
-        putKV(ret, EXEC_LAT_TOTAL, weightAvgAndSum(id2execAvg, id2numExec));
-        putKV(ret, PROC_LAT_TOTAL, weightAvgAndSum(id2procAvg, id2numExec));
-        putKV(ret, EXECUTED, sumValues(id2numExec));
+        ((Map) ret).put(EXEC_LAT_TOTAL, weightAvgAndSum(id2execAvg, id2numExec));
+        ((Map) ret).put(PROC_LAT_TOTAL, weightAvgAndSum(id2procAvg, id2numExec));
+        ((Map) ret).put(EXECUTED, sumValues(id2numExec));
 
         return ret;
     }
@@ -137,8 +137,8 @@ public class StatsUtil {
     public static Map<String, Number> aggSpoutLatAndCount(Map<String, Double> id2compAvg,
                                                           Map<String, Long> id2numAcked) {
         Map<String, Number> ret = new HashMap<>();
-        putKV(ret, COMP_LAT_TOTAL, weightAvgAndSum(id2compAvg, id2numAcked));
-        putKV(ret, ACKED, sumValues(id2numAcked));
+        ((Map) ret).put(COMP_LAT_TOTAL, weightAvgAndSum(id2compAvg, id2numAcked));
+        ((Map) ret).put(ACKED, sumValues(id2numAcked));
 
         return ret;
     }
@@ -155,9 +155,9 @@ public class StatsUtil {
         }
         for (K k : id2execAvg.keySet()) {
             Map<String, Object> subMap = new HashMap<>();
-            putKV(subMap, EXEC_LAT_TOTAL, weightAvg(id2execAvg, id2numExec, k));
-            putKV(subMap, PROC_LAT_TOTAL, weightAvg(id2procAvg, id2numExec, k));
-            putKV(subMap, EXECUTED, id2numExec.get(k));
+            subMap.put(EXEC_LAT_TOTAL, weightAvg(id2execAvg, id2numExec, k));
+            subMap.put(PROC_LAT_TOTAL, weightAvg(id2procAvg, id2numExec, k));
+            subMap.put(EXECUTED, id2numExec.get(k));
             ret.put(k, subMap);
         }
         return ret;
@@ -174,8 +174,8 @@ public class StatsUtil {
         }
         for (K k : id2compAvg.keySet()) {
             Map subMap = new HashMap();
-            putKV(subMap, COMP_LAT_TOTAL, weightAvg(id2compAvg, id2acked, k));
-            putKV(subMap, ACKED, id2acked.get(k));
+            subMap.put(COMP_LAT_TOTAL, weightAvg(id2compAvg, id2acked, k));
+            subMap.put(ACKED, id2acked.get(k));
             ret.put(k, subMap);
         }
         return ret;
@@ -193,22 +193,24 @@ public class StatsUtil {
     public static Map<String, Object> aggPreMergeCompPageBolt(Map<String, Object> beat, String window, boolean includeSys) {
         Map<String, Object> ret = new HashMap<>();
 
-        putKV(ret, EXECUTOR_ID, beat.get("exec-id"));
-        putKV(ret, HOST, beat.get(HOST));
-        putKV(ret, PORT, beat.get(PORT));
-        putKV(ret, ClientStatsUtil.UPTIME, beat.get(ClientStatsUtil.UPTIME));
-        putKV(ret, NUM_EXECUTORS, 1);
-        putKV(ret, NUM_TASKS, beat.get(NUM_TASKS));
+        ret.put(EXECUTOR_ID, beat.get("exec-id"));
+        ret.put(HOST, beat.get(HOST));
+        ret.put(PORT, beat.get(PORT));
+        ret.put(ClientStatsUtil.UPTIME, beat.get(ClientStatsUtil.UPTIME));
+        ret.put(NUM_EXECUTORS, 1);
+        ret.put(NUM_TASKS, beat.get(NUM_TASKS));
 
         Map stat2win2sid2num = ClientStatsUtil.getMapByKey(beat, STATS);
-        putKV(ret, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeyOr0(beat, ClientStatsUtil.UPTIME).intValue()));
+        ret.put(CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeyOr0(beat, ClientStatsUtil.UPTIME).intValue()));
 
         // calc cid+sid->input_stats
         Map inputStats = new HashMap();
         Map sid2acked = (Map) windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, ACKED), TO_STRING).get(window);
         Map sid2failed = (Map) windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, FAILED), TO_STRING).get(window);
-        putKV(inputStats, ACKED, sid2acked != null ? sid2acked : new HashMap());
-        putKV(inputStats, FAILED, sid2failed != null ? sid2failed : new HashMap());
+        Object v1 = sid2acked != null ? sid2acked : new HashMap();
+        inputStats.put(ACKED, v1);
+        Object v = sid2failed != null ? sid2failed : new HashMap();
+        inputStats.put(FAILED, v);
 
         inputStats = swapMapOrder(inputStats);
 
@@ -216,24 +218,24 @@ public class StatsUtil {
         Map sid2procLat = (Map) windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, PROC_LATENCIES), TO_STRING).get(window);
         Map sid2exec = (Map) windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, EXECUTED), TO_STRING).get(window);
         mergeMaps(inputStats, aggBoltStreamsLatAndCount(sid2execLat, sid2procLat, sid2exec));
-        putKV(ret, CID_SID_TO_IN_STATS, inputStats);
+        ret.put(CID_SID_TO_IN_STATS, inputStats);
 
         // calc sid->output_stats
         Map outputStats = new HashMap();
         Map sid2emitted = (Map) windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, EMITTED), TO_STRING).get(window);
         Map sid2transferred = (Map) windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, TRANSFERRED), TO_STRING).get(window);
         if (sid2emitted != null) {
-            putKV(outputStats, EMITTED, filterSysStreams2Stat(sid2emitted, includeSys));
+            outputStats.put(EMITTED, filterSysStreams2Stat(sid2emitted, includeSys));
         } else {
-            putKV(outputStats, EMITTED, new HashMap());
+            outputStats.put(EMITTED, new HashMap());
         }
         if (sid2transferred != null) {
-            putKV(outputStats, TRANSFERRED, filterSysStreams2Stat(sid2transferred, includeSys));
+            outputStats.put(TRANSFERRED, filterSysStreams2Stat(sid2transferred, includeSys));
         } else {
-            putKV(outputStats, TRANSFERRED, new HashMap());
+            outputStats.put(TRANSFERRED, new HashMap());
         }
         outputStats = swapMapOrder(outputStats);
-        putKV(ret, SID_TO_OUT_STATS, outputStats);
+        ret.put(SID_TO_OUT_STATS, outputStats);
 
         return ret;
     }
@@ -249,12 +251,12 @@ public class StatsUtil {
      */
     public static Map<String, Object> aggPreMergeCompPageSpout(Map<String, Object> beat, String window, boolean includeSys) {
         Map<String, Object> ret = new HashMap<>();
-        putKV(ret, EXECUTOR_ID, beat.get("exec-id"));
-        putKV(ret, HOST, beat.get(HOST));
-        putKV(ret, PORT, beat.get(PORT));
-        putKV(ret, ClientStatsUtil.UPTIME, beat.get(ClientStatsUtil.UPTIME));
-        putKV(ret, NUM_EXECUTORS, 1);
-        putKV(ret, NUM_TASKS, beat.get(NUM_TASKS));
+        ret.put(EXECUTOR_ID, beat.get("exec-id"));
+        ret.put(HOST, beat.get(HOST));
+        ret.put(PORT, beat.get(PORT));
+        ret.put(ClientStatsUtil.UPTIME, beat.get(ClientStatsUtil.UPTIME));
+        ret.put(NUM_EXECUTORS, 1);
+        ret.put(NUM_TASKS, beat.get(NUM_TASKS));
 
         Map stat2win2sid2num = ClientStatsUtil.getMapByKey(beat, STATS);
 
@@ -266,31 +268,31 @@ public class StatsUtil {
         Map win2sid2transferred = windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, TRANSFERRED), TO_STRING);
         Map win2sid2compLat = windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, COMP_LATENCIES), TO_STRING);
 
-        putKV(outputStats, ACKED, win2sid2acked.get(window));
-        putKV(outputStats, FAILED, win2sid2failed.get(window));
+        outputStats.put(ACKED, win2sid2acked.get(window));
+        outputStats.put(FAILED, win2sid2failed.get(window));
         Map<String, Long> sid2emitted = (Map) win2sid2emitted.get(window);
         if (sid2emitted == null) {
             sid2emitted = new HashMap<>();
         }
-        putKV(outputStats, EMITTED, filterSysStreams2Stat(sid2emitted, includeSys));
+        outputStats.put(EMITTED, filterSysStreams2Stat(sid2emitted, includeSys));
 
         Map<String, Long> sid2transferred = (Map) win2sid2transferred.get(window);
         if (sid2transferred == null) {
             sid2transferred = new HashMap<>();
         }
-        putKV(outputStats, TRANSFERRED, filterSysStreams2Stat(sid2transferred, includeSys));
+        outputStats.put(TRANSFERRED, filterSysStreams2Stat(sid2transferred, includeSys));
         outputStats = swapMapOrder(outputStats);
 
         Map sid2compLat = (Map) win2sid2compLat.get(window);
         Map sid2acked = (Map) win2sid2acked.get(window);
         mergeMaps(outputStats, aggSpoutStreamsLatAndCount(sid2compLat, sid2acked));
-        putKV(ret, SID_TO_OUT_STATS, outputStats);
+        ret.put(SID_TO_OUT_STATS, outputStats);
 
         return ret;
     }
 
     /**
-     * pre-merge component stats of specified bolt id
+     * pre-merge component stats of specified bolt id.
      *
      * @param beat       executor heartbeat data
      * @param window     specified window
@@ -302,11 +304,11 @@ public class StatsUtil {
         Map<String, Object> ret = new HashMap<>();
 
         Map<String, Object> subRet = new HashMap<>();
-        putKV(subRet, NUM_EXECUTORS, 1);
-        putKV(subRet, NUM_TASKS, beat.get(NUM_TASKS));
+        subRet.put(NUM_EXECUTORS, 1);
+        subRet.put(NUM_TASKS, beat.get(NUM_TASKS));
 
         Map<String, Object> stat2win2sid2num = ClientStatsUtil.getMapByKey(beat, STATS);
-        putKV(subRet, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeyOr0(beat, ClientStatsUtil.UPTIME).intValue()));
+        subRet.put(CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeyOr0(beat, ClientStatsUtil.UPTIME).intValue()));
 
         for (String key : new String[]{ EMITTED, TRANSFERRED, ACKED, FAILED }) {
             Map<String, Map<K, V>> stat = windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, key), TO_STRING);
@@ -320,7 +322,7 @@ public class StatsUtil {
                     sum += v.longValue();
                 }
             }
-            putKV(subRet, key, sum);
+            subRet.put(key, sum);
         }
 
         Map<String, Map<List<String>, Double>> win2sid2execLat =
@@ -337,15 +339,15 @@ public class StatsUtil {
     }
 
     /**
-     * pre-merge component stats of specified spout id and returns { comp id -> comp-stats }
+     * pre-merge component stats of specified spout id and returns { comp id -> comp-stats }.
      */
     public static <K, V extends Number> Map<String, Object> aggPreMergeTopoPageSpout(
         Map<String, Object> m, String window, boolean includeSys) {
         Map<String, Object> ret = new HashMap<>();
 
         Map<String, Object> subRet = new HashMap<>();
-        putKV(subRet, NUM_EXECUTORS, 1);
-        putKV(subRet, NUM_TASKS, m.get(NUM_TASKS));
+        subRet.put(NUM_EXECUTORS, 1);
+        subRet.put(NUM_TASKS, m.get(NUM_TASKS));
 
         // no capacity for spout
         Map<String, Map<String, Map<String, V>>> stat2win2sid2num = ClientStatsUtil.getMapByKey(m, STATS);
@@ -361,7 +363,7 @@ public class StatsUtil {
                     sum += v.longValue();
                 }
             }
-            putKV(subRet, key, sum);
+            subRet.put(key, sum);
         }
 
         Map<String, Map<String, Double>> win2sid2compLat =
@@ -375,7 +377,7 @@ public class StatsUtil {
     }
 
     /**
-     * merge accumulated bolt stats with pre-merged component stats
+     * merge accumulated bolt stats with pre-merged component stats.
      *
      * @param accBoltStats accumulated bolt stats
      * @param boltStats    pre-merged component stats
@@ -391,47 +393,47 @@ public class StatsUtil {
         Map<String, Map<String, ?>> boltOut = ClientStatsUtil.getMapByKey(boltStats, SID_TO_OUT_STATS);
 
         int numExecutors = getByKeyOr0(accBoltStats, NUM_EXECUTORS).intValue();
-        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
-        putKV(ret, NUM_TASKS, sumOr0(
+        ret.put(NUM_EXECUTORS, numExecutors + 1);
+        ret.put(NUM_TASKS, sumOr0(
             getByKeyOr0(accBoltStats, NUM_TASKS), getByKeyOr0(boltStats, NUM_TASKS)));
 
         // (merge-with (partial merge-with sum-or-0) acc-out spout-out)
-        putKV(ret, SID_TO_OUT_STATS, fullMergeWithSum(accOut, boltOut));
+        ret.put(SID_TO_OUT_STATS, fullMergeWithSum(accOut, boltOut));
         // {component id -> metric -> value}, note that input may contain both long and double values
-        putKV(ret, CID_SID_TO_IN_STATS, fullMergeWithSum(accIn, boltIn));
+        ret.put(CID_SID_TO_IN_STATS, fullMergeWithSum(accIn, boltIn));
 
         long executed = sumStreamsLong(boltIn, EXECUTED);
-        putKV(ret, EXECUTED, executed);
+        ret.put(EXECUTED, executed);
 
         Map<String, Object> executorStats = new HashMap<>();
-        putKV(executorStats, EXECUTOR_ID, boltStats.get(EXECUTOR_ID));
-        putKV(executorStats, ClientStatsUtil.UPTIME, boltStats.get(ClientStatsUtil.UPTIME));
-        putKV(executorStats, HOST, boltStats.get(HOST));
-        putKV(executorStats, PORT, boltStats.get(PORT));
-        putKV(executorStats, CAPACITY, boltStats.get(CAPACITY));
-
-        putKV(executorStats, EMITTED, sumStreamsLong(boltOut, EMITTED));
-        putKV(executorStats, TRANSFERRED, sumStreamsLong(boltOut, TRANSFERRED));
-        putKV(executorStats, ACKED, sumStreamsLong(boltIn, ACKED));
-        putKV(executorStats, FAILED, sumStreamsLong(boltIn, FAILED));
-        putKV(executorStats, EXECUTED, executed);
+        executorStats.put(EXECUTOR_ID, boltStats.get(EXECUTOR_ID));
+        executorStats.put(ClientStatsUtil.UPTIME, boltStats.get(ClientStatsUtil.UPTIME));
+        executorStats.put(HOST, boltStats.get(HOST));
+        executorStats.put(PORT, boltStats.get(PORT));
+        executorStats.put(CAPACITY, boltStats.get(CAPACITY));
+
+        executorStats.put(EMITTED, sumStreamsLong(boltOut, EMITTED));
+        executorStats.put(TRANSFERRED, sumStreamsLong(boltOut, TRANSFERRED));
+        executorStats.put(ACKED, sumStreamsLong(boltIn, ACKED));
+        executorStats.put(FAILED, sumStreamsLong(boltIn, FAILED));
+        executorStats.put(EXECUTED, executed);
 
         if (executed > 0) {
-            putKV(executorStats, EXEC_LATENCY, sumStreamsDouble(boltIn, EXEC_LAT_TOTAL) / executed);
-            putKV(executorStats, PROC_LATENCY, sumStreamsDouble(boltIn, PROC_LAT_TOTAL) / executed);
+            executorStats.put(EXEC_LATENCY, sumStreamsDouble(boltIn, EXEC_LAT_TOTAL) / executed);
+            executorStats.put(PROC_LATENCY, sumStreamsDouble(boltIn, PROC_LAT_TOTAL) / executed);
         } else {
-            putKV(executorStats, EXEC_LATENCY, null);
-            putKV(executorStats, PROC_LATENCY, null);
+            executorStats.put(EXEC_LATENCY, null);
+            executorStats.put(PROC_LATENCY, null);
         }
         List executorStatsList = ((List) accBoltStats.get(ClientStatsUtil.EXECUTOR_STATS));
         executorStatsList.add(executorStats);
-        putKV(ret, ClientStatsUtil.EXECUTOR_STATS, executorStatsList);
+        ret.put(ClientStatsUtil.EXECUTOR_STATS, executorStatsList);
 
         return ret;
     }
 
     /**
-     * merge accumulated bolt stats with pre-merged component stats
+     * merge accumulated bolt stats with pre-merged component stats.
      */
     public static Map<String, Object> mergeAggCompStatsCompPageSpout(
         Map<String, Object> accSpoutStats, Map<String, Object> spoutStats) {
@@ -442,36 +444,36 @@ public class StatsUtil {
         Map<String, Map<String, ?>> spoutOut = ClientStatsUtil.getMapByKey(spoutStats, SID_TO_OUT_STATS);
 
         int numExecutors = getByKeyOr0(accSpoutStats, NUM_EXECUTORS).intValue();
-        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
-        putKV(ret, NUM_TASKS, sumOr0(
+        ret.put(NUM_EXECUTORS, numExecutors + 1);
+        ret.put(NUM_TASKS, sumOr0(
             getByKeyOr0(accSpoutStats, NUM_TASKS), getByKeyOr0(spoutStats, NUM_TASKS)));
-        putKV(ret, SID_TO_OUT_STATS, fullMergeWithSum(accOut, spoutOut));
+        ret.put(SID_TO_OUT_STATS, fullMergeWithSum(accOut, spoutOut));
 
         Map executorStats = new HashMap();
-        putKV(executorStats, EXECUTOR_ID, spoutStats.get(EXECUTOR_ID));
-        putKV(executorStats, ClientStatsUtil.UPTIME, spoutStats.get(ClientStatsUtil.UPTIME));
-        putKV(executorStats, HOST, spoutStats.get(HOST));
-        putKV(executorStats, PORT, spoutStats.get(PORT));
-
-        putKV(executorStats, EMITTED, sumStreamsLong(spoutOut, EMITTED));
-        putKV(executorStats, TRANSFERRED, sumStreamsLong(spoutOut, TRANSFERRED));
-        putKV(executorStats, FAILED, sumStreamsLong(spoutOut, FAILED));
+        executorStats.put(EXECUTOR_ID, spoutStats.get(EXECUTOR_ID));
+        executorStats.put(ClientStatsUtil.UPTIME, spoutStats.get(ClientStatsUtil.UPTIME));
+        executorStats.put(HOST, spoutStats.get(HOST));
+        executorStats.put(PORT, spoutStats.get(PORT));
+
+        executorStats.put(EMITTED, sumStreamsLong(spoutOut, EMITTED));
+        executorStats.put(TRANSFERRED, sumStreamsLong(spoutOut, TRANSFERRED));
+        executorStats.put(FAILED, sumStreamsLong(spoutOut, FAILED));
         long acked = sumStreamsLong(spoutOut, ACKED);
-        putKV(executorStats, ACKED, acked);
+        executorStats.put(ACKED, acked);
         if (acked > 0) {
-            putKV(executorStats, COMP_LATENCY, sumStreamsDouble(spoutOut, COMP_LAT_TOTAL) / acked);
+            executorStats.put(COMP_LATENCY, sumStreamsDouble(spoutOut, COMP_LAT_TOTAL) / acked);
         } else {
-            putKV(executorStats, COMP_LATENCY, null);
+            executorStats.put(COMP_LATENCY, null);
         }
         List executorStatsList = ((List) accSpoutStats.get(ClientStatsUtil.EXECUTOR_STATS));
         executorStatsList.add(executorStats);
-        putKV(ret, ClientStatsUtil.EXECUTOR_STATS, executorStatsList);
+        ret.put(ClientStatsUtil.EXECUTOR_STATS, executorStatsList);
 
         return ret;
     }
 
     /**
-     * merge accumulated bolt stats with new bolt stats
+     * merge accumulated bolt stats with new bolt stats.
      *
      * @param accBoltStats accumulated bolt stats
      * @param boltStats    new input bolt stats
@@ -482,50 +484,35 @@ public class StatsUtil {
         Map<String, Object> ret = new HashMap<>();
 
         Integer numExecutors = getByKeyOr0(accBoltStats, NUM_EXECUTORS).intValue();
-        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
-        putKV(ret, NUM_TASKS,
-              sumOr0(getByKeyOr0(accBoltStats, NUM_TASKS), getByKeyOr0(boltStats, NUM_TASKS)));
-        putKV(ret, EMITTED,
-              sumOr0(getByKeyOr0(accBoltStats, EMITTED), getByKeyOr0(boltStats, EMITTED)));
-        putKV(ret, TRANSFERRED,
-              sumOr0(getByKeyOr0(accBoltStats, TRANSFERRED), getByKeyOr0(boltStats, TRANSFERRED)));
-        putKV(ret, EXEC_LAT_TOTAL,
-              sumOr0(getByKeyOr0(accBoltStats, EXEC_LAT_TOTAL), getByKeyOr0(boltStats, EXEC_LAT_TOTAL)));
-        putKV(ret, PROC_LAT_TOTAL,
-              sumOr0(getByKeyOr0(accBoltStats, PROC_LAT_TOTAL), getByKeyOr0(boltStats, PROC_LAT_TOTAL)));
-        putKV(ret, EXECUTED,
-              sumOr0(getByKeyOr0(accBoltStats, EXECUTED), getByKeyOr0(boltStats, EXECUTED)));
-        putKV(ret, ACKED,
-              sumOr0(getByKeyOr0(accBoltStats, ACKED), getByKeyOr0(boltStats, ACKED)));
-        putKV(ret, FAILED,
-              sumOr0(getByKeyOr0(accBoltStats, FAILED), getByKeyOr0(boltStats, FAILED)));
-        putKV(ret, CAPACITY,
-              maxOr0(getByKeyOr0(accBoltStats, CAPACITY), getByKeyOr0(boltStats, CAPACITY)));
+        ret.put(NUM_EXECUTORS, numExecutors + 1);
+        ret.put(NUM_TASKS, sumOr0(getByKeyOr0(accBoltStats, NUM_TASKS), getByKeyOr0(boltStats, NUM_TASKS)));
+        ret.put(EMITTED, sumOr0(getByKeyOr0(accBoltStats, EMITTED), getByKeyOr0(boltStats, EMITTED)));
+        ret.put(TRANSFERRED, sumOr0(getByKeyOr0(accBoltStats, TRANSFERRED), getByKeyOr0(boltStats, TRANSFERRED)));
+        ret.put(EXEC_LAT_TOTAL, sumOr0(getByKeyOr0(accBoltStats, EXEC_LAT_TOTAL), getByKeyOr0(boltStats, EXEC_LAT_TOTAL)));
+        ret.put(PROC_LAT_TOTAL, sumOr0(getByKeyOr0(accBoltStats, PROC_LAT_TOTAL), getByKeyOr0(boltStats, PROC_LAT_TOTAL)));
+        ret.put(EXECUTED, sumOr0(getByKeyOr0(accBoltStats, EXECUTED), getByKeyOr0(boltStats, EXECUTED)));
+        ret.put(ACKED, sumOr0(getByKeyOr0(accBoltStats, ACKED), getByKeyOr0(boltStats, ACKED)));
+        ret.put(FAILED, sumOr0(getByKeyOr0(accBoltStats, FAILED), getByKeyOr0(boltStats, FAILED)));
+        ret.put(CAPACITY, maxOr0(getByKeyOr0(accBoltStats, CAPACITY), getByKeyOr0(boltStats, CAPACITY)));
 
         return ret;
     }
 
     /**
-     * merge accumulated bolt stats with new bolt stats
+     * merge accumulated bolt stats with new bolt stats.
      */
     public static Map<String, Object> mergeAggCompStatsTopoPageSpout(Map<String, Object> accSpoutStats,
                                                                      Map<String, Object> spoutStats) {
         Map<String, Object> ret = new HashMap<>();
 
         Integer numExecutors = getByKeyOr0(accSpoutStats, NUM_EXECUTORS).intValue();
-        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
-        putKV(ret, NUM_TASKS,
-              sumOr0(getByKeyOr0(accSpoutStats, NUM_TASKS), getByKeyOr0(spoutStats, NUM_TASKS)));
-        putKV(ret, EMITTED,
-              sumOr0(getByKeyOr0(accSpoutStats, EMITTED), getByKeyOr0(spoutStats, EMITTED)));
-        putKV(ret, TRANSFERRED,
-              sumOr0(getByKeyOr0(accSpoutStats, TRANSFERRED), getByKeyOr0(spoutStats, TRANSFERRED)));
-        putKV(ret, COMP_LAT_TOTAL,
-              sumOr0(getByKeyOr0(accSpoutStats, COMP_LAT_TOTAL), getByKeyOr0(spoutStats, COMP_LAT_TOTAL)));
-        putKV(ret, ACKED,
-              sumOr0(getByKeyOr0(accSpoutStats, ACKED), getByKeyOr0(spoutStats, ACKED)));
-        putKV(ret, FAILED,
-              sumOr0(getByKeyOr0(accSpoutStats, FAILED), getByKeyOr0(spoutStats, FAILED)));
+        ret.put(NUM_EXECUTORS, numExecutors + 1);
+        ret.put(NUM_TASKS, sumOr0(getByKeyOr0(accSpoutStats, NUM_TASKS), getByKeyOr0(spoutStats, NUM_TASKS)));
+        ret.put(EMITTED, sumOr0(getByKeyOr0(accSpoutStats, EMITTED), getByKeyOr0(spoutStats, EMITTED)));
+        ret.put(TRANSFERRED, sumOr0(getByKeyOr0(accSpoutStats, TRANSFERRED), getByKeyOr0(spoutStats, TRANSFERRED)));
+        ret.put(COMP_LAT_TOTAL, sumOr0(getByKeyOr0(accSpoutStats, COMP_LAT_TOTAL), getByKeyOr0(spoutStats, COMP_LAT_TOTAL)));
+        ret.put(ACKED, sumOr0(getByKeyOr0(accSpoutStats, ACKED), getByKeyOr0(spoutStats, ACKED)));
+        ret.put(FAILED, sumOr0(getByKeyOr0(accSpoutStats, FAILED), getByKeyOr0(spoutStats, FAILED)));
 
         return ret;
     }
@@ -537,15 +524,6 @@ public class StatsUtil {
         String window, boolean includeSys, Map<String, Object> accStats, Map<String, Object> beat, String compType) {
         Map<String, Object> ret = new HashMap<>();
 
-        Set workerSet = (Set) accStats.get(WORKERS_SET);
-        Map bolt2stats = ClientStatsUtil.getMapByKey(accStats, BOLT_TO_STATS);
-        Map spout2stats = ClientStatsUtil.getMapByKey(accStats, SPOUT_TO_STATS);
-        Map win2emitted = ClientStatsUtil.getMapByKey(accStats, WIN_TO_EMITTED);
-        Map win2transferred = ClientStatsUtil.getMapByKey(accStats, WIN_TO_TRANSFERRED);
-        Map win2compLatWgtAvg = ClientStatsUtil.getMapByKey(accStats, WIN_TO_COMP_LAT_WGT_AVG);
-        Map win2acked = ClientStatsUtil.getMapByKey(accStats, WIN_TO_ACKED);
-        Map win2failed = ClientStatsUtil.getMapByKey(accStats, WIN_TO_FAILED);
-
         boolean isSpout = compType.equals(ClientStatsUtil.SPOUT);
         // component id -> stats
         Map<String, Object> cid2stats;
@@ -556,7 +534,8 @@ public class StatsUtil {
         }
 
         Map stats = ClientStatsUtil.getMapByKey(beat, STATS);
-        Map w2compLatWgtAvg, w2acked;
+        Map w2compLatWgtAvg;
+        Map w2acked;
         Map compLatStats = ClientStatsUtil.getMapByKey(stats, COMP_LATENCIES);
         if (isSpout) { // agg spout stats
             Map mm = new HashMap();
@@ -573,21 +552,37 @@ public class StatsUtil {
             w2acked = aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, ACKED));
         }
 
+        Set workerSet = (Set) accStats.get(WORKERS_SET);
         workerSet.add(Lists.newArrayList(beat.get(HOST), beat.get(PORT)));
-        putKV(ret, WORKERS_SET, workerSet);
-        putKV(ret, BOLT_TO_STATS, bolt2stats);
-        putKV(ret, SPOUT_TO_STATS, spout2stats);
-        putKV(ret, WIN_TO_EMITTED, mergeWithSumLong(win2emitted, aggregateCountStreams(
+        ret.put(WORKERS_SET, workerSet);
+
+        Map bolt2stats = ClientStatsUtil.getMapByKey(accStats, BOLT_TO_STATS);
+        ret.put(BOLT_TO_STATS, bolt2stats);
+
+        Map spout2stats = ClientStatsUtil.getMapByKey(accStats, SPOUT_TO_STATS);
+        ret.put(SPOUT_TO_STATS, spout2stats);
+
+        Map win2emitted = ClientStatsUtil.getMapByKey(accStats, WIN_TO_EMITTED);
+        ret.put(WIN_TO_EMITTED, mergeWithSumLong(win2emitted, aggregateCountStreams(
             filterSysStreams(ClientStatsUtil.getMapByKey(stats, EMITTED), includeSys))));
-        putKV(ret, WIN_TO_TRANSFERRED, mergeWithSumLong(win2transferred, aggregateCountStreams(
+
+        Map win2transferred = ClientStatsUtil.getMapByKey(accStats, WIN_TO_TRANSFERRED);
+        ret.put(WIN_TO_TRANSFERRED, mergeWithSumLong(win2transferred, aggregateCountStreams(
             filterSysStreams(ClientStatsUtil.getMapByKey(stats, TRANSFERRED), includeSys))));
-        putKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSumDouble(win2compLatWgtAvg, w2compLatWgtAvg));
 
-        //boolean isSpoutStat = SPOUT.equals(((Keyword) getByKey(stats, TYPE)).getName());
-        putKV(ret, WIN_TO_ACKED, isSpout ? mergeWithSumLong(win2acked, w2acked) : win2acked);
-        putKV(ret, WIN_TO_FAILED, isSpout ?
-            mergeWithSumLong(aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, FAILED)), win2failed) : win2failed);
-        putKV(ret, TYPE, stats.get(TYPE));
+        Map win2compLatWgtAvg = ClientStatsUtil.getMapByKey(accStats, WIN_TO_COMP_LAT_WGT_AVG);
+        ret.put(WIN_TO_COMP_LAT_WGT_AVG, mergeWithSumDouble(win2compLatWgtAvg, w2compLatWgtAvg));
+
+        Map win2acked = ClientStatsUtil.getMapByKey(accStats, WIN_TO_ACKED);
+        Object v1 = isSpout ? mergeWithSumLong(win2acked, w2acked) : win2acked;
+        ret.put(WIN_TO_ACKED, v1);
+
+        Map win2failed = ClientStatsUtil.getMapByKey(accStats, WIN_TO_FAILED);
+        Object v = isSpout
+            ? mergeWithSumLong(aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, FAILED)), win2failed) : win2failed;
+
+        ret.put(WIN_TO_FAILED, v);
+        ret.put(TYPE, stats.get(TYPE));
 
         // (merge-with merge-agg-comp-stats-topo-page-bolt/spout (acc-stats comp-key) cid->statk->num)
         // (acc-stats comp-key) ==> bolt2stats/spout2stats
@@ -605,7 +600,7 @@ public class StatsUtil {
     }
 
     /**
-     * aggregate topo executors stats
+     * aggregate topo executors stats.
      *
      * @param topologyId     topology id
      * @param exec2nodePort  executor -> host+port
@@ -625,16 +620,16 @@ public class StatsUtil {
         return postAggregateTopoStats(task2component, exec2nodePort, topoStats, topologyId, clusterState);
     }
 
-    public static Map<String, Object> aggregateTopoStats(String win, boolean includeSys, List<Map<String, Object>> heartbeats) {
+    private static Map<String, Object> aggregateTopoStats(String win, boolean includeSys, List<Map<String, Object>> heartbeats) {
         Map<String, Object> initVal = new HashMap<>();
-        putKV(initVal, WORKERS_SET, new HashSet());
-        putKV(initVal, BOLT_TO_STATS, new HashMap());
-        putKV(initVal, SPOUT_TO_STATS, new HashMap());
-        putKV(initVal, WIN_TO_EMITTED, new HashMap());
-        putKV(initVal, WIN_TO_TRANSFERRED, new HashMap());
-        putKV(initVal, WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
-        putKV(initVal, WIN_TO_ACKED, new HashMap());
-        putKV(initVal, WIN_TO_FAILED, new HashMap());
+        initVal.put(WORKERS_SET, new HashSet());
+        initVal.put(BOLT_TO_STATS, new HashMap());
+        initVal.put(SPOUT_TO_STATS, new HashMap());
+        initVal.put(WIN_TO_EMITTED, new HashMap());
+        initVal.put(WIN_TO_TRANSFERRED, new HashMap());
+        initVal.put(WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
+        initVal.put(WIN_TO_ACKED, new HashMap());
+        initVal.put(WIN_TO_FAILED, new HashMap());
 
         for (Map<String, Object> heartbeat : heartbeats) {
             String compType = (String) heartbeat.get(TYPE);
@@ -644,7 +639,7 @@ public class StatsUtil {
         return initVal;
     }
 
-    public static TopologyPageInfo postAggregateTopoStats(Map task2comp, Map exec2nodePort, Map<String, Object> accData,
+    private static TopologyPageInfo postAggregateTopoStats(Map task2comp, Map exec2nodePort, Map<String, Object> accData,
                                                           String topologyId, IStormClusterState clusterState) {
         TopologyPageInfo ret = new TopologyPageInfo(topologyId);
 
@@ -656,19 +651,19 @@ public class StatsUtil {
         Map<String, ComponentAggregateStats> aggBolt2stats = new HashMap<>();
         for (Object o : bolt2stats.entrySet()) {
             Map.Entry e = (Map.Entry) o;
-            String id = (String) e.getKey();
             Map m = (Map) e.getValue();
             long executed = getByKeyOr0(m, EXECUTED).longValue();
             if (executed > 0) {
                 double execLatencyTotal = getByKeyOr0(m, EXEC_LAT_TOTAL).doubleValue();
-                putKV(m, EXEC_LATENCY, execLatencyTotal / executed);
+                m.put(EXEC_LATENCY, execLatencyTotal / executed);
 
                 double procLatencyTotal = getByKeyOr0(m, PROC_LAT_TOTAL).doubleValue();
-                putKV(m, PROC_LATENCY, procLatencyTotal / executed);
+                m.put(PROC_LATENCY, procLatencyTotal / executed);
             }
-            remove(m, EXEC_LAT_TOTAL);
-            remove(m, PROC_LAT_TOTAL);
-            putKV(m, "last-error", getLastError(clusterState, topologyId, id));
+            m.remove(EXEC_LAT_TOTAL);
+            m.remove(PROC_LAT_TOTAL);
+            String id = (String) e.getKey();
+            m.put("last-error", getLastError(clusterState, topologyId, id));
 
             aggBolt2stats.put(id, thriftifyBoltAggStats(m));
         }
@@ -682,10 +677,10 @@ public class StatsUtil {
             long acked = getByKeyOr0(m, ACKED).longValue();
             if (acked > 0) {
                 double compLatencyTotal = getByKeyOr0(m, COMP_LAT_TOTAL).doubleValue();
-                putKV(m, COMP_LATENCY, compLatencyTotal / acked);
+                m.put(COMP_LATENCY, compLatencyTotal / acked);
             }
-            remove(m, COMP_LAT_TOTAL);
-            putKV(m, "last-error", getLastError(clusterState, topologyId, id));
+            m.remove(COMP_LAT_TOTAL);
+            m.put("last-error", getLastError(clusterState, topologyId, id));
 
             aggSpout2stats.put(id, thriftifySpoutAggStats(m));
         }
@@ -706,7 +701,7 @@ public class StatsUtil {
     }
 
     /**
-     * aggregate bolt stats
+     * aggregate bolt stats.
      *
      * @param statsSeq   a seq of ExecutorStats
      * @param includeSys whether to include system streams
@@ -733,17 +728,17 @@ public class StatsUtil {
             executeLatencies.add(stat.get_specific().get_bolt().get_execute_ms_avg());
         }
         mergeMaps(ret, commonStats);
-        putKV(ret, ACKED, aggregateCounts(acked));
-        putKV(ret, FAILED, aggregateCounts(failed));
-        putKV(ret, EXECUTED, aggregateCounts(executed));
-        putKV(ret, PROC_LATENCIES, aggregateAverages(processLatencies, acked));
-        putKV(ret, EXEC_LATENCIES, aggregateAverages(executeLatencies, executed));
+        ((Map) ret).put(ACKED, aggregateCounts(acked));
+        ((Map) ret).put(FAILED, aggregateCounts(failed));
+        ((Map) ret).put(EXECUTED, aggregateCounts(executed));
+        ((Map) ret).put(PROC_LATENCIES, aggregateAverages(processLatencies, acked));
+        ((Map) ret).put(EXEC_LATENCIES, aggregateAverages(executeLatencies, executed));
 
         return ret;
     }
 
     /**
-     * aggregate spout stats
+     * aggregate spout stats.
      *
      * @param statsSeq   a seq of ExecutorStats
      * @param includeSys whether to include system streams
@@ -767,15 +762,15 @@ public class StatsUtil {
             completeLatencies.add(stats.get_specific().get_spout().get_complete_ms_avg());
         }
         ret.putAll(commonStats);
-        putKV(ret, ACKED, aggregateCounts(acked));
-        putKV(ret, FAILED, aggregateCounts(failed));
-        putKV(ret, COMP_LATENCIES, aggregateAverages(completeLatencies, acked));
+        ((Map) ret).put(ACKED, aggregateCounts(acked));
+        ((Map) ret).put(FAILED, aggregateCounts(failed));
+        ((Map) ret).put(COMP_LATENCIES, aggregateAverages(completeLatencies, acked));
 
         return ret;
     }
 
     /**
-     * aggregate common stats from a spout/bolt, called in aggregateSpoutStats/aggregateBoltStats
+     * aggregate common stats from a spout/bolt, called in aggregateSpoutStats/aggregateBoltStats.
      */
     public static <T> Map<String, Map<String, Map<T, Long>>> aggregateCommonStats(List<ExecutorSummary> statsSeq) {
         Map<String, Map<String, Map<T, Long>>> ret = new HashMap<>();
@@ -786,28 +781,28 @@ public class StatsUtil {
             emitted.add(summ.get_stats().get_emitted());
             transferred.add(summ.get_stats().get_transferred());
         }
-        putKV(ret, EMITTED, aggregateCounts(emitted));
-        putKV(ret, TRANSFERRED, aggregateCounts(transferred));
+        ((Map) ret).put(EMITTED, aggregateCounts(emitted));
+        ((Map) ret).put(TRANSFERRED, aggregateCounts(transferred));
 
         return ret;
     }
 
     /**
-     * filter system streams of aggregated spout/bolt stats if necessary
+     * filter system streams of aggregated spout/bolt stats if necessary.
      */
     public static <T> Map<String, Map<String, Map<T, Long>>> preProcessStreamSummary(
         Map<String, Map<String, Map<T, Long>>> streamSummary, boolean includeSys) {
         Map<String, Map<T, Long>> emitted = ClientStatsUtil.getMapByKey(streamSummary, EMITTED);
         Map<String, Map<T, Long>> transferred = ClientStatsUtil.getMapByKey(streamSummary, TRANSFERRED);
 
-        putKV(streamSummary, EMITTED, filterSysStreams(emitted, includeSys));
-        putKV(streamSummary, TRANSFERRED, filterSysStreams(transferred, includeSys));
+        ((Map) streamSummary).put(EMITTED, filterSysStreams(emitted, includeSys));
+        ((Map) streamSummary).put(TRANSFERRED, filterSysStreams(transferred, includeSys));
 
         return streamSummary;
     }
 
     /**
-     * aggregate count streams by window
+     * aggregate count streams by window.
      *
      * @param stats a Map of value: {win -> stream -> value}
      * @return a Map of value: {win -> value}
@@ -817,7 +812,7 @@ public class StatsUtil {
         Map<String, Long> ret = new HashMap<>();
         for (Map.Entry<String, Map<K, V>> entry : stats.entrySet()) {
             Map<K, V> value = entry.getValue();
-            long sum = 0l;
+            long sum = 0L;
             for (V num : value.values()) {
                 sum += num.longValue();
             }
@@ -827,7 +822,7 @@ public class StatsUtil {
     }
 
     /**
-     * compute an weighted average from a list of average maps and a corresponding count maps extracted from a list of ExecutorSummary
+     * compute an weighted average from a list of average maps and a corresponding count maps extracted from a list of ExecutorSummary.
      *
      * @param avgSeq   a list of {win -> global stream id -> avg value}
      * @param countSeq a list of {win -> global stream id -> count value}
@@ -854,7 +849,7 @@ public class StatsUtil {
     }
 
     /**
-     * aggregate weighted average of all streams
+     * aggregate weighted average of all streams.
      *
      * @param avgs   a Map of {win -> stream -> average value}
      * @param counts a Map of {win -> stream -> count value}
@@ -869,7 +864,7 @@ public class StatsUtil {
             String win = entry.getKey();
 
             double avgTotal = 0.0;
-            long cntTotal = 0l;
+            long cntTotal = 0L;
             Map<K, List> inner = entry.getValue();
             for (K kk : inner.keySet()) {
                 List vv = inner.get(kk);
@@ -883,7 +878,7 @@ public class StatsUtil {
     }
 
     /**
-     * aggregates spout stream stats, returns a Map of {metric -> win -> aggregated value}
+     * aggregates spout stream stats, returns a Map of {metric -> win -> aggregated value}.
      */
     public static Map<String, Map> spoutStreamsStats(List<ExecutorSummary> summs, boolean includeSys) {
         if (summs == null) {
@@ -895,7 +890,7 @@ public class StatsUtil {
     }
 
     /**
-     * aggregates bolt stream stats, returns a Map of {metric -> win -> aggregated value}
+     * aggregates bolt stream stats, returns a Map of {metric -> win -> aggregated value}.
      */
     public static Map<String, Map> boltStreamsStats(List<ExecutorSummary> summs, boolean includeSys) {
         if (summs == null) {
@@ -906,7 +901,7 @@ public class StatsUtil {
     }
 
     /**
-     * aggregate all spout streams
+     * aggregate all spout streams.
      *
      * @param stats a Map of {metric -> win -> stream id -> value}
      * @return a Map of {metric -> win -> aggregated value}
@@ -914,37 +909,37 @@ public class StatsUtil {
     public static Map<String, Map> aggregateSpoutStreams(Map<String, Map> stats) {
         // actual ret is Map<String, Map<String, Long/Double>>
         Map<String, Map> ret = new HashMap<>();
-        putKV(ret, ACKED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, ACKED)));
-        putKV(ret, FAILED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, FAILED)));
-        putKV(ret, EMITTED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, EMITTED)));
-        putKV(ret, TRANSFERRED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, TRANSFERRED)));
-        putKV(ret, COMP_LATENCIES, aggregateAvgStreams(
+        ((Map) ret).put(ACKED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, ACKED)));
+        ((Map) ret).put(FAILED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, FAILED)));
+        ((Map) ret).put(EMITTED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, EMITTED)));
+        ((Map) ret).put(TRANSFERRED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, TRANSFERRED)));
+        ((Map) ret).put(COMP_LATENCIES, aggregateAvgStreams(
             ClientStatsUtil.getMapByKey(stats, COMP_LATENCIES), ClientStatsUtil.getMapByKey(stats, ACKED)));
         return ret;
     }
 
     /**
-     * aggregate all bolt streams
+     * aggregate all bolt streams.
      *
      * @param stats a Map of {metric -> win -> stream id -> value}
      * @return a Map of {metric -> win -> aggregated value}
      */
     public static Map<String, Map> aggregateBoltStreams(Map<String, Map> stats) {
         Map<String, Map> ret = new HashMap<>();
-        putKV(ret, ACKED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, ACKED)));
-        putKV(ret, FAILED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, FAILED)));
-        putKV(ret, EMITTED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, EMITTED)));
-        putKV(ret, TRANSFERRED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, TRANSFERRED)));
-        putKV(ret, EXECUTED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, EXECUTED)));
-        putKV(ret, PROC_LATENCIES, aggregateAvgStreams(
+        ((Map) ret).put(ACKED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, ACKED)));
+        ((Map) ret).put(FAILED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, FAILED)));
+        ((Map) ret).put(EMITTED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, EMITTED)));
+        ((Map) ret).put(TRANSFERRED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, TRANSFERRED)));
+        ((Map) ret).put(EXECUTED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, EXECUTED)));
+        ((Map) ret).put(PROC_LATENCIES, aggregateAvgStreams(
             ClientStatsUtil.getMapByKey(stats, PROC_LATENCIES), ClientStatsUtil.getMapByKey(stats, ACKED)));
-        putKV(ret, EXEC_LATENCIES, aggregateAvgStreams(
+        ((Map) ret).put(EXEC_LATENCIES, aggregateAvgStreams(
             ClientStatsUtil.getMapByKey(stats, EXEC_LATENCIES), ClientStatsUtil.getMapByKey(stats, EXECUTED)));
         return ret;
     }
 
     /**
-     * aggregate windowed stats from a bolt executor stats with a Map of accumulated stats
+     * aggregate windowed stats from a bolt executor stats with a Map of accumulated stats.
      */
     public static Map<String, Object> aggBoltExecWinStats(
         Map<String, Object> accStats, Map<String, Object> newStats, boolean includeSys) {
@@ -966,29 +961,29 @@ public class StatsUtil {
         Map<String, Map<String, Long>> emitted = ClientStatsUtil.getMapByKey(newStats, EMITTED);
         Map<String, Long> win2emitted = mergeWithSumLong(aggregateCountStreams(filterSysStreams(emitted, includeSys)),
                                                          ClientStatsUtil.getMapByKey(accStats, WIN_TO_EMITTED));
-        putKV(ret, WIN_TO_EMITTED, win2emitted);
+        ret.put(WIN_TO_EMITTED, win2emitted);
 
         Map<String, Map<String, Long>> transferred = ClientStatsUtil.getMapByKey(newStats, TRANSFERRED);
         Map<String, Long> win2transferred = mergeWithSumLong(aggregateCountStreams(filterSysStreams(transferred, includeSys)),
                                                              ClientStatsUtil.getMapByKey(accStats, WIN_TO_TRANSFERRED));
-        putKV(ret, WIN_TO_TRANSFERRED, win2transferred);
+        ret.put(WIN_TO_TRANSFERRED, win2transferred);
 
-        putKV(ret, WIN_TO_EXEC_LAT_WGT_AVG, mergeWithSumDouble(
+        ret.put(WIN_TO_EXEC_LAT_WGT_AVG, mergeWithSumDouble(
             ClientStatsUtil.getMapByKey(accStats, WIN_TO_EXEC_LAT_WGT_AVG), win2execLatWgtAvg));
-        putKV(ret, WIN_TO_PROC_LAT_WGT_AVG, mergeWithSumDouble(
+        ret.put(WIN_TO_PROC_LAT_WGT_AVG, mergeWithSumDouble(
             ClientStatsUtil.getMapByKey(accStats, WIN_TO_PROC_LAT_WGT_AVG), win2procLatWgtAvg));
-        putKV(ret, WIN_TO_EXECUTED, mergeWithSumLong(
+        ret.put(WIN_TO_EXECUTED, mergeWithSumLong(
             ClientStatsUtil.getMapByKey(accStats, WIN_TO_EXECUTED), win2executed));
-        putKV(ret, WIN_TO_ACKED, mergeWithSumLong(
+        ret.put(WIN_TO_ACKED, mergeWithSumLong(
             aggregateCountStreams(ClientStatsUtil.getMapByKey(newStats, ACKED)), ClientStatsUtil.getMapByKey(accStats, WIN_TO_ACKED)));
-        putKV(ret, WIN_TO_FAILED, mergeWithSumLong(
+        ret.put(WIN_TO_FAILED, mergeWithSumLong(
             aggregateCountStreams(ClientStatsUtil.getMapByKey(newStats, FAILED)), ClientStatsUtil.getMapByKey(accStats, WIN_TO_FAILED)));
 
         return ret;
     }
 
     /**
-     * aggregate windowed stats from a spout executor stats with a Map of accumulated stats
+     * aggregate windowed stats from a spout executor stats with a Map of accumulated stats.
      */
     public static Map<String, Object> aggSpoutExecWinStats(
         Map<String, Object> accStats, Map<String, Object> beat, boolean includeSys) {
@@ -1008,18 +1003,18 @@ public class StatsUtil {
         Map<String, Map<String, Long>> emitted = ClientStatsUtil.getMapByKey(beat, EMITTED);
         Map<String, Long> win2emitted = mergeWithSumLong(aggregateCountStreams(filterSysStreams(emitted, includeSys)),
                                                          ClientStatsUtil.getMapByKey(accStats, WIN_TO_EMITTED));
-        putKV(ret, WIN_TO_EMITTED, win2emitted);
+        ret.put(WIN_TO_EMITTED, win2emitted);
 
         Map<String, Map<String, Long>> transferred = ClientStatsUtil.getMapByKey(beat, TRANSFERRED);
         Map<String, Long> win2transferred = mergeWithSumLong(aggregateCountStreams(filterSysStreams(transferred, includeSys)),
                                                              ClientStatsUtil.getMapByKey(accStats, WIN_TO_TRANSFERRED));
-        putKV(ret, WIN_TO_TRANSFERRED, win2transferred);
+        ret.put(WIN_TO_TRANSFERRED, win2transferred);
 
-        putKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSumDouble(
+        ret.put(WIN_TO_COMP_LAT_WGT_AVG, mergeWithSumDouble(
             ClientStatsUtil.getMapByKey(accStats, WIN_TO_COMP_LAT_WGT_AVG), win2compLatWgtAvg));
-        putKV(ret, WIN_TO_ACKED, mergeWithSumLong(
+        ret.put(WIN_TO_ACKED, mergeWithSumLong(
             ClientStatsUtil.getMapByKey(accStats, WIN_TO_ACKED), win2acked));
-        putKV(ret, WIN_TO_FAILED, mergeWithSumLong(
+        ret.put(WIN_TO_FAILED, mergeWithSumLong(
             aggregateCountStreams(ClientStatsUtil.getMapByKey(beat, FAILED)), ClientStatsUtil.getMapByKey(accStats, WIN_TO_FAILED)));
 
         return ret;
@@ -1027,7 +1022,7 @@ public class StatsUtil {
 
 
     /**
-     * aggregate a list of count maps into one map
+     * aggregate a list of count maps into one map.
      *
      * @param countsSeq a seq of {win -> GlobalStreamId -> value}
      */
@@ -1056,30 +1051,33 @@ public class StatsUtil {
         return ret;
     }
 
+    /**
+     * Aggregate the stats for a component over a given window of time.
+     */
     public static Map<String, Object> aggregateCompStats(
         String window, boolean includeSys, List<Map<String, Object>> beats, String compType) {
-        boolean isSpout = ClientStatsUtil.SPOUT.equals(compType);
 
         Map<String, Object> initVal = new HashMap<>();
-        putKV(initVal, WIN_TO_ACKED, new HashMap());
-        putKV(initVal, WIN_TO_FAILED, new HashMap());
-        putKV(initVal, WIN_TO_EMITTED, new HashMap());
-        putKV(initVal, WIN_TO_TRANSFERRED, new HashMap());
+        initVal.put(WIN_TO_ACKED, new HashMap());
+        initVal.put(WIN_TO_FAILED, new HashMap());
+        initVal.put(WIN_TO_EMITTED, new HashMap());
+        initVal.put(WIN_TO_TRANSFERRED, new HashMap());
 
         Map<String, Object> stats = new HashMap();
-        putKV(stats, ClientStatsUtil.EXECUTOR_STATS, new ArrayList());
-        putKV(stats, SID_TO_OUT_STATS, new HashMap());
+        stats.put(ClientStatsUtil.EXECUTOR_STATS, new ArrayList());
+        stats.put(SID_TO_OUT_STATS, new HashMap());
+        boolean isSpout = ClientStatsUtil.SPOUT.equals(compType);
         if (isSpout) {
-            putKV(initVal, TYPE, ClientStatsUtil.SPOUT);
-            putKV(initVal, WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
+            initVal.put(TYPE, ClientStatsUtil.SPOUT);
+            initVal.put(WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
         } else {
-            putKV(initVal, TYPE, ClientStatsUtil.BOLT);
-            putKV(initVal, WIN_TO_EXECUTED, new HashMap());
-            putKV(stats, CID_SID_TO_IN_STATS, new HashMap());
-            putKV(initVal, WIN_TO_EXEC_LAT_WGT_AVG, new HashMap());
-            putKV(initVal, WIN_TO_PROC_LAT_WGT_AVG, new HashMap());
+            initVal.put(TYPE, ClientStatsUtil.BOLT);
+            initVal.put(WIN_TO_EXECUTED, new HashMap());
+            stats.put(CID_SID_TO_IN_STATS, new HashMap());
+            initVal.put(WIN_TO_EXEC_LAT_WGT_AVG, new HashMap());
+            initVal.put(WIN_TO_PROC_LAT_WGT_AVG, new HashMap());
         }
-        putKV(initVal, STATS, stats);
+        initVal.put(STATS, stats);
 
         // iterate through all executor heartbeats
         for (Map<String, Object> beat : beats) {
@@ -1098,16 +1096,16 @@ public class StatsUtil {
         Map<String, Object> ret = new HashMap<>();
         if (ClientStatsUtil.SPOUT.equals(compType)) {
             ret.putAll(aggSpoutExecWinStats(accStats, ClientStatsUtil.getMapByKey(beat, STATS), includeSys));
-            putKV(ret, STATS, mergeAggCompStatsCompPageSpout(
-                ClientStatsUtil.getMapByKey(accStats, STATS),
-                aggPreMergeCompPageSpout(beat, window, includeSys)));
+            ret.put(STATS, mergeAggCompStatsCompPageSpout(
+                    ClientStatsUtil.getMapByKey(accStats, STATS),
+                    aggPreMergeCompPageSpout(beat, window, includeSys)));
         } else {
             ret.putAll(aggBoltExecWinStats(accStats, ClientStatsUtil.getMapByKey(beat, STATS), includeSys));
-            putKV(ret, STATS, mergeAggCompStatsCompPageBolt(
-                ClientStatsUtil.getMapByKey(accStats, STATS),
-                aggPreMergeCompPageBolt(beat, window, includeSys)));
+            ret.put(STATS, mergeAggCompStatsCompPageBolt(
+                    ClientStatsUtil.getMapByKey(accStats, STATS),
+                    aggPreMergeCompPageBolt(beat, window, includeSys)));
         }
-        putKV(ret, TYPE, compType);
+        ret.put(TYPE, compType);
 
         return ret;
     }
@@ -1127,14 +1125,14 @@ public class StatsUtil {
         Integer numExecutors = getByKeyOr0(stats, NUM_EXECUTORS).intValue();
         Map outStats = ClientStatsUtil.getMapByKey(stats, SID_TO_OUT_STATS);
 
-        putKV(ret, TYPE, compType);
-        putKV(ret, NUM_TASKS, numTasks);
-        putKV(ret, NUM_EXECUTORS, numExecutors);
-        putKV(ret, ClientStatsUtil.EXECUTOR_STATS, stats.get(ClientStatsUtil.EXECUTOR_STATS));
-        putKV(ret, WIN_TO_EMITTED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_EMITTED)));
-        putKV(ret, WIN_TO_TRANSFERRED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_TRANSFERRED)));
-        putKV(ret, WIN_TO_ACKED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_ACKED)));
-        putKV(ret, WIN_TO_FAILED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_FAILED)));
+        ret.put(TYPE, compType);
+        ret.put(NUM_TASKS, numTasks);
+        ret.put(NUM_EXECUTORS, numExecutors);
+        ret.put(ClientStatsUtil.EXECUTOR_STATS, stats.get(ClientStatsUtil.EXECUTOR_STATS));
+        ret.put(WIN_TO_EMITTED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_EMITTED)));
+        ret.put(WIN_TO_TRANSFERRED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_TRANSFERRED)));
+        ret.put(WIN_TO_ACKED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_ACKED)));
+        ret.put(WIN_TO_FAILED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_FAILED)));
 
         if (ClientStatsUtil.BOLT.equals(compType)) {
             Map inStats = ClientStatsUtil.getMapByKey(stats, CID_SID_TO_IN_STATS);
@@ -1142,30 +1140,29 @@ public class StatsUtil {
             Map inStats2 = new HashMap();
             for (Object o : inStats.entrySet()) {
                 Map.Entry e = (Map.Entry) o;
-                Object k = e.getKey();
                 Map v = (Map) e.getValue();
                 long executed = getByKeyOr0(v, EXECUTED).longValue();
                 if (executed > 0) {
                     double executeLatencyTotal = getByKeyOr0(v, EXEC_LAT_TOTAL).doubleValue();
                     double processLatencyTotal = getByKeyOr0(v, PROC_LAT_TOTAL).doubleValue();
-                    putKV(v, EXEC_LATENCY, executeLatencyTotal / executed);
-                    putKV(v, PROC_LATENCY, processLatencyTotal / executed);
+                    v.put(EXEC_LATENCY, executeLatencyTotal / executed);
+                    v.put(PROC_LATENCY, processLatencyTotal / executed);
                 } else {
-                    putKV(v, EXEC_LATENCY, 0.0);
-                    putKV(v, PROC_LATENCY, 0.0);
+                    v.put(EXEC_LATENCY, 0.0);
+                    v.put(PROC_LATENCY, 0.0);
                 }
-                remove(v, EXEC_LAT_TOTAL);
-                remove(v, PROC_LAT_TOTAL);
-                inStats2.put(k, v);
+                v.remove(EXEC_LAT_TOTAL);
+                v.remove(PROC_LAT_TOTAL);
+                inStats2.put(e.getKey(), v);
             }
-            putKV(ret, CID_SID_TO_IN_STATS, inStats2);
-
-            putKV(ret, SID_TO_OUT_STATS, outStats);
-            putKV(ret, WIN_TO_EXECUTED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_EXECUTED)));
-            putKV(ret, WIN_TO_EXEC_LAT, computeWeightedAveragesPerWindow(
-                compStats, WIN_TO_EXEC_LAT_WGT_AVG, WIN_TO_EXECUTED));
-            putKV(ret, WIN_TO_PROC_LAT, computeWeightedAveragesPerWindow(
-                compStats, WIN_TO_PROC_LAT_WGT_AVG, WIN_TO_EXECUTED));
+            ret.put(CID_SID_TO_IN_STATS, inStats2);
+
+            ret.put(SID_TO_OUT_STATS, outStats);
+            ret.put(WIN_TO_EXECUTED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_EXECUTED)));
+            ret.put(WIN_TO_EXEC_LAT, computeWeightedAveragesPerWindow(
+                    compStats, WIN_TO_EXEC_LAT_WGT_AVG, WIN_TO_EXECUTED));
+            ret.put(WIN_TO_PROC_LAT, computeWeightedAveragesPerWindow(
+                    compStats, WIN_TO_PROC_LAT_WGT_AVG, WIN_TO_EXECUTED));
         } else {
             Map outStats2 = new HashMap();
             for (Object o : outStats.entrySet()) {
@@ -1175,23 +1172,23 @@ public class StatsUtil {
                 long acked = getByKeyOr0(v, ACKED).longValue();
                 if (acked > 0) {
                     double compLatencyTotal = getByKeyOr0(v, COMP_LAT_TOTAL).doubleValue();
-                    putKV(v, COMP_LATENCY, compLatencyTotal / acked);
+                    v.put(COMP_LATENCY, compLatencyTotal / acked);
                 } else {
-                    putKV(v, COMP_LATENCY, 0.0);
+                    v.put(COMP_LATENCY, 0.0);
                 }
-                remove(v, COMP_LAT_TOTAL);
+                v.remove(COMP_LAT_TOTAL);
                 outStats2.put(k, v);
             }
-            putKV(ret, SID_TO_OUT_STATS, outStats2);
-            putKV(ret, WIN_TO_COMP_LAT, computeWeightedAveragesPerWindow(
-                compStats, WIN_TO_COMP_LAT_WGT_AVG, WIN_TO_ACKED));
+            ret.put(SID_TO_OUT_STATS, outStats2);
+            ret.put(WIN_TO_COMP_LAT, computeWeightedAveragesPerWindow(
+                    compStats, WIN_TO_COMP_LAT_WGT_AVG, WIN_TO_ACKED));
         }
 
         return ret;
     }
 
     /**
-     * aggregate component executor stats
+     * aggregate component executor stats.
      *
      * @param exec2hostPort  a Map of {executor -> host+port}
      * @param task2component a Map of {task id -> component}
@@ -1225,7 +1222,6 @@ public class StatsUtil {
      * @param includeSys       whether to include system streams
      * @param userAuthorized   whether the user is authorized to view topology info
      * @param filterSupervisor if not null, only return WorkerSummaries for that supervisor
-     * @return List<WorkerSummary> thrift structures
      */
     public static List<WorkerSummary> aggWorkerStats(String stormId, String stormName,
                                                      Map<Integer, String> task2Component,
@@ -1313,8 +1309,8 @@ public class StatsUtil {
     }
 
     /**
-     * Aggregate statistics per worker for a topology. Optionally filtering on specific supervisors
-     *
+     * Aggregate statistics per worker for a topology. Optionally filtering on specific supervisors.
+     * <br/>
      * Convenience overload when called from the topology page code (in that case we want data for all workers in the topology, not filtered
      * by supervisor)
      *
@@ -1325,7 +1321,6 @@ public class StatsUtil {
      * @param exec2NodePort  a Map of {executor -> host+port}
      * @param includeSys     whether to include system streams
      * @param userAuthorized whether the user is authorized to view topology info
-     * @return List<WorkerSummary> thrift structures
      */
     public static List<WorkerSummary> aggWorkerStats(String stormId, String stormName,
                                                      Map<Integer, String> task2Component,
@@ -1344,7 +1339,7 @@ public class StatsUtil {
     // =====================================================================================
 
     /**
-     * convert thrift executor heartbeats into a java HashMap
+     * convert thrift executor heartbeats into a java HashMap.
      */
     public static Map<List<Integer>, Map<String, Object>> convertExecutorBeats(Map<ExecutorInfo, ExecutorBeat> beats) {
         Map<List<Integer>, Map<String, Object>> ret = new HashMap<>();
@@ -1359,10 +1354,7 @@ public class StatsUtil {
     }
 
     /**
-     * convert {@link SupervisorWorkerHeartbeat} to nimbus local report executor heartbeats
-     *
-     * @param workerHeartbeat
-     * @return
+     * convert {@link SupervisorWorkerHeartbeat} to nimbus local report executor heartbeats.
      */
     public static Map<List<Integer>, Map<String, Object>> convertWorkerBeats(SupervisorWorkerHeartbeat workerHeartbeat) {
         Map<List<Integer>, Map<String, Object>> ret = new HashMap<>();
@@ -1377,7 +1369,7 @@ public class StatsUtil {
     }
 
     /**
-     * convert thrift ExecutorBeat into a java HashMap
+     * convert thrift ExecutorBeat into a java HashMap.
      */
     public static Map<String, Object> convertZkExecutorHb(ExecutorBeat beat) {
         Map<String, Object> ret = new HashMap<>();
@@ -1391,10 +1383,7 @@ public class StatsUtil {
     }
 
     /**
-     * convert a thrift worker heartbeat into a java HashMap
-     *
-     * @param workerHb
-     * @return
+     * convert a thrift worker heartbeat into a java HashMap.
      */
     public static Map<String, Object> convertZkWorkerHb(ClusterWorkerHeartbeat workerHb) {
         Map<String, Object> ret = new HashMap<>();
@@ -1408,7 +1397,7 @@ public class StatsUtil {
     }
 
     /**
-     * convert executors stats into a HashMap, note that ExecutorStats are remained unchanged
+     * convert executors stats into a HashMap, note that ExecutorStats are remained unchanged.
      */
     public static Map<List<Integer>, ExecutorStats> convertExecutorsStats(Map<ExecutorInfo, ExecutorStats> stats) {
         Map<List<Integer>, ExecutorStats> ret = new HashMap<>();
@@ -1423,21 +1412,21 @@ public class StatsUtil {
     }
 
     /**
-     * convert thrift ExecutorStats structure into a java HashMap
+     * convert thrift ExecutorStats structure into a java HashMap.
      */
     public static Map<String, Object> convertExecutorStats(ExecutorStats stats) {
         Map<String, Object> ret = new HashMap<>();
 
-        putKV(ret, EMITTED, stats.get_emitted());
-        putKV(ret, TRANSFERRED, stats.get_transferred());
-        putKV(ret, RATE, stats.get_rate());
+        ret.put(EMITTED, stats.get_emitted());
+        ret.put(TRANSFERRED, stats.get_transferred());
+        ret.put(RATE, stats.get_rate());
 
         if (stats.get_specific().is_set_bolt()) {
             ret.putAll(convertSpecificStats(stats.get_specific().get_bolt()));
-            putKV(ret, TYPE, ClientStatsUtil.BOLT);
+            ret.put(TYPE, ClientStatsUtil.BOLT);
         } else {
             ret.putAll(convertSpecificStats(stats.get_specific().get_spout()));
-            putKV(ret, TYPE, ClientStatsUtil.SPOUT);
+            ret.put(TYPE, ClientStatsUtil.SPOUT);
         }
 
         return ret;
@@ -1445,9 +1434,9 @@ public class StatsUtil {
 
     private static Map<String, Object> convertSpecificStats(SpoutStats stats) {
         Map<String, Object> ret = new HashMap<>();
-        putKV(ret, ACKED, stats.get_acked());
-        putKV(ret, FAILED, stats.get_failed());
-        putKV(ret, COMP_LATENCIES, stats.get_complete_ms_avg());
+        ret.put(ACKED, stats.get_acked());
+        ret.put(FAILED, stats.get_failed());
+        ret.put(COMP_LATENCIES, stats.get_complete_ms_avg());
 
         return ret;
     }
@@ -1461,17 +1450,17 @@ public class StatsUtil {
         Map executed = ClientStatsUtil.windowSetConverter(stats.get_executed(), FROM_GSID, ClientStatsUtil.IDENTITY);
         Map executeAvg = ClientStatsUtil.windowSetConverter(stats.get_execute_ms_avg(), FROM_GSID, ClientStatsUtil.IDENTITY);
 
-        putKV(ret, ACKED, acked);
-        putKV(ret, FAILED, failed);
-        putKV(ret, PROC_LATENCIES, processAvg);
-        putKV(ret, EXECUTED, executed);
-        putKV(ret, EXEC_LATENCIES, executeAvg);
+        ret.put(ACKED, acked);
+        ret.put(FAILED, failed);
+        ret.put(PROC_LATENCIES, processAvg);
+        ret.put(EXECUTED, executed);
+        ret.put(EXEC_LATENCIES, executeAvg);
 
         return ret;
     }
 
     /**
-     * extract a list of host port info for specified component
+     * extract a list of host port info for specified component.
      *
      * @param exec2hostPort  {executor -> host+port}
      * @param task2component {task id -> component}
@@ -1499,8 +1488,8 @@ public class StatsUtil {
 
         for (List hostPort : hostPorts) {
             Map<String, Object> m = new HashMap<>();
-            putKV(m, HOST, hostPort.get(0));
-            putKV(m, PORT, hostPort.get(1));
+            m.put(HOST, hostPort.get(0));
+            m.put(PORT, hostPort.get(1));
             ret.add(m);
         }
 
@@ -1513,7 +1502,7 @@ public class StatsUtil {
     // =====================================================================================
 
     /**
-     * extracts a list of executor data from heart beats
+     * extracts a list of executor data from heart beats.
      */
     public static List<Map<String, Object>> extractDataFromHb(Map executor2hostPort, Map task2component,
                                                               Map<List<Integer>, Map<String, Object>> beats,
@@ -1521,6 +1510,9 @@ public class StatsUtil {
         return extractDataFromHb(executor2hostPort, task2component, beats, includeSys, topology, null);
     }
 
+    /**
+     * extracts a list of executor data from heart beats.
+     */
     public static List<Map<String, Object>> extractDataFromHb(Map executor2hostPort, Map task2component,
                                                               Map<List<Integer>, Map<String, Object>> beats,
                                                               boolean includeSys, StormTopology topology, String compId) {
@@ -1547,21 +1539,21 @@ public class StatsUtil {
 
             Map<String, Object> m = new HashMap<>();
             if ((compId == null || compId.equals(id)) && (includeSys || !Utils.isSystemId(id))) {
-                putKV(m, "exec-id", entry.getKey());
-                putKV(m, "comp-id", id);
-                putKV(m, NUM_TASKS, end - start + 1);
-                putKV(m, HOST, host);
-                putKV(m, PORT, port);
+                m.put("exec-id", entry.getKey());
+                m.put("comp-id", id);
+                m.put(NUM_TASKS, end - start + 1);
+                m.put(HOST, host);
+                m.put(PORT, port);
 
                 Map stats = ClientStatsUtil.getMapByKey(beat, STATS);
-                putKV(m, ClientStatsUtil.UPTIME, beat.get(ClientStatsUtil.UPTIME));
-                putKV(m, STATS, stats);
+                m.put(ClientStatsUtil.UPTIME, beat.get(ClientStatsUtil.UPTIME));
+                m.put(STATS, stats);
 
                 String type = componentType(topology, compId);
                 if (type != null) {
-                    putKV(m, TYPE, type);
+                    m.put(TYPE, type);
                 } else {
-                    putKV(m, TYPE, stats.get(TYPE));
+                    m.put(TYPE, stats.get(TYPE));
                 }
                 ret.add(m);
             }
@@ -1570,7 +1562,7 @@ public class StatsUtil {
     }
 
     /**
-     * compute weighted avg from a Map of stats and given avg/count keys
+     * compute weighted avg from a Map of stats and given avg/count keys.
      *
      * @param accData    a Map of {win -> key -> value}
      * @param wgtAvgKey  weighted average key
@@ -1593,7 +1585,7 @@ public class StatsUtil {
     }
 
     /**
-     * computes max bolt capacity
+     * computes max bolt capacity.
      *
      * @param executorSumms a list of ExecutorSummary
      * @return max bolt capacity
@@ -1609,6 +1601,11 @@ public class StatsUtil {
         return max;
     }
 
+    /**
+     * Compute the capacity of a executor. approximation of the % of time spent doing real work.
+     * @param summary the stats for the executor.
+     * @return the capacity of the executor.
+     */
     public static double computeExecutorCapacity(ExecutorSummary summary) {
         ExecutorStats stats = summary.get_stats();
         if (stats == null) {
@@ -1633,7 +1630,7 @@ public class StatsUtil {
     }
 
     /**
-     * filter ExecutorSummary whose stats is null
+     * filter ExecutorSummary whose stats is null.
      *
      * @param summs a list of ExecutorSummary
      * @return filtered summs
@@ -1687,7 +1684,7 @@ public class StatsUtil {
     }
 
     /**
-     * same as clojure's (merge-with merge m1 m2)
+     * same as clojure's (merge-with merge m1 m2).
      */
     private static Map mergeMaps(Map m1, Map m2) {
         if (m2 == null) {
@@ -1709,7 +1706,7 @@ public class StatsUtil {
 
 
     /**
-     * filter system streams from stats
+     * filter system streams from stats.
      *
      * @param stream2stat { stream id -> value }
      * @param includeSys  whether to filter system streams
@@ -1729,7 +1726,7 @@ public class StatsUtil {
     }
 
     /**
-     * filter system streams from stats
+     * filter system streams from stats.
      *
      * @param stats      { win -> stream id -> value }
      * @param includeSys whether to filter system streams
@@ -1753,7 +1750,7 @@ public class StatsUtil {
     }
 
     /**
-     * equals to clojure's: (merge-with (partial merge-with sum-or-0) acc-out spout-out)
+     * equals to clojure's: (merge-with (partial merge-with sum-or-0) acc-out spout-out).
      */
     private static <K1, K2> Map<K1, Map<K2, Number>> fullMergeWithSum(Map<K1, Map<K2, ?>> m1,
                                                                       Map<K1, Map<K2, ?>> m2) {
@@ -1767,7 +1764,8 @@ public class StatsUtil {
 
         Map<K1, Map<K2, Number>> ret = new HashMap<>();
         for (K1 k : allKeys) {
-            Map<K2, ?> mm1 = null, mm2 = null;
+            Map<K2, ?> mm1 = null;
+            Map<K2, ?> mm2 = null;
             if (m1 != null) {
                 mm1 = m1.get(k);
             }
@@ -1842,7 +1840,7 @@ public class StatsUtil {
     }
 
     /**
-     * this method merges 2 two-level-deep maps, which is different from mergeWithSum, and we expect the two maps have the same keys
+     * this method merges 2 two-level-deep maps, which is different from mergeWithSum, and we expect the two maps have the same keys.
      */
     private static <K> Map<String, Map<K, List>> mergeWithAddPair(Map<String, Map<K, List>> m1,
                                                                   Map<String, Map<K, List>> m2) {
@@ -1892,10 +1890,8 @@ public class StatsUtil {
 
     /**
      * Used for local test.
-     *
-     * @return
      */
-    public static SupervisorWorkerHeartbeat thriftifyRPCWorkerHb(String stormId, List<Long> executorId) {
+    public static SupervisorWorkerHeartbeat thriftifyRpcWorkerHb(String stormId, List<Long> executorId) {
         SupervisorWorkerHeartbeat supervisorWorkerHeartbeat = new SupervisorWorkerHeartbeat();
         supervisorWorkerHeartbeat.set_storm_id(stormId);
         supervisorWorkerHeartbeat
@@ -1936,8 +1932,6 @@ public class StatsUtil {
     }
 
     private static ExecutorAggregateStats thriftifyExecAggStats(String compId, String compType, Map m) {
-        ExecutorAggregateStats stats = new ExecutorAggregateStats();
-
         ExecutorSummary executorSummary = new ExecutorSummary();
         List executor = (List) m.get(EXECUTOR_ID);
         executorSummary.set_executor_info(new ExecutorInfo(((Number) executor.get(0)).intValue(),
@@ -1947,6 +1941,7 @@ public class StatsUtil {
         executorSummary.set_port(getByKeyOr0(m, PORT).intValue());
         int uptime = getByKeyOr0(m, ClientStatsUtil.UPTIME).intValue();
         executorSummary.set_uptime_secs(uptime);
+        ExecutorAggregateStats stats = new ExecutorAggregateStats();
         stats.set_exec_summary(executorSummary);
 
         if (compType.equals(ClientStatsUtil.SPOUT)) {
@@ -2003,20 +1998,20 @@ public class StatsUtil {
         ret.set_component_id(compId);
 
         Map win2stats = new HashMap();
-        putKV(win2stats, EMITTED, ClientStatsUtil.getMapByKey(data, WIN_TO_EMITTED));
-        putKV(win2stats, TRANSFERRED, ClientStatsUtil.getMapByKey(data, WIN_TO_TRANSFERRED));
-        putKV(win2stats, ACKED, ClientStatsUtil.getMapByKey(data, WIN_TO_ACKED));
-        putKV(win2stats, FAILED, ClientStatsUtil.getMapByKey(data, WIN_TO_FAILED));
+        win2stats.put(EMITTED, ClientStatsUtil.getMapByKey(data, WIN_TO_EMITTED));
+        win2stats.put(TRANSFERRED, ClientStatsUtil.getMapByKey(data, WIN_TO_TRANSFERRED));
+        win2stats.put(ACKED, ClientStatsUtil.getMapByKey(data, WIN_TO_ACKED));
+        win2stats.put(FAILED, ClientStatsUtil.getMapByKey(data, WIN_TO_FAILED));
 
         String compType = (String) data.get(TYPE);
         if (compType.equals(ClientStatsUtil.SPOUT)) {
             ret.set_component_type(ComponentType.SPOUT);
-            putKV(win2stats, COMP_LATENCY, ClientStatsUtil.getMapByKey(data, WIN_TO_COMP_LAT));
+            win2stats.put(COMP_LATENCY, ClientStatsUtil.getMapByKey(data, WIN_TO_COMP_LAT));
         } else {
             ret.set_component_type(ComponentType.BOLT);
-            putKV(win2stats, EXEC_LATENCY, ClientStatsUtil.getMapByKey(data, WIN_TO_EXEC_LAT));
-            putKV(win2stats, PROC_LATENCY, ClientStatsUtil.getMapByKey(data, WIN_TO_PROC_LAT));
-            putKV(win2stats, EXECUTED, ClientStatsUtil.getMapByKey(data, WIN_TO_EXECUTED));
+            win2stats.put(EXEC_LATENCY, ClientStatsUtil.getMapByKey(data, WIN_TO_EXEC_LAT));
+            win2stats.put(PROC_LATENCY, ClientStatsUtil.getMapByKey(data, WIN_TO_PROC_LAT));
+            win2stats.put(EXECUTED, ClientStatsUtil.getMapByKey(data, WIN_TO_EXECUTED));
         }
         win2stats = swapMapOrder(win2stats);
 
@@ -2028,7 +2023,8 @@ public class StatsUtil {
             }
         }
 
-        Map gsid2inputStats, sid2outputStats;
+        Map gsid2inputStats;
+        Map sid2outputStats;
         if (compType.equals(ClientStatsUtil.SPOUT)) {
             Map tmp = new HashMap();
             for (Object k : win2stats.keySet()) {
@@ -2058,20 +2054,11 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map thriftifyStats(List stats) {
-        Map ret = new HashMap();
-        for (Object o : stats) {
-            List stat = (List) o;
-            List executor = (List) stat.get(0);
-            int start = ((Number) executor.get(0)).intValue();
-            int end = ((Number) executor.get(1)).intValue();
-            Map executorStat = (Map) stat.get(1);
-            ExecutorInfo executorInfo = new ExecutorInfo(start, end);
-            ret.put(executorInfo, thriftifyExecutorStats(executorStat));
-        }
-        return ret;
-    }
-
+    /**
+     * Convert Executor stats to thrift data structure.
+     * @param stats the stats in the form of a map.
+     * @return teh thrift structure for the stats.
+     */
     public static ExecutorStats thriftifyExecutorStats(Map stats) {
         ExecutorStats ret = new ExecutorStats();
         ExecutorSpecificStats specificStats = thriftifySpecificStats(stats);
@@ -2090,17 +2077,23 @@ public class StatsUtil {
         String compType = (String) stats.get(TYPE);
         if (ClientStatsUtil.BOLT.equals(compType)) {
             BoltStats boltStats = new BoltStats();
-            boltStats.set_acked(ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, ACKED), ClientStatsUtil.TO_GSID, TO_STRING));
-            boltStats.set_executed(ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, EXECUTED), ClientStatsUtil.TO_GSID, TO_STRING));
-            boltStats.set_execute_ms_avg(ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, EXEC_LATENCIES), ClientStatsUtil.TO_GSID, TO_STRING));
-            boltStats.set_failed(ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, FAILED), ClientStatsUtil.TO_GSID, TO_STRING));
-            boltStats.set_process_ms_avg(ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, PROC_LATENCIES), ClientStatsUtil.TO_GSID, TO_STRING));
+            boltStats.set_acked(
+                ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, ACKED), ClientStatsUtil.TO_GSID, TO_STRING));
+            boltStats.set_executed(
+                ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, EXECUTED), ClientStatsUtil.TO_GSID, TO_STRING));
+            boltStats.set_execute_ms_avg(
+                ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, EXEC_LATENCIES), ClientStatsUtil.TO_GSID, TO_STRING));
+            boltStats.set_failed(
+                ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, FAILED), ClientStatsUtil.TO_GSID, TO_STRING));
+            boltStats.set_process_ms_avg(
+                ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, PROC_LATENCIES), ClientStatsUtil.TO_GSID, TO_STRING));
             specificStats.set_bolt(boltStats);
         } else {
             SpoutStats spoutStats = new SpoutStats();
             spoutStats.set_acked(ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, ACKED), TO_STRING, TO_STRING));
             spoutStats.set_failed(ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, FAILED), TO_STRING, TO_STRING));
-            spoutStats.set_complete_ms_avg(ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, COMP_LATENCIES), TO_STRING, TO_STRING));
+            spoutStats.set_complete_ms_avg(
+                ClientStatsUtil.windowSetConverter(ClientStatsUtil.getMapByKey(stats, COMP_LATENCIES), TO_STRING, TO_STRING));
             specificStats.set_spout(spoutStats);
         }
         return specificStats;
@@ -2116,15 +2109,15 @@ public class StatsUtil {
     }
 
     /**
-     * Returns true if x is a number that is not NaN or Infinity, false otherwise
+     * Returns true if x is a number that is not NaN or Infinity, false otherwise.
      */
     private static boolean isValidNumber(Object x) {
-        return x != null && x instanceof Number &&
-               !Double.isNaN(((Number) x).doubleValue()) &&
-               !Double.isInfinite(((Number) x).doubleValue());
+        return x != null && x instanceof Number
+            && !Double.isNaN(((Number) x).doubleValue())
+            && !Double.isInfinite(((Number) x).doubleValue());
     }
 
-    /**
+    /*
      * the value of m is as follows:
      * <pre>
      * #org.apache.storm.stats.CommonStats {
@@ -2209,6 +2202,12 @@ public class StatsUtil {
         return productOr0(id2Avg.get(key), id2num.get(key));
     }
 
+    /**
+     * Get the coponenet type for a give id.
+     * @param topology the topology this is a part of.
+     * @param compId the id of the component.
+     * @return the type as a String "BOLT" or "SPOUT".
+     */
     public static String componentType(StormTopology topology, String compId) {
         if (compId == null) {
             return null;
@@ -2221,14 +2220,6 @@ public class StatsUtil {
         return ClientStatsUtil.SPOUT;
     }
 
-    public static void putKV(Map map, String k, Object v) {
-        map.put(k, v);
-    }
-
-    private static void remove(Map map, String k) {
-        map.remove(k);
-    }
-
     private static <K, V extends Number> long sumValues(Map<K, V> m) {
         long ret = 0L;
         if (m == null) {
@@ -2293,6 +2284,7 @@ public class StatsUtil {
     }
 
     /**
+     * Expand the count/average out into total, count.
      * @param avgs   a HashMap of values: { win -> GlobalStreamId -> value }
      * @param counts a HashMap of values: { win -> GlobalStreamId -> value }
      * @return a HashMap of values: {win -> GlobalStreamId -> [cnt*avg, cnt]}
@@ -2320,7 +2312,7 @@ public class StatsUtil {
     }
 
     /**
-     * first zip the two seqs, then do expand-average, then merge with sum
+     * first zip the two seqs, then do expand-average, then merge with sum.
      *
      * @param avgSeq   list of avgs like: [{win -> GlobalStreamId -> value}, ...]
      * @param countSeq list of counts like [{win -> GlobalStreamId -> value}, ...]
@@ -2347,6 +2339,11 @@ public class StatsUtil {
         return t / c;
     }
 
+    /**
+     * Convert a float to a string for display.
+     * @param n the value to format.
+     * @return the string ready for display.
+     */
     public static String floatStr(Double n) {
         if (n == null) {
             return "0";


[5/7] storm git commit: STORM-3162: Cleanup heartbeats cache and make it thread safe

Posted by bo...@apache.org.
STORM-3162: Cleanup heartbeats cache and make it thread safe


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

Branch: refs/heads/master
Commit: 9d3feb0e18a96a90cdce9c2ff1727c9c9ecca4a1
Parents: eaed3cb
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Fri Sep 14 14:48:33 2018 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Mon Sep 17 14:56:56 2018 -0500

----------------------------------------------------------------------
 storm-client/pom.xml                            |    2 +-
 .../org/apache/storm/daemon/worker/Worker.java  |   10 +-
 .../jvm/org/apache/storm/executor/Executor.java |    8 +-
 .../storm/executor/bolt/BoltExecutor.java       |    4 +-
 .../storm/executor/spout/SpoutExecutor.java     |    4 +-
 .../apache/storm/stats/BoltExecutorStats.java   |   10 +-
 .../org/apache/storm/stats/ClientStatsUtil.java |  202 ++
 .../jvm/org/apache/storm/stats/StatsUtil.java   | 2610 ------------------
 .../test/clj/org/apache/storm/nimbus_test.clj   |   22 +-
 storm-server/pom.xml                            |    2 +-
 .../storm/daemon/nimbus/HeartbeatCache.java     |  229 ++
 .../org/apache/storm/daemon/nimbus/Nimbus.java  |   87 +-
 .../java/org/apache/storm/stats/StatsUtil.java  | 2388 ++++++++++++++++
 13 files changed, 2866 insertions(+), 2712 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9d3feb0e/storm-client/pom.xml
----------------------------------------------------------------------
diff --git a/storm-client/pom.xml b/storm-client/pom.xml
index 1a120d8..2e60d19 100644
--- a/storm-client/pom.xml
+++ b/storm-client/pom.xml
@@ -164,7 +164,7 @@
                 <!--Note - the version would be inherited-->
                 <configuration>
                     <excludes>**/generated/**</excludes>
-                    <maxAllowedViolations>3285</maxAllowedViolations>
+                    <maxAllowedViolations>3067</maxAllowedViolations>
                 </configuration>
             </plugin>
             <plugin>

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3feb0e/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java b/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
index d02de9b..048425e 100644
--- a/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
+++ b/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java
@@ -55,7 +55,7 @@ import org.apache.storm.shade.com.google.common.base.Preconditions;
 import org.apache.storm.shade.org.apache.commons.io.FileUtils;
 import org.apache.storm.shade.org.apache.commons.lang.ObjectUtils;
 import org.apache.storm.shade.uk.org.lidalia.sysoutslf4j.context.SysOutOverSLF4J;
-import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.stats.ClientStatsUtil;
 import org.apache.storm.utils.ConfigUtils;
 import org.apache.storm.utils.LocalState;
 import org.apache.storm.utils.NimbusClient;
@@ -346,17 +346,17 @@ public class Worker implements Shutdownable, DaemonCommon {
         Map<List<Integer>, ExecutorStats> stats;
         List<IRunningExecutor> executors = this.executorsAtom.get();
         if (null == executors) {
-            stats = StatsUtil.mkEmptyExecutorZkHbs(workerState.localExecutors);
+            stats = ClientStatsUtil.mkEmptyExecutorZkHbs(workerState.localExecutors);
         } else {
-            stats = StatsUtil.convertExecutorZkHbs(executors.stream().collect(Collectors
+            stats = ClientStatsUtil.convertExecutorZkHbs(executors.stream().collect(Collectors
                                                                                   .toMap(IRunningExecutor::getExecutorId,
                                                                                          IRunningExecutor::renderStats)));
         }
-        Map<String, Object> zkHB = StatsUtil.mkZkWorkerHb(workerState.topologyId, stats, workerState.uptime.upTime());
+        Map<String, Object> zkHB = ClientStatsUtil.mkZkWorkerHb(workerState.topologyId, stats, workerState.uptime.upTime());
         try {
             workerState.stormClusterState
                 .workerHeartbeat(workerState.topologyId, workerState.assignmentId, (long) workerState.port,
-                                 StatsUtil.thriftifyZkWorkerHb(zkHB));
+                                 ClientStatsUtil.thriftifyZkWorkerHb(zkHB));
         } catch (Exception ex) {
             LOG.error("Worker failed to write heartbeats to ZK or Pacemaker...will retry", ex);
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3feb0e/storm-client/src/jvm/org/apache/storm/executor/Executor.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/Executor.java b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
index e3de2e1..2352eec 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/Executor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/Executor.java
@@ -60,8 +60,8 @@ import org.apache.storm.shade.com.google.common.collect.Lists;
 import org.apache.storm.shade.org.jctools.queues.MpscChunkedArrayQueue;
 import org.apache.storm.shade.org.json.simple.JSONValue;
 import org.apache.storm.shade.org.json.simple.parser.ParseException;
+import org.apache.storm.stats.ClientStatsUtil;
 import org.apache.storm.stats.CommonStats;
-import org.apache.storm.stats.StatsUtil;
 import org.apache.storm.task.WorkerTopologyContext;
 import org.apache.storm.tuple.AddressedTuple;
 import org.apache.storm.tuple.Fields;
@@ -177,7 +177,7 @@ public abstract class Executor implements Callable, JCQueue.Consumer {
         String componentId = workerTopologyContext.getComponentId(taskIds.get(0));
 
         String type = getExecutorType(workerTopologyContext, componentId);
-        if (StatsUtil.SPOUT.equals(type)) {
+        if (ClientStatsUtil.SPOUT.equals(type)) {
             executor = new SpoutExecutor(workerState, executorId, credentials);
         } else {
             executor = new BoltExecutor(workerState, executorId, credentials);
@@ -205,9 +205,9 @@ public abstract class Executor implements Callable, JCQueue.Consumer {
         Map<String, SpoutSpec> spouts = topology.get_spouts();
         Map<String, Bolt> bolts = topology.get_bolts();
         if (spouts.containsKey(componentId)) {
-            return StatsUtil.SPOUT;
+            return ClientStatsUtil.SPOUT;
         } else if (bolts.containsKey(componentId)) {
-            return StatsUtil.BOLT;
+            return ClientStatsUtil.BOLT;
         } else {
             throw new RuntimeException("Could not find " + componentId + " in " + topology);
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3feb0e/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
index 1008eba..48d39c6 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/bolt/BoltExecutor.java
@@ -38,7 +38,7 @@ import org.apache.storm.policy.WaitStrategyPark;
 import org.apache.storm.security.auth.IAutoCredentials;
 import org.apache.storm.shade.com.google.common.collect.ImmutableMap;
 import org.apache.storm.stats.BoltExecutorStats;
-import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.stats.ClientStatsUtil;
 import org.apache.storm.task.IBolt;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
@@ -68,7 +68,7 @@ public class BoltExecutor extends Executor {
     private BoltOutputCollectorImpl outputCollector;
 
     public BoltExecutor(WorkerState workerData, List<Long> executorId, Map<String, String> credentials) {
-        super(workerData, executorId, credentials, StatsUtil.BOLT);
+        super(workerData, executorId, credentials, ClientStatsUtil.BOLT);
         this.executeSampler = ConfigUtils.mkStatsSampler(topoConf);
         this.isSystemBoltExecutor = (executorId == Constants.SYSTEM_EXECUTOR_ID);
         if (isSystemBoltExecutor) {

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3feb0e/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java b/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java
index cd4bb09..c46a7b9 100644
--- a/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java
+++ b/storm-client/src/jvm/org/apache/storm/executor/spout/SpoutExecutor.java
@@ -37,8 +37,8 @@ import org.apache.storm.policy.IWaitStrategy.WAIT_SITUATION;
 import org.apache.storm.shade.com.google.common.collect.ImmutableMap;
 import org.apache.storm.spout.ISpout;
 import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.stats.ClientStatsUtil;
 import org.apache.storm.stats.SpoutExecutorStats;
-import org.apache.storm.stats.StatsUtil;
 import org.apache.storm.tuple.AddressedTuple;
 import org.apache.storm.tuple.TupleImpl;
 import org.apache.storm.utils.ConfigUtils;
@@ -73,7 +73,7 @@ public class SpoutExecutor extends Executor {
     private long threadId = 0;
 
     public SpoutExecutor(final WorkerState workerData, final List<Long> executorId, Map<String, String> credentials) {
-        super(workerData, executorId, credentials, StatsUtil.SPOUT);
+        super(workerData, executorId, credentials, ClientStatsUtil.SPOUT);
         this.spoutWaitStrategy = ReflectionUtils.newInstance((String) topoConf.get(Config.TOPOLOGY_SPOUT_WAIT_STRATEGY));
         this.spoutWaitStrategy.prepare(topoConf, WAIT_SITUATION.SPOUT_WAIT);
         this.backPressureWaitStrategy = ReflectionUtils.newInstance((String) topoConf.get(Config.TOPOLOGY_BACKPRESSURE_WAIT_STRATEGY));

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3feb0e/storm-client/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/stats/BoltExecutorStats.java b/storm-client/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
index 6fea28a..26b2d74 100644
--- a/storm-client/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
+++ b/storm-client/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
@@ -83,11 +83,11 @@ public class BoltExecutorStats extends CommonStats {
 
         // bolt stats
         BoltStats boltStats = new BoltStats(
-            StatsUtil.windowSetConverter(valueStat(getAcked()), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
-            StatsUtil.windowSetConverter(valueStat(getFailed()), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
-            StatsUtil.windowSetConverter(valueStat(processLatencyStats), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
-            StatsUtil.windowSetConverter(valueStat(executedStats), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
-            StatsUtil.windowSetConverter(valueStat(executeLatencyStats), StatsUtil.TO_GSID, StatsUtil.IDENTITY));
+            ClientStatsUtil.windowSetConverter(valueStat(getAcked()), ClientStatsUtil.TO_GSID, ClientStatsUtil.IDENTITY),
+            ClientStatsUtil.windowSetConverter(valueStat(getFailed()), ClientStatsUtil.TO_GSID, ClientStatsUtil.IDENTITY),
+            ClientStatsUtil.windowSetConverter(valueStat(processLatencyStats), ClientStatsUtil.TO_GSID, ClientStatsUtil.IDENTITY),
+            ClientStatsUtil.windowSetConverter(valueStat(executedStats), ClientStatsUtil.TO_GSID, ClientStatsUtil.IDENTITY),
+            ClientStatsUtil.windowSetConverter(valueStat(executeLatencyStats), ClientStatsUtil.TO_GSID, ClientStatsUtil.IDENTITY));
         ret.set_specific(ExecutorSpecificStats.bolt(boltStats));
 
         return ret;

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3feb0e/storm-client/src/jvm/org/apache/storm/stats/ClientStatsUtil.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/stats/ClientStatsUtil.java b/storm-client/src/jvm/org/apache/storm/stats/ClientStatsUtil.java
new file mode 100644
index 0000000..3f63db3
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/stats/ClientStatsUtil.java
@@ -0,0 +1,202 @@
+/*
+ * 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.
+ */
+
+package org.apache.storm.stats;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.storm.generated.ClusterWorkerHeartbeat;
+import org.apache.storm.generated.ExecutorInfo;
+import org.apache.storm.generated.ExecutorStats;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.shade.com.google.common.collect.Lists;
+import org.apache.storm.utils.Time;
+
+/**
+ * Stats calculations needed by storm client code.
+ */
+public class ClientStatsUtil {
+    public static final String SPOUT = "spout";
+    public static final String BOLT = "bolt";
+    static final String EXECUTOR_STATS = "executor-stats";
+    static final String UPTIME = "uptime";
+    public static final String TIME_SECS = "time-secs";
+    public static final ToGlobalStreamIdTransformer TO_GSID = new ToGlobalStreamIdTransformer();
+    public static final IdentityTransformer IDENTITY = new IdentityTransformer();
+
+    /**
+     * Convert a List<Long> executor to java List<Integer>.
+     */
+    public static List<Integer> convertExecutor(List<Long> executor) {
+        return Lists.newArrayList(executor.get(0).intValue(), executor.get(1).intValue());
+    }
+
+    /**
+     * Make and map of executors to empty stats.
+     * @param executors the executors as keys of the map.
+     * @return and empty map of executors to stats.
+     */
+    public static Map<List<Integer>, ExecutorStats> mkEmptyExecutorZkHbs(Set<List<Long>> executors) {
+        Map<List<Integer>, ExecutorStats> ret = new HashMap<>();
+        for (Object executor : executors) {
+            List startEnd = (List) executor;
+            ret.put(convertExecutor(startEnd), null);
+        }
+        return ret;
+    }
+
+    /**
+     * Convert Long Executor Ids in ZkHbs to Integer ones structure to java maps.
+     */
+    public static Map<List<Integer>, ExecutorStats> convertExecutorZkHbs(Map<List<Long>, ExecutorStats> executorBeats) {
+        Map<List<Integer>, ExecutorStats> ret = new HashMap<>();
+        for (Map.Entry<List<Long>, ExecutorStats> entry : executorBeats.entrySet()) {
+            ret.put(convertExecutor(entry.getKey()), entry.getValue());
+        }
+        return ret;
+    }
+
+    /**
+     * Create a new worker heartbeat for zookeeper.
+     * @param topoId the topology id
+     * @param executorStats the stats for the executors
+     * @param uptime the uptime for the worker.
+     * @return the heartbeat map.
+     */
+    public static Map<String, Object> mkZkWorkerHb(String topoId, Map<List<Integer>, ExecutorStats> executorStats, Integer uptime) {
+        Map<String, Object> ret = new HashMap<>();
+        ret.put("storm-id", topoId);
+        ret.put(EXECUTOR_STATS, executorStats);
+        ret.put(UPTIME, uptime);
+        ret.put(TIME_SECS, Time.currentTimeSecs());
+
+        return ret;
+    }
+
+    private static Number getByKeyOr0(Map<String, Object> m, String k) {
+        if (m == null) {
+            return 0;
+        }
+
+        Number n = (Number) m.get(k);
+        if (n == null) {
+            return 0;
+        }
+        return n;
+    }
+
+    /**
+     * Get a sub-map by a given key.
+     * @param map the original map
+     * @param key the key to get it from.
+     * @return the map stored under key.
+     */
+    public static <K, V> Map<K, V> getMapByKey(Map map, String key) {
+        if (map == null) {
+            return null;
+        }
+        return (Map<K, V>) map.get(key);
+    }
+
+    public static ClusterWorkerHeartbeat thriftifyZkWorkerHb(Map<String, Object> heartbeat) {
+        ClusterWorkerHeartbeat ret = new ClusterWorkerHeartbeat();
+        ret.set_uptime_secs(getByKeyOr0(heartbeat, UPTIME).intValue());
+        ret.set_storm_id((String) heartbeat.get("storm-id"));
+        ret.set_time_secs(getByKeyOr0(heartbeat, TIME_SECS).intValue());
+
+        Map<ExecutorInfo, ExecutorStats> convertedStats = new HashMap<>();
+
+        Map<List<Integer>, ExecutorStats> executorStats = getMapByKey(heartbeat, EXECUTOR_STATS);
+        if (executorStats != null) {
+            for (Map.Entry<List<Integer>, ExecutorStats> entry : executorStats.entrySet()) {
+                List<Integer> executor = entry.getKey();
+                ExecutorStats stats = entry.getValue();
+                if (null != stats) {
+                    convertedStats.put(new ExecutorInfo(executor.get(0), executor.get(1)), stats);
+                }
+            }
+        }
+        ret.set_executor_stats(convertedStats);
+
+        return ret;
+    }
+
+    /**
+     * Converts stats to be over given windows of time.
+     * @param stats the stats
+     * @param secKeyFunc transform the sub-key
+     * @param firstKeyFunc transform the main key
+     */
+    public static <K1, K2> Map windowSetConverter(
+        Map stats, KeyTransformer<K2> secKeyFunc, KeyTransformer<K1> firstKeyFunc) {
+        Map ret = new HashMap();
+
+        for (Object o : stats.entrySet()) {
+            Map.Entry entry = (Map.Entry) o;
+            K1 key1 = firstKeyFunc.transform(entry.getKey());
+
+            Map subRetMap = (Map) ret.get(key1);
+            if (subRetMap == null) {
+                subRetMap = new HashMap();
+            }
+            ret.put(key1, subRetMap);
+
+            Map value = (Map) entry.getValue();
+            for (Object oo : value.entrySet()) {
+                Map.Entry subEntry = (Map.Entry) oo;
+                K2 key2 = secKeyFunc.transform(subEntry.getKey());
+                subRetMap.put(key2, subEntry.getValue());
+            }
+        }
+        return ret;
+    }
+
+    // =====================================================================================
+    // key transformers
+    // =====================================================================================
+
+    /**
+     * Provides a way to transform one key into another.
+     * @param <T>
+     */
+    interface KeyTransformer<T> {
+        T transform(Object key);
+    }
+
+    static class ToGlobalStreamIdTransformer implements KeyTransformer<GlobalStreamId> {
+        @Override
+        public GlobalStreamId transform(Object key) {
+            if (key instanceof List) {
+                List l = (List) key;
+                if (l.size() > 1) {
+                    return new GlobalStreamId((String) l.get(0), (String) l.get(1));
+                }
+            }
+            return new GlobalStreamId("", key.toString());
+        }
+    }
+
+    static class IdentityTransformer implements KeyTransformer<Object> {
+        @Override
+        public Object transform(Object key) {
+            return key;
+        }
+    }
+}


[3/7] storm git commit: STORM-3162: Cleanup heartbeats cache and make it thread safe

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/9d3feb0e/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
index 696cb2b..8f9f061 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -24,7 +24,7 @@
            [org.apache.storm.daemon.nimbus TopoCache Nimbus Nimbus$StandaloneINimbus]
            [org.apache.storm.generated GlobalStreamId TopologyStatus SupervisorInfo StormTopology StormBase]
            [org.apache.storm LocalCluster LocalCluster$Builder Thrift MockAutoCred Testing Testing$Condition]
-           [org.apache.storm.stats BoltExecutorStats StatsUtil]
+           [org.apache.storm.stats BoltExecutorStats StatsUtil ClientStatsUtil]
            [org.apache.storm.security.auth IGroupMappingServiceProvider IAuthorizer])
   (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
   (:import [org.apache.storm.testing TmpPath])
@@ -166,11 +166,11 @@
                 (HashMap.))]
     (log-warn "curr-beat:" (prn-str curr-beat) ",stats:" (prn-str stats))
     (log-warn "stats type:" (type stats))
-    (.put stats (StatsUtil/convertExecutor executor) (.renderStats (BoltExecutorStats. 20 (*STORM-CONF* NUM-STAT-BUCKETS))))
+    (.put stats (ClientStatsUtil/convertExecutor executor) (.renderStats (BoltExecutorStats. 20 (*STORM-CONF* NUM-STAT-BUCKETS))))
     (log-warn "merged:" stats)
 
     (.workerHeartbeat state storm-id node port
-      (StatsUtil/thriftifyZkWorkerHb (StatsUtil/mkZkWorkerHb storm-id stats (int 10))))
+      (ClientStatsUtil/thriftifyZkWorkerHb (ClientStatsUtil/mkZkWorkerHb storm-id stats (int 10))))
     (.sendSupervisorWorkerHeartbeat (.getNimbus cluster) (StatsUtil/thriftifyRPCWorkerHb storm-id executor))))
 
 (defn slot-assignments [cluster storm-id]
@@ -1876,14 +1876,14 @@
 
 (deftest do-cleanup-removes-inactive-znodes
   (let [inactive-topos (list "topo2" "topo3")
-        hb-cache (into {}(map vector inactive-topos '(nil nil)))
         mock-state (mock-cluster-state)
         mock-blob-store (Mockito/mock BlobStore)
         conf {NIMBUS-MONITOR-FREQ-SECS 10 NIMBUS-TOPOLOGY-BLOBSTORE-DELETION-DELAY-MS 0}]
     (with-open [_ (MockedZookeeper. (proxy [Zookeeper] []
                     (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls metrics-registry] (MockLeaderElector. ))))]
       (let [nimbus (Mockito/spy (Nimbus. conf nil mock-state nil mock-blob-store nil nil (StormMetricsRegistry.)))]
-        (.set (.getHeartbeatsCache nimbus) hb-cache)
+        (.addEmptyTopoForTests (.getHeartbeatsCache nimbus) "topo2")
+        (.addEmptyTopoForTests (.getHeartbeatsCache nimbus) "topo3")
         (.thenReturn (Mockito/when (.storedTopoIds mock-blob-store)) (HashSet. inactive-topos))
 
           (.doCleanup nimbus)
@@ -1909,19 +1909,19 @@
           (.rmDependencyJarsInTopology (Mockito/verify nimbus) "topo3")
 
           ;; remove topos from heartbeat cache
-          (is (= (count (.get (.getHeartbeatsCache nimbus))) 0))))))
+          (is (= (.getNumToposCached (.getHeartbeatsCache nimbus)) 0))))))
 
 
 (deftest do-cleanup-does-not-teardown-active-topos
   (let [inactive-topos ()
-        hb-cache {"topo1" nil "topo2" nil}
         mock-state (mock-cluster-state)
         mock-blob-store (Mockito/mock BlobStore)
         conf {NIMBUS-MONITOR-FREQ-SECS 10}]
     (with-open [_ (MockedZookeeper. (proxy [Zookeeper] []
                     (zkLeaderElectorImpl [conf zk blob-store tc cluster-state acls metrics-registry] (MockLeaderElector. ))))]
       (let [nimbus (Mockito/spy (Nimbus. conf nil mock-state nil mock-blob-store nil nil (StormMetricsRegistry.)))]
-        (.set (.getHeartbeatsCache nimbus) hb-cache)
+        (.addEmptyTopoForTests (.getHeartbeatsCache nimbus) "topo1")
+        (.addEmptyTopoForTests (.getHeartbeatsCache nimbus) "topo2")
         (.thenReturn (Mockito/when (.storedTopoIds mock-blob-store)) (set inactive-topos))
 
           (.doCleanup nimbus)
@@ -1932,9 +1932,9 @@
           (.rmTopologyKeys (Mockito/verify nimbus (Mockito/times 0)) (Mockito/anyObject))
 
           ;; hb-cache goes down to 1 because only one topo was inactive
-          (is (= (count (.get (.getHeartbeatsCache nimbus))) 2))
-          (is (contains? (.get (.getHeartbeatsCache nimbus)) "topo1"))
-          (is (contains? (.get (.getHeartbeatsCache nimbus)) "topo2"))))))
+          (is (= (.getNumToposCached (.getHeartbeatsCache nimbus)) 2))
+          (is (contains? (.getTopologyIds (.getHeartbeatsCache nimbus)) "topo1"))
+          (is (contains? (.getTopologyIds (.getHeartbeatsCache nimbus)) "topo2"))))))
 
 (deftest user-topologies-for-supervisor
   (let [assignment (doto (Assignment.)

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3feb0e/storm-server/pom.xml
----------------------------------------------------------------------
diff --git a/storm-server/pom.xml b/storm-server/pom.xml
index 43d2e19..b3d74b3 100644
--- a/storm-server/pom.xml
+++ b/storm-server/pom.xml
@@ -171,7 +171,7 @@
                 <artifactId>maven-checkstyle-plugin</artifactId>
                 <!--Note - the version would be inherited-->
                 <configuration>
-                    <maxAllowedViolations>780</maxAllowedViolations>
+                    <maxAllowedViolations>853</maxAllowedViolations>
                 </configuration>
             </plugin>
             <plugin>

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3feb0e/storm-server/src/main/java/org/apache/storm/daemon/nimbus/HeartbeatCache.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/nimbus/HeartbeatCache.java b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/HeartbeatCache.java
new file mode 100644
index 0000000..320f4fb
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/HeartbeatCache.java
@@ -0,0 +1,229 @@
+/*
+ * 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.
+ */
+
+package org.apache.storm.daemon.nimbus;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Function;
+import org.apache.storm.generated.Assignment;
+import org.apache.storm.generated.ExecutorInfo;
+import org.apache.storm.generated.SupervisorWorkerHeartbeat;
+import org.apache.storm.shade.com.google.common.annotations.VisibleForTesting;
+import org.apache.storm.stats.ClientStatsUtil;
+import org.apache.storm.stats.StatsUtil;
+import org.apache.storm.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Holds a cache of heartbeats from the workers.
+ */
+public class HeartbeatCache {
+    private static final Logger LOG = LoggerFactory.getLogger(HeartbeatCache.class);
+    private static final Function<String, ConcurrentHashMap<List<Integer>, ExecutorCache>> MAKE_MAP = (k) -> new ConcurrentHashMap<>();
+
+    private static class ExecutorCache {
+        private Boolean isTimedOut;
+        private Integer nimbusTime;
+        private Integer executorReportedTime;
+
+        public ExecutorCache(Map<String, Object> newBeat) {
+            if (newBeat != null) {
+                executorReportedTime = (Integer) newBeat.getOrDefault(ClientStatsUtil.TIME_SECS, 0);
+            } else {
+                executorReportedTime = 0;
+            }
+
+            nimbusTime = Time.currentTimeSecs();
+            isTimedOut = false;
+        }
+
+        public ExecutorCache(boolean isTimedOut, Integer nimbusTime, Integer executorReportedTime) {
+            this.isTimedOut = isTimedOut;
+            this.nimbusTime = nimbusTime;
+            this.executorReportedTime = executorReportedTime;
+        }
+
+        public synchronized Boolean isTimedOut() {
+            return isTimedOut;
+        }
+
+        public synchronized Integer getNimbusTime() {
+            return nimbusTime;
+        }
+
+        public synchronized Integer getExecutorReportedTime() {
+            return executorReportedTime;
+        }
+
+        public synchronized void updateTimeout(Integer timeout) {
+            isTimedOut = Time.deltaSecs(getNimbusTime()) >= timeout;
+        }
+
+        public synchronized void updateFromHb(Integer timeout, Map<String,Object> newBeat) {
+            if (newBeat != null) {
+                Integer newReportedTime = (Integer) newBeat.getOrDefault(ClientStatsUtil.TIME_SECS, 0);
+                if (!newReportedTime.equals(executorReportedTime)) {
+                    nimbusTime = Time.currentTimeSecs();
+                }
+                executorReportedTime = newReportedTime;
+            }
+            updateTimeout(timeout);
+        }
+    }
+
+    //Topology Id -> executor ids -> component -> stats(...)
+    private final ConcurrentHashMap<String, ConcurrentHashMap<List<Integer>, ExecutorCache>> cache;
+
+    /**
+     * Create an empty cache.
+     */
+    public HeartbeatCache() {
+        this.cache = new ConcurrentHashMap<>();
+    }
+
+    /**
+     * Add an empty topology to the cache for testing purposes.
+     * @param topoId the id of the topology to add.
+     */
+    @VisibleForTesting
+    public void addEmptyTopoForTests(String topoId) {
+        cache.put(topoId, new ConcurrentHashMap<>());
+    }
+
+    /**
+     * Get the number of topologies with cached heartbeats.
+     * @return the number of topologies with cached heartbeats.
+     */
+    @VisibleForTesting
+    public int getNumToposCached() {
+        return cache.size();
+    }
+
+    /**
+     * Get the topology ids with cached heartbeats.
+     * @return the set of topology ids with cached heartbeats.
+     */
+    @VisibleForTesting
+    public Set<String> getTopologyIds() {
+        return cache.keySet();
+    }
+
+    /**
+     * Remove a specific topology from the cache.
+     * @param topoId the id of the topology to remove.
+     */
+    public void removeTopo(String topoId) {
+        cache.remove(topoId);
+    }
+
+    /**
+     * Update the heartbeats for a topology with no heartbeats that came in.
+     * @param topoId the id of the topology to look at.
+     * @param taskTimeoutSecs the timeout to know if they are too old.
+     */
+    public void timeoutOldHeartbeats(String topoId, Integer taskTimeoutSecs) {
+        Map<List<Integer>, ExecutorCache> topoCache = cache.computeIfAbsent(topoId, MAKE_MAP);
+        //if not executor beats, refresh is-timed-out of the cache which is done by master
+        for (ExecutorCache ec : topoCache.values()) {
+            ec.updateTimeout(taskTimeoutSecs);
+        }
+    }
+
+    /**
+     * Update the cache with heartbeats from a worker through zookeeper.
+     * @param topoId the id to the topology.
+     * @param executorBeats the HB data.
+     * @param allExecutors the executors.
+     * @param timeout the timeout.
+     */
+    public void updateFromZkHeartbeat(String topoId, Map<List<Integer>, Map<String,Object>> executorBeats,
+                                      Set<List<Integer>> allExecutors, Integer timeout) {
+        Map<List<Integer>, ExecutorCache> topoCache = cache.computeIfAbsent(topoId, MAKE_MAP);
+        if (executorBeats == null) {
+            executorBeats = new HashMap<>();
+        }
+
+        for (List<Integer> executor : allExecutors) {
+            final Map<String, Object> newBeat = executorBeats.get(executor);
+            ExecutorCache currBeat = topoCache.computeIfAbsent(executor, (k) -> new ExecutorCache(newBeat));
+            currBeat.updateFromHb(timeout, newBeat);
+        }
+    }
+
+    /**
+     * Update the heartbeats for a given worker.
+     * @param workerHeartbeat the heartbeats from the worker.
+     * @param taskTimeoutSecs the timeout we should be looking at.
+     */
+    public void updateHeartbeat(SupervisorWorkerHeartbeat workerHeartbeat, Integer taskTimeoutSecs) {
+        Map<List<Integer>, Map<String, Object>> executorBeats = StatsUtil.convertWorkerBeats(workerHeartbeat);
+        String topoId = workerHeartbeat.get_storm_id();
+        Map<List<Integer>, ExecutorCache> topoCache = cache.computeIfAbsent(topoId, MAKE_MAP);
+
+        for (ExecutorInfo executorInfo : workerHeartbeat.get_executors()) {
+            List<Integer> executor = Arrays.asList(executorInfo.get_task_start(), executorInfo.get_task_end());
+            final Map<String, Object> newBeat = executorBeats.get(executor);
+            ExecutorCache currBeat = topoCache.computeIfAbsent(executor, (k) -> new ExecutorCache(newBeat));
+            currBeat.updateFromHb(taskTimeoutSecs, newBeat);
+        }
+    }
+
+    /**
+     * Get all of the alive executors for a given topology.
+     * @param topoId the id of the topology we are looking for.
+     * @param allExecutors all of the executors for this topology.
+     * @param assignment the current topology assignment.
+     * @param taskLaunchSecs timeout for right after a worker is launched.
+     * @return the set of tasks that are alive.
+     */
+    public Set<List<Integer>> getAliveExecutors(String topoId, Set<List<Integer>> allExecutors, Assignment assignment, int taskLaunchSecs) {
+        Map<List<Integer>, ExecutorCache> topoCache = cache.computeIfAbsent(topoId, MAKE_MAP);
+        LOG.debug("Computing alive executors for {}\nExecutors: {}\nAssignment: {}\nHeartbeat cache: {}",
+            topoId, allExecutors, assignment, topoCache);
+
+        Set<List<Integer>> ret = new HashSet<>();
+        Map<List<Long>, Long> execToStartTimes = assignment.get_executor_start_time_secs();
+
+        for (List<Integer> exec : allExecutors) {
+            List<Long> longExec = new ArrayList<>(exec.size());
+            for (Integer num : exec) {
+                longExec.add(num.longValue());
+            }
+
+            Long startTime = execToStartTimes.get(longExec);
+            ExecutorCache executorCache = topoCache.get(exec);
+            //null isTimedOut means worker never reported any heartbeat
+            Boolean isTimedOut = executorCache == null ? null : executorCache.isTimedOut();
+            Integer delta = startTime == null ? null : Time.deltaSecs(startTime.intValue());
+            if (startTime != null && ((delta < taskLaunchSecs) || (isTimedOut != null && !isTimedOut))) {
+                ret.add(exec);
+            } else {
+                LOG.info("Executor {}:{} not alive", topoId, exec);
+            }
+        }
+        return ret;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/9d3feb0e/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
index 5c48271..a33a4d0 100644
--- a/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
+++ b/storm-server/src/main/java/org/apache/storm/daemon/nimbus/Nimbus.java
@@ -190,6 +190,7 @@ import org.apache.storm.shade.com.google.common.collect.Maps;
 import org.apache.storm.shade.org.apache.curator.framework.CuratorFramework;
 import org.apache.storm.shade.org.apache.zookeeper.ZooDefs;
 import org.apache.storm.shade.org.apache.zookeeper.data.ACL;
+import org.apache.storm.stats.ClientStatsUtil;
 import org.apache.storm.stats.StatsUtil;
 import org.apache.storm.thrift.TException;
 import org.apache.storm.utils.BufferInputStream;
@@ -297,7 +298,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
         Assignment oldAssignment = state.assignmentInfo(topoId, null);
         state.removeStorm(topoId);
         notifySupervisorsAsKilled(state, oldAssignment, nimbus.getAssignmentsDistributer());
-        nimbus.getHeartbeatsCache().getAndUpdate(new Dissoc<>(topoId));
+        nimbus.heartbeatsCache.removeTopo(topoId);
         nimbus.getIdToExecutors().getAndUpdate(new Dissoc<>(topoId));
         return null;
     };
@@ -404,7 +405,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
     private final Object submitLock = new Object();
     private final Object schedLock = new Object();
     private final Object credUpdateLock = new Object();
-    private final AtomicReference<Map<String, Map<List<Integer>, Map<String, Object>>>> heartbeatsCache;
+    private final HeartbeatCache heartbeatsCache;
     private final AtomicBoolean heartbeatsReadyFlag;
     private final IWorkerHeartbeatsRecoveryStrategy heartbeatsRecoveryStrategy;
     @SuppressWarnings("deprecation")
@@ -541,7 +542,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
             stormClusterState = makeStormClusterState(conf);
         }
         this.stormClusterState = stormClusterState;
-        this.heartbeatsCache = new AtomicReference<>(new HashMap<>());
+        this.heartbeatsCache = new HeartbeatCache();
         this.heartbeatsReadyFlag = new AtomicBoolean(false);
         this.heartbeatsRecoveryStrategy = WorkerHeartbeatsRecoveryStrategyFactory.getStrategy(conf);
         this.downloaders = fileCacheMap(conf);
@@ -1464,7 +1465,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
     }
 
     @VisibleForTesting
-    public AtomicReference<Map<String, Map<List<Integer>, Map<String, Object>>>> getHeartbeatsCache() {
+    public HeartbeatCache getHeartbeatsCache() {
         return heartbeatsCache;
     }
 
@@ -1719,23 +1720,8 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
         IStormClusterState state = stormClusterState;
         Map<List<Integer>, Map<String, Object>> executorBeats =
             StatsUtil.convertExecutorBeats(state.executorBeats(topoId, existingAssignment.get_executor_node_port()));
-        Map<List<Integer>, Map<String, Object>> cache = StatsUtil.updateHeartbeatCacheFromZkHeartbeat(heartbeatsCache.get().get(topoId),
-                                                                                                      executorBeats, allExecutors,
-                                                                                                      ObjectReader.getInt(conf.get(
-                                                                                                          DaemonConfig
-                                                                                                              .NIMBUS_TASK_TIMEOUT_SECS)));
-        heartbeatsCache.getAndUpdate(new Assoc<>(topoId, cache));
-    }
-
-    private void updateHeartbeats(String topoId, Set<List<Integer>> allExecutors, Assignment existingAssignment) {
-        LOG.debug("Updating heartbeats for {} {}", topoId, allExecutors);
-        Map<List<Integer>, Map<String, Object>> cache = heartbeatsCache.get().get(topoId);
-        if (cache == null) {
-            cache = new HashMap<>();
-            heartbeatsCache.getAndUpdate(new Assoc<>(topoId, cache));
-        }
-        StatsUtil.updateHeartbeatCache(heartbeatsCache.get().get(topoId),
-                                       null, allExecutors, ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_TASK_TIMEOUT_SECS)));
+        heartbeatsCache.updateFromZkHeartbeat(topoId, executorBeats, allExecutors,
+            ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_TASK_TIMEOUT_SECS)));
     }
 
     /**
@@ -1751,27 +1737,14 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
             if (zkHeartbeatTopologies.contains(topoId)) {
                 updateHeartbeatsFromZkHeartbeat(topoId, topologyToExecutors.get(topoId), entry.getValue());
             } else {
-                updateHeartbeats(topoId, topologyToExecutors.get(topoId), entry.getValue());
+                LOG.debug("Timing out old heartbeats for {}", topoId);
+                heartbeatsCache.timeoutOldHeartbeats(topoId, ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_TASK_TIMEOUT_SECS)));
             }
         }
     }
 
     private void updateCachedHeartbeatsFromWorker(SupervisorWorkerHeartbeat workerHeartbeat) {
-        Map<List<Integer>, Map<String, Object>> executorBeats = StatsUtil.convertWorkerBeats(workerHeartbeat);
-        String topoId = workerHeartbeat.get_storm_id();
-        Map<List<Integer>, Map<String, Object>> cache = heartbeatsCache.get().get(topoId);
-        if (cache == null) {
-            cache = new HashMap<>();
-            heartbeatsCache.getAndUpdate(new Assoc<>(topoId, cache));
-        }
-        Set<List<Integer>> executors = new HashSet<>();
-        for (ExecutorInfo executorInfo : workerHeartbeat.get_executors()) {
-            executors.add(Arrays.asList(executorInfo.get_task_start(), executorInfo.get_task_end()));
-        }
-
-        StatsUtil.updateHeartbeatCache(heartbeatsCache.get().get(topoId), executorBeats, executors,
-                                       ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_TASK_TIMEOUT_SECS)));
-
+        heartbeatsCache.updateHeartbeat(workerHeartbeat, ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_TASK_TIMEOUT_SECS)));
     }
 
     private void updateCachedHeartbeatsFromSupervisor(SupervisorWorkerHeartbeats workerHeartbeats) {
@@ -1812,36 +1785,8 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
     }
 
     private Set<List<Integer>> aliveExecutors(String topoId, Set<List<Integer>> allExecutors, Assignment assignment) {
-        Map<List<Integer>, Map<String, Object>> hbCache = heartbeatsCache.get().get(topoId);
-        //in case that no workers report any heartbeats yet.
-        if (null == hbCache) {
-            hbCache = new HashMap<>();
-        }
-        LOG.debug("NEW  Computing alive executors for {}\nExecutors: {}\nAssignment: {}\nHeartbeat cache: {}",
-                  topoId, allExecutors, assignment, hbCache);
-
-        int taskLaunchSecs = ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_TASK_LAUNCH_SECS));
-        Set<List<Integer>> ret = new HashSet<>();
-        Map<List<Long>, Long> execToStartTimes = assignment.get_executor_start_time_secs();
-
-        for (List<Integer> exec : allExecutors) {
-            List<Long> longExec = new ArrayList<Long>(exec.size());
-            for (Integer num : exec) {
-                longExec.add(num.longValue());
-            }
-
-            Long startTime = execToStartTimes.get(longExec);
-            Map<String, Object> executorCache = hbCache.get(StatsUtil.convertExecutor(longExec));
-            //null isTimedOut means worker never reported any heartbeat
-            Boolean isTimedOut = executorCache == null ? null : (Boolean) executorCache.get("is-timed-out");
-            Integer delta = startTime == null ? null : Time.deltaSecs(startTime.intValue());
-            if (startTime != null && ((delta < taskLaunchSecs) || (isTimedOut != null && !isTimedOut))) {
-                ret.add(exec);
-            } else {
-                LOG.info("Executor {}:{} not alive", topoId, exec);
-            }
-        }
-        return ret;
+        return heartbeatsCache.getAliveExecutors(topoId, allExecutors, assignment,
+            ObjectReader.getInt(conf.get(DaemonConfig.NIMBUS_TASK_LAUNCH_SECS)));
     }
 
     private List<List<Integer>> computeExecutors(String topoId, StormBase base, Map<String, Object> topoConf,
@@ -2591,7 +2536,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
                 rmDependencyJarsInTopology(topoId);
                 forceDeleteTopoDistDir(topoId);
                 rmTopologyKeys(topoId);
-                heartbeatsCache.getAndUpdate(new Dissoc<>(topoId));
+                heartbeatsCache.removeTopo(topoId);
                 idToExecutors.getAndUpdate(new Dissoc<>(topoId));
             }
         }
@@ -3956,7 +3901,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
                     NodeInfo ni = entry.getValue();
                     ExecutorInfo execInfo = toExecInfo(entry.getKey());
                     Map<String, String> nodeToHost = common.assignment.get_node_host();
-                    Map<String, Object> heartbeat = common.beats.get(StatsUtil.convertExecutor(entry.getKey()));
+                    Map<String, Object> heartbeat = common.beats.get(ClientStatsUtil.convertExecutor(entry.getKey()));
                     if (heartbeat == null) {
                         heartbeat = Collections.emptyMap();
                     }
@@ -4675,7 +4620,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
         return true;
     }
 
-    private static final class Assoc<K, V> implements UnaryOperator<Map<K, V>> {
+    static final class Assoc<K, V> implements UnaryOperator<Map<K, V>> {
         private final K key;
         private final V value;
 
@@ -4694,7 +4639,7 @@ public class Nimbus implements Iface, Shutdownable, DaemonCommon {
 
     // Shutdownable methods
 
-    private static final class Dissoc<K, V> implements UnaryOperator<Map<K, V>> {
+    static final class Dissoc<K, V> implements UnaryOperator<Map<K, V>> {
         private final K key;
 
         public Dissoc(K key) {


[2/7] storm git commit: STORM-3162: Cleanup heartbeats cache and make it thread safe

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/9d3feb0e/storm-server/src/main/java/org/apache/storm/stats/StatsUtil.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/stats/StatsUtil.java b/storm-server/src/main/java/org/apache/storm/stats/StatsUtil.java
new file mode 100644
index 0000000..1e7d4fa
--- /dev/null
+++ b/storm-server/src/main/java/org/apache/storm/stats/StatsUtil.java
@@ -0,0 +1,2388 @@
+/*
+ * 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.
+ */
+
+package org.apache.storm.stats;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import org.apache.storm.cluster.ExecutorBeat;
+import org.apache.storm.cluster.IStormClusterState;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.BoltAggregateStats;
+import org.apache.storm.generated.BoltStats;
+import org.apache.storm.generated.ClusterWorkerHeartbeat;
+import org.apache.storm.generated.CommonAggregateStats;
+import org.apache.storm.generated.ComponentAggregateStats;
+import org.apache.storm.generated.ComponentPageInfo;
+import org.apache.storm.generated.ComponentType;
+import org.apache.storm.generated.ErrorInfo;
+import org.apache.storm.generated.ExecutorAggregateStats;
+import org.apache.storm.generated.ExecutorInfo;
+import org.apache.storm.generated.ExecutorSpecificStats;
+import org.apache.storm.generated.ExecutorStats;
+import org.apache.storm.generated.ExecutorSummary;
+import org.apache.storm.generated.GlobalStreamId;
+import org.apache.storm.generated.SpecificAggregateStats;
+import org.apache.storm.generated.SpoutAggregateStats;
+import org.apache.storm.generated.SpoutStats;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.generated.SupervisorWorkerHeartbeat;
+import org.apache.storm.generated.TopologyPageInfo;
+import org.apache.storm.generated.TopologyStats;
+import org.apache.storm.generated.WorkerResources;
+import org.apache.storm.generated.WorkerSummary;
+import org.apache.storm.scheduler.WorkerSlot;
+import org.apache.storm.shade.com.google.common.collect.Lists;
+import org.apache.storm.utils.Time;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("unchecked")
+public class StatsUtil {
+    public static final String TYPE = "type";
+    public static final int TEN_MIN_IN_SECONDS = 60 * 10;
+    public static final String TEN_MIN_IN_SECONDS_STR = TEN_MIN_IN_SECONDS + "";
+    private static final Logger LOG = LoggerFactory.getLogger(StatsUtil.class);
+    private static final String HOST = "host";
+    private static final String PORT = "port";
+    private static final String NUM_TASKS = "num-tasks";
+    private static final String NUM_EXECUTORS = "num-executors";
+    private static final String CAPACITY = "capacity";
+    private static final String STATS = "stats";
+    private static final String EXECUTOR_ID = "executor-id";
+    private static final String LAST_ERROR = "lastError";
+    private static final String RATE = "rate";
+    private static final String ACKED = "acked";
+    private static final String FAILED = "failed";
+    private static final String EXECUTED = "executed";
+    private static final String EMITTED = "emitted";
+    private static final String TRANSFERRED = "transferred";
+    private static final String EXEC_LATENCIES = "execute-latencies";
+    private static final String PROC_LATENCIES = "process-latencies";
+    private static final String COMP_LATENCIES = "complete-latencies";
+    private static final String EXEC_LATENCY = "execute-latency";
+    private static final String PROC_LATENCY = "process-latency";
+    private static final String COMP_LATENCY = "complete-latency";
+    private static final String EXEC_LAT_TOTAL = "executeLatencyTotal";
+    private static final String PROC_LAT_TOTAL = "processLatencyTotal";
+    private static final String COMP_LAT_TOTAL = "completeLatencyTotal";
+    private static final String WIN_TO_EMITTED = "window->emitted";
+    private static final String WIN_TO_ACKED = "window->acked";
+    private static final String WIN_TO_FAILED = "window->failed";
+    private static final String WIN_TO_EXECUTED = "window->executed";
+    private static final String WIN_TO_TRANSFERRED = "window->transferred";
+    private static final String WIN_TO_EXEC_LAT = "window->execute-latency";
+    private static final String WIN_TO_PROC_LAT = "window->process-latency";
+    private static final String WIN_TO_COMP_LAT = "window->complete-latency";
+    private static final String WIN_TO_COMP_LAT_WGT_AVG = "window->comp-lat-wgt-avg";
+    private static final String WIN_TO_EXEC_LAT_WGT_AVG = "window->exec-lat-wgt-avg";
+    private static final String WIN_TO_PROC_LAT_WGT_AVG = "window->proc-lat-wgt-avg";
+    private static final String BOLT_TO_STATS = "bolt-id->stats";
+    private static final String SPOUT_TO_STATS = "spout-id->stats";
+    private static final String SID_TO_OUT_STATS = "sid->output-stats";
+    private static final String CID_SID_TO_IN_STATS = "cid+sid->input-stats";
+    private static final String WORKERS_SET = "workers-set";
+    private static final ToStringTransformer TO_STRING = new ToStringTransformer();
+    private static final FromGlobalStreamIdTransformer FROM_GSID = new FromGlobalStreamIdTransformer();
+
+
+    // =====================================================================================
+    // aggregation stats methods
+    // =====================================================================================
+
+    /**
+     * Aggregates number executed, process latency, and execute latency across all streams.
+     *
+     * @param id2execAvg { global stream id -> exec avg value }, e.g., {["split" "default"] 0.44313}
+     * @param id2procAvg { global stream id -> proc avg value }
+     * @param id2numExec { global stream id -> executed }
+     */
+    public static Map<String, Number> aggBoltLatAndCount(Map<List<String>, Double> id2execAvg,
+                                                         Map<List<String>, Double> id2procAvg,
+                                                         Map<List<String>, Long> id2numExec) {
+        Map<String, Number> ret = new HashMap<>();
+        putKV(ret, EXEC_LAT_TOTAL, weightAvgAndSum(id2execAvg, id2numExec));
+        putKV(ret, PROC_LAT_TOTAL, weightAvgAndSum(id2procAvg, id2numExec));
+        putKV(ret, EXECUTED, sumValues(id2numExec));
+
+        return ret;
+    }
+
+    /**
+     * aggregate number acked and complete latencies across all streams.
+     */
+    public static Map<String, Number> aggSpoutLatAndCount(Map<String, Double> id2compAvg,
+                                                          Map<String, Long> id2numAcked) {
+        Map<String, Number> ret = new HashMap<>();
+        putKV(ret, COMP_LAT_TOTAL, weightAvgAndSum(id2compAvg, id2numAcked));
+        putKV(ret, ACKED, sumValues(id2numAcked));
+
+        return ret;
+    }
+
+    /**
+     * aggregate number executed and process & execute latencies.
+     */
+    public static <K> Map<K, Map> aggBoltStreamsLatAndCount(Map<K, Double> id2execAvg,
+                                                            Map<K, Double> id2procAvg,
+                                                            Map<K, Long> id2numExec) {
+        Map<K, Map> ret = new HashMap<>();
+        if (id2execAvg == null || id2procAvg == null || id2numExec == null) {
+            return ret;
+        }
+        for (K k : id2execAvg.keySet()) {
+            Map<String, Object> subMap = new HashMap<>();
+            putKV(subMap, EXEC_LAT_TOTAL, weightAvg(id2execAvg, id2numExec, k));
+            putKV(subMap, PROC_LAT_TOTAL, weightAvg(id2procAvg, id2numExec, k));
+            putKV(subMap, EXECUTED, id2numExec.get(k));
+            ret.put(k, subMap);
+        }
+        return ret;
+    }
+
+    /**
+     * Aggregates number acked and complete latencies.
+     */
+    public static <K> Map<K, Map> aggSpoutStreamsLatAndCount(Map<K, Double> id2compAvg,
+                                                             Map<K, Long> id2acked) {
+        Map<K, Map> ret = new HashMap<>();
+        if (id2compAvg == null || id2acked == null) {
+            return ret;
+        }
+        for (K k : id2compAvg.keySet()) {
+            Map subMap = new HashMap();
+            putKV(subMap, COMP_LAT_TOTAL, weightAvg(id2compAvg, id2acked, k));
+            putKV(subMap, ACKED, id2acked.get(k));
+            ret.put(k, subMap);
+        }
+        return ret;
+    }
+
+    /**
+     * pre-merge component page bolt stats from an executor heartbeat 1. computes component capacity 2. converts map keys of stats 3.
+     * filters streams if necessary
+     *
+     * @param beat       executor heartbeat data
+     * @param window     specified window
+     * @param includeSys whether to include system streams
+     * @return per-merged stats
+     */
+    public static Map<String, Object> aggPreMergeCompPageBolt(Map<String, Object> beat, String window, boolean includeSys) {
+        Map<String, Object> ret = new HashMap<>();
+
+        putKV(ret, EXECUTOR_ID, beat.get("exec-id"));
+        putKV(ret, HOST, beat.get(HOST));
+        putKV(ret, PORT, beat.get(PORT));
+        putKV(ret, ClientStatsUtil.UPTIME, beat.get(ClientStatsUtil.UPTIME));
+        putKV(ret, NUM_EXECUTORS, 1);
+        putKV(ret, NUM_TASKS, beat.get(NUM_TASKS));
+
+        Map stat2win2sid2num = ClientStatsUtil.getMapByKey(beat, STATS);
+        putKV(ret, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeyOr0(beat, ClientStatsUtil.UPTIME).intValue()));
+
+        // calc cid+sid->input_stats
+        Map inputStats = new HashMap();
+        Map sid2acked = (Map) windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, ACKED), TO_STRING).get(window);
+        Map sid2failed = (Map) windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, FAILED), TO_STRING).get(window);
+        putKV(inputStats, ACKED, sid2acked != null ? sid2acked : new HashMap());
+        putKV(inputStats, FAILED, sid2failed != null ? sid2failed : new HashMap());
+
+        inputStats = swapMapOrder(inputStats);
+
+        Map sid2execLat = (Map) windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, EXEC_LATENCIES), TO_STRING).get(window);
+        Map sid2procLat = (Map) windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, PROC_LATENCIES), TO_STRING).get(window);
+        Map sid2exec = (Map) windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, EXECUTED), TO_STRING).get(window);
+        mergeMaps(inputStats, aggBoltStreamsLatAndCount(sid2execLat, sid2procLat, sid2exec));
+        putKV(ret, CID_SID_TO_IN_STATS, inputStats);
+
+        // calc sid->output_stats
+        Map outputStats = new HashMap();
+        Map sid2emitted = (Map) windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, EMITTED), TO_STRING).get(window);
+        Map sid2transferred = (Map) windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, TRANSFERRED), TO_STRING).get(window);
+        if (sid2emitted != null) {
+            putKV(outputStats, EMITTED, filterSysStreams2Stat(sid2emitted, includeSys));
+        } else {
+            putKV(outputStats, EMITTED, new HashMap());
+        }
+        if (sid2transferred != null) {
+            putKV(outputStats, TRANSFERRED, filterSysStreams2Stat(sid2transferred, includeSys));
+        } else {
+            putKV(outputStats, TRANSFERRED, new HashMap());
+        }
+        outputStats = swapMapOrder(outputStats);
+        putKV(ret, SID_TO_OUT_STATS, outputStats);
+
+        return ret;
+    }
+
+    /**
+     * pre-merge component page spout stats from an executor heartbeat 1. computes component capacity 2. converts map keys of stats 3.
+     * filters streams if necessary
+     *
+     * @param beat       executor heartbeat data
+     * @param window     specified window
+     * @param includeSys whether to include system streams
+     * @return per-merged stats
+     */
+    public static Map<String, Object> aggPreMergeCompPageSpout(Map<String, Object> beat, String window, boolean includeSys) {
+        Map<String, Object> ret = new HashMap<>();
+        putKV(ret, EXECUTOR_ID, beat.get("exec-id"));
+        putKV(ret, HOST, beat.get(HOST));
+        putKV(ret, PORT, beat.get(PORT));
+        putKV(ret, ClientStatsUtil.UPTIME, beat.get(ClientStatsUtil.UPTIME));
+        putKV(ret, NUM_EXECUTORS, 1);
+        putKV(ret, NUM_TASKS, beat.get(NUM_TASKS));
+
+        Map stat2win2sid2num = ClientStatsUtil.getMapByKey(beat, STATS);
+
+        // calc sid->output-stats
+        Map outputStats = new HashMap();
+        Map win2sid2acked = windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, ACKED), TO_STRING);
+        Map win2sid2failed = windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, FAILED), TO_STRING);
+        Map win2sid2emitted = windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, EMITTED), TO_STRING);
+        Map win2sid2transferred = windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, TRANSFERRED), TO_STRING);
+        Map win2sid2compLat = windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, COMP_LATENCIES), TO_STRING);
+
+        putKV(outputStats, ACKED, win2sid2acked.get(window));
+        putKV(outputStats, FAILED, win2sid2failed.get(window));
+        Map<String, Long> sid2emitted = (Map) win2sid2emitted.get(window);
+        if (sid2emitted == null) {
+            sid2emitted = new HashMap<>();
+        }
+        putKV(outputStats, EMITTED, filterSysStreams2Stat(sid2emitted, includeSys));
+
+        Map<String, Long> sid2transferred = (Map) win2sid2transferred.get(window);
+        if (sid2transferred == null) {
+            sid2transferred = new HashMap<>();
+        }
+        putKV(outputStats, TRANSFERRED, filterSysStreams2Stat(sid2transferred, includeSys));
+        outputStats = swapMapOrder(outputStats);
+
+        Map sid2compLat = (Map) win2sid2compLat.get(window);
+        Map sid2acked = (Map) win2sid2acked.get(window);
+        mergeMaps(outputStats, aggSpoutStreamsLatAndCount(sid2compLat, sid2acked));
+        putKV(ret, SID_TO_OUT_STATS, outputStats);
+
+        return ret;
+    }
+
+    /**
+     * pre-merge component stats of specified bolt id
+     *
+     * @param beat       executor heartbeat data
+     * @param window     specified window
+     * @param includeSys whether to include system streams
+     * @return { comp id -> comp-stats }
+     */
+    public static <K, V extends Number> Map<String, Object> aggPreMergeTopoPageBolt(
+        Map<String, Object> beat, String window, boolean includeSys) {
+        Map<String, Object> ret = new HashMap<>();
+
+        Map<String, Object> subRet = new HashMap<>();
+        putKV(subRet, NUM_EXECUTORS, 1);
+        putKV(subRet, NUM_TASKS, beat.get(NUM_TASKS));
+
+        Map<String, Object> stat2win2sid2num = ClientStatsUtil.getMapByKey(beat, STATS);
+        putKV(subRet, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeyOr0(beat, ClientStatsUtil.UPTIME).intValue()));
+
+        for (String key : new String[]{ EMITTED, TRANSFERRED, ACKED, FAILED }) {
+            Map<String, Map<K, V>> stat = windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, key), TO_STRING);
+            if (EMITTED.equals(key) || TRANSFERRED.equals(key)) {
+                stat = filterSysStreams(stat, includeSys);
+            }
+            Map<K, V> winStat = stat.get(window);
+            long sum = 0;
+            if (winStat != null) {
+                for (V v : winStat.values()) {
+                    sum += v.longValue();
+                }
+            }
+            putKV(subRet, key, sum);
+        }
+
+        Map<String, Map<List<String>, Double>> win2sid2execLat =
+            windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, EXEC_LATENCIES), TO_STRING);
+        Map<String, Map<List<String>, Double>> win2sid2procLat =
+            windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, PROC_LATENCIES), TO_STRING);
+        Map<String, Map<List<String>, Long>> win2sid2exec =
+            windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, EXECUTED), TO_STRING);
+        subRet.putAll(aggBoltLatAndCount(
+            win2sid2execLat.get(window), win2sid2procLat.get(window), win2sid2exec.get(window)));
+
+        ret.put((String) beat.get("comp-id"), subRet);
+        return ret;
+    }
+
+    /**
+     * pre-merge component stats of specified spout id and returns { comp id -> comp-stats }
+     */
+    public static <K, V extends Number> Map<String, Object> aggPreMergeTopoPageSpout(
+        Map<String, Object> m, String window, boolean includeSys) {
+        Map<String, Object> ret = new HashMap<>();
+
+        Map<String, Object> subRet = new HashMap<>();
+        putKV(subRet, NUM_EXECUTORS, 1);
+        putKV(subRet, NUM_TASKS, m.get(NUM_TASKS));
+
+        // no capacity for spout
+        Map<String, Map<String, Map<String, V>>> stat2win2sid2num = ClientStatsUtil.getMapByKey(m, STATS);
+        for (String key : new String[]{ EMITTED, TRANSFERRED, FAILED }) {
+            Map<String, Map<K, V>> stat = windowSetConverter(stat2win2sid2num.get(key), TO_STRING);
+            if (EMITTED.equals(key) || TRANSFERRED.equals(key)) {
+                stat = filterSysStreams(stat, includeSys);
+            }
+            Map<K, V> winStat = stat.get(window);
+            long sum = 0;
+            if (winStat != null) {
+                for (V v : winStat.values()) {
+                    sum += v.longValue();
+                }
+            }
+            putKV(subRet, key, sum);
+        }
+
+        Map<String, Map<String, Double>> win2sid2compLat =
+            windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, COMP_LATENCIES), TO_STRING);
+        Map<String, Map<String, Long>> win2sid2acked =
+            windowSetConverter(ClientStatsUtil.getMapByKey(stat2win2sid2num, ACKED), TO_STRING);
+        subRet.putAll(aggSpoutLatAndCount(win2sid2compLat.get(window), win2sid2acked.get(window)));
+
+        ret.put((String) m.get("comp-id"), subRet);
+        return ret;
+    }
+
+    /**
+     * merge accumulated bolt stats with pre-merged component stats
+     *
+     * @param accBoltStats accumulated bolt stats
+     * @param boltStats    pre-merged component stats
+     * @return merged stats
+     */
+    public static Map<String, Object> mergeAggCompStatsCompPageBolt(
+        Map<String, Object> accBoltStats, Map<String, Object> boltStats) {
+        Map<String, Object> ret = new HashMap<>();
+
+        Map<List<String>, Map<String, ?>> accIn = ClientStatsUtil.getMapByKey(accBoltStats, CID_SID_TO_IN_STATS);
+        Map<String, Map<String, ?>> accOut = ClientStatsUtil.getMapByKey(accBoltStats, SID_TO_OUT_STATS);
+        Map<List<String>, Map<String, ?>> boltIn = ClientStatsUtil.getMapByKey(boltStats, CID_SID_TO_IN_STATS);
+        Map<String, Map<String, ?>> boltOut = ClientStatsUtil.getMapByKey(boltStats, SID_TO_OUT_STATS);
+
+        int numExecutors = getByKeyOr0(accBoltStats, NUM_EXECUTORS).intValue();
+        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
+        putKV(ret, NUM_TASKS, sumOr0(
+            getByKeyOr0(accBoltStats, NUM_TASKS), getByKeyOr0(boltStats, NUM_TASKS)));
+
+        // (merge-with (partial merge-with sum-or-0) acc-out spout-out)
+        putKV(ret, SID_TO_OUT_STATS, fullMergeWithSum(accOut, boltOut));
+        // {component id -> metric -> value}, note that input may contain both long and double values
+        putKV(ret, CID_SID_TO_IN_STATS, fullMergeWithSum(accIn, boltIn));
+
+        long executed = sumStreamsLong(boltIn, EXECUTED);
+        putKV(ret, EXECUTED, executed);
+
+        Map<String, Object> executorStats = new HashMap<>();
+        putKV(executorStats, EXECUTOR_ID, boltStats.get(EXECUTOR_ID));
+        putKV(executorStats, ClientStatsUtil.UPTIME, boltStats.get(ClientStatsUtil.UPTIME));
+        putKV(executorStats, HOST, boltStats.get(HOST));
+        putKV(executorStats, PORT, boltStats.get(PORT));
+        putKV(executorStats, CAPACITY, boltStats.get(CAPACITY));
+
+        putKV(executorStats, EMITTED, sumStreamsLong(boltOut, EMITTED));
+        putKV(executorStats, TRANSFERRED, sumStreamsLong(boltOut, TRANSFERRED));
+        putKV(executorStats, ACKED, sumStreamsLong(boltIn, ACKED));
+        putKV(executorStats, FAILED, sumStreamsLong(boltIn, FAILED));
+        putKV(executorStats, EXECUTED, executed);
+
+        if (executed > 0) {
+            putKV(executorStats, EXEC_LATENCY, sumStreamsDouble(boltIn, EXEC_LAT_TOTAL) / executed);
+            putKV(executorStats, PROC_LATENCY, sumStreamsDouble(boltIn, PROC_LAT_TOTAL) / executed);
+        } else {
+            putKV(executorStats, EXEC_LATENCY, null);
+            putKV(executorStats, PROC_LATENCY, null);
+        }
+        List executorStatsList = ((List) accBoltStats.get(ClientStatsUtil.EXECUTOR_STATS));
+        executorStatsList.add(executorStats);
+        putKV(ret, ClientStatsUtil.EXECUTOR_STATS, executorStatsList);
+
+        return ret;
+    }
+
+    /**
+     * merge accumulated bolt stats with pre-merged component stats
+     */
+    public static Map<String, Object> mergeAggCompStatsCompPageSpout(
+        Map<String, Object> accSpoutStats, Map<String, Object> spoutStats) {
+        Map<String, Object> ret = new HashMap<>();
+
+        // {stream id -> metric -> value}, note that sid->out-stats may contain both long and double values
+        Map<String, Map<String, ?>> accOut = ClientStatsUtil.getMapByKey(accSpoutStats, SID_TO_OUT_STATS);
+        Map<String, Map<String, ?>> spoutOut = ClientStatsUtil.getMapByKey(spoutStats, SID_TO_OUT_STATS);
+
+        int numExecutors = getByKeyOr0(accSpoutStats, NUM_EXECUTORS).intValue();
+        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
+        putKV(ret, NUM_TASKS, sumOr0(
+            getByKeyOr0(accSpoutStats, NUM_TASKS), getByKeyOr0(spoutStats, NUM_TASKS)));
+        putKV(ret, SID_TO_OUT_STATS, fullMergeWithSum(accOut, spoutOut));
+
+        Map executorStats = new HashMap();
+        putKV(executorStats, EXECUTOR_ID, spoutStats.get(EXECUTOR_ID));
+        putKV(executorStats, ClientStatsUtil.UPTIME, spoutStats.get(ClientStatsUtil.UPTIME));
+        putKV(executorStats, HOST, spoutStats.get(HOST));
+        putKV(executorStats, PORT, spoutStats.get(PORT));
+
+        putKV(executorStats, EMITTED, sumStreamsLong(spoutOut, EMITTED));
+        putKV(executorStats, TRANSFERRED, sumStreamsLong(spoutOut, TRANSFERRED));
+        putKV(executorStats, FAILED, sumStreamsLong(spoutOut, FAILED));
+        long acked = sumStreamsLong(spoutOut, ACKED);
+        putKV(executorStats, ACKED, acked);
+        if (acked > 0) {
+            putKV(executorStats, COMP_LATENCY, sumStreamsDouble(spoutOut, COMP_LAT_TOTAL) / acked);
+        } else {
+            putKV(executorStats, COMP_LATENCY, null);
+        }
+        List executorStatsList = ((List) accSpoutStats.get(ClientStatsUtil.EXECUTOR_STATS));
+        executorStatsList.add(executorStats);
+        putKV(ret, ClientStatsUtil.EXECUTOR_STATS, executorStatsList);
+
+        return ret;
+    }
+
+    /**
+     * merge accumulated bolt stats with new bolt stats
+     *
+     * @param accBoltStats accumulated bolt stats
+     * @param boltStats    new input bolt stats
+     * @return merged bolt stats
+     */
+    public static Map<String, Object> mergeAggCompStatsTopoPageBolt(Map<String, Object> accBoltStats,
+                                                                    Map<String, Object> boltStats) {
+        Map<String, Object> ret = new HashMap<>();
+
+        Integer numExecutors = getByKeyOr0(accBoltStats, NUM_EXECUTORS).intValue();
+        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
+        putKV(ret, NUM_TASKS,
+              sumOr0(getByKeyOr0(accBoltStats, NUM_TASKS), getByKeyOr0(boltStats, NUM_TASKS)));
+        putKV(ret, EMITTED,
+              sumOr0(getByKeyOr0(accBoltStats, EMITTED), getByKeyOr0(boltStats, EMITTED)));
+        putKV(ret, TRANSFERRED,
+              sumOr0(getByKeyOr0(accBoltStats, TRANSFERRED), getByKeyOr0(boltStats, TRANSFERRED)));
+        putKV(ret, EXEC_LAT_TOTAL,
+              sumOr0(getByKeyOr0(accBoltStats, EXEC_LAT_TOTAL), getByKeyOr0(boltStats, EXEC_LAT_TOTAL)));
+        putKV(ret, PROC_LAT_TOTAL,
+              sumOr0(getByKeyOr0(accBoltStats, PROC_LAT_TOTAL), getByKeyOr0(boltStats, PROC_LAT_TOTAL)));
+        putKV(ret, EXECUTED,
+              sumOr0(getByKeyOr0(accBoltStats, EXECUTED), getByKeyOr0(boltStats, EXECUTED)));
+        putKV(ret, ACKED,
+              sumOr0(getByKeyOr0(accBoltStats, ACKED), getByKeyOr0(boltStats, ACKED)));
+        putKV(ret, FAILED,
+              sumOr0(getByKeyOr0(accBoltStats, FAILED), getByKeyOr0(boltStats, FAILED)));
+        putKV(ret, CAPACITY,
+              maxOr0(getByKeyOr0(accBoltStats, CAPACITY), getByKeyOr0(boltStats, CAPACITY)));
+
+        return ret;
+    }
+
+    /**
+     * merge accumulated bolt stats with new bolt stats
+     */
+    public static Map<String, Object> mergeAggCompStatsTopoPageSpout(Map<String, Object> accSpoutStats,
+                                                                     Map<String, Object> spoutStats) {
+        Map<String, Object> ret = new HashMap<>();
+
+        Integer numExecutors = getByKeyOr0(accSpoutStats, NUM_EXECUTORS).intValue();
+        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
+        putKV(ret, NUM_TASKS,
+              sumOr0(getByKeyOr0(accSpoutStats, NUM_TASKS), getByKeyOr0(spoutStats, NUM_TASKS)));
+        putKV(ret, EMITTED,
+              sumOr0(getByKeyOr0(accSpoutStats, EMITTED), getByKeyOr0(spoutStats, EMITTED)));
+        putKV(ret, TRANSFERRED,
+              sumOr0(getByKeyOr0(accSpoutStats, TRANSFERRED), getByKeyOr0(spoutStats, TRANSFERRED)));
+        putKV(ret, COMP_LAT_TOTAL,
+              sumOr0(getByKeyOr0(accSpoutStats, COMP_LAT_TOTAL), getByKeyOr0(spoutStats, COMP_LAT_TOTAL)));
+        putKV(ret, ACKED,
+              sumOr0(getByKeyOr0(accSpoutStats, ACKED), getByKeyOr0(spoutStats, ACKED)));
+        putKV(ret, FAILED,
+              sumOr0(getByKeyOr0(accSpoutStats, FAILED), getByKeyOr0(spoutStats, FAILED)));
+
+        return ret;
+    }
+
+    /**
+     * A helper function that does the common work to aggregate stats of one executor with the given map for the topology page.
+     */
+    public static Map<String, Object> aggTopoExecStats(
+        String window, boolean includeSys, Map<String, Object> accStats, Map<String, Object> beat, String compType) {
+        Map<String, Object> ret = new HashMap<>();
+
+        Set workerSet = (Set) accStats.get(WORKERS_SET);
+        Map bolt2stats = ClientStatsUtil.getMapByKey(accStats, BOLT_TO_STATS);
+        Map spout2stats = ClientStatsUtil.getMapByKey(accStats, SPOUT_TO_STATS);
+        Map win2emitted = ClientStatsUtil.getMapByKey(accStats, WIN_TO_EMITTED);
+        Map win2transferred = ClientStatsUtil.getMapByKey(accStats, WIN_TO_TRANSFERRED);
+        Map win2compLatWgtAvg = ClientStatsUtil.getMapByKey(accStats, WIN_TO_COMP_LAT_WGT_AVG);
+        Map win2acked = ClientStatsUtil.getMapByKey(accStats, WIN_TO_ACKED);
+        Map win2failed = ClientStatsUtil.getMapByKey(accStats, WIN_TO_FAILED);
+
+        boolean isSpout = compType.equals(ClientStatsUtil.SPOUT);
+        // component id -> stats
+        Map<String, Object> cid2stats;
+        if (isSpout) {
+            cid2stats = aggPreMergeTopoPageSpout(beat, window, includeSys);
+        } else {
+            cid2stats = aggPreMergeTopoPageBolt(beat, window, includeSys);
+        }
+
+        Map stats = ClientStatsUtil.getMapByKey(beat, STATS);
+        Map w2compLatWgtAvg, w2acked;
+        Map compLatStats = ClientStatsUtil.getMapByKey(stats, COMP_LATENCIES);
+        if (isSpout) { // agg spout stats
+            Map mm = new HashMap();
+
+            Map acked = ClientStatsUtil.getMapByKey(stats, ACKED);
+            for (Object win : acked.keySet()) {
+                mm.put(win, aggSpoutLatAndCount((Map) compLatStats.get(win), (Map) acked.get(win)));
+            }
+            mm = swapMapOrder(mm);
+            w2compLatWgtAvg = ClientStatsUtil.getMapByKey(mm, COMP_LAT_TOTAL);
+            w2acked = ClientStatsUtil.getMapByKey(mm, ACKED);
+        } else {
+            w2compLatWgtAvg = null;
+            w2acked = aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, ACKED));
+        }
+
+        workerSet.add(Lists.newArrayList(beat.get(HOST), beat.get(PORT)));
+        putKV(ret, WORKERS_SET, workerSet);
+        putKV(ret, BOLT_TO_STATS, bolt2stats);
+        putKV(ret, SPOUT_TO_STATS, spout2stats);
+        putKV(ret, WIN_TO_EMITTED, mergeWithSumLong(win2emitted, aggregateCountStreams(
+            filterSysStreams(ClientStatsUtil.getMapByKey(stats, EMITTED), includeSys))));
+        putKV(ret, WIN_TO_TRANSFERRED, mergeWithSumLong(win2transferred, aggregateCountStreams(
+            filterSysStreams(ClientStatsUtil.getMapByKey(stats, TRANSFERRED), includeSys))));
+        putKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSumDouble(win2compLatWgtAvg, w2compLatWgtAvg));
+
+        //boolean isSpoutStat = SPOUT.equals(((Keyword) getByKey(stats, TYPE)).getName());
+        putKV(ret, WIN_TO_ACKED, isSpout ? mergeWithSumLong(win2acked, w2acked) : win2acked);
+        putKV(ret, WIN_TO_FAILED, isSpout ?
+            mergeWithSumLong(aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, FAILED)), win2failed) : win2failed);
+        putKV(ret, TYPE, stats.get(TYPE));
+
+        // (merge-with merge-agg-comp-stats-topo-page-bolt/spout (acc-stats comp-key) cid->statk->num)
+        // (acc-stats comp-key) ==> bolt2stats/spout2stats
+        if (isSpout) {
+            for (String spout : cid2stats.keySet()) {
+                spout2stats.put(spout, mergeAggCompStatsTopoPageSpout((Map) spout2stats.get(spout), (Map) cid2stats.get(spout)));
+            }
+        } else {
+            for (String bolt : cid2stats.keySet()) {
+                bolt2stats.put(bolt, mergeAggCompStatsTopoPageBolt((Map) bolt2stats.get(bolt), (Map) cid2stats.get(bolt)));
+            }
+        }
+
+        return ret;
+    }
+
+    /**
+     * aggregate topo executors stats
+     *
+     * @param topologyId     topology id
+     * @param exec2nodePort  executor -> host+port
+     * @param task2component task -> component
+     * @param beats          executor[start, end] -> executor heartbeat
+     * @param topology       storm topology
+     * @param window         the window to be aggregated
+     * @param includeSys     whether to include system streams
+     * @param clusterState   cluster state
+     * @return TopologyPageInfo thrift structure
+     */
+    public static TopologyPageInfo aggTopoExecsStats(
+        String topologyId, Map exec2nodePort, Map task2component, Map<List<Integer>, Map<String, Object>> beats,
+        StormTopology topology, String window, boolean includeSys, IStormClusterState clusterState) {
+        List<Map<String, Object>> beatList = extractDataFromHb(exec2nodePort, task2component, beats, includeSys, topology);
+        Map<String, Object> topoStats = aggregateTopoStats(window, includeSys, beatList);
+        return postAggregateTopoStats(task2component, exec2nodePort, topoStats, topologyId, clusterState);
+    }
+
+    public static Map<String, Object> aggregateTopoStats(String win, boolean includeSys, List<Map<String, Object>> heartbeats) {
+        Map<String, Object> initVal = new HashMap<>();
+        putKV(initVal, WORKERS_SET, new HashSet());
+        putKV(initVal, BOLT_TO_STATS, new HashMap());
+        putKV(initVal, SPOUT_TO_STATS, new HashMap());
+        putKV(initVal, WIN_TO_EMITTED, new HashMap());
+        putKV(initVal, WIN_TO_TRANSFERRED, new HashMap());
+        putKV(initVal, WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
+        putKV(initVal, WIN_TO_ACKED, new HashMap());
+        putKV(initVal, WIN_TO_FAILED, new HashMap());
+
+        for (Map<String, Object> heartbeat : heartbeats) {
+            String compType = (String) heartbeat.get(TYPE);
+            initVal = aggTopoExecStats(win, includeSys, initVal, heartbeat, compType);
+        }
+
+        return initVal;
+    }
+
+    public static TopologyPageInfo postAggregateTopoStats(Map task2comp, Map exec2nodePort, Map<String, Object> accData,
+                                                          String topologyId, IStormClusterState clusterState) {
+        TopologyPageInfo ret = new TopologyPageInfo(topologyId);
+
+        ret.set_num_tasks(task2comp.size());
+        ret.set_num_workers(((Set) accData.get(WORKERS_SET)).size());
+        ret.set_num_executors(exec2nodePort != null ? exec2nodePort.size() : 0);
+
+        Map bolt2stats = ClientStatsUtil.getMapByKey(accData, BOLT_TO_STATS);
+        Map<String, ComponentAggregateStats> aggBolt2stats = new HashMap<>();
+        for (Object o : bolt2stats.entrySet()) {
+            Map.Entry e = (Map.Entry) o;
+            String id = (String) e.getKey();
+            Map m = (Map) e.getValue();
+            long executed = getByKeyOr0(m, EXECUTED).longValue();
+            if (executed > 0) {
+                double execLatencyTotal = getByKeyOr0(m, EXEC_LAT_TOTAL).doubleValue();
+                putKV(m, EXEC_LATENCY, execLatencyTotal / executed);
+
+                double procLatencyTotal = getByKeyOr0(m, PROC_LAT_TOTAL).doubleValue();
+                putKV(m, PROC_LATENCY, procLatencyTotal / executed);
+            }
+            remove(m, EXEC_LAT_TOTAL);
+            remove(m, PROC_LAT_TOTAL);
+            putKV(m, "last-error", getLastError(clusterState, topologyId, id));
+
+            aggBolt2stats.put(id, thriftifyBoltAggStats(m));
+        }
+
+        Map spout2stats = ClientStatsUtil.getMapByKey(accData, SPOUT_TO_STATS);
+        Map<String, ComponentAggregateStats> aggSpout2stats = new HashMap<>();
+        for (Object o : spout2stats.entrySet()) {
+            Map.Entry e = (Map.Entry) o;
+            String id = (String) e.getKey();
+            Map m = (Map) e.getValue();
+            long acked = getByKeyOr0(m, ACKED).longValue();
+            if (acked > 0) {
+                double compLatencyTotal = getByKeyOr0(m, COMP_LAT_TOTAL).doubleValue();
+                putKV(m, COMP_LATENCY, compLatencyTotal / acked);
+            }
+            remove(m, COMP_LAT_TOTAL);
+            putKV(m, "last-error", getLastError(clusterState, topologyId, id));
+
+            aggSpout2stats.put(id, thriftifySpoutAggStats(m));
+        }
+
+        TopologyStats topologyStats = new TopologyStats();
+        topologyStats.set_window_to_acked(mapKeyStr(ClientStatsUtil.getMapByKey(accData, WIN_TO_ACKED)));
+        topologyStats.set_window_to_emitted(mapKeyStr(ClientStatsUtil.getMapByKey(accData, WIN_TO_EMITTED)));
+        topologyStats.set_window_to_failed(mapKeyStr(ClientStatsUtil.getMapByKey(accData, WIN_TO_FAILED)));
+        topologyStats.set_window_to_transferred(mapKeyStr(ClientStatsUtil.getMapByKey(accData, WIN_TO_TRANSFERRED)));
+        topologyStats.set_window_to_complete_latencies_ms(computeWeightedAveragesPerWindow(
+            accData, WIN_TO_COMP_LAT_WGT_AVG, WIN_TO_ACKED));
+
+        ret.set_topology_stats(topologyStats);
+        ret.set_id_to_spout_agg_stats(aggSpout2stats);
+        ret.set_id_to_bolt_agg_stats(aggBolt2stats);
+
+        return ret;
+    }
+
+    /**
+     * aggregate bolt stats
+     *
+     * @param statsSeq   a seq of ExecutorStats
+     * @param includeSys whether to include system streams
+     * @return aggregated bolt stats: {metric -> win -> global stream id -> value}
+     */
+    public static <T> Map<String, Map> aggregateBoltStats(List<ExecutorSummary> statsSeq, boolean includeSys) {
+        Map<String, Map> ret = new HashMap<>();
+
+        Map<String, Map<String, Map<T, Long>>> commonStats = aggregateCommonStats(statsSeq);
+        // filter sys streams if necessary
+        commonStats = preProcessStreamSummary(commonStats, includeSys);
+
+        List<Map<String, Map<GlobalStreamId, Long>>> acked = new ArrayList<>();
+        List<Map<String, Map<GlobalStreamId, Long>>> failed = new ArrayList<>();
+        List<Map<String, Map<GlobalStreamId, Long>>> executed = new ArrayList<>();
+        List<Map<String, Map<GlobalStreamId, Double>>> processLatencies = new ArrayList<>();
+        List<Map<String, Map<GlobalStreamId, Double>>> executeLatencies = new ArrayList<>();
+        for (ExecutorSummary summary : statsSeq) {
+            ExecutorStats stat = summary.get_stats();
+            acked.add(stat.get_specific().get_bolt().get_acked());
+            failed.add(stat.get_specific().get_bolt().get_failed());
+            executed.add(stat.get_specific().get_bolt().get_executed());
+            processLatencies.add(stat.get_specific().get_bolt().get_process_ms_avg());
+            executeLatencies.add(stat.get_specific().get_bolt().get_execute_ms_avg());
+        }
+        mergeMaps(ret, commonStats);
+        putKV(ret, ACKED, aggregateCounts(acked));
+        putKV(ret, FAILED, aggregateCounts(failed));
+        putKV(ret, EXECUTED, aggregateCounts(executed));
+        putKV(ret, PROC_LATENCIES, aggregateAverages(processLatencies, acked));
+        putKV(ret, EXEC_LATENCIES, aggregateAverages(executeLatencies, executed));
+
+        return ret;
+    }
+
+    /**
+     * aggregate spout stats
+     *
+     * @param statsSeq   a seq of ExecutorStats
+     * @param includeSys whether to include system streams
+     * @return aggregated spout stats: {metric -> win -> global stream id -> value}
+     */
+    public static Map<String, Map> aggregateSpoutStats(List<ExecutorSummary> statsSeq, boolean includeSys) {
+        // actually Map<String, Map<String, Map<String, Long/Double>>>
+        Map<String, Map> ret = new HashMap<>();
+
+        Map<String, Map<String, Map<String, Long>>> commonStats = aggregateCommonStats(statsSeq);
+        // filter sys streams if necessary
+        commonStats = preProcessStreamSummary(commonStats, includeSys);
+
+        List<Map<String, Map<String, Long>>> acked = new ArrayList<>();
+        List<Map<String, Map<String, Long>>> failed = new ArrayList<>();
+        List<Map<String, Map<String, Double>>> completeLatencies = new ArrayList<>();
+        for (ExecutorSummary summary : statsSeq) {
+            ExecutorStats stats = summary.get_stats();
+            acked.add(stats.get_specific().get_spout().get_acked());
+            failed.add(stats.get_specific().get_spout().get_failed());
+            completeLatencies.add(stats.get_specific().get_spout().get_complete_ms_avg());
+        }
+        ret.putAll(commonStats);
+        putKV(ret, ACKED, aggregateCounts(acked));
+        putKV(ret, FAILED, aggregateCounts(failed));
+        putKV(ret, COMP_LATENCIES, aggregateAverages(completeLatencies, acked));
+
+        return ret;
+    }
+
+    /**
+     * aggregate common stats from a spout/bolt, called in aggregateSpoutStats/aggregateBoltStats
+     */
+    public static <T> Map<String, Map<String, Map<T, Long>>> aggregateCommonStats(List<ExecutorSummary> statsSeq) {
+        Map<String, Map<String, Map<T, Long>>> ret = new HashMap<>();
+
+        List<Map<String, Map<String, Long>>> emitted = new ArrayList<>();
+        List<Map<String, Map<String, Long>>> transferred = new ArrayList<>();
+        for (ExecutorSummary summ : statsSeq) {
+            emitted.add(summ.get_stats().get_emitted());
+            transferred.add(summ.get_stats().get_transferred());
+        }
+        putKV(ret, EMITTED, aggregateCounts(emitted));
+        putKV(ret, TRANSFERRED, aggregateCounts(transferred));
+
+        return ret;
+    }
+
+    /**
+     * filter system streams of aggregated spout/bolt stats if necessary
+     */
+    public static <T> Map<String, Map<String, Map<T, Long>>> preProcessStreamSummary(
+        Map<String, Map<String, Map<T, Long>>> streamSummary, boolean includeSys) {
+        Map<String, Map<T, Long>> emitted = ClientStatsUtil.getMapByKey(streamSummary, EMITTED);
+        Map<String, Map<T, Long>> transferred = ClientStatsUtil.getMapByKey(streamSummary, TRANSFERRED);
+
+        putKV(streamSummary, EMITTED, filterSysStreams(emitted, includeSys));
+        putKV(streamSummary, TRANSFERRED, filterSysStreams(transferred, includeSys));
+
+        return streamSummary;
+    }
+
+    /**
+     * aggregate count streams by window
+     *
+     * @param stats a Map of value: {win -> stream -> value}
+     * @return a Map of value: {win -> value}
+     */
+    public static <K, V extends Number> Map<String, Long> aggregateCountStreams(
+        Map<String, Map<K, V>> stats) {
+        Map<String, Long> ret = new HashMap<>();
+        for (Map.Entry<String, Map<K, V>> entry : stats.entrySet()) {
+            Map<K, V> value = entry.getValue();
+            long sum = 0l;
+            for (V num : value.values()) {
+                sum += num.longValue();
+            }
+            ret.put(entry.getKey(), sum);
+        }
+        return ret;
+    }
+
+    /**
+     * compute an weighted average from a list of average maps and a corresponding count maps extracted from a list of ExecutorSummary
+     *
+     * @param avgSeq   a list of {win -> global stream id -> avg value}
+     * @param countSeq a list of {win -> global stream id -> count value}
+     * @return a Map of {win -> global stream id -> weighted avg value}
+     */
+    public static <K> Map<String, Map<K, Double>> aggregateAverages(List<Map<String, Map<K, Double>>> avgSeq,
+                                                                    List<Map<String, Map<K, Long>>> countSeq) {
+        Map<String, Map<K, Double>> ret = new HashMap<>();
+
+        Map<String, Map<K, List>> expands = expandAveragesSeq(avgSeq, countSeq);
+        for (Map.Entry<String, Map<K, List>> entry : expands.entrySet()) {
+            String k = entry.getKey();
+
+            Map<K, Double> tmp = new HashMap<>();
+            Map<K, List> inner = entry.getValue();
+            for (K kk : inner.keySet()) {
+                List vv = inner.get(kk);
+                tmp.put(kk, valAvg(((Number) vv.get(0)).doubleValue(), ((Number) vv.get(1)).longValue()));
+            }
+            ret.put(k, tmp);
+        }
+
+        return ret;
+    }
+
+    /**
+     * aggregate weighted average of all streams
+     *
+     * @param avgs   a Map of {win -> stream -> average value}
+     * @param counts a Map of {win -> stream -> count value}
+     * @return a Map of {win -> aggregated value}
+     */
+    public static <K> Map<String, Double> aggregateAvgStreams(Map<String, Map<K, Double>> avgs,
+                                                              Map<String, Map<K, Long>> counts) {
+        Map<String, Double> ret = new HashMap<>();
+
+        Map<String, Map<K, List>> expands = expandAverages(avgs, counts);
+        for (Map.Entry<String, Map<K, List>> entry : expands.entrySet()) {
+            String win = entry.getKey();
+
+            double avgTotal = 0.0;
+            long cntTotal = 0l;
+            Map<K, List> inner = entry.getValue();
+            for (K kk : inner.keySet()) {
+                List vv = inner.get(kk);
+                avgTotal += ((Number) vv.get(0)).doubleValue();
+                cntTotal += ((Number) vv.get(1)).longValue();
+            }
+            ret.put(win, valAvg(avgTotal, cntTotal));
+        }
+
+        return ret;
+    }
+
+    /**
+     * aggregates spout stream stats, returns a Map of {metric -> win -> aggregated value}
+     */
+    public static Map<String, Map> spoutStreamsStats(List<ExecutorSummary> summs, boolean includeSys) {
+        if (summs == null) {
+            return new HashMap<>();
+        }
+        // filter ExecutorSummary's with empty stats
+        List<ExecutorSummary> statsSeq = getFilledStats(summs);
+        return aggregateSpoutStreams(aggregateSpoutStats(statsSeq, includeSys));
+    }
+
+    /**
+     * aggregates bolt stream stats, returns a Map of {metric -> win -> aggregated value}
+     */
+    public static Map<String, Map> boltStreamsStats(List<ExecutorSummary> summs, boolean includeSys) {
+        if (summs == null) {
+            return new HashMap<>();
+        }
+        List<ExecutorSummary> statsSeq = getFilledStats(summs);
+        return aggregateBoltStreams(aggregateBoltStats(statsSeq, includeSys));
+    }
+
+    /**
+     * aggregate all spout streams
+     *
+     * @param stats a Map of {metric -> win -> stream id -> value}
+     * @return a Map of {metric -> win -> aggregated value}
+     */
+    public static Map<String, Map> aggregateSpoutStreams(Map<String, Map> stats) {
+        // actual ret is Map<String, Map<String, Long/Double>>
+        Map<String, Map> ret = new HashMap<>();
+        putKV(ret, ACKED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, ACKED)));
+        putKV(ret, FAILED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, FAILED)));
+        putKV(ret, EMITTED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, EMITTED)));
+        putKV(ret, TRANSFERRED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, TRANSFERRED)));
+        putKV(ret, COMP_LATENCIES, aggregateAvgStreams(
+            ClientStatsUtil.getMapByKey(stats, COMP_LATENCIES), ClientStatsUtil.getMapByKey(stats, ACKED)));
+        return ret;
+    }
+
+    /**
+     * aggregate all bolt streams
+     *
+     * @param stats a Map of {metric -> win -> stream id -> value}
+     * @return a Map of {metric -> win -> aggregated value}
+     */
+    public static Map<String, Map> aggregateBoltStreams(Map<String, Map> stats) {
+        Map<String, Map> ret = new HashMap<>();
+        putKV(ret, ACKED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, ACKED)));
+        putKV(ret, FAILED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, FAILED)));
+        putKV(ret, EMITTED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, EMITTED)));
+        putKV(ret, TRANSFERRED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, TRANSFERRED)));
+        putKV(ret, EXECUTED, aggregateCountStreams(ClientStatsUtil.getMapByKey(stats, EXECUTED)));
+        putKV(ret, PROC_LATENCIES, aggregateAvgStreams(
+            ClientStatsUtil.getMapByKey(stats, PROC_LATENCIES), ClientStatsUtil.getMapByKey(stats, ACKED)));
+        putKV(ret, EXEC_LATENCIES, aggregateAvgStreams(
+            ClientStatsUtil.getMapByKey(stats, EXEC_LATENCIES), ClientStatsUtil.getMapByKey(stats, EXECUTED)));
+        return ret;
+    }
+
+    /**
+     * aggregate windowed stats from a bolt executor stats with a Map of accumulated stats
+     */
+    public static Map<String, Object> aggBoltExecWinStats(
+        Map<String, Object> accStats, Map<String, Object> newStats, boolean includeSys) {
+        Map<String, Object> ret = new HashMap<>();
+
+        Map<String, Map<String, Number>> m = new HashMap<>();
+        for (Object win : ClientStatsUtil.getMapByKey(newStats, EXECUTED).keySet()) {
+            m.put((String) win, aggBoltLatAndCount(
+                (Map) (ClientStatsUtil.getMapByKey(newStats, EXEC_LATENCIES)).get(win),
+                (Map) (ClientStatsUtil.getMapByKey(newStats, PROC_LATENCIES)).get(win),
+                (Map) (ClientStatsUtil.getMapByKey(newStats, EXECUTED)).get(win)));
+        }
+        m = swapMapOrder(m);
+
+        Map<String, Double> win2execLatWgtAvg = ClientStatsUtil.getMapByKey(m, EXEC_LAT_TOTAL);
+        Map<String, Double> win2procLatWgtAvg = ClientStatsUtil.getMapByKey(m, PROC_LAT_TOTAL);
+        Map<String, Long> win2executed = ClientStatsUtil.getMapByKey(m, EXECUTED);
+
+        Map<String, Map<String, Long>> emitted = ClientStatsUtil.getMapByKey(newStats, EMITTED);
+        Map<String, Long> win2emitted = mergeWithSumLong(aggregateCountStreams(filterSysStreams(emitted, includeSys)),
+                                                         ClientStatsUtil.getMapByKey(accStats, WIN_TO_EMITTED));
+        putKV(ret, WIN_TO_EMITTED, win2emitted);
+
+        Map<String, Map<String, Long>> transferred = ClientStatsUtil.getMapByKey(newStats, TRANSFERRED);
+        Map<String, Long> win2transferred = mergeWithSumLong(aggregateCountStreams(filterSysStreams(transferred, includeSys)),
+                                                             ClientStatsUtil.getMapByKey(accStats, WIN_TO_TRANSFERRED));
+        putKV(ret, WIN_TO_TRANSFERRED, win2transferred);
+
+        putKV(ret, WIN_TO_EXEC_LAT_WGT_AVG, mergeWithSumDouble(
+            ClientStatsUtil.getMapByKey(accStats, WIN_TO_EXEC_LAT_WGT_AVG), win2execLatWgtAvg));
+        putKV(ret, WIN_TO_PROC_LAT_WGT_AVG, mergeWithSumDouble(
+            ClientStatsUtil.getMapByKey(accStats, WIN_TO_PROC_LAT_WGT_AVG), win2procLatWgtAvg));
+        putKV(ret, WIN_TO_EXECUTED, mergeWithSumLong(
+            ClientStatsUtil.getMapByKey(accStats, WIN_TO_EXECUTED), win2executed));
+        putKV(ret, WIN_TO_ACKED, mergeWithSumLong(
+            aggregateCountStreams(ClientStatsUtil.getMapByKey(newStats, ACKED)), ClientStatsUtil.getMapByKey(accStats, WIN_TO_ACKED)));
+        putKV(ret, WIN_TO_FAILED, mergeWithSumLong(
+            aggregateCountStreams(ClientStatsUtil.getMapByKey(newStats, FAILED)), ClientStatsUtil.getMapByKey(accStats, WIN_TO_FAILED)));
+
+        return ret;
+    }
+
+    /**
+     * aggregate windowed stats from a spout executor stats with a Map of accumulated stats
+     */
+    public static Map<String, Object> aggSpoutExecWinStats(
+        Map<String, Object> accStats, Map<String, Object> beat, boolean includeSys) {
+        Map<String, Object> ret = new HashMap<>();
+
+        Map<String, Map<String, Number>> m = new HashMap<>();
+        for (Object win : ClientStatsUtil.getMapByKey(beat, ACKED).keySet()) {
+            m.put((String) win, aggSpoutLatAndCount(
+                (Map<String, Double>) (ClientStatsUtil.getMapByKey(beat, COMP_LATENCIES)).get(win),
+                (Map<String, Long>) (ClientStatsUtil.getMapByKey(beat, ACKED)).get(win)));
+        }
+        m = swapMapOrder(m);
+
+        Map<String, Double> win2compLatWgtAvg = ClientStatsUtil.getMapByKey(m, COMP_LAT_TOTAL);
+        Map<String, Long> win2acked = ClientStatsUtil.getMapByKey(m, ACKED);
+
+        Map<String, Map<String, Long>> emitted = ClientStatsUtil.getMapByKey(beat, EMITTED);
+        Map<String, Long> win2emitted = mergeWithSumLong(aggregateCountStreams(filterSysStreams(emitted, includeSys)),
+                                                         ClientStatsUtil.getMapByKey(accStats, WIN_TO_EMITTED));
+        putKV(ret, WIN_TO_EMITTED, win2emitted);
+
+        Map<String, Map<String, Long>> transferred = ClientStatsUtil.getMapByKey(beat, TRANSFERRED);
+        Map<String, Long> win2transferred = mergeWithSumLong(aggregateCountStreams(filterSysStreams(transferred, includeSys)),
+                                                             ClientStatsUtil.getMapByKey(accStats, WIN_TO_TRANSFERRED));
+        putKV(ret, WIN_TO_TRANSFERRED, win2transferred);
+
+        putKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSumDouble(
+            ClientStatsUtil.getMapByKey(accStats, WIN_TO_COMP_LAT_WGT_AVG), win2compLatWgtAvg));
+        putKV(ret, WIN_TO_ACKED, mergeWithSumLong(
+            ClientStatsUtil.getMapByKey(accStats, WIN_TO_ACKED), win2acked));
+        putKV(ret, WIN_TO_FAILED, mergeWithSumLong(
+            aggregateCountStreams(ClientStatsUtil.getMapByKey(beat, FAILED)), ClientStatsUtil.getMapByKey(accStats, WIN_TO_FAILED)));
+
+        return ret;
+    }
+
+
+    /**
+     * aggregate a list of count maps into one map
+     *
+     * @param countsSeq a seq of {win -> GlobalStreamId -> value}
+     */
+    public static <T> Map<String, Map<T, Long>> aggregateCounts(List<Map<String, Map<T, Long>>> countsSeq) {
+        Map<String, Map<T, Long>> ret = new HashMap<>();
+        for (Map<String, Map<T, Long>> counts : countsSeq) {
+            for (Map.Entry<String, Map<T, Long>> entry : counts.entrySet()) {
+                String win = entry.getKey();
+                Map<T, Long> stream2count = entry.getValue();
+
+                if (!ret.containsKey(win)) {
+                    ret.put(win, stream2count);
+                } else {
+                    Map<T, Long> existing = ret.get(win);
+                    for (Map.Entry<T, Long> subEntry : stream2count.entrySet()) {
+                        T stream = subEntry.getKey();
+                        if (!existing.containsKey(stream)) {
+                            existing.put(stream, subEntry.getValue());
+                        } else {
+                            existing.put(stream, subEntry.getValue() + existing.get(stream));
+                        }
+                    }
+                }
+            }
+        }
+        return ret;
+    }
+
+    public static Map<String, Object> aggregateCompStats(
+        String window, boolean includeSys, List<Map<String, Object>> beats, String compType) {
+        boolean isSpout = ClientStatsUtil.SPOUT.equals(compType);
+
+        Map<String, Object> initVal = new HashMap<>();
+        putKV(initVal, WIN_TO_ACKED, new HashMap());
+        putKV(initVal, WIN_TO_FAILED, new HashMap());
+        putKV(initVal, WIN_TO_EMITTED, new HashMap());
+        putKV(initVal, WIN_TO_TRANSFERRED, new HashMap());
+
+        Map<String, Object> stats = new HashMap();
+        putKV(stats, ClientStatsUtil.EXECUTOR_STATS, new ArrayList());
+        putKV(stats, SID_TO_OUT_STATS, new HashMap());
+        if (isSpout) {
+            putKV(initVal, TYPE, ClientStatsUtil.SPOUT);
+            putKV(initVal, WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
+        } else {
+            putKV(initVal, TYPE, ClientStatsUtil.BOLT);
+            putKV(initVal, WIN_TO_EXECUTED, new HashMap());
+            putKV(stats, CID_SID_TO_IN_STATS, new HashMap());
+            putKV(initVal, WIN_TO_EXEC_LAT_WGT_AVG, new HashMap());
+            putKV(initVal, WIN_TO_PROC_LAT_WGT_AVG, new HashMap());
+        }
+        putKV(initVal, STATS, stats);
+
+        // iterate through all executor heartbeats
+        for (Map<String, Object> beat : beats) {
+            initVal = aggCompExecStats(window, includeSys, initVal, beat, compType);
+        }
+
+        return initVal;
+    }
+
+    /**
+     * Combines the aggregate stats of one executor with the given map, selecting the appropriate window and including system components as
+     * specified.
+     */
+    public static Map<String, Object> aggCompExecStats(String window, boolean includeSys, Map<String, Object> accStats,
+                                                       Map<String, Object> beat, String compType) {
+        Map<String, Object> ret = new HashMap<>();
+        if (ClientStatsUtil.SPOUT.equals(compType)) {
+            ret.putAll(aggSpoutExecWinStats(accStats, ClientStatsUtil.getMapByKey(beat, STATS), includeSys));
+            putKV(ret, STATS, mergeAggCompStatsCompPageSpout(
+                ClientStatsUtil.getMapByKey(accStats, STATS),
+                aggPreMergeCompPageSpout(beat, window, includeSys)));
+        } else {
+            ret.putAll(aggBoltExecWinStats(accStats, ClientStatsUtil.getMapByKey(beat, STATS), includeSys));
+            putKV(ret, STATS, mergeAggCompStatsCompPageBolt(
+                ClientStatsUtil.getMapByKey(accStats, STATS),
+                aggPreMergeCompPageBolt(beat, window, includeSys)));
+        }
+        putKV(ret, TYPE, compType);
+
+        return ret;
+    }
+
+    /**
+     * post aggregate component stats: 1. computes execute-latency/process-latency from execute/process latency total 2. computes windowed
+     * weight avgs 3. transform Map keys
+     *
+     * @param compStats accumulated comp stats
+     */
+    public static Map<String, Object> postAggregateCompStats(Map<String, Object> compStats) {
+        Map<String, Object> ret = new HashMap<>();
+
+        String compType = (String) compStats.get(TYPE);
+        Map stats = ClientStatsUtil.getMapByKey(compStats, STATS);
+        Integer numTasks = getByKeyOr0(stats, NUM_TASKS).intValue();
+        Integer numExecutors = getByKeyOr0(stats, NUM_EXECUTORS).intValue();
+        Map outStats = ClientStatsUtil.getMapByKey(stats, SID_TO_OUT_STATS);
+
+        putKV(ret, TYPE, compType);
+        putKV(ret, NUM_TASKS, numTasks);
+        putKV(ret, NUM_EXECUTORS, numExecutors);
+        putKV(ret, ClientStatsUtil.EXECUTOR_STATS, stats.get(ClientStatsUtil.EXECUTOR_STATS));
+        putKV(ret, WIN_TO_EMITTED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_EMITTED)));
+        putKV(ret, WIN_TO_TRANSFERRED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_TRANSFERRED)));
+        putKV(ret, WIN_TO_ACKED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_ACKED)));
+        putKV(ret, WIN_TO_FAILED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_FAILED)));
+
+        if (ClientStatsUtil.BOLT.equals(compType)) {
+            Map inStats = ClientStatsUtil.getMapByKey(stats, CID_SID_TO_IN_STATS);
+
+            Map inStats2 = new HashMap();
+            for (Object o : inStats.entrySet()) {
+                Map.Entry e = (Map.Entry) o;
+                Object k = e.getKey();
+                Map v = (Map) e.getValue();
+                long executed = getByKeyOr0(v, EXECUTED).longValue();
+                if (executed > 0) {
+                    double executeLatencyTotal = getByKeyOr0(v, EXEC_LAT_TOTAL).doubleValue();
+                    double processLatencyTotal = getByKeyOr0(v, PROC_LAT_TOTAL).doubleValue();
+                    putKV(v, EXEC_LATENCY, executeLatencyTotal / executed);
+                    putKV(v, PROC_LATENCY, processLatencyTotal / executed);
+                } else {
+                    putKV(v, EXEC_LATENCY, 0.0);
+                    putKV(v, PROC_LATENCY, 0.0);
+                }
+                remove(v, EXEC_LAT_TOTAL);
+                remove(v, PROC_LAT_TOTAL);
+                inStats2.put(k, v);
+            }
+            putKV(ret, CID_SID_TO_IN_STATS, inStats2);
+
+            putKV(ret, SID_TO_OUT_STATS, outStats);
+            putKV(ret, WIN_TO_EXECUTED, mapKeyStr(ClientStatsUtil.getMapByKey(compStats, WIN_TO_EXECUTED)));
+            putKV(ret, WIN_TO_EXEC_LAT, computeWeightedAveragesPerWindow(
+                compStats, WIN_TO_EXEC_LAT_WGT_AVG, WIN_TO_EXECUTED));
+            putKV(ret, WIN_TO_PROC_LAT, computeWeightedAveragesPerWindow(
+                compStats, WIN_TO_PROC_LAT_WGT_AVG, WIN_TO_EXECUTED));
+        } else {
+            Map outStats2 = new HashMap();
+            for (Object o : outStats.entrySet()) {
+                Map.Entry e = (Map.Entry) o;
+                Object k = e.getKey();
+                Map v = (Map) e.getValue();
+                long acked = getByKeyOr0(v, ACKED).longValue();
+                if (acked > 0) {
+                    double compLatencyTotal = getByKeyOr0(v, COMP_LAT_TOTAL).doubleValue();
+                    putKV(v, COMP_LATENCY, compLatencyTotal / acked);
+                } else {
+                    putKV(v, COMP_LATENCY, 0.0);
+                }
+                remove(v, COMP_LAT_TOTAL);
+                outStats2.put(k, v);
+            }
+            putKV(ret, SID_TO_OUT_STATS, outStats2);
+            putKV(ret, WIN_TO_COMP_LAT, computeWeightedAveragesPerWindow(
+                compStats, WIN_TO_COMP_LAT_WGT_AVG, WIN_TO_ACKED));
+        }
+
+        return ret;
+    }
+
+    /**
+     * aggregate component executor stats
+     *
+     * @param exec2hostPort  a Map of {executor -> host+port}
+     * @param task2component a Map of {task id -> component}
+     * @param beats          a converted HashMap of executor heartbeats, {executor -> heartbeat}
+     * @param window         specified window
+     * @param includeSys     whether to include system streams
+     * @param topologyId     topology id
+     * @param topology       storm topology
+     * @param componentId    component id
+     * @return ComponentPageInfo thrift structure
+     */
+    public static ComponentPageInfo aggCompExecsStats(
+        Map exec2hostPort, Map task2component, Map<List<Integer>, Map<String, Object>> beats,
+        String window, boolean includeSys, String topologyId, StormTopology topology, String componentId) {
+
+        List<Map<String, Object>> beatList =
+            extractDataFromHb(exec2hostPort, task2component, beats, includeSys, topology, componentId);
+        Map<String, Object> compStats = aggregateCompStats(window, includeSys, beatList, componentType(topology, componentId));
+        compStats = postAggregateCompStats(compStats);
+        return thriftifyCompPageData(topologyId, topology, componentId, compStats);
+    }
+
+    /**
+     * aggregate statistics per worker for a topology. Optionally filtering on specific supervisors
+     *
+     * @param stormId          topology id
+     * @param stormName        storm topology
+     * @param task2Component   a Map of {task id -> component}
+     * @param beats            a converted HashMap of executor heartbeats, {executor -> heartbeat}
+     * @param exec2NodePort    a Map of {executor -> host+port}
+     * @param includeSys       whether to include system streams
+     * @param userAuthorized   whether the user is authorized to view topology info
+     * @param filterSupervisor if not null, only return WorkerSummaries for that supervisor
+     * @return List<WorkerSummary> thrift structures
+     */
+    public static List<WorkerSummary> aggWorkerStats(String stormId, String stormName,
+                                                     Map<Integer, String> task2Component,
+                                                     Map<List<Integer>, Map<String, Object>> beats,
+                                                     Map<List<Long>, List<Object>> exec2NodePort,
+                                                     Map<String, String> nodeHost,
+                                                     Map<WorkerSlot, WorkerResources> worker2Resources,
+                                                     boolean includeSys, boolean userAuthorized, String filterSupervisor) {
+
+        // host,port => WorkerSummary
+        HashMap<WorkerSlot, WorkerSummary> workerSummaryMap = new HashMap<>();
+
+        if (exec2NodePort != null) {
+            // for each executor -> node+port pair
+            for (Map.Entry<List<Long>, List<Object>> execNodePort : exec2NodePort.entrySet()) {
+                List<Object> nodePort = execNodePort.getValue();
+                String node = (String) nodePort.get(0);
+                Long port = (Long) nodePort.get(1);
+                String host = nodeHost.get(node);
+                WorkerSlot slot = new WorkerSlot(node, port);
+                WorkerResources resources = worker2Resources.get(slot);
+
+                if (filterSupervisor == null || node.equals(filterSupervisor)) {
+                    WorkerSummary ws = workerSummaryMap.get(slot);
+
+                    if (ws == null) {
+                        ws = new WorkerSummary();
+                        ws.set_host(host);
+                        ws.set_port(port.intValue());
+                        ws.set_supervisor_id(node);
+                        ws.set_topology_id(stormId);
+                        ws.set_topology_name(stormName);
+                        ws.set_num_executors(0);
+                        if (resources != null) {
+                            ws.set_assigned_memonheap(resources.get_mem_on_heap());
+                            ws.set_assigned_memoffheap(resources.get_mem_off_heap());
+                            ws.set_assigned_cpu(resources.get_cpu());
+                        } else {
+                            ws.set_assigned_memonheap(0);
+                            ws.set_assigned_memoffheap(0);
+                            ws.set_assigned_cpu(0);
+                        }
+                        ws.set_component_to_num_tasks(new HashMap<String, Long>());
+                        workerSummaryMap.put(slot, ws);
+                    }
+                    Map<String, Long> componentToNumTasks = ws.get_component_to_num_tasks();
+
+                    // gets min/max task pairs (executors): [1 1] [2 3] ...
+                    List<Long> exec = execNodePort.getKey();
+                    // get executor heartbeat
+                    int hbeatSecs = 0;
+                    if (beats != null) {
+                        Map<String, Object> beat = beats.get(ClientStatsUtil.convertExecutor(exec));
+                        hbeatSecs = beat == null ? 0 : (int) beat.get("uptime");
+                    }
+                    ws.set_uptime_secs(hbeatSecs);
+                    ws.set_num_executors(ws.get_num_executors() + 1);
+
+                    // get tasks if the user is authorized for this topology
+                    if (userAuthorized) {
+                        int firstTask = exec.get(0).intValue();
+                        int lastTask = exec.get(1).intValue();
+
+                        // get per task components
+                        for (int task = firstTask; task <= lastTask; task++) {
+                            String component = task2Component.get(task);
+                            // if the component is a system (__*) component and we are hiding
+                            // them in UI, keep going
+                            if (!includeSys && Utils.isSystemId(component)) {
+                                continue;
+                            }
+
+                            // good to go, increment # of tasks this component is being executed on
+                            Long counter = componentToNumTasks.get(component);
+                            if (counter == null) {
+                                counter = new Long(0);
+                            }
+                            componentToNumTasks.put(component, counter + 1);
+                        }
+                    }
+                }
+            }
+        }
+        return new ArrayList<WorkerSummary>(workerSummaryMap.values());
+    }
+
+    /**
+     * Aggregate statistics per worker for a topology. Optionally filtering on specific supervisors
+     *
+     * Convenience overload when called from the topology page code (in that case we want data for all workers in the topology, not filtered
+     * by supervisor)
+     *
+     * @param stormId        topology id
+     * @param stormName      storm topology
+     * @param task2Component a Map of {task id -> component}
+     * @param beats          a converted HashMap of executor heartbeats, {executor -> heartbeat}
+     * @param exec2NodePort  a Map of {executor -> host+port}
+     * @param includeSys     whether to include system streams
+     * @param userAuthorized whether the user is authorized to view topology info
+     * @return List<WorkerSummary> thrift structures
+     */
+    public static List<WorkerSummary> aggWorkerStats(String stormId, String stormName,
+                                                     Map<Integer, String> task2Component,
+                                                     Map<List<Integer>, Map<String, Object>> beats,
+                                                     Map<List<Long>, List<Object>> exec2NodePort,
+                                                     Map<String, String> nodeHost,
+                                                     Map<WorkerSlot, WorkerResources> worker2Resources,
+                                                     boolean includeSys, boolean userAuthorized) {
+        return aggWorkerStats(stormId, stormName,
+                              task2Component, beats, exec2NodePort, nodeHost, worker2Resources,
+                              includeSys, userAuthorized, null);
+    }
+
+    // =====================================================================================
+    // convert thrift stats to java maps
+    // =====================================================================================
+
+    /**
+     * convert thrift executor heartbeats into a java HashMap
+     */
+    public static Map<List<Integer>, Map<String, Object>> convertExecutorBeats(Map<ExecutorInfo, ExecutorBeat> beats) {
+        Map<List<Integer>, Map<String, Object>> ret = new HashMap<>();
+        for (Map.Entry<ExecutorInfo, ExecutorBeat> beat : beats.entrySet()) {
+            ExecutorInfo executorInfo = beat.getKey();
+            ExecutorBeat executorBeat = beat.getValue();
+            ret.put(Lists.newArrayList(executorInfo.get_task_start(), executorInfo.get_task_end()),
+                    convertZkExecutorHb(executorBeat));
+        }
+
+        return ret;
+    }
+
+    /**
+     * convert {@link SupervisorWorkerHeartbeat} to nimbus local report executor heartbeats
+     *
+     * @param workerHeartbeat
+     * @return
+     */
+    public static Map<List<Integer>, Map<String, Object>> convertWorkerBeats(SupervisorWorkerHeartbeat workerHeartbeat) {
+        Map<List<Integer>, Map<String, Object>> ret = new HashMap<>();
+        for (ExecutorInfo executorInfo : workerHeartbeat.get_executors()) {
+            Map<String, Object> reportBeat = new HashMap<>();
+            reportBeat.put(ClientStatsUtil.TIME_SECS, workerHeartbeat.get_time_secs());
+            ret.put(Lists.newArrayList(executorInfo.get_task_start(), executorInfo.get_task_end()),
+                    reportBeat);
+        }
+
+        return ret;
+    }
+
+    /**
+     * convert thrift ExecutorBeat into a java HashMap
+     */
+    public static Map<String, Object> convertZkExecutorHb(ExecutorBeat beat) {
+        Map<String, Object> ret = new HashMap<>();
+        if (beat != null) {
+            ret.put(ClientStatsUtil.TIME_SECS, beat.getTimeSecs());
+            ret.put(ClientStatsUtil.UPTIME, beat.getUptime());
+            ret.put(STATS, convertExecutorStats(beat.getStats()));
+        }
+
+        return ret;
+    }
+
+    /**
+     * convert a thrift worker heartbeat into a java HashMap
+     *
+     * @param workerHb
+     * @return
+     */
+    public static Map<String, Object> convertZkWorkerHb(ClusterWorkerHeartbeat workerHb) {
+        Map<String, Object> ret = new HashMap<>();
+        if (workerHb != null) {
+            ret.put("storm-id", workerHb.get_storm_id());
+            ret.put(ClientStatsUtil.EXECUTOR_STATS, convertExecutorsStats(workerHb.get_executor_stats()));
+            ret.put(ClientStatsUtil.UPTIME, workerHb.get_uptime_secs());
+            ret.put(ClientStatsUtil.TIME_SECS, workerHb.get_time_secs());
+        }
+        return ret;
+    }
+
+    /**
+     * convert executors stats into a HashMap, note that ExecutorStats are remained unchanged
+     */
+    public static Map<List<Integer>, ExecutorStats> convertExecutorsStats(Map<ExecutorInfo, ExecutorStats> stats) {
+        Map<List<Integer>, ExecutorStats> ret = new HashMap<>();
+        for (Map.Entry<ExecutorInfo, ExecutorStats> entry : stats.entrySet()) {
+            ExecutorInfo executorInfo = entry.getKey();
+            ExecutorStats executorStats = entry.getValue();
+
+            ret.put(Lists.newArrayList(executorInfo.get_task_start(), executorInfo.get_task_end()),
+                    executorStats);
+        }
+        return ret;
+    }
+
+    /**
+     * convert thrift ExecutorStats structure into a java HashMap
+     */
+    public static Map<String, Object> convertExecutorStats(ExecutorStats stats) {
+        Map<String, Object> ret = new HashMap<>();
+
+        putKV(ret, EMITTED, stats.get_emitted());
+        putKV(ret, TRANSFERRED, stats.get_transferred());
+        putKV(ret, RATE, stats.get_rate());
+
+        if (stats.get_specific().is_set_bolt()) {
+            ret.putAll(convertSpecificStats(stats.get_specific().get_bolt()));
+            putKV(ret, TYPE, ClientStatsUtil.BOLT);
+        } else {
+            ret.putAll(convertSpecificStats(stats.get_specific().get_spout()));
+            putKV(ret, TYPE, ClientStatsUtil.SPOUT);
+        }
+
+        return ret;
+    }
+
+    private static Map<String, Object> convertSpecificStats(SpoutStats stats) {
+        Map<String, Object> ret = new HashMap<>();
+        putKV(ret, ACKED, stats.get_acked());
+        putKV(ret, FAILED, stats.get_failed());
+        putKV(ret, COMP_LATENCIES, stats.get_complete_ms_avg());
+
+        return ret;
+    }
+
+    private static Map<String, Object> convertSpecificStats(BoltStats stats) {
+        Map<String, Object> ret = new HashMap<>();
+
+        Map acked = ClientStatsUtil.windowSetConverter(stats.get_acked(), FROM_GSID, ClientStatsUtil.IDENTITY);
+        Map failed = ClientStatsUtil.windowSetConverter(stats.get_failed(), FROM_GSID, ClientStatsUtil.IDENTITY);
+        Map processAvg = ClientStatsUtil.windowSetConverter(stats.get_process_ms_avg(), FROM_GSID, ClientStatsUtil.IDENTITY);
+        Map executed = ClientStatsUtil.windowSetConverter(stats.get_executed(), FROM_GSID, ClientStatsUtil.IDENTITY);
+        Map executeAvg = ClientStatsUtil.windowSetConverter(stats.get_execute_ms_avg(), FROM_GSID, ClientStatsUtil.IDENTITY);
+
+        putKV(ret, ACKED, acked);
+        putKV(ret, FAILED, failed);
+        putKV(ret, PROC_LATENCIES, processAvg);
+        putKV(ret, EXECUTED, executed);
+        putKV(ret, EXEC_LATENCIES, executeAvg);
+
+        return ret;
+    }
+
+    /**
+     * extract a list of host port info for specified component
+     *
+     * @param exec2hostPort  {executor -> host+port}
+     * @param task2component {task id -> component}
+     * @param includeSys     whether to include system streams
+     * @param compId         component id
+     * @return a list of host+port
+     */
+    public static List<Map<String, Object>> extractNodeInfosFromHbForComp(
+        Map<List<? extends Number>, List<Object>> exec2hostPort, Map<Integer, String> task2component, boolean includeSys, String compId) {
+        List<Map<String, Object>> ret = new ArrayList<>();
+
+        Set<List> hostPorts = new HashSet<>();
+        for (Entry<List<? extends Number>, List<Object>> entry : exec2hostPort.entrySet()) {
+            List<? extends Number> key = entry.getKey();
+            List<Object> value = entry.getValue();
+
+            Integer start = key.get(0).intValue();
+            String host = (String) value.get(0);
+            Integer port = (Integer) value.get(1);
+            String comp = task2component.get(start);
+            if ((compId == null || compId.equals(comp)) && (includeSys || !Utils.isSystemId(comp))) {
+                hostPorts.add(Lists.newArrayList(host, port));
+            }
+        }
+
+        for (List hostPort : hostPorts) {
+            Map<String, Object> m = new HashMap<>();
+            putKV(m, HOST, hostPort.get(0));
+            putKV(m, PORT, hostPort.get(1));
+            ret.add(m);
+        }
+
+        return ret;
+    }
+
+
+    // =====================================================================================
+    // heartbeats related
+    // =====================================================================================
+
+    /**
+     * extracts a list of executor data from heart beats
+     */
+    public static List<Map<String, Object>> extractDataFromHb(Map executor2hostPort, Map task2component,
+                                                              Map<List<Integer>, Map<String, Object>> beats,
+                                                              boolean includeSys, StormTopology topology) {
+        return extractDataFromHb(executor2hostPort, task2component, beats, includeSys, topology, null);
+    }
+
+    public static List<Map<String, Object>> extractDataFromHb(Map executor2hostPort, Map task2component,
+                                                              Map<List<Integer>, Map<String, Object>> beats,
+                                                              boolean includeSys, StormTopology topology, String compId) {
+        List<Map<String, Object>> ret = new ArrayList<>();
+        if (executor2hostPort == null || beats == null) {
+            return ret;
+        }
+        for (Object o : executor2hostPort.entrySet()) {
+            Map.Entry entry = (Map.Entry) o;
+            List executor = (List) entry.getKey();
+            List hostPort = (List) entry.getValue();
+
+            Integer start = ((Number) executor.get(0)).intValue();
+            Integer end = ((Number) executor.get(1)).intValue();
+
+            String host = (String) hostPort.get(0);
+            Integer port = ((Number) hostPort.get(1)).intValue();
+
+            Map<String, Object> beat = beats.get(ClientStatsUtil.convertExecutor(executor));
+            if (beat == null) {
+                continue;
+            }
+            String id = (String) task2component.get(start);
+
+            Map<String, Object> m = new HashMap<>();
+            if ((compId == null || compId.equals(id)) && (includeSys || !Utils.isSystemId(id))) {
+                putKV(m, "exec-id", entry.getKey());
+                putKV(m, "comp-id", id);
+                putKV(m, NUM_TASKS, end - start + 1);
+                putKV(m, HOST, host);
+                putKV(m, PORT, port);
+
+                Map stats = ClientStatsUtil.getMapByKey(beat, STATS);
+                putKV(m, ClientStatsUtil.UPTIME, beat.get(ClientStatsUtil.UPTIME));
+                putKV(m, STATS, stats);
+
+                String type = componentType(topology, compId);
+                if (type != null) {
+                    putKV(m, TYPE, type);
+                } else {
+                    putKV(m, TYPE, stats.get(TYPE));
+                }
+                ret.add(m);
+            }
+        }
+        return ret;
+    }
+
+    /**
+     * compute weighted avg from a Map of stats and given avg/count keys
+     *
+     * @param accData    a Map of {win -> key -> value}
+     * @param wgtAvgKey  weighted average key
+     * @param divisorKey count key
+     * @return a Map of {win -> weighted avg value}
+     */
+    private static Map<String, Double> computeWeightedAveragesPerWindow(Map<String, Object> accData,
+                                                                        String wgtAvgKey, String divisorKey) {
+        Map<String, Double> ret = new HashMap<>();
+        for (Object o : ClientStatsUtil.getMapByKey(accData, wgtAvgKey).entrySet()) {
+            Map.Entry e = (Map.Entry) o;
+            Object window = e.getKey();
+            double wgtAvg = ((Number) e.getValue()).doubleValue();
+            long divisor = ((Number) ClientStatsUtil.getMapByKey(accData, divisorKey).get(window)).longValue();
+            if (divisor > 0) {
+                ret.put(window.toString(), wgtAvg / divisor);
+            }
+        }
+        return ret;
+    }
+
+    /**
+     * computes max bolt capacity
+     *
+     * @param executorSumms a list of ExecutorSummary
+     * @return max bolt capacity
+     */
+    public static double computeBoltCapacity(List<ExecutorSummary> executorSumms) {
+        double max = 0.0;
+        for (ExecutorSummary summary : executorSumms) {
+            double capacity = computeExecutorCapacity(summary);
+            if (capacity > max) {
+                max = capacity;
+            }
+        }
+        return max;
+    }
+
+    public static double computeExecutorCapacity(ExecutorSummary summary) {
+        ExecutorStats stats = summary.get_stats();
+        if (stats == null) {
+            return 0.0;
+        } else {
+            // actual value of m is: Map<String, Map<String/GlobalStreamId, Long/Double>> ({win -> stream -> value})
+            Map<String, Map> m = aggregateBoltStats(Lists.newArrayList(summary), true);
+            // {metric -> win -> value} ==> {win -> metric -> value}
+            m = swapMapOrder(aggregateBoltStreams(m));
+            // {metric -> value}
+            Map data = ClientStatsUtil.getMapByKey(m, TEN_MIN_IN_SECONDS_STR);
+
+            int uptime = summary.get_uptime_secs();
+            int win = Math.min(uptime, TEN_MIN_IN_SECONDS);
+            long executed = getByKeyOr0(data, EXECUTED).longValue();
+            double latency = getByKeyOr0(data, EXEC_LATENCIES).doubleValue();
+            if (win > 0) {
+                return executed * latency / (1000 * win);
+            }
+            return 0.0;
+        }
+    }
+
+    /**
+     * filter ExecutorSummary whose stats is null
+     *
+     * @param summs a list of ExecutorSummary
+     * @return filtered summs
+     */
+    public static List<ExecutorSummary> getFilledStats(List<ExecutorSummary> summs) {
+        List<ExecutorSummary> ret = new ArrayList<>();
+        for (ExecutorSummary summ : summs) {
+            if (summ.get_stats() != null) {
+                ret.add(summ);
+            }
+        }
+        return ret;
+    }
+
+    private static <K, V> Map<String, V> mapKeyStr(Map<K, V> m) {
+        Map<String, V> ret = new HashMap<>();
+        for (Map.Entry<K, V> entry : m.entrySet()) {
+            ret.put(entry.getKey().toString(), entry.getValue());
+        }
+        return ret;
+    }
+
+    private static <K1, K2> long sumStreamsLong(Map<K1, Map<K2, ?>> m, String key) {
+        long sum = 0;
+        if (m == null) {
+            return sum;
+        }
+        for (Map<K2, ?> v : m.values()) {
+            for (Map.Entry<K2, ?> entry : v.entrySet()) {
+                if (entry.getKey().equals(key)) {
+                    sum += ((Number) entry.getValue()).longValue();
+                }
+            }
+        }
+        return sum;
+    }
+
+    private static <K1, K2> double sumStreamsDouble(Map<K1, Map<K2, ?>> m, String key) {
+        double sum = 0;
+        if (m == null) {
+            return sum;
+        }
+        for (Map<K2, ?> v : m.values()) {
+            for (Map.Entry<K2, ?> entry : v.entrySet()) {
+                if (entry.getKey().equals(key)) {
+                    sum += ((Number) entry.getValue()).doubleValue();
+                }
+            }
+        }
+        return sum;
+    }
+
+    /**
+     * same as clojure's (merge-with merge m1 m2)
+     */
+    private static Map mergeMaps(Map m1, Map m2) {
+        if (m2 == null) {
+            return m1;
+        }
+        for (Object o : m2.entrySet()) {
+            Map.Entry entry = (Map.Entry) o;
+            Object k = entry.getKey();
+
+            Map existing = (Map) m1.get(k);
+            if (existing == null) {
+                m1.put(k, entry.getValue());
+            } else {
+                existing.putAll((Map) m2.get(k));
+            }
+        }
+        return m1;
+    }
+
+
+    /**
+     * filter system streams from stats
+     *
+     * @param stream2stat { stream id -> value }
+     * @param includeSys  whether to filter system streams
+     * @return filtered stats
+     */
+    private static <K, V> Map<K, V> filterSysStreams2Stat(Map<K, V> stream2stat, boolean includeSys) {
+        LOG.trace("Filter Sys Streams2Stat {}", stream2stat);
+        if (!includeSys) {
+            for (Iterator itr = stream2stat.keySet().iterator(); itr.hasNext(); ) {
+                Object key = itr.next();
+                if (key instanceof String && Utils.isSystemId((String) key)) {
+                    itr.remove();
+                }
+            }
+        }
+        return stream2stat;
+    }
+
+    /**
+     * filter system streams from stats
+     *
+     * @param stats      { win -> stream id -> value }
+     * @param includeSys whether to filter system streams
+     * @return filtered stats
+     */
+    private static <K, V> Map<String, Map<K, V>> filterSysStreams(Map<String, Map<K, V>> stats, boolean includeSys) {
+        LOG.trace("Filter Sys Streams {}", stats);
+        if (!includeSys) {
+            for (Iterator<String> itr = stats.keySet().iterator(); itr.hasNext(); ) {
+                String winOrStream = itr.next();
+                Map<K, V> stream2stat = stats.get(winOrStream);
+                for (Iterator subItr = stream2stat.keySet().iterator(); subItr.hasNext(); ) {
+                    Object key = subItr.next();
+                    if (key instanceof String && Utils.isSystemId((String) key)) {
+                        subItr.remove();
+                    }
+                }
+            }
+        }
+        return stats;
+    }
+
+    /**
+     * equals to clojure's: (merge-with (partial merge-with sum-or-0) acc-out spout-out)
+     */
+    private static <K1, K2> Map<K1, Map<K2, Number>> fullMergeWithSum(Map<K1, Map<K2, ?>> m1,
+                                                                      Map<K1, Map<K2, ?>> m2) {
+        Set<K1> allKeys = new HashSet<>();
+        if (m1 != null) {
+            allKeys.addAll(m1.keySet());
+        }
+        if (m2 != null) {
+            allKeys.addAll(m2.keySet());
+        }
+
+        Map<K1, Map<K2, Number>> ret = new HashMap<>();
+        for (K1 k : allKeys) {
+            Map<K2, ?> mm1 = null, mm2 = null;
+            if (m1 != null) {
+                mm1 = m1.get(k);
+            }
+            if (m2 != null) {
+                mm2 = m2.get(k);
+            }
+            ret.put(k, mergeWithSum(mm1, mm2));
+        }
+
+        return ret;
+    }
+
+    private static <K> Map<K, Number> mergeWithSum(Map<K, ?> m1, Map<K, ?> m2) {
+        Map<K, Number> ret = new HashMap<>();
+
+        Set<K> allKeys = new HashSet<>();
+        if (m1 != null) {
+            allKeys.addAll(m1.keySet());
+        }
+        if (m2 != null) {
+            allKeys.addAll(m2.keySet());
+        }
+
+        for (K k : allKeys) {
+            Number n1 = getOr0(m1, k);
+            Number n2 = getOr0(m2, k);
+            if (n1 instanceof Long) {
+                ret.put(k, n1.longValue() + n2.longValue());
+            } else {
+                ret.put(k, n1.doubleValue() + n2.doubleValue());
+            }
+        }
+        return ret;
+    }
+
+    private static <K> Map<K, Long> mergeWithSumLong(Map<K, Long> m1, Map<K, Long> m2) {
+        Map<K, Long> ret = new HashMap<>();
+
+        Set<K> allKeys = new HashSet<>();
+        if (m1 != null) {
+            allKeys.addAll(m1.keySet());
+        }
+        if (m2 != null) {
+            allKeys.addAll(m2.keySet());
+        }
+
+        for (K k : allKeys) {
+            Number n1 = getOr0(m1, k);
+            Number n2 = getOr0(m2, k);
+            ret.put(k, n1.longValue() + n2.longValue());
+        }
+        return ret;
+    }
+
+    private static <K> Map<K, Double> mergeWithSumDouble(Map<K, Double> m1, Map<K, Double> m2) {
+        Map<K, Double> ret = new HashMap<>();
+
+        Set<K> allKeys = new HashSet<>();
+        if (m1 != null) {
+            allKeys.addAll(m1.keySet());
+        }
+        if (m2 != null) {
+            allKeys.addAll(m2.keySet());
+        }
+
+        for (K k : allKeys) {
+            Number n1 = getOr0(m1, k);
+            Number n2 = getOr0(m2, k);
+            ret.put(k, n1.doubleValue() + n2.doubleValue());
+        }
+        return ret;
+    }
+
+    /**
+     * this method merges 2 two-level-deep maps, which is different from mergeWithSum, and we expect the two maps have the same keys
+     */
+    private static <K> Map<String, Map<K, List>> mergeWithAddPair(Map<String, Map<K, List>> m1,
+                                                                  Map<String, Map<K, List>> m2) {
+        Map<String, Map<K, List>> ret = new HashMap<>();
+
+        Set<String> allKeys = new HashSet<>();
+        if (m1 != null) {
+            allKeys.addAll(m1.keySet());
+        }
+        if (m2 != null) {
+            allKeys.addAll(m2.keySet());
+        }
+
+        for (String k : allKeys) {
+            Map<K, List> mm1 = (m1 != null) ? m1.get(k) : null;
+            Map<K, List> mm2 = (m2 != null) ? m2.get(k) : null;
+            if (mm1 == null && mm2 == null) {
+                continue;
+            } else if (mm1 == null) {
+                ret.put(k, mm2);
+            } else if (mm2 == null) {
+                ret.put(k, mm1);
+            } else {
+                Map<K, List> tmp = new HashMap<>();
+                for (K kk : mm1.keySet()) {
+                    List seq1 = mm1.get(kk);
+                    List seq2 = mm2.get(kk);
+                    List sums =

<TRUNCATED>

[6/7] storm git commit: New violations were added by other pull requests

Posted by bo...@apache.org.
New violations were added by other pull requests


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

Branch: refs/heads/master
Commit: 4c1ba85a6d1547ce2424902dca9aed8856ddbf69
Parents: 15a62dd
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Mon Sep 17 15:01:41 2018 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Mon Sep 17 15:01:41 2018 -0500

----------------------------------------------------------------------
 storm-server/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/4c1ba85a/storm-server/pom.xml
----------------------------------------------------------------------
diff --git a/storm-server/pom.xml b/storm-server/pom.xml
index e05e16f..67276a9 100644
--- a/storm-server/pom.xml
+++ b/storm-server/pom.xml
@@ -171,7 +171,7 @@
                 <artifactId>maven-checkstyle-plugin</artifactId>
                 <!--Note - the version would be inherited-->
                 <configuration>
-                    <maxAllowedViolations>757</maxAllowedViolations>
+                    <maxAllowedViolations>763</maxAllowedViolations>
                 </configuration>
             </plugin>
             <plugin>


[4/7] storm git commit: STORM-3162: Cleanup heartbeats cache and make it thread safe

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/9d3feb0e/storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java b/storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java
deleted file mode 100644
index d2af216..0000000
--- a/storm-client/src/jvm/org/apache/storm/stats/StatsUtil.java
+++ /dev/null
@@ -1,2610 +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.
- */
-
-package org.apache.storm.stats;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import org.apache.storm.cluster.ExecutorBeat;
-import org.apache.storm.cluster.IStormClusterState;
-import org.apache.storm.generated.Bolt;
-import org.apache.storm.generated.BoltAggregateStats;
-import org.apache.storm.generated.BoltStats;
-import org.apache.storm.generated.ClusterWorkerHeartbeat;
-import org.apache.storm.generated.CommonAggregateStats;
-import org.apache.storm.generated.ComponentAggregateStats;
-import org.apache.storm.generated.ComponentPageInfo;
-import org.apache.storm.generated.ComponentType;
-import org.apache.storm.generated.ErrorInfo;
-import org.apache.storm.generated.ExecutorAggregateStats;
-import org.apache.storm.generated.ExecutorInfo;
-import org.apache.storm.generated.ExecutorSpecificStats;
-import org.apache.storm.generated.ExecutorStats;
-import org.apache.storm.generated.ExecutorSummary;
-import org.apache.storm.generated.GlobalStreamId;
-import org.apache.storm.generated.SpecificAggregateStats;
-import org.apache.storm.generated.SpoutAggregateStats;
-import org.apache.storm.generated.SpoutStats;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.generated.SupervisorWorkerHeartbeat;
-import org.apache.storm.generated.TopologyPageInfo;
-import org.apache.storm.generated.TopologyStats;
-import org.apache.storm.generated.WorkerResources;
-import org.apache.storm.generated.WorkerSummary;
-import org.apache.storm.scheduler.WorkerSlot;
-import org.apache.storm.shade.com.google.common.collect.Lists;
-import org.apache.storm.utils.Time;
-import org.apache.storm.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@SuppressWarnings("unchecked")
-public class StatsUtil {
-    public static final String TYPE = "type";
-    public static final String SPOUT = "spout";
-    public static final String BOLT = "bolt";
-    public static final int TEN_MIN_IN_SECONDS = 60 * 10;
-    public static final String TEN_MIN_IN_SECONDS_STR = TEN_MIN_IN_SECONDS + "";
-    public static final IdentityTransformer IDENTITY = new IdentityTransformer();
-    public static final ToGlobalStreamIdTransformer TO_GSID = new ToGlobalStreamIdTransformer();
-    private static final Logger LOG = LoggerFactory.getLogger(StatsUtil.class);
-    private static final String UPTIME = "uptime";
-    private static final String HOST = "host";
-    private static final String PORT = "port";
-    private static final String NUM_TASKS = "num-tasks";
-    private static final String NUM_EXECUTORS = "num-executors";
-    private static final String NUM_WORKERS = "num-workers";
-    private static final String CAPACITY = "capacity";
-    private static final String STATS = "stats";
-    private static final String EXECUTOR_STATS = "executor-stats";
-    private static final String EXECUTOR_ID = "executor-id";
-    private static final String LAST_ERROR = "lastError";
-    private static final String HEARTBEAT = "heartbeat";
-    private static final String TIME_SECS = "time-secs";
-    private static final String RATE = "rate";
-    private static final String ACKED = "acked";
-    private static final String FAILED = "failed";
-    private static final String EXECUTED = "executed";
-    private static final String EMITTED = "emitted";
-    private static final String TRANSFERRED = "transferred";
-    private static final String EXEC_LATENCIES = "execute-latencies";
-    private static final String PROC_LATENCIES = "process-latencies";
-    private static final String COMP_LATENCIES = "complete-latencies";
-    private static final String EXEC_LATENCY = "execute-latency";
-    private static final String PROC_LATENCY = "process-latency";
-    private static final String COMP_LATENCY = "complete-latency";
-    private static final String EXEC_LAT_TOTAL = "executeLatencyTotal";
-    private static final String PROC_LAT_TOTAL = "processLatencyTotal";
-    private static final String COMP_LAT_TOTAL = "completeLatencyTotal";
-    private static final String WIN_TO_EMITTED = "window->emitted";
-    private static final String WIN_TO_ACKED = "window->acked";
-    private static final String WIN_TO_FAILED = "window->failed";
-    private static final String WIN_TO_EXECUTED = "window->executed";
-    private static final String WIN_TO_TRANSFERRED = "window->transferred";
-    private static final String WIN_TO_EXEC_LAT = "window->execute-latency";
-    private static final String WIN_TO_PROC_LAT = "window->process-latency";
-    private static final String WIN_TO_COMP_LAT = "window->complete-latency";
-    private static final String WIN_TO_COMP_LAT_WGT_AVG = "window->comp-lat-wgt-avg";
-    private static final String WIN_TO_EXEC_LAT_WGT_AVG = "window->exec-lat-wgt-avg";
-    private static final String WIN_TO_PROC_LAT_WGT_AVG = "window->proc-lat-wgt-avg";
-    private static final String BOLT_TO_STATS = "bolt-id->stats";
-    private static final String SPOUT_TO_STATS = "spout-id->stats";
-    private static final String SID_TO_OUT_STATS = "sid->output-stats";
-    private static final String CID_SID_TO_IN_STATS = "cid+sid->input-stats";
-    private static final String WORKERS_SET = "workers-set";
-    private static final ToStringTransformer TO_STRING = new ToStringTransformer();
-    private static final FromGlobalStreamIdTransformer FROM_GSID = new FromGlobalStreamIdTransformer();
-
-
-    // =====================================================================================
-    // aggregation stats methods
-    // =====================================================================================
-
-    /**
-     * Aggregates number executed, process latency, and execute latency across all streams.
-     *
-     * @param id2execAvg { global stream id -> exec avg value }, e.g., {["split" "default"] 0.44313}
-     * @param id2procAvg { global stream id -> proc avg value }
-     * @param id2numExec { global stream id -> executed }
-     */
-    public static Map<String, Number> aggBoltLatAndCount(Map<List<String>, Double> id2execAvg,
-                                                         Map<List<String>, Double> id2procAvg,
-                                                         Map<List<String>, Long> id2numExec) {
-        Map<String, Number> ret = new HashMap<>();
-        putKV(ret, EXEC_LAT_TOTAL, weightAvgAndSum(id2execAvg, id2numExec));
-        putKV(ret, PROC_LAT_TOTAL, weightAvgAndSum(id2procAvg, id2numExec));
-        putKV(ret, EXECUTED, sumValues(id2numExec));
-
-        return ret;
-    }
-
-    /**
-     * aggregate number acked and complete latencies across all streams.
-     */
-    public static Map<String, Number> aggSpoutLatAndCount(Map<String, Double> id2compAvg,
-                                                          Map<String, Long> id2numAcked) {
-        Map<String, Number> ret = new HashMap<>();
-        putKV(ret, COMP_LAT_TOTAL, weightAvgAndSum(id2compAvg, id2numAcked));
-        putKV(ret, ACKED, sumValues(id2numAcked));
-
-        return ret;
-    }
-
-    /**
-     * aggregate number executed and process & execute latencies.
-     */
-    public static <K> Map<K, Map> aggBoltStreamsLatAndCount(Map<K, Double> id2execAvg,
-                                                            Map<K, Double> id2procAvg,
-                                                            Map<K, Long> id2numExec) {
-        Map<K, Map> ret = new HashMap<>();
-        if (id2execAvg == null || id2procAvg == null || id2numExec == null) {
-            return ret;
-        }
-        for (K k : id2execAvg.keySet()) {
-            Map<String, Object> subMap = new HashMap<>();
-            putKV(subMap, EXEC_LAT_TOTAL, weightAvg(id2execAvg, id2numExec, k));
-            putKV(subMap, PROC_LAT_TOTAL, weightAvg(id2procAvg, id2numExec, k));
-            putKV(subMap, EXECUTED, id2numExec.get(k));
-            ret.put(k, subMap);
-        }
-        return ret;
-    }
-
-    /**
-     * Aggregates number acked and complete latencies.
-     */
-    public static <K> Map<K, Map> aggSpoutStreamsLatAndCount(Map<K, Double> id2compAvg,
-                                                             Map<K, Long> id2acked) {
-        Map<K, Map> ret = new HashMap<>();
-        if (id2compAvg == null || id2acked == null) {
-            return ret;
-        }
-        for (K k : id2compAvg.keySet()) {
-            Map subMap = new HashMap();
-            putKV(subMap, COMP_LAT_TOTAL, weightAvg(id2compAvg, id2acked, k));
-            putKV(subMap, ACKED, id2acked.get(k));
-            ret.put(k, subMap);
-        }
-        return ret;
-    }
-
-    /**
-     * pre-merge component page bolt stats from an executor heartbeat 1. computes component capacity 2. converts map keys of stats 3.
-     * filters streams if necessary
-     *
-     * @param beat       executor heartbeat data
-     * @param window     specified window
-     * @param includeSys whether to include system streams
-     * @return per-merged stats
-     */
-    public static Map<String, Object> aggPreMergeCompPageBolt(Map<String, Object> beat, String window, boolean includeSys) {
-        Map<String, Object> ret = new HashMap<>();
-
-        putKV(ret, EXECUTOR_ID, getByKey(beat, "exec-id"));
-        putKV(ret, HOST, getByKey(beat, HOST));
-        putKV(ret, PORT, getByKey(beat, PORT));
-        putKV(ret, UPTIME, getByKey(beat, UPTIME));
-        putKV(ret, NUM_EXECUTORS, 1);
-        putKV(ret, NUM_TASKS, getByKey(beat, NUM_TASKS));
-
-        Map stat2win2sid2num = getMapByKey(beat, STATS);
-        putKV(ret, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeyOr0(beat, UPTIME).intValue()));
-
-        // calc cid+sid->input_stats
-        Map inputStats = new HashMap();
-        Map sid2acked = (Map) windowSetConverter(getMapByKey(stat2win2sid2num, ACKED), TO_STRING).get(window);
-        Map sid2failed = (Map) windowSetConverter(getMapByKey(stat2win2sid2num, FAILED), TO_STRING).get(window);
-        putKV(inputStats, ACKED, sid2acked != null ? sid2acked : new HashMap());
-        putKV(inputStats, FAILED, sid2failed != null ? sid2failed : new HashMap());
-
-        inputStats = swapMapOrder(inputStats);
-
-        Map sid2execLat = (Map) windowSetConverter(getMapByKey(stat2win2sid2num, EXEC_LATENCIES), TO_STRING).get(window);
-        Map sid2procLat = (Map) windowSetConverter(getMapByKey(stat2win2sid2num, PROC_LATENCIES), TO_STRING).get(window);
-        Map sid2exec = (Map) windowSetConverter(getMapByKey(stat2win2sid2num, EXECUTED), TO_STRING).get(window);
-        mergeMaps(inputStats, aggBoltStreamsLatAndCount(sid2execLat, sid2procLat, sid2exec));
-        putKV(ret, CID_SID_TO_IN_STATS, inputStats);
-
-        // calc sid->output_stats
-        Map outputStats = new HashMap();
-        Map sid2emitted = (Map) windowSetConverter(getMapByKey(stat2win2sid2num, EMITTED), TO_STRING).get(window);
-        Map sid2transferred = (Map) windowSetConverter(getMapByKey(stat2win2sid2num, TRANSFERRED), TO_STRING).get(window);
-        if (sid2emitted != null) {
-            putKV(outputStats, EMITTED, filterSysStreams2Stat(sid2emitted, includeSys));
-        } else {
-            putKV(outputStats, EMITTED, new HashMap());
-        }
-        if (sid2transferred != null) {
-            putKV(outputStats, TRANSFERRED, filterSysStreams2Stat(sid2transferred, includeSys));
-        } else {
-            putKV(outputStats, TRANSFERRED, new HashMap());
-        }
-        outputStats = swapMapOrder(outputStats);
-        putKV(ret, SID_TO_OUT_STATS, outputStats);
-
-        return ret;
-    }
-
-    /**
-     * pre-merge component page spout stats from an executor heartbeat 1. computes component capacity 2. converts map keys of stats 3.
-     * filters streams if necessary
-     *
-     * @param beat       executor heartbeat data
-     * @param window     specified window
-     * @param includeSys whether to include system streams
-     * @return per-merged stats
-     */
-    public static Map<String, Object> aggPreMergeCompPageSpout(Map<String, Object> beat, String window, boolean includeSys) {
-        Map<String, Object> ret = new HashMap<>();
-        putKV(ret, EXECUTOR_ID, getByKey(beat, "exec-id"));
-        putKV(ret, HOST, getByKey(beat, HOST));
-        putKV(ret, PORT, getByKey(beat, PORT));
-        putKV(ret, UPTIME, getByKey(beat, UPTIME));
-        putKV(ret, NUM_EXECUTORS, 1);
-        putKV(ret, NUM_TASKS, getByKey(beat, NUM_TASKS));
-
-        Map stat2win2sid2num = getMapByKey(beat, STATS);
-
-        // calc sid->output-stats
-        Map outputStats = new HashMap();
-        Map win2sid2acked = windowSetConverter(getMapByKey(stat2win2sid2num, ACKED), TO_STRING);
-        Map win2sid2failed = windowSetConverter(getMapByKey(stat2win2sid2num, FAILED), TO_STRING);
-        Map win2sid2emitted = windowSetConverter(getMapByKey(stat2win2sid2num, EMITTED), TO_STRING);
-        Map win2sid2transferred = windowSetConverter(getMapByKey(stat2win2sid2num, TRANSFERRED), TO_STRING);
-        Map win2sid2compLat = windowSetConverter(getMapByKey(stat2win2sid2num, COMP_LATENCIES), TO_STRING);
-
-        putKV(outputStats, ACKED, win2sid2acked.get(window));
-        putKV(outputStats, FAILED, win2sid2failed.get(window));
-        Map<String, Long> sid2emitted = (Map) win2sid2emitted.get(window);
-        if (sid2emitted == null) {
-            sid2emitted = new HashMap<>();
-        }
-        putKV(outputStats, EMITTED, filterSysStreams2Stat(sid2emitted, includeSys));
-
-        Map<String, Long> sid2transferred = (Map) win2sid2transferred.get(window);
-        if (sid2transferred == null) {
-            sid2transferred = new HashMap<>();
-        }
-        putKV(outputStats, TRANSFERRED, filterSysStreams2Stat(sid2transferred, includeSys));
-        outputStats = swapMapOrder(outputStats);
-
-        Map sid2compLat = (Map) win2sid2compLat.get(window);
-        Map sid2acked = (Map) win2sid2acked.get(window);
-        mergeMaps(outputStats, aggSpoutStreamsLatAndCount(sid2compLat, sid2acked));
-        putKV(ret, SID_TO_OUT_STATS, outputStats);
-
-        return ret;
-    }
-
-    /**
-     * pre-merge component stats of specified bolt id
-     *
-     * @param beat       executor heartbeat data
-     * @param window     specified window
-     * @param includeSys whether to include system streams
-     * @return { comp id -> comp-stats }
-     */
-    public static <K, V extends Number> Map<String, Object> aggPreMergeTopoPageBolt(
-        Map<String, Object> beat, String window, boolean includeSys) {
-        Map<String, Object> ret = new HashMap<>();
-
-        Map<String, Object> subRet = new HashMap<>();
-        putKV(subRet, NUM_EXECUTORS, 1);
-        putKV(subRet, NUM_TASKS, getByKey(beat, NUM_TASKS));
-
-        Map<String, Object> stat2win2sid2num = getMapByKey(beat, STATS);
-        putKV(subRet, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeyOr0(beat, UPTIME).intValue()));
-
-        for (String key : new String[]{ EMITTED, TRANSFERRED, ACKED, FAILED }) {
-            Map<String, Map<K, V>> stat = windowSetConverter(getMapByKey(stat2win2sid2num, key), TO_STRING);
-            if (EMITTED.equals(key) || TRANSFERRED.equals(key)) {
-                stat = filterSysStreams(stat, includeSys);
-            }
-            Map<K, V> winStat = stat.get(window);
-            long sum = 0;
-            if (winStat != null) {
-                for (V v : winStat.values()) {
-                    sum += v.longValue();
-                }
-            }
-            putKV(subRet, key, sum);
-        }
-
-        Map<String, Map<List<String>, Double>> win2sid2execLat =
-            windowSetConverter(getMapByKey(stat2win2sid2num, EXEC_LATENCIES), TO_STRING);
-        Map<String, Map<List<String>, Double>> win2sid2procLat =
-            windowSetConverter(getMapByKey(stat2win2sid2num, PROC_LATENCIES), TO_STRING);
-        Map<String, Map<List<String>, Long>> win2sid2exec =
-            windowSetConverter(getMapByKey(stat2win2sid2num, EXECUTED), TO_STRING);
-        subRet.putAll(aggBoltLatAndCount(
-            win2sid2execLat.get(window), win2sid2procLat.get(window), win2sid2exec.get(window)));
-
-        ret.put((String) getByKey(beat, "comp-id"), subRet);
-        return ret;
-    }
-
-    /**
-     * pre-merge component stats of specified spout id and returns { comp id -> comp-stats }
-     */
-    public static <K, V extends Number> Map<String, Object> aggPreMergeTopoPageSpout(
-        Map<String, Object> m, String window, boolean includeSys) {
-        Map<String, Object> ret = new HashMap<>();
-
-        Map<String, Object> subRet = new HashMap<>();
-        putKV(subRet, NUM_EXECUTORS, 1);
-        putKV(subRet, NUM_TASKS, getByKey(m, NUM_TASKS));
-
-        // no capacity for spout
-        Map<String, Map<String, Map<String, V>>> stat2win2sid2num = getMapByKey(m, STATS);
-        for (String key : new String[]{ EMITTED, TRANSFERRED, FAILED }) {
-            Map<String, Map<K, V>> stat = windowSetConverter(stat2win2sid2num.get(key), TO_STRING);
-            if (EMITTED.equals(key) || TRANSFERRED.equals(key)) {
-                stat = filterSysStreams(stat, includeSys);
-            }
-            Map<K, V> winStat = stat.get(window);
-            long sum = 0;
-            if (winStat != null) {
-                for (V v : winStat.values()) {
-                    sum += v.longValue();
-                }
-            }
-            putKV(subRet, key, sum);
-        }
-
-        Map<String, Map<String, Double>> win2sid2compLat =
-            windowSetConverter(getMapByKey(stat2win2sid2num, COMP_LATENCIES), TO_STRING);
-        Map<String, Map<String, Long>> win2sid2acked =
-            windowSetConverter(getMapByKey(stat2win2sid2num, ACKED), TO_STRING);
-        subRet.putAll(aggSpoutLatAndCount(win2sid2compLat.get(window), win2sid2acked.get(window)));
-
-        ret.put((String) getByKey(m, "comp-id"), subRet);
-        return ret;
-    }
-
-    /**
-     * merge accumulated bolt stats with pre-merged component stats
-     *
-     * @param accBoltStats accumulated bolt stats
-     * @param boltStats    pre-merged component stats
-     * @return merged stats
-     */
-    public static Map<String, Object> mergeAggCompStatsCompPageBolt(
-        Map<String, Object> accBoltStats, Map<String, Object> boltStats) {
-        Map<String, Object> ret = new HashMap<>();
-
-        Map<List<String>, Map<String, ?>> accIn = getMapByKey(accBoltStats, CID_SID_TO_IN_STATS);
-        Map<String, Map<String, ?>> accOut = getMapByKey(accBoltStats, SID_TO_OUT_STATS);
-        Map<List<String>, Map<String, ?>> boltIn = getMapByKey(boltStats, CID_SID_TO_IN_STATS);
-        Map<String, Map<String, ?>> boltOut = getMapByKey(boltStats, SID_TO_OUT_STATS);
-
-        int numExecutors = getByKeyOr0(accBoltStats, NUM_EXECUTORS).intValue();
-        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
-        putKV(ret, NUM_TASKS, sumOr0(
-            getByKeyOr0(accBoltStats, NUM_TASKS), getByKeyOr0(boltStats, NUM_TASKS)));
-
-        // (merge-with (partial merge-with sum-or-0) acc-out spout-out)
-        putKV(ret, SID_TO_OUT_STATS, fullMergeWithSum(accOut, boltOut));
-        // {component id -> metric -> value}, note that input may contain both long and double values
-        putKV(ret, CID_SID_TO_IN_STATS, fullMergeWithSum(accIn, boltIn));
-
-        long executed = sumStreamsLong(boltIn, EXECUTED);
-        putKV(ret, EXECUTED, executed);
-
-        Map<String, Object> executorStats = new HashMap<>();
-        putKV(executorStats, EXECUTOR_ID, boltStats.get(EXECUTOR_ID));
-        putKV(executorStats, UPTIME, boltStats.get(UPTIME));
-        putKV(executorStats, HOST, boltStats.get(HOST));
-        putKV(executorStats, PORT, boltStats.get(PORT));
-        putKV(executorStats, CAPACITY, boltStats.get(CAPACITY));
-
-        putKV(executorStats, EMITTED, sumStreamsLong(boltOut, EMITTED));
-        putKV(executorStats, TRANSFERRED, sumStreamsLong(boltOut, TRANSFERRED));
-        putKV(executorStats, ACKED, sumStreamsLong(boltIn, ACKED));
-        putKV(executorStats, FAILED, sumStreamsLong(boltIn, FAILED));
-        putKV(executorStats, EXECUTED, executed);
-
-        if (executed > 0) {
-            putKV(executorStats, EXEC_LATENCY, sumStreamsDouble(boltIn, EXEC_LAT_TOTAL) / executed);
-            putKV(executorStats, PROC_LATENCY, sumStreamsDouble(boltIn, PROC_LAT_TOTAL) / executed);
-        } else {
-            putKV(executorStats, EXEC_LATENCY, null);
-            putKV(executorStats, PROC_LATENCY, null);
-        }
-        List executorStatsList = ((List) getByKey(accBoltStats, EXECUTOR_STATS));
-        executorStatsList.add(executorStats);
-        putKV(ret, EXECUTOR_STATS, executorStatsList);
-
-        return ret;
-    }
-
-    /**
-     * merge accumulated bolt stats with pre-merged component stats
-     */
-    public static Map<String, Object> mergeAggCompStatsCompPageSpout(
-        Map<String, Object> accSpoutStats, Map<String, Object> spoutStats) {
-        Map<String, Object> ret = new HashMap<>();
-
-        // {stream id -> metric -> value}, note that sid->out-stats may contain both long and double values
-        Map<String, Map<String, ?>> accOut = getMapByKey(accSpoutStats, SID_TO_OUT_STATS);
-        Map<String, Map<String, ?>> spoutOut = getMapByKey(spoutStats, SID_TO_OUT_STATS);
-
-        int numExecutors = getByKeyOr0(accSpoutStats, NUM_EXECUTORS).intValue();
-        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
-        putKV(ret, NUM_TASKS, sumOr0(
-            getByKeyOr0(accSpoutStats, NUM_TASKS), getByKeyOr0(spoutStats, NUM_TASKS)));
-        putKV(ret, SID_TO_OUT_STATS, fullMergeWithSum(accOut, spoutOut));
-
-        Map executorStats = new HashMap();
-        putKV(executorStats, EXECUTOR_ID, getByKey(spoutStats, EXECUTOR_ID));
-        putKV(executorStats, UPTIME, getByKey(spoutStats, UPTIME));
-        putKV(executorStats, HOST, getByKey(spoutStats, HOST));
-        putKV(executorStats, PORT, getByKey(spoutStats, PORT));
-
-        putKV(executorStats, EMITTED, sumStreamsLong(spoutOut, EMITTED));
-        putKV(executorStats, TRANSFERRED, sumStreamsLong(spoutOut, TRANSFERRED));
-        putKV(executorStats, FAILED, sumStreamsLong(spoutOut, FAILED));
-        long acked = sumStreamsLong(spoutOut, ACKED);
-        putKV(executorStats, ACKED, acked);
-        if (acked > 0) {
-            putKV(executorStats, COMP_LATENCY, sumStreamsDouble(spoutOut, COMP_LAT_TOTAL) / acked);
-        } else {
-            putKV(executorStats, COMP_LATENCY, null);
-        }
-        List executorStatsList = ((List) getByKey(accSpoutStats, EXECUTOR_STATS));
-        executorStatsList.add(executorStats);
-        putKV(ret, EXECUTOR_STATS, executorStatsList);
-
-        return ret;
-    }
-
-    /**
-     * merge accumulated bolt stats with new bolt stats
-     *
-     * @param accBoltStats accumulated bolt stats
-     * @param boltStats    new input bolt stats
-     * @return merged bolt stats
-     */
-    public static Map<String, Object> mergeAggCompStatsTopoPageBolt(Map<String, Object> accBoltStats,
-                                                                    Map<String, Object> boltStats) {
-        Map<String, Object> ret = new HashMap<>();
-
-        Integer numExecutors = getByKeyOr0(accBoltStats, NUM_EXECUTORS).intValue();
-        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
-        putKV(ret, NUM_TASKS,
-              sumOr0(getByKeyOr0(accBoltStats, NUM_TASKS), getByKeyOr0(boltStats, NUM_TASKS)));
-        putKV(ret, EMITTED,
-              sumOr0(getByKeyOr0(accBoltStats, EMITTED), getByKeyOr0(boltStats, EMITTED)));
-        putKV(ret, TRANSFERRED,
-              sumOr0(getByKeyOr0(accBoltStats, TRANSFERRED), getByKeyOr0(boltStats, TRANSFERRED)));
-        putKV(ret, EXEC_LAT_TOTAL,
-              sumOr0(getByKeyOr0(accBoltStats, EXEC_LAT_TOTAL), getByKeyOr0(boltStats, EXEC_LAT_TOTAL)));
-        putKV(ret, PROC_LAT_TOTAL,
-              sumOr0(getByKeyOr0(accBoltStats, PROC_LAT_TOTAL), getByKeyOr0(boltStats, PROC_LAT_TOTAL)));
-        putKV(ret, EXECUTED,
-              sumOr0(getByKeyOr0(accBoltStats, EXECUTED), getByKeyOr0(boltStats, EXECUTED)));
-        putKV(ret, ACKED,
-              sumOr0(getByKeyOr0(accBoltStats, ACKED), getByKeyOr0(boltStats, ACKED)));
-        putKV(ret, FAILED,
-              sumOr0(getByKeyOr0(accBoltStats, FAILED), getByKeyOr0(boltStats, FAILED)));
-        putKV(ret, CAPACITY,
-              maxOr0(getByKeyOr0(accBoltStats, CAPACITY), getByKeyOr0(boltStats, CAPACITY)));
-
-        return ret;
-    }
-
-    /**
-     * merge accumulated bolt stats with new bolt stats
-     */
-    public static Map<String, Object> mergeAggCompStatsTopoPageSpout(Map<String, Object> accSpoutStats,
-                                                                     Map<String, Object> spoutStats) {
-        Map<String, Object> ret = new HashMap<>();
-
-        Integer numExecutors = getByKeyOr0(accSpoutStats, NUM_EXECUTORS).intValue();
-        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
-        putKV(ret, NUM_TASKS,
-              sumOr0(getByKeyOr0(accSpoutStats, NUM_TASKS), getByKeyOr0(spoutStats, NUM_TASKS)));
-        putKV(ret, EMITTED,
-              sumOr0(getByKeyOr0(accSpoutStats, EMITTED), getByKeyOr0(spoutStats, EMITTED)));
-        putKV(ret, TRANSFERRED,
-              sumOr0(getByKeyOr0(accSpoutStats, TRANSFERRED), getByKeyOr0(spoutStats, TRANSFERRED)));
-        putKV(ret, COMP_LAT_TOTAL,
-              sumOr0(getByKeyOr0(accSpoutStats, COMP_LAT_TOTAL), getByKeyOr0(spoutStats, COMP_LAT_TOTAL)));
-        putKV(ret, ACKED,
-              sumOr0(getByKeyOr0(accSpoutStats, ACKED), getByKeyOr0(spoutStats, ACKED)));
-        putKV(ret, FAILED,
-              sumOr0(getByKeyOr0(accSpoutStats, FAILED), getByKeyOr0(spoutStats, FAILED)));
-
-        return ret;
-    }
-
-    /**
-     * A helper function that does the common work to aggregate stats of one executor with the given map for the topology page.
-     */
-    public static Map<String, Object> aggTopoExecStats(
-        String window, boolean includeSys, Map<String, Object> accStats, Map<String, Object> beat, String compType) {
-        Map<String, Object> ret = new HashMap<>();
-
-        Set workerSet = (Set) accStats.get(WORKERS_SET);
-        Map bolt2stats = getMapByKey(accStats, BOLT_TO_STATS);
-        Map spout2stats = getMapByKey(accStats, SPOUT_TO_STATS);
-        Map win2emitted = getMapByKey(accStats, WIN_TO_EMITTED);
-        Map win2transferred = getMapByKey(accStats, WIN_TO_TRANSFERRED);
-        Map win2compLatWgtAvg = getMapByKey(accStats, WIN_TO_COMP_LAT_WGT_AVG);
-        Map win2acked = getMapByKey(accStats, WIN_TO_ACKED);
-        Map win2failed = getMapByKey(accStats, WIN_TO_FAILED);
-
-        boolean isSpout = compType.equals(SPOUT);
-        // component id -> stats
-        Map<String, Object> cid2stats;
-        if (isSpout) {
-            cid2stats = aggPreMergeTopoPageSpout(beat, window, includeSys);
-        } else {
-            cid2stats = aggPreMergeTopoPageBolt(beat, window, includeSys);
-        }
-
-        Map stats = getMapByKey(beat, STATS);
-        Map w2compLatWgtAvg, w2acked;
-        Map compLatStats = getMapByKey(stats, COMP_LATENCIES);
-        if (isSpout) { // agg spout stats
-            Map mm = new HashMap();
-
-            Map acked = getMapByKey(stats, ACKED);
-            for (Object win : acked.keySet()) {
-                mm.put(win, aggSpoutLatAndCount((Map) compLatStats.get(win), (Map) acked.get(win)));
-            }
-            mm = swapMapOrder(mm);
-            w2compLatWgtAvg = getMapByKey(mm, COMP_LAT_TOTAL);
-            w2acked = getMapByKey(mm, ACKED);
-        } else {
-            w2compLatWgtAvg = null;
-            w2acked = aggregateCountStreams(getMapByKey(stats, ACKED));
-        }
-
-        workerSet.add(Lists.newArrayList(getByKey(beat, HOST), getByKey(beat, PORT)));
-        putKV(ret, WORKERS_SET, workerSet);
-        putKV(ret, BOLT_TO_STATS, bolt2stats);
-        putKV(ret, SPOUT_TO_STATS, spout2stats);
-        putKV(ret, WIN_TO_EMITTED, mergeWithSumLong(win2emitted, aggregateCountStreams(
-            filterSysStreams(getMapByKey(stats, EMITTED), includeSys))));
-        putKV(ret, WIN_TO_TRANSFERRED, mergeWithSumLong(win2transferred, aggregateCountStreams(
-            filterSysStreams(getMapByKey(stats, TRANSFERRED), includeSys))));
-        putKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSumDouble(win2compLatWgtAvg, w2compLatWgtAvg));
-
-        //boolean isSpoutStat = SPOUT.equals(((Keyword) getByKey(stats, TYPE)).getName());
-        putKV(ret, WIN_TO_ACKED, isSpout ? mergeWithSumLong(win2acked, w2acked) : win2acked);
-        putKV(ret, WIN_TO_FAILED, isSpout ?
-            mergeWithSumLong(aggregateCountStreams(getMapByKey(stats, FAILED)), win2failed) : win2failed);
-        putKV(ret, TYPE, getByKey(stats, TYPE));
-
-        // (merge-with merge-agg-comp-stats-topo-page-bolt/spout (acc-stats comp-key) cid->statk->num)
-        // (acc-stats comp-key) ==> bolt2stats/spout2stats
-        if (isSpout) {
-            for (String spout : cid2stats.keySet()) {
-                spout2stats.put(spout, mergeAggCompStatsTopoPageSpout((Map) spout2stats.get(spout), (Map) cid2stats.get(spout)));
-            }
-        } else {
-            for (String bolt : cid2stats.keySet()) {
-                bolt2stats.put(bolt, mergeAggCompStatsTopoPageBolt((Map) bolt2stats.get(bolt), (Map) cid2stats.get(bolt)));
-            }
-        }
-
-        return ret;
-    }
-
-    /**
-     * aggregate topo executors stats
-     *
-     * @param topologyId     topology id
-     * @param exec2nodePort  executor -> host+port
-     * @param task2component task -> component
-     * @param beats          executor[start, end] -> executor heartbeat
-     * @param topology       storm topology
-     * @param window         the window to be aggregated
-     * @param includeSys     whether to include system streams
-     * @param clusterState   cluster state
-     * @return TopologyPageInfo thrift structure
-     */
-    public static TopologyPageInfo aggTopoExecsStats(
-        String topologyId, Map exec2nodePort, Map task2component, Map<List<Integer>, Map<String, Object>> beats,
-        StormTopology topology, String window, boolean includeSys, IStormClusterState clusterState) {
-        List<Map<String, Object>> beatList = extractDataFromHb(exec2nodePort, task2component, beats, includeSys, topology);
-        Map<String, Object> topoStats = aggregateTopoStats(window, includeSys, beatList);
-        return postAggregateTopoStats(task2component, exec2nodePort, topoStats, topologyId, clusterState);
-    }
-
-    public static Map<String, Object> aggregateTopoStats(String win, boolean includeSys, List<Map<String, Object>> heartbeats) {
-        Map<String, Object> initVal = new HashMap<>();
-        putKV(initVal, WORKERS_SET, new HashSet());
-        putKV(initVal, BOLT_TO_STATS, new HashMap());
-        putKV(initVal, SPOUT_TO_STATS, new HashMap());
-        putKV(initVal, WIN_TO_EMITTED, new HashMap());
-        putKV(initVal, WIN_TO_TRANSFERRED, new HashMap());
-        putKV(initVal, WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
-        putKV(initVal, WIN_TO_ACKED, new HashMap());
-        putKV(initVal, WIN_TO_FAILED, new HashMap());
-
-        for (Map<String, Object> heartbeat : heartbeats) {
-            String compType = (String) getByKey(heartbeat, TYPE);
-            initVal = aggTopoExecStats(win, includeSys, initVal, heartbeat, compType);
-        }
-
-        return initVal;
-    }
-
-    public static TopologyPageInfo postAggregateTopoStats(Map task2comp, Map exec2nodePort, Map<String, Object> accData,
-                                                          String topologyId, IStormClusterState clusterState) {
-        TopologyPageInfo ret = new TopologyPageInfo(topologyId);
-
-        ret.set_num_tasks(task2comp.size());
-        ret.set_num_workers(((Set) getByKey(accData, WORKERS_SET)).size());
-        ret.set_num_executors(exec2nodePort != null ? exec2nodePort.size() : 0);
-
-        Map bolt2stats = getMapByKey(accData, BOLT_TO_STATS);
-        Map<String, ComponentAggregateStats> aggBolt2stats = new HashMap<>();
-        for (Object o : bolt2stats.entrySet()) {
-            Map.Entry e = (Map.Entry) o;
-            String id = (String) e.getKey();
-            Map m = (Map) e.getValue();
-            long executed = getByKeyOr0(m, EXECUTED).longValue();
-            if (executed > 0) {
-                double execLatencyTotal = getByKeyOr0(m, EXEC_LAT_TOTAL).doubleValue();
-                putKV(m, EXEC_LATENCY, execLatencyTotal / executed);
-
-                double procLatencyTotal = getByKeyOr0(m, PROC_LAT_TOTAL).doubleValue();
-                putKV(m, PROC_LATENCY, procLatencyTotal / executed);
-            }
-            remove(m, EXEC_LAT_TOTAL);
-            remove(m, PROC_LAT_TOTAL);
-            putKV(m, "last-error", getLastError(clusterState, topologyId, id));
-
-            aggBolt2stats.put(id, thriftifyBoltAggStats(m));
-        }
-
-        Map spout2stats = getMapByKey(accData, SPOUT_TO_STATS);
-        Map<String, ComponentAggregateStats> aggSpout2stats = new HashMap<>();
-        for (Object o : spout2stats.entrySet()) {
-            Map.Entry e = (Map.Entry) o;
-            String id = (String) e.getKey();
-            Map m = (Map) e.getValue();
-            long acked = getByKeyOr0(m, ACKED).longValue();
-            if (acked > 0) {
-                double compLatencyTotal = getByKeyOr0(m, COMP_LAT_TOTAL).doubleValue();
-                putKV(m, COMP_LATENCY, compLatencyTotal / acked);
-            }
-            remove(m, COMP_LAT_TOTAL);
-            putKV(m, "last-error", getLastError(clusterState, topologyId, id));
-
-            aggSpout2stats.put(id, thriftifySpoutAggStats(m));
-        }
-
-        TopologyStats topologyStats = new TopologyStats();
-        topologyStats.set_window_to_acked(mapKeyStr(getMapByKey(accData, WIN_TO_ACKED)));
-        topologyStats.set_window_to_emitted(mapKeyStr(getMapByKey(accData, WIN_TO_EMITTED)));
-        topologyStats.set_window_to_failed(mapKeyStr(getMapByKey(accData, WIN_TO_FAILED)));
-        topologyStats.set_window_to_transferred(mapKeyStr(getMapByKey(accData, WIN_TO_TRANSFERRED)));
-        topologyStats.set_window_to_complete_latencies_ms(computeWeightedAveragesPerWindow(
-            accData, WIN_TO_COMP_LAT_WGT_AVG, WIN_TO_ACKED));
-
-        ret.set_topology_stats(topologyStats);
-        ret.set_id_to_spout_agg_stats(aggSpout2stats);
-        ret.set_id_to_bolt_agg_stats(aggBolt2stats);
-
-        return ret;
-    }
-
-    /**
-     * aggregate bolt stats
-     *
-     * @param statsSeq   a seq of ExecutorStats
-     * @param includeSys whether to include system streams
-     * @return aggregated bolt stats: {metric -> win -> global stream id -> value}
-     */
-    public static <T> Map<String, Map> aggregateBoltStats(List<ExecutorSummary> statsSeq, boolean includeSys) {
-        Map<String, Map> ret = new HashMap<>();
-
-        Map<String, Map<String, Map<T, Long>>> commonStats = aggregateCommonStats(statsSeq);
-        // filter sys streams if necessary
-        commonStats = preProcessStreamSummary(commonStats, includeSys);
-
-        List<Map<String, Map<GlobalStreamId, Long>>> acked = new ArrayList<>();
-        List<Map<String, Map<GlobalStreamId, Long>>> failed = new ArrayList<>();
-        List<Map<String, Map<GlobalStreamId, Long>>> executed = new ArrayList<>();
-        List<Map<String, Map<GlobalStreamId, Double>>> processLatencies = new ArrayList<>();
-        List<Map<String, Map<GlobalStreamId, Double>>> executeLatencies = new ArrayList<>();
-        for (ExecutorSummary summary : statsSeq) {
-            ExecutorStats stat = summary.get_stats();
-            acked.add(stat.get_specific().get_bolt().get_acked());
-            failed.add(stat.get_specific().get_bolt().get_failed());
-            executed.add(stat.get_specific().get_bolt().get_executed());
-            processLatencies.add(stat.get_specific().get_bolt().get_process_ms_avg());
-            executeLatencies.add(stat.get_specific().get_bolt().get_execute_ms_avg());
-        }
-        mergeMaps(ret, commonStats);
-        putKV(ret, ACKED, aggregateCounts(acked));
-        putKV(ret, FAILED, aggregateCounts(failed));
-        putKV(ret, EXECUTED, aggregateCounts(executed));
-        putKV(ret, PROC_LATENCIES, aggregateAverages(processLatencies, acked));
-        putKV(ret, EXEC_LATENCIES, aggregateAverages(executeLatencies, executed));
-
-        return ret;
-    }
-
-    /**
-     * aggregate spout stats
-     *
-     * @param statsSeq   a seq of ExecutorStats
-     * @param includeSys whether to include system streams
-     * @return aggregated spout stats: {metric -> win -> global stream id -> value}
-     */
-    public static Map<String, Map> aggregateSpoutStats(List<ExecutorSummary> statsSeq, boolean includeSys) {
-        // actually Map<String, Map<String, Map<String, Long/Double>>>
-        Map<String, Map> ret = new HashMap<>();
-
-        Map<String, Map<String, Map<String, Long>>> commonStats = aggregateCommonStats(statsSeq);
-        // filter sys streams if necessary
-        commonStats = preProcessStreamSummary(commonStats, includeSys);
-
-        List<Map<String, Map<String, Long>>> acked = new ArrayList<>();
-        List<Map<String, Map<String, Long>>> failed = new ArrayList<>();
-        List<Map<String, Map<String, Double>>> completeLatencies = new ArrayList<>();
-        for (ExecutorSummary summary : statsSeq) {
-            ExecutorStats stats = summary.get_stats();
-            acked.add(stats.get_specific().get_spout().get_acked());
-            failed.add(stats.get_specific().get_spout().get_failed());
-            completeLatencies.add(stats.get_specific().get_spout().get_complete_ms_avg());
-        }
-        ret.putAll(commonStats);
-        putKV(ret, ACKED, aggregateCounts(acked));
-        putKV(ret, FAILED, aggregateCounts(failed));
-        putKV(ret, COMP_LATENCIES, aggregateAverages(completeLatencies, acked));
-
-        return ret;
-    }
-
-    /**
-     * aggregate common stats from a spout/bolt, called in aggregateSpoutStats/aggregateBoltStats
-     */
-    public static <T> Map<String, Map<String, Map<T, Long>>> aggregateCommonStats(List<ExecutorSummary> statsSeq) {
-        Map<String, Map<String, Map<T, Long>>> ret = new HashMap<>();
-
-        List<Map<String, Map<String, Long>>> emitted = new ArrayList<>();
-        List<Map<String, Map<String, Long>>> transferred = new ArrayList<>();
-        for (ExecutorSummary summ : statsSeq) {
-            emitted.add(summ.get_stats().get_emitted());
-            transferred.add(summ.get_stats().get_transferred());
-        }
-        putKV(ret, EMITTED, aggregateCounts(emitted));
-        putKV(ret, TRANSFERRED, aggregateCounts(transferred));
-
-        return ret;
-    }
-
-    /**
-     * filter system streams of aggregated spout/bolt stats if necessary
-     */
-    public static <T> Map<String, Map<String, Map<T, Long>>> preProcessStreamSummary(
-        Map<String, Map<String, Map<T, Long>>> streamSummary, boolean includeSys) {
-        Map<String, Map<T, Long>> emitted = getMapByKey(streamSummary, EMITTED);
-        Map<String, Map<T, Long>> transferred = getMapByKey(streamSummary, TRANSFERRED);
-
-        putKV(streamSummary, EMITTED, filterSysStreams(emitted, includeSys));
-        putKV(streamSummary, TRANSFERRED, filterSysStreams(transferred, includeSys));
-
-        return streamSummary;
-    }
-
-    /**
-     * aggregate count streams by window
-     *
-     * @param stats a Map of value: {win -> stream -> value}
-     * @return a Map of value: {win -> value}
-     */
-    public static <K, V extends Number> Map<String, Long> aggregateCountStreams(
-        Map<String, Map<K, V>> stats) {
-        Map<String, Long> ret = new HashMap<>();
-        for (Map.Entry<String, Map<K, V>> entry : stats.entrySet()) {
-            Map<K, V> value = entry.getValue();
-            long sum = 0l;
-            for (V num : value.values()) {
-                sum += num.longValue();
-            }
-            ret.put(entry.getKey(), sum);
-        }
-        return ret;
-    }
-
-    /**
-     * compute an weighted average from a list of average maps and a corresponding count maps extracted from a list of ExecutorSummary
-     *
-     * @param avgSeq   a list of {win -> global stream id -> avg value}
-     * @param countSeq a list of {win -> global stream id -> count value}
-     * @return a Map of {win -> global stream id -> weighted avg value}
-     */
-    public static <K> Map<String, Map<K, Double>> aggregateAverages(List<Map<String, Map<K, Double>>> avgSeq,
-                                                                    List<Map<String, Map<K, Long>>> countSeq) {
-        Map<String, Map<K, Double>> ret = new HashMap<>();
-
-        Map<String, Map<K, List>> expands = expandAveragesSeq(avgSeq, countSeq);
-        for (Map.Entry<String, Map<K, List>> entry : expands.entrySet()) {
-            String k = entry.getKey();
-
-            Map<K, Double> tmp = new HashMap<>();
-            Map<K, List> inner = entry.getValue();
-            for (K kk : inner.keySet()) {
-                List vv = inner.get(kk);
-                tmp.put(kk, valAvg(((Number) vv.get(0)).doubleValue(), ((Number) vv.get(1)).longValue()));
-            }
-            ret.put(k, tmp);
-        }
-
-        return ret;
-    }
-
-    /**
-     * aggregate weighted average of all streams
-     *
-     * @param avgs   a Map of {win -> stream -> average value}
-     * @param counts a Map of {win -> stream -> count value}
-     * @return a Map of {win -> aggregated value}
-     */
-    public static <K> Map<String, Double> aggregateAvgStreams(Map<String, Map<K, Double>> avgs,
-                                                              Map<String, Map<K, Long>> counts) {
-        Map<String, Double> ret = new HashMap<>();
-
-        Map<String, Map<K, List>> expands = expandAverages(avgs, counts);
-        for (Map.Entry<String, Map<K, List>> entry : expands.entrySet()) {
-            String win = entry.getKey();
-
-            double avgTotal = 0.0;
-            long cntTotal = 0l;
-            Map<K, List> inner = entry.getValue();
-            for (K kk : inner.keySet()) {
-                List vv = inner.get(kk);
-                avgTotal += ((Number) vv.get(0)).doubleValue();
-                cntTotal += ((Number) vv.get(1)).longValue();
-            }
-            ret.put(win, valAvg(avgTotal, cntTotal));
-        }
-
-        return ret;
-    }
-
-    /**
-     * aggregates spout stream stats, returns a Map of {metric -> win -> aggregated value}
-     */
-    public static Map<String, Map> spoutStreamsStats(List<ExecutorSummary> summs, boolean includeSys) {
-        if (summs == null) {
-            return new HashMap<>();
-        }
-        // filter ExecutorSummary's with empty stats
-        List<ExecutorSummary> statsSeq = getFilledStats(summs);
-        return aggregateSpoutStreams(aggregateSpoutStats(statsSeq, includeSys));
-    }
-
-    /**
-     * aggregates bolt stream stats, returns a Map of {metric -> win -> aggregated value}
-     */
-    public static Map<String, Map> boltStreamsStats(List<ExecutorSummary> summs, boolean includeSys) {
-        if (summs == null) {
-            return new HashMap<>();
-        }
-        List<ExecutorSummary> statsSeq = getFilledStats(summs);
-        return aggregateBoltStreams(aggregateBoltStats(statsSeq, includeSys));
-    }
-
-    /**
-     * aggregate all spout streams
-     *
-     * @param stats a Map of {metric -> win -> stream id -> value}
-     * @return a Map of {metric -> win -> aggregated value}
-     */
-    public static Map<String, Map> aggregateSpoutStreams(Map<String, Map> stats) {
-        // actual ret is Map<String, Map<String, Long/Double>>
-        Map<String, Map> ret = new HashMap<>();
-        putKV(ret, ACKED, aggregateCountStreams(getMapByKey(stats, ACKED)));
-        putKV(ret, FAILED, aggregateCountStreams(getMapByKey(stats, FAILED)));
-        putKV(ret, EMITTED, aggregateCountStreams(getMapByKey(stats, EMITTED)));
-        putKV(ret, TRANSFERRED, aggregateCountStreams(getMapByKey(stats, TRANSFERRED)));
-        putKV(ret, COMP_LATENCIES, aggregateAvgStreams(
-            getMapByKey(stats, COMP_LATENCIES), getMapByKey(stats, ACKED)));
-        return ret;
-    }
-
-    /**
-     * aggregate all bolt streams
-     *
-     * @param stats a Map of {metric -> win -> stream id -> value}
-     * @return a Map of {metric -> win -> aggregated value}
-     */
-    public static Map<String, Map> aggregateBoltStreams(Map<String, Map> stats) {
-        Map<String, Map> ret = new HashMap<>();
-        putKV(ret, ACKED, aggregateCountStreams(getMapByKey(stats, ACKED)));
-        putKV(ret, FAILED, aggregateCountStreams(getMapByKey(stats, FAILED)));
-        putKV(ret, EMITTED, aggregateCountStreams(getMapByKey(stats, EMITTED)));
-        putKV(ret, TRANSFERRED, aggregateCountStreams(getMapByKey(stats, TRANSFERRED)));
-        putKV(ret, EXECUTED, aggregateCountStreams(getMapByKey(stats, EXECUTED)));
-        putKV(ret, PROC_LATENCIES, aggregateAvgStreams(
-            getMapByKey(stats, PROC_LATENCIES), getMapByKey(stats, ACKED)));
-        putKV(ret, EXEC_LATENCIES, aggregateAvgStreams(
-            getMapByKey(stats, EXEC_LATENCIES), getMapByKey(stats, EXECUTED)));
-        return ret;
-    }
-
-    /**
-     * aggregate windowed stats from a bolt executor stats with a Map of accumulated stats
-     */
-    public static Map<String, Object> aggBoltExecWinStats(
-        Map<String, Object> accStats, Map<String, Object> newStats, boolean includeSys) {
-        Map<String, Object> ret = new HashMap<>();
-
-        Map<String, Map<String, Number>> m = new HashMap<>();
-        for (Object win : getMapByKey(newStats, EXECUTED).keySet()) {
-            m.put((String) win, aggBoltLatAndCount(
-                (Map) (getMapByKey(newStats, EXEC_LATENCIES)).get(win),
-                (Map) (getMapByKey(newStats, PROC_LATENCIES)).get(win),
-                (Map) (getMapByKey(newStats, EXECUTED)).get(win)));
-        }
-        m = swapMapOrder(m);
-
-        Map<String, Double> win2execLatWgtAvg = getMapByKey(m, EXEC_LAT_TOTAL);
-        Map<String, Double> win2procLatWgtAvg = getMapByKey(m, PROC_LAT_TOTAL);
-        Map<String, Long> win2executed = getMapByKey(m, EXECUTED);
-
-        Map<String, Map<String, Long>> emitted = getMapByKey(newStats, EMITTED);
-        Map<String, Long> win2emitted = mergeWithSumLong(aggregateCountStreams(filterSysStreams(emitted, includeSys)),
-                                                         getMapByKey(accStats, WIN_TO_EMITTED));
-        putKV(ret, WIN_TO_EMITTED, win2emitted);
-
-        Map<String, Map<String, Long>> transferred = getMapByKey(newStats, TRANSFERRED);
-        Map<String, Long> win2transferred = mergeWithSumLong(aggregateCountStreams(filterSysStreams(transferred, includeSys)),
-                                                             getMapByKey(accStats, WIN_TO_TRANSFERRED));
-        putKV(ret, WIN_TO_TRANSFERRED, win2transferred);
-
-        putKV(ret, WIN_TO_EXEC_LAT_WGT_AVG, mergeWithSumDouble(
-            getMapByKey(accStats, WIN_TO_EXEC_LAT_WGT_AVG), win2execLatWgtAvg));
-        putKV(ret, WIN_TO_PROC_LAT_WGT_AVG, mergeWithSumDouble(
-            getMapByKey(accStats, WIN_TO_PROC_LAT_WGT_AVG), win2procLatWgtAvg));
-        putKV(ret, WIN_TO_EXECUTED, mergeWithSumLong(
-            getMapByKey(accStats, WIN_TO_EXECUTED), win2executed));
-        putKV(ret, WIN_TO_ACKED, mergeWithSumLong(
-            aggregateCountStreams(getMapByKey(newStats, ACKED)), getMapByKey(accStats, WIN_TO_ACKED)));
-        putKV(ret, WIN_TO_FAILED, mergeWithSumLong(
-            aggregateCountStreams(getMapByKey(newStats, FAILED)), getMapByKey(accStats, WIN_TO_FAILED)));
-
-        return ret;
-    }
-
-    /**
-     * aggregate windowed stats from a spout executor stats with a Map of accumulated stats
-     */
-    public static Map<String, Object> aggSpoutExecWinStats(
-        Map<String, Object> accStats, Map<String, Object> beat, boolean includeSys) {
-        Map<String, Object> ret = new HashMap<>();
-
-        Map<String, Map<String, Number>> m = new HashMap<>();
-        for (Object win : getMapByKey(beat, ACKED).keySet()) {
-            m.put((String) win, aggSpoutLatAndCount(
-                (Map<String, Double>) (getMapByKey(beat, COMP_LATENCIES)).get(win),
-                (Map<String, Long>) (getMapByKey(beat, ACKED)).get(win)));
-        }
-        m = swapMapOrder(m);
-
-        Map<String, Double> win2compLatWgtAvg = getMapByKey(m, COMP_LAT_TOTAL);
-        Map<String, Long> win2acked = getMapByKey(m, ACKED);
-
-        Map<String, Map<String, Long>> emitted = getMapByKey(beat, EMITTED);
-        Map<String, Long> win2emitted = mergeWithSumLong(aggregateCountStreams(filterSysStreams(emitted, includeSys)),
-                                                         getMapByKey(accStats, WIN_TO_EMITTED));
-        putKV(ret, WIN_TO_EMITTED, win2emitted);
-
-        Map<String, Map<String, Long>> transferred = getMapByKey(beat, TRANSFERRED);
-        Map<String, Long> win2transferred = mergeWithSumLong(aggregateCountStreams(filterSysStreams(transferred, includeSys)),
-                                                             getMapByKey(accStats, WIN_TO_TRANSFERRED));
-        putKV(ret, WIN_TO_TRANSFERRED, win2transferred);
-
-        putKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSumDouble(
-            getMapByKey(accStats, WIN_TO_COMP_LAT_WGT_AVG), win2compLatWgtAvg));
-        putKV(ret, WIN_TO_ACKED, mergeWithSumLong(
-            getMapByKey(accStats, WIN_TO_ACKED), win2acked));
-        putKV(ret, WIN_TO_FAILED, mergeWithSumLong(
-            aggregateCountStreams(getMapByKey(beat, FAILED)), getMapByKey(accStats, WIN_TO_FAILED)));
-
-        return ret;
-    }
-
-
-    /**
-     * aggregate a list of count maps into one map
-     *
-     * @param countsSeq a seq of {win -> GlobalStreamId -> value}
-     */
-    public static <T> Map<String, Map<T, Long>> aggregateCounts(List<Map<String, Map<T, Long>>> countsSeq) {
-        Map<String, Map<T, Long>> ret = new HashMap<>();
-        for (Map<String, Map<T, Long>> counts : countsSeq) {
-            for (Map.Entry<String, Map<T, Long>> entry : counts.entrySet()) {
-                String win = entry.getKey();
-                Map<T, Long> stream2count = entry.getValue();
-
-                if (!ret.containsKey(win)) {
-                    ret.put(win, stream2count);
-                } else {
-                    Map<T, Long> existing = ret.get(win);
-                    for (Map.Entry<T, Long> subEntry : stream2count.entrySet()) {
-                        T stream = subEntry.getKey();
-                        if (!existing.containsKey(stream)) {
-                            existing.put(stream, subEntry.getValue());
-                        } else {
-                            existing.put(stream, subEntry.getValue() + existing.get(stream));
-                        }
-                    }
-                }
-            }
-        }
-        return ret;
-    }
-
-    public static Map<String, Object> aggregateCompStats(
-        String window, boolean includeSys, List<Map<String, Object>> beats, String compType) {
-        boolean isSpout = SPOUT.equals(compType);
-
-        Map<String, Object> initVal = new HashMap<>();
-        putKV(initVal, WIN_TO_ACKED, new HashMap());
-        putKV(initVal, WIN_TO_FAILED, new HashMap());
-        putKV(initVal, WIN_TO_EMITTED, new HashMap());
-        putKV(initVal, WIN_TO_TRANSFERRED, new HashMap());
-
-        Map<String, Object> stats = new HashMap();
-        putKV(stats, EXECUTOR_STATS, new ArrayList());
-        putKV(stats, SID_TO_OUT_STATS, new HashMap());
-        if (isSpout) {
-            putKV(initVal, TYPE, SPOUT);
-            putKV(initVal, WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
-        } else {
-            putKV(initVal, TYPE, BOLT);
-            putKV(initVal, WIN_TO_EXECUTED, new HashMap());
-            putKV(stats, CID_SID_TO_IN_STATS, new HashMap());
-            putKV(initVal, WIN_TO_EXEC_LAT_WGT_AVG, new HashMap());
-            putKV(initVal, WIN_TO_PROC_LAT_WGT_AVG, new HashMap());
-        }
-        putKV(initVal, STATS, stats);
-
-        // iterate through all executor heartbeats
-        for (Map<String, Object> beat : beats) {
-            initVal = aggCompExecStats(window, includeSys, initVal, beat, compType);
-        }
-
-        return initVal;
-    }
-
-    /**
-     * Combines the aggregate stats of one executor with the given map, selecting the appropriate window and including system components as
-     * specified.
-     */
-    public static Map<String, Object> aggCompExecStats(String window, boolean includeSys, Map<String, Object> accStats,
-                                                       Map<String, Object> beat, String compType) {
-        Map<String, Object> ret = new HashMap<>();
-        if (SPOUT.equals(compType)) {
-            ret.putAll(aggSpoutExecWinStats(accStats, getMapByKey(beat, STATS), includeSys));
-            putKV(ret, STATS, mergeAggCompStatsCompPageSpout(
-                getMapByKey(accStats, STATS),
-                aggPreMergeCompPageSpout(beat, window, includeSys)));
-        } else {
-            ret.putAll(aggBoltExecWinStats(accStats, getMapByKey(beat, STATS), includeSys));
-            putKV(ret, STATS, mergeAggCompStatsCompPageBolt(
-                getMapByKey(accStats, STATS),
-                aggPreMergeCompPageBolt(beat, window, includeSys)));
-        }
-        putKV(ret, TYPE, compType);
-
-        return ret;
-    }
-
-    /**
-     * post aggregate component stats: 1. computes execute-latency/process-latency from execute/process latency total 2. computes windowed
-     * weight avgs 3. transform Map keys
-     *
-     * @param compStats accumulated comp stats
-     */
-    public static Map<String, Object> postAggregateCompStats(Map<String, Object> compStats) {
-        Map<String, Object> ret = new HashMap<>();
-
-        String compType = (String) compStats.get(TYPE);
-        Map stats = getMapByKey(compStats, STATS);
-        Integer numTasks = getByKeyOr0(stats, NUM_TASKS).intValue();
-        Integer numExecutors = getByKeyOr0(stats, NUM_EXECUTORS).intValue();
-        Map outStats = getMapByKey(stats, SID_TO_OUT_STATS);
-
-        putKV(ret, TYPE, compType);
-        putKV(ret, NUM_TASKS, numTasks);
-        putKV(ret, NUM_EXECUTORS, numExecutors);
-        putKV(ret, EXECUTOR_STATS, getByKey(stats, EXECUTOR_STATS));
-        putKV(ret, WIN_TO_EMITTED, mapKeyStr(getMapByKey(compStats, WIN_TO_EMITTED)));
-        putKV(ret, WIN_TO_TRANSFERRED, mapKeyStr(getMapByKey(compStats, WIN_TO_TRANSFERRED)));
-        putKV(ret, WIN_TO_ACKED, mapKeyStr(getMapByKey(compStats, WIN_TO_ACKED)));
-        putKV(ret, WIN_TO_FAILED, mapKeyStr(getMapByKey(compStats, WIN_TO_FAILED)));
-
-        if (BOLT.equals(compType)) {
-            Map inStats = getMapByKey(stats, CID_SID_TO_IN_STATS);
-
-            Map inStats2 = new HashMap();
-            for (Object o : inStats.entrySet()) {
-                Map.Entry e = (Map.Entry) o;
-                Object k = e.getKey();
-                Map v = (Map) e.getValue();
-                long executed = getByKeyOr0(v, EXECUTED).longValue();
-                if (executed > 0) {
-                    double executeLatencyTotal = getByKeyOr0(v, EXEC_LAT_TOTAL).doubleValue();
-                    double processLatencyTotal = getByKeyOr0(v, PROC_LAT_TOTAL).doubleValue();
-                    putKV(v, EXEC_LATENCY, executeLatencyTotal / executed);
-                    putKV(v, PROC_LATENCY, processLatencyTotal / executed);
-                } else {
-                    putKV(v, EXEC_LATENCY, 0.0);
-                    putKV(v, PROC_LATENCY, 0.0);
-                }
-                remove(v, EXEC_LAT_TOTAL);
-                remove(v, PROC_LAT_TOTAL);
-                inStats2.put(k, v);
-            }
-            putKV(ret, CID_SID_TO_IN_STATS, inStats2);
-
-            putKV(ret, SID_TO_OUT_STATS, outStats);
-            putKV(ret, WIN_TO_EXECUTED, mapKeyStr(getMapByKey(compStats, WIN_TO_EXECUTED)));
-            putKV(ret, WIN_TO_EXEC_LAT, computeWeightedAveragesPerWindow(
-                compStats, WIN_TO_EXEC_LAT_WGT_AVG, WIN_TO_EXECUTED));
-            putKV(ret, WIN_TO_PROC_LAT, computeWeightedAveragesPerWindow(
-                compStats, WIN_TO_PROC_LAT_WGT_AVG, WIN_TO_EXECUTED));
-        } else {
-            Map outStats2 = new HashMap();
-            for (Object o : outStats.entrySet()) {
-                Map.Entry e = (Map.Entry) o;
-                Object k = e.getKey();
-                Map v = (Map) e.getValue();
-                long acked = getByKeyOr0(v, ACKED).longValue();
-                if (acked > 0) {
-                    double compLatencyTotal = getByKeyOr0(v, COMP_LAT_TOTAL).doubleValue();
-                    putKV(v, COMP_LATENCY, compLatencyTotal / acked);
-                } else {
-                    putKV(v, COMP_LATENCY, 0.0);
-                }
-                remove(v, COMP_LAT_TOTAL);
-                outStats2.put(k, v);
-            }
-            putKV(ret, SID_TO_OUT_STATS, outStats2);
-            putKV(ret, WIN_TO_COMP_LAT, computeWeightedAveragesPerWindow(
-                compStats, WIN_TO_COMP_LAT_WGT_AVG, WIN_TO_ACKED));
-        }
-
-        return ret;
-    }
-
-    /**
-     * aggregate component executor stats
-     *
-     * @param exec2hostPort  a Map of {executor -> host+port}
-     * @param task2component a Map of {task id -> component}
-     * @param beats          a converted HashMap of executor heartbeats, {executor -> heartbeat}
-     * @param window         specified window
-     * @param includeSys     whether to include system streams
-     * @param topologyId     topology id
-     * @param topology       storm topology
-     * @param componentId    component id
-     * @return ComponentPageInfo thrift structure
-     */
-    public static ComponentPageInfo aggCompExecsStats(
-        Map exec2hostPort, Map task2component, Map<List<Integer>, Map<String, Object>> beats,
-        String window, boolean includeSys, String topologyId, StormTopology topology, String componentId) {
-
-        List<Map<String, Object>> beatList =
-            extractDataFromHb(exec2hostPort, task2component, beats, includeSys, topology, componentId);
-        Map<String, Object> compStats = aggregateCompStats(window, includeSys, beatList, componentType(topology, componentId));
-        compStats = postAggregateCompStats(compStats);
-        return thriftifyCompPageData(topologyId, topology, componentId, compStats);
-    }
-
-    /**
-     * aggregate statistics per worker for a topology. Optionally filtering on specific supervisors
-     *
-     * @param stormId          topology id
-     * @param stormName        storm topology
-     * @param task2Component   a Map of {task id -> component}
-     * @param beats            a converted HashMap of executor heartbeats, {executor -> heartbeat}
-     * @param exec2NodePort    a Map of {executor -> host+port}
-     * @param includeSys       whether to include system streams
-     * @param userAuthorized   whether the user is authorized to view topology info
-     * @param filterSupervisor if not null, only return WorkerSummaries for that supervisor
-     * @return List<WorkerSummary> thrift structures
-     */
-    public static List<WorkerSummary> aggWorkerStats(String stormId, String stormName,
-                                                     Map<Integer, String> task2Component,
-                                                     Map<List<Integer>, Map<String, Object>> beats,
-                                                     Map<List<Long>, List<Object>> exec2NodePort,
-                                                     Map<String, String> nodeHost,
-                                                     Map<WorkerSlot, WorkerResources> worker2Resources,
-                                                     boolean includeSys, boolean userAuthorized, String filterSupervisor) {
-
-        // host,port => WorkerSummary
-        HashMap<WorkerSlot, WorkerSummary> workerSummaryMap = new HashMap<>();
-
-        if (exec2NodePort != null) {
-            // for each executor -> node+port pair
-            for (Map.Entry<List<Long>, List<Object>> execNodePort : exec2NodePort.entrySet()) {
-                List<Object> nodePort = execNodePort.getValue();
-                String node = (String) nodePort.get(0);
-                Long port = (Long) nodePort.get(1);
-                String host = nodeHost.get(node);
-                WorkerSlot slot = new WorkerSlot(node, port);
-                WorkerResources resources = worker2Resources.get(slot);
-
-                if (filterSupervisor == null || node.equals(filterSupervisor)) {
-                    WorkerSummary ws = workerSummaryMap.get(slot);
-
-                    if (ws == null) {
-                        ws = new WorkerSummary();
-                        ws.set_host(host);
-                        ws.set_port(port.intValue());
-                        ws.set_supervisor_id(node);
-                        ws.set_topology_id(stormId);
-                        ws.set_topology_name(stormName);
-                        ws.set_num_executors(0);
-                        if (resources != null) {
-                            ws.set_assigned_memonheap(resources.get_mem_on_heap());
-                            ws.set_assigned_memoffheap(resources.get_mem_off_heap());
-                            ws.set_assigned_cpu(resources.get_cpu());
-                        } else {
-                            ws.set_assigned_memonheap(0);
-                            ws.set_assigned_memoffheap(0);
-                            ws.set_assigned_cpu(0);
-                        }
-                        ws.set_component_to_num_tasks(new HashMap<String, Long>());
-                        workerSummaryMap.put(slot, ws);
-                    }
-                    Map<String, Long> componentToNumTasks = ws.get_component_to_num_tasks();
-
-                    // gets min/max task pairs (executors): [1 1] [2 3] ...
-                    List<Long> exec = execNodePort.getKey();
-                    // get executor heartbeat
-                    int hbeatSecs = 0;
-                    if (beats != null) {
-                        Map<String, Object> beat = beats.get(convertExecutor(exec));
-                        hbeatSecs = beat == null ? 0 : (int) beat.get("uptime");
-                    }
-                    ws.set_uptime_secs(hbeatSecs);
-                    ws.set_num_executors(ws.get_num_executors() + 1);
-
-                    // get tasks if the user is authorized for this topology
-                    if (userAuthorized) {
-                        int firstTask = exec.get(0).intValue();
-                        int lastTask = exec.get(1).intValue();
-
-                        // get per task components
-                        for (int task = firstTask; task <= lastTask; task++) {
-                            String component = task2Component.get(task);
-                            // if the component is a system (__*) component and we are hiding
-                            // them in UI, keep going
-                            if (!includeSys && Utils.isSystemId(component)) {
-                                continue;
-                            }
-
-                            // good to go, increment # of tasks this component is being executed on
-                            Long counter = componentToNumTasks.get(component);
-                            if (counter == null) {
-                                counter = new Long(0);
-                            }
-                            componentToNumTasks.put(component, counter + 1);
-                        }
-                    }
-                }
-            }
-        }
-        return new ArrayList<WorkerSummary>(workerSummaryMap.values());
-    }
-
-    /**
-     * Aggregate statistics per worker for a topology. Optionally filtering on specific supervisors
-     *
-     * Convenience overload when called from the topology page code (in that case we want data for all workers in the topology, not filtered
-     * by supervisor)
-     *
-     * @param stormId        topology id
-     * @param stormName      storm topology
-     * @param task2Component a Map of {task id -> component}
-     * @param beats          a converted HashMap of executor heartbeats, {executor -> heartbeat}
-     * @param exec2NodePort  a Map of {executor -> host+port}
-     * @param includeSys     whether to include system streams
-     * @param userAuthorized whether the user is authorized to view topology info
-     * @return List<WorkerSummary> thrift structures
-     */
-    public static List<WorkerSummary> aggWorkerStats(String stormId, String stormName,
-                                                     Map<Integer, String> task2Component,
-                                                     Map<List<Integer>, Map<String, Object>> beats,
-                                                     Map<List<Long>, List<Object>> exec2NodePort,
-                                                     Map<String, String> nodeHost,
-                                                     Map<WorkerSlot, WorkerResources> worker2Resources,
-                                                     boolean includeSys, boolean userAuthorized) {
-        return aggWorkerStats(stormId, stormName,
-                              task2Component, beats, exec2NodePort, nodeHost, worker2Resources,
-                              includeSys, userAuthorized, null);
-    }
-
-    // =====================================================================================
-    // convert thrift stats to java maps
-    // =====================================================================================
-
-    /**
-     * convert thrift executor heartbeats into a java HashMap
-     */
-    public static Map<List<Integer>, Map<String, Object>> convertExecutorBeats(Map<ExecutorInfo, ExecutorBeat> beats) {
-        Map<List<Integer>, Map<String, Object>> ret = new HashMap<>();
-        for (Map.Entry<ExecutorInfo, ExecutorBeat> beat : beats.entrySet()) {
-            ExecutorInfo executorInfo = beat.getKey();
-            ExecutorBeat executorBeat = beat.getValue();
-            ret.put(Lists.newArrayList(executorInfo.get_task_start(), executorInfo.get_task_end()),
-                    convertZkExecutorHb(executorBeat));
-        }
-
-        return ret;
-    }
-
-    /**
-     * convert {@link SupervisorWorkerHeartbeat} to nimbus local report executor heartbeats
-     *
-     * @param workerHeartbeat
-     * @return
-     */
-    public static Map<List<Integer>, Map<String, Object>> convertWorkerBeats(SupervisorWorkerHeartbeat workerHeartbeat) {
-        Map<List<Integer>, Map<String, Object>> ret = new HashMap<>();
-        for (ExecutorInfo executorInfo : workerHeartbeat.get_executors()) {
-            Map<String, Object> reportBeat = new HashMap<>();
-            reportBeat.put(TIME_SECS, workerHeartbeat.get_time_secs());
-            ret.put(Lists.newArrayList(executorInfo.get_task_start(), executorInfo.get_task_end()),
-                    reportBeat);
-        }
-
-        return ret;
-    }
-
-    /**
-     * convert thrift ExecutorBeat into a java HashMap
-     */
-    public static Map<String, Object> convertZkExecutorHb(ExecutorBeat beat) {
-        Map<String, Object> ret = new HashMap<>();
-        if (beat != null) {
-            ret.put(TIME_SECS, beat.getTimeSecs());
-            ret.put(UPTIME, beat.getUptime());
-            ret.put(STATS, convertExecutorStats(beat.getStats()));
-        }
-
-        return ret;
-    }
-
-    /**
-     * convert a thrift worker heartbeat into a java HashMap
-     *
-     * @param workerHb
-     * @return
-     */
-    public static Map<String, Object> convertZkWorkerHb(ClusterWorkerHeartbeat workerHb) {
-        Map<String, Object> ret = new HashMap<>();
-        if (workerHb != null) {
-            ret.put("storm-id", workerHb.get_storm_id());
-            ret.put(EXECUTOR_STATS, convertExecutorsStats(workerHb.get_executor_stats()));
-            ret.put(UPTIME, workerHb.get_uptime_secs());
-            ret.put(TIME_SECS, workerHb.get_time_secs());
-        }
-        return ret;
-    }
-
-    /**
-     * convert executors stats into a HashMap, note that ExecutorStats are remained unchanged
-     */
-    public static Map<List<Integer>, ExecutorStats> convertExecutorsStats(Map<ExecutorInfo, ExecutorStats> stats) {
-        Map<List<Integer>, ExecutorStats> ret = new HashMap<>();
-        for (Map.Entry<ExecutorInfo, ExecutorStats> entry : stats.entrySet()) {
-            ExecutorInfo executorInfo = entry.getKey();
-            ExecutorStats executorStats = entry.getValue();
-
-            ret.put(Lists.newArrayList(executorInfo.get_task_start(), executorInfo.get_task_end()),
-                    executorStats);
-        }
-        return ret;
-    }
-
-    /**
-     * convert thrift ExecutorStats structure into a java HashMap
-     */
-    public static Map<String, Object> convertExecutorStats(ExecutorStats stats) {
-        Map<String, Object> ret = new HashMap<>();
-
-        putKV(ret, EMITTED, stats.get_emitted());
-        putKV(ret, TRANSFERRED, stats.get_transferred());
-        putKV(ret, RATE, stats.get_rate());
-
-        if (stats.get_specific().is_set_bolt()) {
-            ret.putAll(convertSpecificStats(stats.get_specific().get_bolt()));
-            putKV(ret, TYPE, BOLT);
-        } else {
-            ret.putAll(convertSpecificStats(stats.get_specific().get_spout()));
-            putKV(ret, TYPE, SPOUT);
-        }
-
-        return ret;
-    }
-
-    private static Map<String, Object> convertSpecificStats(SpoutStats stats) {
-        Map<String, Object> ret = new HashMap<>();
-        putKV(ret, ACKED, stats.get_acked());
-        putKV(ret, FAILED, stats.get_failed());
-        putKV(ret, COMP_LATENCIES, stats.get_complete_ms_avg());
-
-        return ret;
-    }
-
-    private static Map<String, Object> convertSpecificStats(BoltStats stats) {
-        Map<String, Object> ret = new HashMap<>();
-
-        Map acked = windowSetConverter(stats.get_acked(), FROM_GSID, IDENTITY);
-        Map failed = windowSetConverter(stats.get_failed(), FROM_GSID, IDENTITY);
-        Map processAvg = windowSetConverter(stats.get_process_ms_avg(), FROM_GSID, IDENTITY);
-        Map executed = windowSetConverter(stats.get_executed(), FROM_GSID, IDENTITY);
-        Map executeAvg = windowSetConverter(stats.get_execute_ms_avg(), FROM_GSID, IDENTITY);
-
-        putKV(ret, ACKED, acked);
-        putKV(ret, FAILED, failed);
-        putKV(ret, PROC_LATENCIES, processAvg);
-        putKV(ret, EXECUTED, executed);
-        putKV(ret, EXEC_LATENCIES, executeAvg);
-
-        return ret;
-    }
-
-    /**
-     * extract a list of host port info for specified component
-     *
-     * @param exec2hostPort  {executor -> host+port}
-     * @param task2component {task id -> component}
-     * @param includeSys     whether to include system streams
-     * @param compId         component id
-     * @return a list of host+port
-     */
-    public static List<Map<String, Object>> extractNodeInfosFromHbForComp(
-        Map<List<? extends Number>, List<Object>> exec2hostPort, Map<Integer, String> task2component, boolean includeSys, String compId) {
-        List<Map<String, Object>> ret = new ArrayList<>();
-
-        Set<List> hostPorts = new HashSet<>();
-        for (Entry<List<? extends Number>, List<Object>> entry : exec2hostPort.entrySet()) {
-            List<? extends Number> key = entry.getKey();
-            List<Object> value = entry.getValue();
-
-            Integer start = key.get(0).intValue();
-            String host = (String) value.get(0);
-            Integer port = (Integer) value.get(1);
-            String comp = task2component.get(start);
-            if ((compId == null || compId.equals(comp)) && (includeSys || !Utils.isSystemId(comp))) {
-                hostPorts.add(Lists.newArrayList(host, port));
-            }
-        }
-
-        for (List hostPort : hostPorts) {
-            Map<String, Object> m = new HashMap<>();
-            putKV(m, HOST, hostPort.get(0));
-            putKV(m, PORT, hostPort.get(1));
-            ret.add(m);
-        }
-
-        return ret;
-    }
-
-    /**
-     * update all executor heart beats (legacy ZK heartbeat compatibility)
-     *
-     * @param cache         existing heart beats cache
-     * @param executorBeats new heart beats
-     * @param executors     all executors
-     * @param timeout       timeout
-     * @return a HashMap of updated executor heart beats
-     */
-    public static Map<List<Integer>, Map<String, Object>> updateHeartbeatCacheFromZkHeartbeat(Map<List<Integer>, Map<String, Object>> cache,
-                                                                                              Map<List<Integer>, Map<String, Object>>
-                                                                                                  executorBeats,
-                                                                                              Set<List<Integer>> executors,
-                                                                                              Integer timeout) {
-        Map<List<Integer>, Map<String, Object>> ret = new HashMap<>();
-        if (cache == null && executorBeats == null) {
-            return ret;
-        }
-
-        if (cache == null) {
-            cache = new HashMap<>();
-        }
-        if (executorBeats == null) {
-            executorBeats = new HashMap<>();
-        }
-
-        for (List<Integer> executor : executors) {
-            ret.put(executor, updateExecutorCache(cache.get(executor), executorBeats.get(executor), timeout));
-        }
-
-        return ret;
-    }
-
-
-    // =====================================================================================
-    // heartbeats related
-    // =====================================================================================
-
-    /**
-     * update all executor heart beats. TODO: should move this method to nimbus when nimbus.clj is translated
-     *
-     * @param cache         existing heart beats cache
-     * @param executorBeats new heart beats
-     * @param executors     all executors
-     * @param timeout       timeout
-     */
-    public static void updateHeartbeatCache(Map<List<Integer>, Map<String, Object>> cache,
-                                            Map<List<Integer>, Map<String, Object>> executorBeats, Set<List<Integer>> executors,
-                                            Integer timeout) {
-        //if not executor beats, refresh is-timed-out of the cache which is done by master
-        if (executorBeats == null) {
-            for (Map.Entry<List<Integer>, Map<String, Object>> executorbeat : cache.entrySet()) {
-                Map<String, Object> beat = executorbeat.getValue();
-                beat.put("is-timed-out", Time.deltaSecs((Integer) beat.get("nimbus-time")) >= timeout);
-            }
-            return;
-        }
-        //else refresh nimbus-time and executor-reported-time by heartbeats reporting
-        for (List<Integer> executor : executors) {
-            cache.put(executor, updateExecutorCache(cache.get(executor), executorBeats.get(executor), timeout));
-        }
-    }
-
-    // TODO: should move this method to nimbus when nimbus.clj is translated
-    public static Map<String, Object> updateExecutorCache(
-        Map<String, Object> currBeat, Map<String, Object> newBeat, Integer timeout) {
-        Map<String, Object> ret = new HashMap<>();
-
-        Integer lastNimbusTime = null, lastReportedTime = null;
-        if (currBeat != null) {
-            lastNimbusTime = (Integer) currBeat.get("nimbus-time");
-            lastReportedTime = (Integer) currBeat.get("executor-reported-time");
-        }
-
-        Integer reportedTime = null;
-        if (newBeat != null) {
-            reportedTime = (Integer) newBeat.get(TIME_SECS);
-        }
-
-        if (reportedTime == null) {
-            if (lastReportedTime != null) {
-                reportedTime = lastReportedTime;
-            } else {
-                reportedTime = lastReportedTime = 0;
-            }
-        }
-
-        if (lastNimbusTime == null || !reportedTime.equals(lastReportedTime)) {
-            lastNimbusTime = Time.currentTimeSecs();
-        }
-
-        ret.put("is-timed-out", Time.deltaSecs(lastNimbusTime) >= timeout);
-        ret.put("nimbus-time", lastNimbusTime);
-        ret.put("executor-reported-time", reportedTime);
-
-        return ret;
-    }
-
-
-    /**
-     * extracts a list of executor data from heart beats
-     */
-    public static List<Map<String, Object>> extractDataFromHb(Map executor2hostPort, Map task2component,
-                                                              Map<List<Integer>, Map<String, Object>> beats,
-                                                              boolean includeSys, StormTopology topology) {
-        return extractDataFromHb(executor2hostPort, task2component, beats, includeSys, topology, null);
-    }
-
-    public static List<Map<String, Object>> extractDataFromHb(Map executor2hostPort, Map task2component,
-                                                              Map<List<Integer>, Map<String, Object>> beats,
-                                                              boolean includeSys, StormTopology topology, String compId) {
-        List<Map<String, Object>> ret = new ArrayList<>();
-        if (executor2hostPort == null || beats == null) {
-            return ret;
-        }
-        for (Object o : executor2hostPort.entrySet()) {
-            Map.Entry entry = (Map.Entry) o;
-            List executor = (List) entry.getKey();
-            List hostPort = (List) entry.getValue();
-
-            Integer start = ((Number) executor.get(0)).intValue();
-            Integer end = ((Number) executor.get(1)).intValue();
-
-            String host = (String) hostPort.get(0);
-            Integer port = ((Number) hostPort.get(1)).intValue();
-
-            Map<String, Object> beat = beats.get(convertExecutor(executor));
-            if (beat == null) {
-                continue;
-            }
-            String id = (String) task2component.get(start);
-
-            Map<String, Object> m = new HashMap<>();
-            if ((compId == null || compId.equals(id)) && (includeSys || !Utils.isSystemId(id))) {
-                putKV(m, "exec-id", entry.getKey());
-                putKV(m, "comp-id", id);
-                putKV(m, NUM_TASKS, end - start + 1);
-                putKV(m, HOST, host);
-                putKV(m, PORT, port);
-
-                Map stats = getMapByKey(beat, STATS);
-                putKV(m, UPTIME, beat.get(UPTIME));
-                putKV(m, STATS, stats);
-
-                String type = componentType(topology, compId);
-                if (type != null) {
-                    putKV(m, TYPE, type);
-                } else {
-                    putKV(m, TYPE, stats.get(TYPE));
-                }
-                ret.add(m);
-            }
-        }
-        return ret;
-    }
-
-    /**
-     * compute weighted avg from a Map of stats and given avg/count keys
-     *
-     * @param accData    a Map of {win -> key -> value}
-     * @param wgtAvgKey  weighted average key
-     * @param divisorKey count key
-     * @return a Map of {win -> weighted avg value}
-     */
-    private static Map<String, Double> computeWeightedAveragesPerWindow(Map<String, Object> accData,
-                                                                        String wgtAvgKey, String divisorKey) {
-        Map<String, Double> ret = new HashMap<>();
-        for (Object o : getMapByKey(accData, wgtAvgKey).entrySet()) {
-            Map.Entry e = (Map.Entry) o;
-            Object window = e.getKey();
-            double wgtAvg = ((Number) e.getValue()).doubleValue();
-            long divisor = ((Number) getMapByKey(accData, divisorKey).get(window)).longValue();
-            if (divisor > 0) {
-                ret.put(window.toString(), wgtAvg / divisor);
-            }
-        }
-        return ret;
-    }
-
-    /**
-     * convert a List<Long> executor to java List<Integer>
-     */
-    public static List<Integer> convertExecutor(List<Long> executor) {
-        return Lists.newArrayList(executor.get(0).intValue(), executor.get(1).intValue());
-    }
-
-    /**
-     * computes max bolt capacity
-     *
-     * @param executorSumms a list of ExecutorSummary
-     * @return max bolt capacity
-     */
-    public static double computeBoltCapacity(List<ExecutorSummary> executorSumms) {
-        double max = 0.0;
-        for (ExecutorSummary summary : executorSumms) {
-            double capacity = computeExecutorCapacity(summary);
-            if (capacity > max) {
-                max = capacity;
-            }
-        }
-        return max;
-    }
-
-    public static double computeExecutorCapacity(ExecutorSummary summary) {
-        ExecutorStats stats = summary.get_stats();
-        if (stats == null) {
-            return 0.0;
-        } else {
-            // actual value of m is: Map<String, Map<String/GlobalStreamId, Long/Double>> ({win -> stream -> value})
-            Map<String, Map> m = aggregateBoltStats(Lists.newArrayList(summary), true);
-            // {metric -> win -> value} ==> {win -> metric -> value}
-            m = swapMapOrder(aggregateBoltStreams(m));
-            // {metric -> value}
-            Map data = getMapByKey(m, TEN_MIN_IN_SECONDS_STR);
-
-            int uptime = summary.get_uptime_secs();
-            int win = Math.min(uptime, TEN_MIN_IN_SECONDS);
-            long executed = getByKeyOr0(data, EXECUTED).longValue();
-            double latency = getByKeyOr0(data, EXEC_LATENCIES).doubleValue();
-            if (win > 0) {
-                return executed * latency / (1000 * win);
-            }
-            return 0.0;
-        }
-    }
-
-    /**
-     * filter ExecutorSummary whose stats is null
-     *
-     * @param summs a list of ExecutorSummary
-     * @return filtered summs
-     */
-    public static List<ExecutorSummary> getFilledStats(List<ExecutorSummary> summs) {
-        List<ExecutorSummary> ret = new ArrayList<>();
-        for (ExecutorSummary summ : summs) {
-            if (summ.get_stats() != null) {
-                ret.add(summ);
-            }
-        }
-        return ret;
-    }
-
-    private static <K, V> Map<String, V> mapKeyStr(Map<K, V> m) {
-        Map<String, V> ret = new HashMap<>();
-        for (Map.Entry<K, V> entry : m.entrySet()) {
-            ret.put(entry.getKey().toString(), entry.getValue());
-        }
-        return ret;
-    }
-
-    private static <K1, K2> long sumStreamsLong(Map<K1, Map<K2, ?>> m, String key) {
-        long sum = 0;
-        if (m == null) {
-            return sum;
-        }
-        for (Map<K2, ?> v : m.values()) {
-            for (Map.Entry<K2, ?> entry : v.entrySet()) {
-                if (entry.getKey().equals(key)) {
-                    sum += ((Number) entry.getValue()).longValue();
-                }
-            }
-        }
-        return sum;
-    }
-
-    private static <K1, K2> double sumStreamsDouble(Map<K1, Map<K2, ?>> m, String key) {
-        double sum = 0;
-        if (m == null) {
-            return sum;
-        }
-        for (Map<K2, ?> v : m.values()) {
-            for (Map.Entry<K2, ?> entry : v.entrySet()) {
-                if (entry.getKey().equals(key)) {
-                    sum += ((Number) entry.getValue()).doubleValue();
-                }
-            }
-        }
-        return sum;
-    }
-
-    /**
-     * same as clojure's (merge-with merge m1 m2)
-     */
-    private static Map mergeMaps(Map m1, Map m2) {
-        if (m2 == null) {
-            return m1;
-        }
-        for (Object o : m2.entrySet()) {
-            Map.Entry entry = (Map.Entry) o;
-            Object k = entry.getKey();
-
-            Map existing = (Map) m1.get(k);
-            if (existing == null) {
-                m1.put(k, entry.getValue());
-            } else {
-                existing.putAll((Map) m2.get(k));
-            }
-        }
-        return m1;
-    }
-
-
-    /**
-     * filter system streams from stats
-     *
-     * @param stream2stat { stream id -> value }
-     * @param includeSys  whether to filter system streams
-     * @return filtered stats
-     */
-    private static <K, V> Map<K, V> filterSysStreams2Stat(Map<K, V> stream2stat, boolean includeSys) {
-        LOG.trace("Filter Sys Streams2Stat {}", stream2stat);
-        if (!includeSys) {
-            for (Iterator itr = stream2stat.keySet().iterator(); itr.hasNext(); ) {
-                Object key = itr.next();
-                if (key instanceof String && Utils.isSystemId((String) key)) {
-                    itr.remove();
-                }
-            }
-        }
-        return stream2stat;
-    }
-
-    /**
-     * filter system streams from stats
-     *
-     * @param stats      { win -> stream id -> value }
-     * @param includeSys whether to filter system streams
-     * @return filtered stats
-     */
-    private static <K, V> Map<String, Map<K, V>> filterSysStreams(Map<String, Map<K, V>> stats, boolean includeSys) {
-        LOG.trace("Filter Sys Streams {}", stats);
-        if (!includeSys) {
-            for (Iterator<String> itr = stats.keySet().iterator(); itr.hasNext(); ) {
-                String winOrStream = itr.next();
-                Map<K, V> stream2stat = stats.get(winOrStream);
-                for (Iterator subItr = stream2stat.keySet().iterator(); subItr.hasNext(); ) {
-                    Object key = subItr.next();
-                    if (key instanceof String && Utils.isSystemId((String) key)) {
-                        subItr.remove();
-                    }
-                }
-            }
-        }
-        return stats;
-    }
-
-    /**
-     * equals to clojure's: (merge-with (partial merge-with sum-or-0) acc-out spout-out)
-     */
-    private static <K1, K2> Map<K1, Map<K2, Number>> fullMergeWithSum(Map<K1, Map<K2, ?>> m1,
-                                                                      Map<K1, Map<K2, ?>> m2) {
-        Set<K1> allKeys = new HashSet<>();
-        if (m1 != null) {
-            allKeys.addAll(m1.keySet());
-        }
-        if (m2 != null) {
-            allKeys.addAll(m2.keySet());
-        }
-
-        Map<K1, Map<K2, Number>> ret = new HashMap<>();
-        for (K1 k : allKeys) {
-            Map<K2, ?> mm1 = null, mm2 = null;
-            if (m1 != null) {
-                mm1 = m1.get(k);
-            }
-            if (m2 != null) {
-                mm2 = m2.get(k);
-            }
-            ret.put(k, mergeWithSum(mm1, mm2));
-        }
-
-        return ret;
-    }
-
-    private static <K> Map<K, Number> mergeWithSum(Map<K, ?> m1, Map<K, ?> m2) {
-        Map<K, Number> ret = new HashMap<>();
-
-        Set<K> allKeys = new HashSet<>();
-        if (m1 != null) {
-            allKeys.addAll(m1.keySet());
-        }
-        if (m2 != null) {
-            allKeys.addAll(m2.keySet());
-        }
-
-        for (K k : allKeys) {
-            Number n1 = getOr0(m1, k);
-            Number n2 = getOr0(m2, k);
-            if (n1 instanceof Long) {
-                ret

<TRUNCATED>

[7/7] storm git commit: Merge branch 'STORM-3162' of https://github.com/revans2/incubator-storm into STORM-3162

Posted by bo...@apache.org.
Merge branch 'STORM-3162' of https://github.com/revans2/incubator-storm into STORM-3162

STORM-3162: Cleanup heartbeats cache and make it thread safe

This closes #2836


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

Branch: refs/heads/master
Commit: 83fea15d7bf240453d35a18958bece29df5489c9
Parents: eaed3cb 4c1ba85
Author: Robert Evans <ev...@yahoo-inc.com>
Authored: Mon Sep 17 15:02:58 2018 -0500
Committer: Robert Evans <ev...@yahoo-inc.com>
Committed: Mon Sep 17 15:02:58 2018 -0500

----------------------------------------------------------------------
 storm-client/pom.xml                            |    2 +-
 .../org/apache/storm/daemon/worker/Worker.java  |   10 +-
 .../jvm/org/apache/storm/executor/Executor.java |    8 +-
 .../storm/executor/bolt/BoltExecutor.java       |    4 +-
 .../storm/executor/spout/SpoutExecutor.java     |    4 +-
 .../apache/storm/stats/BoltExecutorStats.java   |   10 +-
 .../org/apache/storm/stats/ClientStatsUtil.java |  202 ++
 .../jvm/org/apache/storm/stats/StatsUtil.java   | 2610 ------------------
 .../test/clj/org/apache/storm/nimbus_test.clj   |   24 +-
 storm-server/pom.xml                            |    2 +-
 .../storm/daemon/nimbus/HeartbeatCache.java     |  218 ++
 .../org/apache/storm/daemon/nimbus/Nimbus.java  |   83 +-
 .../java/org/apache/storm/stats/StatsUtil.java  | 2385 ++++++++++++++++
 13 files changed, 2851 insertions(+), 2711 deletions(-)
----------------------------------------------------------------------